You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by ay...@apache.org on 2020/11/04 03:54:51 UTC

[ozone] branch master updated: HDDS-4413. Container replication should fail in case of import failure. (#1537)

This is an automated email from the ASF dual-hosted git repository.

ayushsaxena pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new ca220f9  HDDS-4413. Container replication should fail in case of import failure. (#1537)
ca220f9 is described below

commit ca220f96516d7f4036963f8cf1e0f501e09a870a
Author: Ayush Saxena <ay...@apache.org>
AuthorDate: Wed Nov 4 09:24:35 2020 +0530

    HDDS-4413. Container replication should fail in case of import failure. (#1537)
---
 .../replication/DownloadAndImportReplicator.java   | 12 ++++----
 .../replication/TestReplicationSupervisor.java     | 35 ++++++++++++++++++++++
 2 files changed, 40 insertions(+), 7 deletions(-)

diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java
index 74733ac..cdab0fd 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.ozone.container.replication;
 
 import java.io.FileInputStream;
+import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.List;
@@ -43,7 +44,7 @@ import org.slf4j.LoggerFactory;
  */
 public class DownloadAndImportReplicator implements ContainerReplicator {
 
-  private static final Logger LOG =
+  public static final Logger LOG =
       LoggerFactory.getLogger(DownloadAndImportReplicator.class);
 
   private final ContainerSet containerSet;
@@ -65,7 +66,8 @@ public class DownloadAndImportReplicator implements ContainerReplicator {
     this.packer = packer;
   }
 
-  public void importContainer(long containerID, Path tarFilePath) {
+  public void importContainer(long containerID, Path tarFilePath)
+      throws IOException {
     try {
       ContainerData originalContainerData;
       try (FileInputStream tempContainerTarStream = new FileInputStream(
@@ -85,10 +87,6 @@ public class DownloadAndImportReplicator implements ContainerReplicator {
         containerSet.addContainer(container);
       }
 
-    } catch (Exception e) {
-      LOG.error(
-          "Can't import the downloaded container data id=" + containerID,
-          e);
     } finally {
       try {
         Files.delete(tarFilePath);
@@ -122,7 +120,7 @@ public class DownloadAndImportReplicator implements ContainerReplicator {
       LOG.info("Container {} is replicated successfully", containerID);
       task.setStatus(Status.DONE);
     } catch (Exception e) {
-      LOG.error("Container replication was unsuccessful .", e);
+      LOG.error("Container {} replication was unsuccessful.", containerID, e);
       task.setStatus(Status.FAILED);
     }
   }
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java
index b7415fa..df9ffbc 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java
@@ -18,9 +18,12 @@
 
 package org.apache.hadoop.ozone.container.replication;
 
+import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.AbstractExecutorService;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
@@ -33,12 +36,14 @@ import org.apache.hadoop.ozone.container.keyvalue.ChunkLayoutTestInfo;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
 
 import javax.annotation.Nonnull;
 
@@ -173,6 +178,36 @@ public class TestReplicationSupervisor {
     }
   }
 
+  @Test
+  public void testDownloadAndImportReplicatorFailure() {
+    ReplicationSupervisor supervisor =
+        new ReplicationSupervisor(set, mutableReplicator,
+            newDirectExecutorService());
+
+    // Mock to fetch an exception in the importContainer method.
+    SimpleContainerDownloader moc =
+        Mockito.mock(SimpleContainerDownloader.class);
+    CompletableFuture<Path> res = new CompletableFuture<>();
+    res.complete(Paths.get("file:/tmp/no-such-file"));
+    Mockito.when(
+        moc.getContainerDataFromReplicas(Mockito.anyLong(), Mockito.anyList()))
+        .thenReturn(res);
+
+    ContainerReplicator replicatorFactory =
+        new DownloadAndImportReplicator(set, null, moc, null);
+
+    replicatorRef.set(replicatorFactory);
+
+    GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
+        .captureLogs(DownloadAndImportReplicator.LOG);
+
+    supervisor.addTask(new ReplicationTask(1L, emptyList()));
+    Assert.assertEquals(1, supervisor.getReplicationFailureCount());
+    Assert.assertEquals(0, supervisor.getReplicationSuccessCount());
+    Assert.assertTrue(logCapturer.getOutput()
+        .contains("Container 1 replication was unsuccessful."));
+  }
+
   private ReplicationSupervisor supervisorWithSuccessfulReplicator() {
     return supervisorWith(FakeReplicator::new, newDirectExecutorService());
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@ozone.apache.org
For additional commands, e-mail: commits-help@ozone.apache.org