Skip to content

Bookies should not queue read request indefinitely#706

Closed
agresch wants to merge 5 commits intoapache:masterfrom
agresch:agresch_cms1254
Closed

Bookies should not queue read request indefinitely#706
agresch wants to merge 5 commits intoapache:masterfrom
agresch:agresch_cms1254

Conversation

@agresch
Copy link
Copy Markdown

@agresch agresch commented Nov 7, 2017

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.

@asfgit
Copy link
Copy Markdown

asfgit commented Nov 7, 2017

FAILURE

--none--


counter.await();

// assertEquals(BKException.Code.OK, rcFirstReadOperation.get());
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldn't be commented out.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why does it get this error? Shouldn't it be TooManyRequestException?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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 {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

define -1 as a constant (NO_LIMIT or something).

* of tasks to be scheduled in the thread's queue.
*
*/
public class QueueAssessibleExecutorService implements ListeningScheduledExecutorService {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You can reduce the number of useless implemented methods, like shutdown, by extending from ForwardingListeningExecutorService.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good tip. Thanks.

import java.util.concurrent.TimeUnit;

/**
* Implements {@link ListeningScheduledExecutorService} and allows limiting the number
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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 ?).

[1] https://github.com/frohoff/jdk8u-jdk/blob/master/src/share/classes/java/util/concurrent/ScheduledThreadPoolExecutor.java#L134

@asfgit
Copy link
Copy Markdown

asfgit commented Nov 8, 2017

FAILURE

--none--

@asfgit
Copy link
Copy Markdown

asfgit commented Nov 8, 2017

SUCCESS

--none--

* Write operations failed due to bookies are readonly.
*/
int WriteOnReadOnlyBookieException = -104;
int TooManyRequestsException = -105;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It may need to add a comments for this exception as other exception here.

Copy link
Copy Markdown
Member

@jiazhai jiazhai left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

made some minor comments

*/
int WriteOnReadOnlyBookieException = -104;
int TooManyRequestsException = -105;
//-105 reserved for TooManyRequestsException
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove this line?

@@ -0,0 +1,106 @@
package org.apache.bookkeeper.client;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please add license header before this line.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it is actually very interesting... this license header is missed in this file, but the travis build passes...

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe because it's a test?

Copy link
Copy Markdown
Member

@sijie sijie left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

overall lgtm +1

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() //
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: removing the trailing "//"

r.getReadRequest().getLedgerId(), r.getReadRequest().getEntryId());
}
BookkeeperProtocol.ReadResponse.Builder readResponse =
BookkeeperProtocol.ReadResponse.newBuilder() //
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: removing trailing "//"

@asfgit
Copy link
Copy Markdown

asfgit commented Nov 9, 2017

FAILURE

--none--

@jiazhai
Copy link
Copy Markdown
Member

jiazhai commented Nov 12, 2017

@agresch , Would you please solve the conflicting file? then we can ship it.

@sijie sijie closed this in ed7696b Nov 13, 2017
sijie pushed a commit that referenced this pull request Nov 13, 2017
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>
@sijie sijie modified the milestones: 4.7.0, 4.6.0 Nov 22, 2017
philipsu522 pushed a commit to philipsu522/bookkeeper that referenced this pull request Dec 8, 2017
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
aojea pushed a commit to aojea/bookkeeper that referenced this pull request Dec 16, 2017
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
athanatos pushed a commit to athanatos/bookkeeper that referenced this pull request Jan 25, 2019
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
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants