comnetwork commented on a change in pull request #4039: URL: https://github.com/apache/hbase/pull/4039#discussion_r793282193
########## File path: hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java ########## @@ -272,4 +279,132 @@ public void testWriteLargeChunk() throws IOException, InterruptedException, Exec } assertArrayEquals(b, actual); } + + /** + * <pre> + * This test is for HBASE-26679. Consider there are two dataNodes: dn1 and dn2,dn2 is a slow DN. + * The threads sequence before HBASE-26679 is: + * 1.We write some data to {@link FanOutOneBlockAsyncDFSOutput} and then flush it, there are one + * {@link FanOutOneBlockAsyncDFSOutput.Callback} in + * {@link FanOutOneBlockAsyncDFSOutput#waitingAckQueue}. + * 2.The ack from dn1 arrives firstly and triggers Netty to invoke + * {@link FanOutOneBlockAsyncDFSOutput#completed} with dn1's channel, then in + * {@link FanOutOneBlockAsyncDFSOutput#completed}, dn1's channel is removed from + * {@link FanOutOneBlockAsyncDFSOutput.Callback#unfinishedReplicas}. + * 3.But dn2 responds slowly, before dn2 sending ack,dn1 is shut down or have a exception, + * so {@link FanOutOneBlockAsyncDFSOutput#failed} is triggered by Netty with dn1's channel, + * and because the {@link FanOutOneBlockAsyncDFSOutput.Callback#unfinishedReplicas} does not + * contain dn1's channel,the {@link FanOutOneBlockAsyncDFSOutput.Callback} is skipped in + * {@link FanOutOneBlockAsyncDFSOutput#failed} method,and + * {@link FanOutOneBlockAsyncDFSOutput#state} is set to + * {@link FanOutOneBlockAsyncDFSOutput.State#BROKEN},and dn1,dn2 are all closed at the end of + * {@link FanOutOneBlockAsyncDFSOutput#failed}. + * 4.{@link FanOutOneBlockAsyncDFSOutput#failed} is triggered again by dn2 because it is closed, + * but because {@link FanOutOneBlockAsyncDFSOutput#state} is already + * {@link FanOutOneBlockAsyncDFSOutput.State#BROKEN},the whole + * {@link FanOutOneBlockAsyncDFSOutput#failed} is skipped. So wait on the future + * returned by {@link FanOutOneBlockAsyncDFSOutput#flush} would be stuck for ever. + * After HBASE-26679, for above step 4,even if the {@link FanOutOneBlockAsyncDFSOutput#state} + * is already {@link FanOutOneBlockAsyncDFSOutput.State#BROKEN}, we would still try to trigger + * {@link FanOutOneBlockAsyncDFSOutput.Callback#future}. + * </pre> + */ + @Test + public void testFlushStuckWhenOneDataNodeFailedBeforeOtherDataNodeAck() throws Exception { Review comment: @Apache9 , I mean it seems not affect the test? The Response for heartbeat is ignored by your added ChannelInboundHandler? and in AckHandler.channelRead0, it is also ignore. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@hbase.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org