-
Notifications
You must be signed in to change notification settings - Fork 3.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
HBASE-24120 Flakey Test: TestReplicationAdminWithClusters timeout #1441
Conversation
It is related with HBASE-20561, @Apache9 and @infraio , can you review the change? Thanks. |
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.
You don't want to log the exception message at least?
Otherwise, great.
🎊 +1 overall
This message was automatically generated. |
throw new RuntimeException( | ||
"Thread is interrupted, the replication source may be terminated"); | ||
LOG.info("Thread is interrupted, the replication source may be terminated"); | ||
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.
You sure you don't want to propagate the InterruptedException
? Looks like run
loop in ReplicationSourceShipper
is designed to handle that exception. return
here would continue the thread executing, which I think is not what we want.
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 interrupt flap has been set up in the zkwatcher layer and upper layer handles this flag nicely.
zkw.interruptedException(ie);
https://github.com/apache/hbase/blob/master/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java#L637, in the outside loop, it will handle this interrupt flag and terminate the thread.
Inside isActive() of the following loop, it will return false if isInterrupted flag is true.
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 manually tested the code path by injecting an InterruptException at zkwatcher and made sure thread exits and no region server abort.
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.
Thanks for checking.
@@ -579,8 +579,8 @@ private void interruptOrAbortWhenFail(ReplicationQueueOperation op) { | |||
if (e.getCause() != null && e.getCause() instanceof KeeperException.SystemErrorException |
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 tried rewriting this big nasty if-expression using an Optional
. Tell me what you think
if (Optional.of(e.getCause())
.filter(c -> c instanceof KeeperException.SystemErrorException)
.map(c -> (KeeperException.SystemErrorException) c)
.map(Exception::getCause)
.filter(c -> c instanceof InterruptedException)
.isPresent()) {
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.
missed this comment, will get back.
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
Please give me sometime to review. Usually I do not like to manually create a RuntimeException and throw it as it is an anti pattern(the only exception is validation), so there must be some reasons why we do it here. Let me try to figure out. Thanks. |
👍 would appreciate your analysis here Sir. |
OK, skimmed HBASE-20561, the goal there was to avoid the region server to be aborted... I think the RuntimeException was used to skip all the following code and propagate it soon to the most top level of code, as the operation is failed, we should not execute the following code any more, a simple return will make the upper layer think the operation is succeeded, which is incorrect. Now the code at the top level will abort the region server? Maybe that's the place need to be fixed. If RuntimeException is too general, maybe we could introduce a special exception and throw it, and at some places we catch it and just ignore it and quit. |
From my read, it seems like propagating the |
Look back the code, you are right, ignore and continue is not the right way to handle it.
I do think RuntimeException is too general, can we just define a special exception which extends RuntimeException and let the top level to catch and handle it? I will put a new patch based on your feedback. Nice comments, I will quote your comments to the code as well, thanks! |
Indeed, that is my first approach. I tried to rethrow the InterruptedException from communication layer, the nit is that there will be lots of methods which needs to throw InterruptedException in their signature. Especially in shipEdits(), it catches all exceptions and it needs to rethrow InterruptedException as a special case. @ndimiduk , what do you think? |
Thanks for the very nice comments/dicussions here, @Apache9 , @ndimiduk and @saintstack , I will put another patch based on the comments. There is another case popped up, which is a NullPointerException caused by InterruptedException, I will take a close look and include that handling in the patch as well. |
1c89328
to
80fb755
Compare
@Apache9 @ndimiduk @saintstack Going to address the other issue separately. |
80fb755
to
328b048
Compare
@@ -445,7 +445,7 @@ private IOException extractHiddenEof(Exception ex) { | |||
&& ex.getCause() != null && ex.getCause() instanceof IOException) { | |||
ioEx = (IOException)ex.getCause(); | |||
} | |||
if (ioEx != null) { | |||
if ((ioEx != null) && (ioEx.getMessage() != null)) { |
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.
Added this check because the flakey test run into the following Nullpointer exception.
2020-04-07 03:30:03,677 WARN [RS_REFRESH_PEER-regionserver/asf905:0-0.replicationSource,2.replicationSource.wal-reader.asf905.gq1.ygridcore.net%2C41391%2C1586230117579,2] impl.BlockReaderFactory(768): I/O error constructing remote block reader. java.nio.channels.ClosedByInterruptException at java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:202) at sun.nio.ch.SocketChannelImpl.connect(SocketChannelImpl.java:659) at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:192) at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:531) at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:2881) at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:825) at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:750) at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:387) at org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:717) at org.apache.hadoop.hdfs.DFSInputStream.blockSeekTo(DFSInputStream.java:665) at org.apache.hadoop.hdfs.DFSInputStream.seekToBlockSource(DFSInputStream.java:1697) at org.apache.hadoop.hdfs.DFSInputStream.readBuffer(DFSInputStream.java:915) at org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:950) at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:996) at java.io.DataInputStream.read(DataInputStream.java:149) at java.io.FilterInputStream.read(FilterInputStream.java:133) at java.io.PushbackInputStream.read(PushbackInputStream.java:186) at org.apache.hadoop.io.IOUtils.readFully(IOUtils.java:209) at org.apache.hadoop.hbase.KeyValueUtil.createKeyValueFromInputStream(KeyValueUtil.java:716) at org.apache.hadoop.hbase.codec.KeyValueCodecWithTags$KeyValueDecoder.parseCell(KeyValueCodecWithTags.java:81) at org.apache.hadoop.hbase.codec.BaseDecoder.advance(BaseDecoder.java:68) at org.apache.hadoop.hbase.wal.WALEdit.readFromCells(WALEdit.java:276) at org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.readNext(ProtobufLogReader.java:382) at org.apache.hadoop.hbase.regionserver.wal.ReaderBase.next(ReaderBase.java:98) at org.apache.hadoop.hbase.regionserver.wal.ReaderBase.next(ReaderBase.java:86) at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.readNextEntryAndRecordReaderPosition(WALEntryStream.java:263) at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.tryAdvanceEntry(WALEntryStream.java:176) at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:101) at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.readWALEntries(ReplicationSourceWALReader.java:221) at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:138) 2020-04-07 03:30:03,678 ERROR [RS_REFRESH_PEER-regionserver/asf905:0-0.replicationSource,2.replicationSource.wal-reader.asf905.gq1.ygridcore.net%2C41391%2C1586230117579,2] regionserver.ReplicationSource(397): Unexpected exception in RS_REFRESH_PEER-regionserver/asf905:0-0.replicationSource,2.replicationSource.wal-reader.asf905.gq1.ygridcore.net%2C41391%2C1586230117579,2 currentPath=hdfs://localhost:37359/user/jenkins/test-data/260e1f0f-a3fd-6192-b1d7-6568614aef58/WALs/asf905.gq1.ygridcore.net,41391,1586230117579/asf905.gq1.ygridcore.net%2C41391%2C1586230117579.1586230122806 java.lang.NullPointerException at org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.extractHiddenEof(ProtobufLogReader.java:449) at org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.readNext(ProtobufLogReader.java:396) at org.apache.hadoop.hbase.regionserver.wal.ReaderBase.next(ReaderBase.java:98) at org.apache.hadoop.hbase.regionserver.wal.ReaderBase.next(ReaderBase.java:86) at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.readNextEntryAndRecordReaderPosition(WALEntryStream.java:263) at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.tryAdvanceEntry(WALEntryStream.java:176) at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:101) at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.readWALEntries(ReplicationSourceWALReader.java:221) at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:138)
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 after fixing the simple nits.
@@ -122,6 +122,10 @@ public final void run() { | |||
} catch (InterruptedException e) { | |||
LOG.trace("Interrupted while waiting for next replication entry batch", e); | |||
Thread.currentThread().interrupt(); | |||
} catch (ReplicationRuntimeException rre) { |
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 think we could do this
catch (InterruptedException | ReplicationRuntimeException e) {
blabla
}
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.
Will do, and updated the patch to fix the NullPointerException as well, thanks.
/** | ||
* This exception is thrown when replication source is terminated and source threads got | ||
* interrupted. | ||
* |
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.
Add a
so it will not be reformated by IDE.
@@ -445,7 +445,7 @@ private IOException extractHiddenEof(Exception ex) { | |||
&& ex.getCause() != null && ex.getCause() instanceof IOException) { | |||
ioEx = (IOException)ex.getCause(); | |||
} | |||
if (ioEx != null) { | |||
if ((ioEx != null) && (ioEx.getMessage() != null)) { |
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.
Good.
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
Test failures are mostly due to "Unable to create native thread", will address the comments and kick off a new run later. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
328b048
to
306ce75
Compare
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
…ache#1441) Signed-off-by: stack <stack@apache.org> Signed-off-by: Nick Dimiduk <ndimiduk@apache.org> Signed-off-by: Duo Zhang <zhangduo@apache.org>
…ache#1441) Signed-off-by: stack <stack@apache.org> Signed-off-by: Nick Dimiduk <ndimiduk@apache.org> Signed-off-by: Duo Zhang <zhangduo@apache.org>
) (#1456) Signed-off-by: stack <stack@apache.org> Signed-off-by: Nick Dimiduk <ndimiduk@apache.org> Signed-off-by: Duo Zhang <zhangduo@apache.org>
) (#1457) Signed-off-by: stack <stack@apache.org> Signed-off-by: Nick Dimiduk <ndimiduk@apache.org> Signed-off-by: Duo Zhang <zhangduo@apache.org>
…ache#1441) Signed-off-by: stack <stack@apache.org> Signed-off-by: Nick Dimiduk <ndimiduk@apache.org> Signed-off-by: Duo Zhang <zhangduo@apache.org>
…ache#1441) Signed-off-by: stack <stack@apache.org> Signed-off-by: Nick Dimiduk <ndimiduk@apache.org> Signed-off-by: Duo Zhang <zhangduo@apache.org>
…ache#1441) Signed-off-by: stack <stack@apache.org> Signed-off-by: Nick Dimiduk <ndimiduk@apache.org> Signed-off-by: Duo Zhang <zhangduo@apache.org>
…ache#1441) (apache#1457) Signed-off-by: stack <stack@apache.org> Signed-off-by: Nick Dimiduk <ndimiduk@apache.org> Signed-off-by: Duo Zhang <zhangduo@apache.org>
When remove peer happens, it may interrupt ReplicationSourceShipper thread.
interruptOrAbortWhenFail will throw out a RunTimeException which is not handled
and will abort Region Server. The solution is not to throw out a RunTimeException.