-
Notifications
You must be signed in to change notification settings - Fork 3.4k
HBASE-28128 Reject requests at RPC layer when RegionServer is aborting #5447
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
Conversation
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
serverWithoutMeta.abort("Abort RS for test"); | ||
|
||
// Wait for our expected exception to be thrown | ||
synchronized (THROWN_EXCEPTION) { |
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.
Usually a wait should be in a while loop. Since this is just in a tests, let's just use a UTIL.waitFor to check the exception, and in this way we could have a timeout here.
} | ||
|
||
private void submitRequestsToRegion(Table table, RegionInfo regionInfo) throws IOException { | ||
byte[] rowKey = getRowKeyWithin(regionInfo); |
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.
Better at least add a sleep here? It could be very small but it will help reduce the pressure a lot...
i++; | ||
table.put(new Put(rowKey).addColumn(CF, Bytes.toBytes(i), Bytes.toBytes(i))); | ||
} catch (IOException e) { | ||
// only catch RegionServerAbortedException once. After that, the next exception thrown |
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.
What if the region has been reassigned to other region servers already?
return Optional.of(this); | ||
} | ||
|
||
@Override |
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.
OK, we have a sleep here.
But synchronized on a AtomiceReference may trigger some checkstyle warnings, but to have another lock object, or just use sleep and check here, since we are in a test.
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
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.
The product change looks good to me -- I'm surprised that it's not already there. We're usually pretty good about checking for abort/stopped
status before processing, but maybe that's at a layer down.
@@ -656,6 +663,15 @@ protected void processRequest(ByteBuff buf) throws IOException, InterruptedExcep | |||
} | |||
} | |||
|
|||
private void sendErrorResponseForCall(int id, long totalRequestSize, Span span, String msg, |
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 cleanup.
} | ||
} catch (Exception e) { | ||
LOG.warn("Failed to load data", e); | ||
synchronized (THROWN_EXCEPTION) { |
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.
Maybe a CountdownLatch is less ham-handed than the synchronized/notify/waitFor dance?
// fail at the handler layer, but worse might result in CallQueueTooBigException if the | ||
// queue is full but server is not properly processing requests. Better to throw an aborted | ||
// exception here so that the client can properly react. | ||
if (rpcServer.server != null && rpcServer.server.isAborted()) { |
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.
rpcServer.server
can be null? I guess it doesn't hurt to check.
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.
This comes from tests actually, a few pass in null and were causing NPEs. I figured it was safest to null check it rather than mock all of those cases.
private static final Logger LOG = | ||
LoggerFactory.getLogger(TestRegionServerRejectDuringAbort.class); | ||
|
||
private static final HBaseTestingUtil UTIL = new HBaseTestingUtil(); |
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.
The HBaseTestingUtil has a library of helper methods that you may find useful. For example, createMultiRegionTable
and loadRegion
.
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.
Yea, that's useful for setting up a particular state (which is often the case when doing lots of writes in a test). In this case I simply chose put
as the api to call to get a CQTBE when i force an abort. I'm not actually caring about or reading the data, I probably could have used get
on an empty instead. Thanks for the pointer!
Yea, we check this in pretty much every RSRpcServices endpoint. The problem is that's post-queue, and this adds the check pre-queue. Otherwise if the RS is fully hosed we will end up throwing CallQueueTooBigException (doesnt clear meta) when it really should be ServerAborted (does clear meta) |
The test failure is unrelated. @Apache9 let me know if you have any other concerns, otherwise I'll merge tomorrow |
#5447) Signed-off-by: Nick Dimiduk <ndimiduk@apache.org> Reviewed-by: Duo Zhang <zhangduo@apache.org>
#5447) Signed-off-by: Nick Dimiduk <ndimiduk@apache.org> Reviewed-by: Duo Zhang <zhangduo@apache.org>
#5447) Signed-off-by: Nick Dimiduk <ndimiduk@apache.org> Reviewed-by: Duo Zhang <zhangduo@apache.org>
…nServer is aborting (apache#5447) Signed-off-by: Nick Dimiduk <ndimiduk@apache.org> Reviewed-by: Duo Zhang <zhangduo@apache.org>
apache#5447) Signed-off-by: Nick Dimiduk <ndimiduk@apache.org> Reviewed-by: Duo Zhang <zhangduo@apache.org>
No description provided.