Bookies should not queue read request indefinitely#706
Bookies should not queue read request indefinitely#706agresch wants to merge 5 commits intoapache:masterfrom
Conversation
|
FAILURE --none-- |
|
|
||
| counter.await(); | ||
|
|
||
| // assertEquals(BKException.Code.OK, rcFirstReadOperation.get()); |
There was a problem hiding this comment.
Shouldn't be commented out.
There was a problem hiding this comment.
Thanks, it's commented out in our code base, but seems to run with the assert fine.
| @Override | ||
| public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) { | ||
| if (rc == BKException.Code.NotEnoughBookiesException) { | ||
| receivedTooManyRequestsException.set(true); |
There was a problem hiding this comment.
Why does it get this error? Shouldn't it be TooManyRequestException?
There was a problem hiding this comment.
See the comment on line 82. Unfortunately, I'm not the author, and I'm new to the code base.
| * of tasks to be scheduled in the thread's queue. | ||
| * | ||
| */ | ||
| public class QueueAssessibleExecutorService implements ListeningScheduledExecutorService { |
There was a problem hiding this comment.
The name isn't very good. Call it something like BoundedScheduledExecutorService
| protected StatsLogger statsLogger = NullStatsLogger.INSTANCE; | ||
| protected boolean traceTaskExecution = false; | ||
| protected long warnTimeMicroSec = WARN_TIME_MICRO_SEC_DEFAULT; | ||
| protected int maxTasksInQueue = -1; |
There was a problem hiding this comment.
define -1 as a constant (NO_LIMIT or something).
| * of tasks to be scheduled in the thread's queue. | ||
| * | ||
| */ | ||
| public class QueueAssessibleExecutorService implements ListeningScheduledExecutorService { |
There was a problem hiding this comment.
You can reduce the number of useless implemented methods, like shutdown, by extending from ForwardingListeningExecutorService.
| import java.util.concurrent.TimeUnit; | ||
|
|
||
| /** | ||
| * Implements {@link ListeningScheduledExecutorService} and allows limiting the number |
There was a problem hiding this comment.
There's something I don't like about extending scheduled executor service to be bounded, but I can't quite put my finger on it. Surely we're not the first people to need something like this, but JDK never implemented it (according to the code to make execution handling easier[1]), guava doesn't implement it.
As a review comment, this is neither here nor there, but I'd like someone else to review to reassure me it's ok (@sijie ?).
|
FAILURE --none-- |
|
SUCCESS --none-- |
| * Write operations failed due to bookies are readonly. | ||
| */ | ||
| int WriteOnReadOnlyBookieException = -104; | ||
| int TooManyRequestsException = -105; |
There was a problem hiding this comment.
It may need to add a comments for this exception as other exception here.
| */ | ||
| int WriteOnReadOnlyBookieException = -104; | ||
| int TooManyRequestsException = -105; | ||
| //-105 reserved for TooManyRequestsException |
| @@ -0,0 +1,106 @@ | |||
| package org.apache.bookkeeper.client; | |||
There was a problem hiding this comment.
Please add license header before this line.
There was a problem hiding this comment.
it is actually very interesting... this license header is missed in this file, but the travis build passes...
There was a problem hiding this comment.
Maybe because it's a test?
| LOG.debug("Failed to process request to add entry at {}:{}. Too many pending requests", | ||
| r.getAddRequest().getLedgerId(), r.getAddRequest().getEntryId()); | ||
| } | ||
| BookkeeperProtocol.AddResponse.Builder addResponse = BookkeeperProtocol.AddResponse.newBuilder() // |
| r.getReadRequest().getLedgerId(), r.getReadRequest().getEntryId()); | ||
| } | ||
| BookkeeperProtocol.ReadResponse.Builder readResponse = | ||
| BookkeeperProtocol.ReadResponse.newBuilder() // |
|
FAILURE --none-- |
|
@agresch , Would you please solve the conflicting file? then we can ship it. |
Integrating some changes made to Yahoo bookkeeper by Matteo Merli. The apache bookkeeper thread pool in use by OrderedScheduler is different, and does not have access to setting a queue size limit, so I created QueueAssessibleExecutorService, and check the queue size instead. Other than that, this code change is pretty similar to Yahoo's. Original bug description for this change: Since we are using a thread pool to handle read requests in bookies, we have seen that when the Bookie read IO is maxed out, the requests are being accumulated in the bookie. Essentially, the bookie is busy serving read requests that have already been timed out on the client side (end to end read latency can reach hours..). The queue keeps growing indefinitely, leading to OOM errors and bookie restart. All unit tests pass for me locally with this change. Author: Aaron Gresch <agresch@yahoo-inc.com> Reviewers: Ivan Kelly <ivan@ivankelly.net>, Jia Zhai <None>, Sijie Guo <sijie@apache.org> This closes #706 from agresch/agresch_cms1254 (cherry picked from commit ed7696b) Signed-off-by: Sijie Guo <sijie@apache.org>
Integrating some changes made to Yahoo bookkeeper by Matteo Merli. The apache bookkeeper thread pool in use by OrderedScheduler is different, and does not have access to setting a queue size limit, so I created QueueAssessibleExecutorService, and check the queue size instead. Other than that, this code change is pretty similar to Yahoo's. Original bug description for this change: Since we are using a thread pool to handle read requests in bookies, we have seen that when the Bookie read IO is maxed out, the requests are being accumulated in the bookie. Essentially, the bookie is busy serving read requests that have already been timed out on the client side (end to end read latency can reach hours..). The queue keeps growing indefinitely, leading to OOM errors and bookie restart. All unit tests pass for me locally with this change. Author: Aaron Gresch <agresch@yahoo-inc.com> Reviewers: Ivan Kelly <ivan@ivankelly.net>, Jia Zhai <None>, Sijie Guo <sijie@apache.org> This closes apache#706 from agresch/agresch_cms1254
Integrating some changes made to Yahoo bookkeeper by Matteo Merli. The apache bookkeeper thread pool in use by OrderedScheduler is different, and does not have access to setting a queue size limit, so I created QueueAssessibleExecutorService, and check the queue size instead. Other than that, this code change is pretty similar to Yahoo's. Original bug description for this change: Since we are using a thread pool to handle read requests in bookies, we have seen that when the Bookie read IO is maxed out, the requests are being accumulated in the bookie. Essentially, the bookie is busy serving read requests that have already been timed out on the client side (end to end read latency can reach hours..). The queue keeps growing indefinitely, leading to OOM errors and bookie restart. All unit tests pass for me locally with this change. Author: Aaron Gresch <agresch@yahoo-inc.com> Reviewers: Ivan Kelly <ivan@ivankelly.net>, Jia Zhai <None>, Sijie Guo <sijie@apache.org> This closes apache#706 from agresch/agresch_cms1254
1. Refactor SASL auth tests to use single file, 2. Use CountDownLatch to sync with watcher More context: https://issues.apache.org/jira/browse/ZOOKEEPER-3193 Author: Andor Molnar <andor@apache.org> Reviewers: andor@apache.org Closes apache#706 from anmolnar/ZOOKEEPER-3193
Integrating some changes made to Yahoo bookkeeper by Matteo Merli.
The apache bookkeeper thread pool in use by OrderedScheduler is different, and does not have access to setting a queue size limit, so I created QueueAssessibleExecutorService, and check the queue size instead. Other than that, this code change is pretty similar to Yahoo's.
Original bug description for this change:
Since we are using a thread pool to handle read requests in bookies, we have seen that when the Bookie read IO is maxed out, the requests are being accumulated in the bookie.
Essentially, the bookie is busy serving read requests that have already been timed out on the client side (end to end read latency can reach hours..). The queue keeps growing indefinitely, leading to OOM errors and bookie restart.
All unit tests pass for me locally with this change.