-
Notifications
You must be signed in to change notification settings - Fork 7.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
ZOOKEEPER-3072: Throttle race condition fix #563
Conversation
e27f95a
to
0561feb
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice catch @bothejjms , thanks for your first contribution. ;)
@@ -1124,6 +1124,7 @@ public void processPacket(ServerCnxn cnxn, ByteBuffer incomingBuffer) throws IOE | |||
} | |||
return; | |||
} else { | |||
cnxn.incrOutstandingRequests(h); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I have 2 observations here which probably don't make a big difference but might worse to consider.
- First, the return statements in the if branches are not required anymore, because there's no more statement at the end of the method anymore,
- Second, moving
cnxn.incrOutstandingRequests(h)
here means that from now on you'll trigger throttling forsasl
requests too, which was not the case previously. Same forauth
packets which I believe was done intentionally.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
hmm right. That return was not there in 3.5.3 where I have spotted the issue. I have missed it when I have moved my change to master.
I see ZOOKEEPER-2785 introduced it. I will update my pr and move incr to the else branch to avoid sasl throttling.
@bothejjms What do you mean by "pretty reliably" exactly?
|
On "pretty reliably" I mean the test has failed for me like 90% of the time with the original code but the result can differ on different machines since it is a race condition. After the fix the test passed on my machine always. I am not sure yet why it fails on jenkins. For me the test takes 40 sec on my VM which is not particularly strong. I am also not satisfied with this test. I just wanted to prove that the race condition is there. Instead of the test I could add a description on how to reproduce and skip permanent testing for it. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think we should include this test with the unit tests. As you mentioned @bothejjms , maybe just write a description about this? This seems unreliable in terms of flakiness and runtime, especially on Apache Jenkins servers, which are often overloaded.
hammers[i].start(); | ||
} | ||
LOG.info("Started hammers"); | ||
Thread.sleep(30000); // allow the clients to run for max 5sec |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: This is 30 seconds not five as in the comment
@bothejjms Sorry, there's a typo in my previous comment: it was 45 seconds on my machine and it literally killed the entire machine which I think isn't acceptable on Jenkins slaves. I'd give it a try with ThreadPoolExecutor at the first place and dig into why it's not 100% reliable currently. If there's no success with a few days work, just skip adding test here. |
a796cad
to
a2eea5b
Compare
I have tweaked the test to use significantly less threads and be faster. Unfortunately it still fails on jenkins. :( I am not sure how ThreadPoolExecutor would help with this. It will spin up the same amount of threads in background, isn't it? |
a2eea5b
to
c68063b
Compare
@bothejjms It would spin up only a limited amount of threads, but that wouldn't help either as you said. You literally want 100 clients simultaneously sending requests until test stops. I don't how to do this properly. I suspect |
c68063b
to
8ae1fcf
Compare
I have removed the test for now as I don't have a good way to test this race condition. I can be reproduced easily by starting a server where the globalOutstandingLimit is 1 and sending a lot exists requests. There is a good chance that one session will stuck in a throttled state despite it has no active requests. |
8ae1fcf
to
3233a64
Compare
Thanks @bothejjms . I think the patch can be accepted now without the test. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 i have a minor refactoring suggestion, but i'm fine if we want to commit as is.
@@ -1128,9 +1128,9 @@ public void processPacket(ServerCnxn cnxn, ByteBuffer incomingBuffer) throws IOE | |||
Record rsp = processSasl(incomingBuffer,cnxn); | |||
ReplyHeader rh = new ReplyHeader(h.getXid(), 0, KeeperException.Code.OK.intValue()); | |||
cnxn.sendResponse(rh,rsp, "response"); // not sure about 3rd arg..what is it? | |||
return; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
it would be nice to keep this return since it matches the handling of the other auth logic above.
it would also be nice if this was an
} else if (h.getType() == OpCode.sasl) {
clause and the
} else {
was done outside of the if since all the other blocks will have returned. i think it makes the logic easier to follow.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I have refactored like that.
Returns are actually unnecessary but I have consistently added them now.
Making the throttle check before passing over the request to the next thread will prevent the possibility of throttling code running after unthrottle
3233a64
to
d1756b9
Compare
I have refactored the branches as suggested. |
thank you @bothejjms ! |
Making the throttle check before passing over the request to the next thread will prevent the possibility of throttling code running after unthrottle Added an additional async hammer thread which is pretty reliably reproduces the race condition. The globalOutstandingLimit is decreased so throttling code is executed. Author: Botond Hejj <[email protected]> Reviewers: Andor Molnár <[email protected]>, Norbert Kalmar <[email protected]>, Benjamin Reed <[email protected]> Closes #563 from bothejjms/ZOOKEEPER-3072 (cherry picked from commit 2a372fc) Signed-off-by: Benjamin Reed <[email protected]>
Making the throttle check before passing over the request to the next thread will prevent the possibility of throttling code running after unthrottle Added an additional async hammer thread which is pretty reliably reproduces the race condition. The globalOutstandingLimit is decreased so throttling code is executed. Author: Botond Hejj <[email protected]> Reviewers: Andor Molnár <[email protected]>, Norbert Kalmar <[email protected]>, Benjamin Reed <[email protected]> Closes apache#563 from bothejjms/ZOOKEEPER-3072
Making the throttle check before passing over the request to the next thread will prevent the possibility of throttling code running after unthrottle
Added an additional async hammer thread which is pretty reliably reproduces the race condition. The globalOutstandingLimit is decreased so throttling code is executed.