瀏覽代碼

HDDS-1955. TestBlockOutputStreamWithFailures#test2DatanodesFailure failing because of assertion error.

Signed-off-by: Nanda kumar <nanda@apache.org>
(cherry picked from commit 2432356570140ec7f55e1ab56e442c373ff05a16)
Mukul Kumar Singh 5 年之前
父節點
當前提交
b4480cca59

+ 12 - 8
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStreamWithFailures.java

@@ -501,14 +501,18 @@ public class TestBlockOutputStreamWithFailures {
     // and one flush for partial chunk
     key.flush();
 
-    // Since, 2 datanodes went down, if the pipeline gets destroyed quickly,
-    // it will hit GroupMismatchException else, it will fail with
-    // RaftRetryFailureException
-    Assert.assertTrue((HddsClientUtils.
-        checkForException(blockOutputStream
-            .getIoException()) instanceof RaftRetryFailureException)
-        || HddsClientUtils.checkForException(
-        blockOutputStream.getIoException()) instanceof GroupMismatchException);
+    Throwable ioException = HddsClientUtils.checkForException(
+        blockOutputStream.getIoException());
+    // Since, 2 datanodes went down,
+    // a) if the pipeline gets destroyed quickly it will hit
+    //    GroupMismatchException.
+    // b) will hit close container exception if the container is closed
+    //    but pipeline is still not destroyed.
+    // c) will fail with RaftRetryFailureException if the leader election
+    //    did not finish before the request retry count finishes.
+    Assert.assertTrue(ioException instanceof RaftRetryFailureException
+        || ioException instanceof GroupMismatchException
+        || ioException instanceof ContainerNotOpenException);
     // Make sure the retryCount is reset after the exception is handled
     Assert.assertTrue(keyOutputStream.getRetryCount() == 0);
     // now close the stream, It will update the ack length after watchForCommit