You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by eh...@apache.org on 2018/07/31 09:25:43 UTC

[01/50] hadoop git commit: HDDS-272. TestBlockDeletingService is failing with DiskOutOfSpaceException. Contributed by Lokesh Jain. [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-12090 bf62e297d -> d52a2afbf (forced update)


HDDS-272. TestBlockDeletingService is failing with DiskOutOfSpaceException. Contributed by Lokesh Jain.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/773d312f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/773d312f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/773d312f

Branch: refs/heads/HDFS-12090
Commit: 773d312f7412d5050c106ed3a1cd0d1934bfa2e0
Parents: ff7c2ed
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Tue Jul 24 21:23:20 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Tue Jul 24 21:23:20 2018 +0530

----------------------------------------------------------------------
 .../container/keyvalue/KeyValueHandler.java     |  2 +-
 .../background/BlockDeletingService.java        |  9 +-
 .../testutils/BlockDeletingServiceTestImpl.java |  3 +-
 .../common/TestBlockDeletingService.java        | 90 ++++++++++++--------
 4 files changed, 60 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/773d312f/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index 9aa3df7..d3a1ca4 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -136,7 +136,7 @@ public class KeyValueHandler extends Handler {
             TimeUnit.MILLISECONDS);
     this.blockDeletingService =
         new BlockDeletingService(containerSet, svcInterval, serviceTimeout,
-            config);
+            TimeUnit.MILLISECONDS, config);
     blockDeletingService.start();
     // TODO: Add supoort for different volumeChoosingPolicies.
     volumeChoosingPolicy = new RoundRobinVolumeChoosingPolicy();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/773d312f/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
index a3e36f4..4a572ca 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
@@ -88,11 +88,10 @@ public class BlockDeletingService extends BackgroundService{
   // Core pool size for container tasks
   private final static int BLOCK_DELETING_SERVICE_CORE_POOL_SIZE = 10;
 
-  public BlockDeletingService(ContainerSet containerSet,
-      long serviceInterval, long serviceTimeout, Configuration conf) {
-    super("BlockDeletingService", serviceInterval,
-        TimeUnit.MILLISECONDS, BLOCK_DELETING_SERVICE_CORE_POOL_SIZE,
-        serviceTimeout);
+  public BlockDeletingService(ContainerSet containerSet, long serviceInterval,
+      long serviceTimeout, TimeUnit timeUnit, Configuration conf) {
+    super("BlockDeletingService", serviceInterval, timeUnit,
+        BLOCK_DELETING_SERVICE_CORE_POOL_SIZE, serviceTimeout);
     this.containerSet = containerSet;
     containerDeletionPolicy = ReflectionUtils.newInstance(conf.getClass(
         ScmConfigKeys.OZONE_SCM_KEY_VALUE_CONTAINER_DELETION_CHOOSING_POLICY,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/773d312f/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
index a87f655..115b5e2 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
@@ -44,7 +44,8 @@ public class BlockDeletingServiceTestImpl
 
   public BlockDeletingServiceTestImpl(ContainerSet containerSet,
       int serviceInterval, Configuration conf) {
-    super(containerSet, serviceInterval, SERVICE_TIMEOUT_IN_MILLISECONDS, conf);
+    super(containerSet, serviceInterval, SERVICE_TIMEOUT_IN_MILLISECONDS,
+        TimeUnit.MILLISECONDS, conf);
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/773d312f/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
index 1ddd39a..a6e53c2 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
@@ -45,17 +45,17 @@ import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.apache.hadoop.utils.BackgroundService;
 import org.apache.hadoop.utils.MetadataKeyFilters;
 import org.apache.hadoop.utils.MetadataStore;
+import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.BeforeClass;
-import org.junit.Before;
-import org.junit.After;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
 import java.nio.charset.Charset;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
@@ -72,35 +72,28 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys
 /**
  * Tests to test block deleting service.
  */
-// TODO: Fix BlockDeletingService to work with new StorageLayer
 public class TestBlockDeletingService {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(TestBlockDeletingService.class);
 
   private static File testRoot;
-  private static File containersDir;
-  private static File chunksDir;
+  private static String scmId;
+  private static String clusterID;
 
   @BeforeClass
-  public static void init() {
+  public static void init() throws IOException {
     testRoot = GenericTestUtils
         .getTestDir(TestBlockDeletingService.class.getSimpleName());
-    chunksDir = new File(testRoot, "chunks");
-    containersDir = new File(testRoot, "containers");
-  }
-
-  @Before
-  public void setup() throws IOException {
-    if (chunksDir.exists()) {
-      FileUtils.deleteDirectory(chunksDir);
+    if (testRoot.exists()) {
+      FileUtils.cleanDirectory(testRoot);
     }
+    scmId = UUID.randomUUID().toString();
+    clusterID = UUID.randomUUID().toString();
   }
 
-  @After
-  public void cleanup() throws IOException {
-    FileUtils.deleteDirectory(chunksDir);
-    FileUtils.deleteDirectory(containersDir);
+  @AfterClass
+  public static void cleanup() throws IOException {
     FileUtils.deleteDirectory(testRoot);
   }
 
@@ -111,14 +104,15 @@ public class TestBlockDeletingService {
    */
   private void createToDeleteBlocks(ContainerSet containerSet,
       Configuration conf, int numOfContainers, int numOfBlocksPerContainer,
-      int numOfChunksPerBlock, File chunkDir) throws IOException {
+      int numOfChunksPerBlock) throws IOException {
     for (int x = 0; x < numOfContainers; x++) {
+      conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, testRoot.getAbsolutePath());
       long containerID = ContainerTestHelper.getTestContainerID();
       KeyValueContainerData data = new KeyValueContainerData(containerID,
           ContainerTestHelper.CONTAINER_MAX_SIZE_GB);
       Container container = new KeyValueContainer(data, conf);
-      container.create(new VolumeSet(UUID.randomUUID().toString(), conf),
-          new RoundRobinVolumeChoosingPolicy(), UUID.randomUUID().toString());
+      container.create(new VolumeSet(scmId, clusterID, conf),
+          new RoundRobinVolumeChoosingPolicy(), scmId);
       containerSet.addContainer(container);
       data = (KeyValueContainerData) containerSet.getContainer(
           containerID).getContainerData();
@@ -133,7 +127,7 @@ public class TestBlockDeletingService {
         for (int k = 0; k<numOfChunksPerBlock; k++) {
           // offset doesn't matter here
           String chunkName = blockID.getLocalID() + "_chunk_" + k;
-          File chunk = new File(chunkDir, chunkName);
+          File chunk = new File(data.getChunksPath(), chunkName);
           FileUtils.writeStringToFile(chunk, "a chunk",
               Charset.defaultCharset());
           LOG.info("Creating file {}", chunk.getAbsolutePath());
@@ -193,7 +187,7 @@ public class TestBlockDeletingService {
     conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
     conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
     ContainerSet containerSet = new ContainerSet();
-    createToDeleteBlocks(containerSet, conf, 1, 3, 1, chunksDir);
+    createToDeleteBlocks(containerSet, conf, 1, 3, 1);
 
     BlockDeletingServiceTestImpl svc =
         new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
@@ -208,12 +202,13 @@ public class TestBlockDeletingService {
     MetadataStore meta = KeyUtils.getDB(
         (KeyValueContainerData) containerData.get(0), conf);
     Map<Long, Container> containerMap = containerSet.getContainerMap();
+    long transactionId = containerMap.get(containerData.get(0).getContainerID())
+        .getContainerData().getDeleteTransactionId();
 
 
     // Number of deleted blocks in container should be equal to 0 before
     // block delete
-    // TODO : Implement deleteTransactionID in ContainerData.
-//    Assert.assertEquals(0, transactionId);
+    Assert.assertEquals(0, transactionId);
 
     // Ensure there are 3 blocks under deletion and 0 deleted blocks
     Assert.assertEquals(3, getUnderDeletionBlocksCount(meta));
@@ -247,7 +242,7 @@ public class TestBlockDeletingService {
     conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 10);
     ContainerSet containerSet = new ContainerSet();
     // Create 1 container with 100 blocks
-    createToDeleteBlocks(containerSet, conf, 1, 100, 1, chunksDir);
+    createToDeleteBlocks(containerSet, conf, 1, 100, 1);
 
     BlockDeletingServiceTestImpl service =
         new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
@@ -275,12 +270,13 @@ public class TestBlockDeletingService {
     conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
     conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
     ContainerSet containerSet = new ContainerSet();
-    createToDeleteBlocks(containerSet, conf, 1, 3, 1, chunksDir);
+    createToDeleteBlocks(containerSet, conf, 1, 3, 1);
 
     // set timeout value as 1ns to trigger timeout behavior
     long timeout  = 1;
-    BlockDeletingService svc =
-        new BlockDeletingService(containerSet, 1000, timeout, conf);
+    BlockDeletingService svc = new BlockDeletingService(containerSet,
+        TimeUnit.MILLISECONDS.toNanos(1000), timeout, TimeUnit.NANOSECONDS,
+        conf);
     svc.start();
 
     LogCapturer log = LogCapturer.captureLogs(BackgroundService.LOG);
@@ -299,8 +295,10 @@ public class TestBlockDeletingService {
 
     // test for normal case that doesn't have timeout limitation
     timeout  = 0;
-    createToDeleteBlocks(containerSet, conf, 1, 3, 1, chunksDir);
-    svc =  new BlockDeletingService(containerSet, 1000, timeout, conf);
+    createToDeleteBlocks(containerSet, conf, 1, 3, 1);
+    svc = new BlockDeletingService(containerSet,
+        TimeUnit.MILLISECONDS.toNanos(1000), timeout, TimeUnit.MILLISECONDS,
+        conf);
     svc.start();
 
     // get container meta data
@@ -347,7 +345,7 @@ public class TestBlockDeletingService {
     conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 1);
     conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 1);
     ContainerSet containerSet = new ContainerSet();
-    createToDeleteBlocks(containerSet, conf, 2, 1, 10, chunksDir);
+    createToDeleteBlocks(containerSet, conf, 2, 1, 10);
 
     BlockDeletingServiceTestImpl service =
         new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
@@ -357,7 +355,12 @@ public class TestBlockDeletingService {
       GenericTestUtils.waitFor(() -> service.isStarted(), 100, 3000);
       // 1st interval processes 1 container 1 block and 10 chunks
       deleteAndWait(service, 1);
-      Assert.assertEquals(10, chunksDir.listFiles().length);
+      Assert.assertEquals(10, getNumberOfChunksInContainers(containerSet));
+      deleteAndWait(service, 2);
+      deleteAndWait(service, 3);
+      deleteAndWait(service, 4);
+      deleteAndWait(service, 5);
+      Assert.assertEquals(0, getNumberOfChunksInContainers(containerSet));
     } finally {
       service.shutdown();
     }
@@ -383,10 +386,10 @@ public class TestBlockDeletingService {
     conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
     conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
     ContainerSet containerSet = new ContainerSet();
-    createToDeleteBlocks(containerSet, conf, 5, 3, 1, chunksDir);
+    createToDeleteBlocks(containerSet, conf, 5, 3, 1);
 
     // Make sure chunks are created
-    Assert.assertEquals(15, chunksDir.listFiles().length);
+    Assert.assertEquals(15, getNumberOfChunksInContainers(containerSet));
 
     BlockDeletingServiceTestImpl service =
         new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
@@ -399,13 +402,26 @@ public class TestBlockDeletingService {
       // number of containers = 5
       // each interval will at most runDeletingTasks 5 * 2 = 10 blocks
       deleteAndWait(service, 1);
-      Assert.assertEquals(5, chunksDir.listFiles().length);
+      Assert.assertEquals(5, getNumberOfChunksInContainers(containerSet));
 
       // There is only 5 blocks left to runDeletingTasks
       deleteAndWait(service, 2);
-      Assert.assertEquals(0, chunksDir.listFiles().length);
+      Assert.assertEquals(0, getNumberOfChunksInContainers(containerSet));
     } finally {
       service.shutdown();
     }
   }
+
+  private int getNumberOfChunksInContainers(ContainerSet containerSet) {
+    Iterator<Container> containerIterator = containerSet.getContainerIterator();
+    int numChunks = 0;
+    while (containerIterator.hasNext()) {
+      Container container = containerIterator.next();
+      File chunkDir = FileUtils.getFile(
+          ((KeyValueContainerData) container.getContainerData())
+              .getChunksPath());
+      numChunks += chunkDir.listFiles().length;
+    }
+    return numChunks;
+  }
 }


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


[50/50] hadoop git commit: HDFS-13310. The DatanodeProtocol should have a DNA_BACKUP to backup blocks. Original patch contributed by Ewan Higgs. Followup work and fixed contributed by Virajith Jalaparthi.

Posted by eh...@apache.org.
HDFS-13310. The DatanodeProtocol should have a DNA_BACKUP to backup blocks. Original patch contributed by Ewan Higgs. Followup work and fixed contributed by Virajith Jalaparthi.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d52a2afb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d52a2afb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d52a2afb

Branch: refs/heads/HDFS-12090
Commit: d52a2afbf0d39b9de9d021e4e38f4ee993fa5fa0
Parents: 3e06a5d
Author: Ewan Higgs <ew...@wdc.com>
Authored: Mon Jul 23 13:14:04 2018 +0200
Committer: Ewan Higgs <ew...@wdc.com>
Committed: Tue Jul 31 11:24:39 2018 +0200

----------------------------------------------------------------------
 .../BlockSyncTaskExecutionFeedback.java         |  67 ++++++
 .../protocol/SyncTaskExecutionOutcome.java      |  25 +++
 .../protocol/SyncTaskExecutionResult.java       |  46 ++++
 .../DatanodeProtocolClientSideTranslatorPB.java |   8 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |   6 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 208 ++++++++++++++++++-
 .../server/blockmanagement/DatanodeManager.java |   4 +-
 .../hdfs/server/datanode/BPServiceActor.java    |   9 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   7 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   7 +-
 .../hdfs/server/protocol/BlockSyncTask.java     |  83 ++++++++
 .../protocol/BulkSyncTaskExecutionFeedback.java |  36 ++++
 .../hdfs/server/protocol/DatanodeProtocol.java  |  22 +-
 .../hdfs/server/protocol/SyncCommand.java       |  39 ++++
 .../src/main/proto/DatanodeProtocol.proto       |  88 +++++++-
 .../blockmanagement/TestDatanodeManager.java    |   2 +-
 .../TestNameNodePrunesMissingStorages.java      |   2 +-
 .../datanode/InternalDataNodeTestUtils.java     |   3 +-
 .../server/datanode/TestBPOfferService.java     |   5 +-
 .../hdfs/server/datanode/TestBlockRecovery.java |   4 +-
 .../server/datanode/TestDataNodeLifeline.java   |   9 +-
 .../TestDatanodeProtocolRetryPolicy.java        |   4 +-
 .../server/datanode/TestFsDatasetCache.java     |   4 +-
 .../hdfs/server/datanode/TestStorageReport.java |   4 +-
 .../server/namenode/NNThroughputBenchmark.java  |   8 +-
 .../hdfs/server/namenode/NameNodeAdapter.java   |   5 +-
 .../hdfs/server/namenode/TestDeadDatanode.java  |   4 +-
 27 files changed, 662 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTaskExecutionFeedback.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTaskExecutionFeedback.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTaskExecutionFeedback.java
new file mode 100644
index 0000000..2e5393e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTaskExecutionFeedback.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.protocol;
+
+import java.util.UUID;
+
+/**
+ * Feedback for a BlockSyncTask.
+ */
+public class BlockSyncTaskExecutionFeedback {
+
+  private UUID syncTaskId;
+  private SyncTaskExecutionOutcome outcome;
+  private SyncTaskExecutionResult result;
+  private String syncMountId;
+
+  public BlockSyncTaskExecutionFeedback(UUID syncTaskId,
+      SyncTaskExecutionOutcome outcome, SyncTaskExecutionResult result,
+      String syncMountId) {
+    this.syncTaskId = syncTaskId;
+    this.outcome = outcome;
+    this.result = result;
+    this.syncMountId = syncMountId;
+  }
+
+  public static BlockSyncTaskExecutionFeedback finishedSuccessfully(
+      UUID syncTaskId, String syncMountId, SyncTaskExecutionResult result) {
+    return new BlockSyncTaskExecutionFeedback(syncTaskId,
+        SyncTaskExecutionOutcome.FINISHED_SUCCESSFULLY, result, syncMountId);
+  }
+
+  public static BlockSyncTaskExecutionFeedback failedWithException(
+      UUID syncTaskId, String syncMountId, Exception e) {
+    return new BlockSyncTaskExecutionFeedback(syncTaskId,
+        SyncTaskExecutionOutcome.EXCEPTION, null, syncMountId);
+  }
+
+  public UUID getSyncTaskId() {
+    return syncTaskId;
+  }
+
+  public SyncTaskExecutionOutcome getOutcome() {
+    return outcome;
+  }
+
+  public SyncTaskExecutionResult getResult() {
+    return result;
+  }
+
+  public String getSyncMountId() {
+    return syncMountId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionOutcome.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionOutcome.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionOutcome.java
new file mode 100644
index 0000000..492575b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionOutcome.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.protocol;
+
+/**
+ * SyncTaskExecutionOutcome is whether the SyncTask was successful or not.
+ */
+public enum SyncTaskExecutionOutcome {
+  FINISHED_SUCCESSFULLY,
+  EXCEPTION
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionResult.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionResult.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionResult.java
new file mode 100644
index 0000000..b623dc5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionResult.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.protocol;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Result of a SyncTask.
+ */
+public class SyncTaskExecutionResult {
+
+  /** result is the opaque byte stream result of a task. e.g. PartHandle */
+  private ByteBuffer result;
+  private Long numberOfBytes;
+
+  public SyncTaskExecutionResult(ByteBuffer result, Long numberOfBytes) {
+    this.result = result;
+    this.numberOfBytes = numberOfBytes;
+  }
+
+  public static SyncTaskExecutionResult emptyResult() {
+    return new SyncTaskExecutionResult(ByteBuffer.wrap(new byte[0]), 0L);
+  }
+
+  public ByteBuffer getResult() {
+    return result;
+  }
+
+  public Long getNumberOfBytes() {
+    return numberOfBytes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
index 9cc4516..20b314c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlock
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -138,7 +139,8 @@ public class DatanodeProtocolClientSideTranslatorPB implements
       VolumeFailureSummary volumeFailureSummary,
       boolean requestFullBlockReportLease,
       @Nonnull SlowPeerReports slowPeers,
-      @Nonnull SlowDiskReports slowDisks) throws IOException {
+      @Nonnull SlowDiskReports slowDisks,
+      BulkSyncTaskExecutionFeedback feedback) throws IOException {
     HeartbeatRequestProto.Builder builder = HeartbeatRequestProto.newBuilder()
         .setRegistration(PBHelper.convert(registration))
         .setXmitsInProgress(xmitsInProgress).setXceiverCount(xceiverCount)
@@ -161,6 +163,10 @@ public class DatanodeProtocolClientSideTranslatorPB implements
     if (slowDisks.haveSlowDisks()) {
       builder.addAllSlowDisks(PBHelper.convertSlowDiskInfo(slowDisks));
     }
+    if(feedback != null && !feedback.getFeedbacks().isEmpty()) {
+      builder.setBulkSyncTaskExecutionFeedback(PBHelper.convert(feedback));
+    }
+
     HeartbeatResponseProto resp;
     try {
       resp = rpcProxy.sendHeartbeat(NULL_CONTROLLER, builder.build());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
index 5cba284..a51ce85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
@@ -109,6 +109,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
   @Override
   public HeartbeatResponseProto sendHeartbeat(RpcController controller,
       HeartbeatRequestProto request) throws ServiceException {
+
     HeartbeatResponse response;
     try {
       final StorageReport[] report = PBHelperClient.convertStorageReports(
@@ -122,7 +123,10 @@ public class DatanodeProtocolServerSideTranslatorPB implements
           request.getXceiverCount(), request.getFailedVolumes(),
           volumeFailureSummary, request.getRequestFullBlockReportLease(),
           PBHelper.convertSlowPeerInfo(request.getSlowPeersList()),
-          PBHelper.convertSlowDiskInfo(request.getSlowDisksList()));
+          PBHelper.convertSlowDiskInfo(request.getSlowDisksList()),
+          PBHelper.convertBulkSyncTaskExecutionFeedback(
+              request.getBulkSyncTaskExecutionFeedback())
+      );
     } catch (IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index ac01348..d520a40 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hdfs.protocolPB;
 
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -24,9 +27,10 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
 
 import com.google.protobuf.ByteString;
-
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSUtilClient;
@@ -43,38 +47,44 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommand
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockECReconstructionCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockSyncTaskProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BulkSyncTaskExecutionFeedbackProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos
-    .SlowDiskReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SlowDiskReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SlowPeerReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SyncCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SyncTaskExecutionFeedbackProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SyncTaskExecutionOutcomeProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SyncTaskExecutionResultProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SyncTaskIdProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECReconstructionInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ProvidedStorageLocationProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ProvidedStorageLocationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlockWithLocationsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlocksWithLocationsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.CheckpointCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.CheckpointSignatureProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.ExportedBlockKeysProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NNHAStatusHeartbeatProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeRegistrationProto.NamenodeRoleProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamespaceInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NNHAStatusHeartbeatProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RecoveringBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RemoteEditLogManifestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RemoteEditLogProto;
@@ -89,18 +99,23 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.hdfs.server.protocol.BlockSyncTask;
+import org.apache.hadoop.hdfs.server.protocol.BlockSyncTaskExecutionFeedback;
+import org.apache.hadoop.hdfs.server.protocol.SyncTaskExecutionOutcome;
+import org.apache.hadoop.hdfs.server.protocol.SyncTaskExecutionResult;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
-import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringStripedBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
@@ -119,7 +134,9 @@ import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import org.apache.hadoop.hdfs.server.protocol.SlowPeerReports;
+import org.apache.hadoop.hdfs.server.protocol.SyncCommand;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
+import org.apache.hadoop.ipc.ClientId;
 
 /**
  * Utilities for converting protobuf classes to and from implementation classes
@@ -469,11 +486,52 @@ public class PBHelper {
       return PBHelper.convert(proto.getBlkIdCmd());
     case BlockECReconstructionCommand:
       return PBHelper.convert(proto.getBlkECReconstructionCmd());
+    case SyncCommand:
+      return PBHelper.convert(proto.getSyncCommand());
     default:
       return null;
     }
   }
-  
+
+  private static SyncCommand convert(SyncCommandProto backupCommand) {
+    List<BlockSyncTaskProto> syncTasksProtoList =
+        backupCommand.getSyncTasksList();
+    List<BlockSyncTask> syncTasksList =
+        new ArrayList(syncTasksProtoList.size());
+    for (BlockSyncTaskProto syncTaskProto : syncTasksProtoList) {
+      syncTasksList.add(convertSyncTask(syncTaskProto));
+    }
+
+    return new SyncCommand(DatanodeProtocol.DNA_BACKUP, syncTasksList);
+  }
+
+  private static BlockSyncTask convertSyncTask(
+      BlockSyncTaskProto syncTaskProto) {
+    SyncTaskIdProto syncTaskIdProto = syncTaskProto.getSyncTaskId();
+    UUID syncTaskId = convert(syncTaskIdProto);
+    try {
+      return new BlockSyncTask(syncTaskId,
+          new URI(syncTaskProto.getUri()),
+          PBHelperClient.convertLocatedBlocks(
+              syncTaskProto.getLocatedBlocksList()),
+          syncTaskProto.getPartNumber(),
+          syncTaskProto.getUploadHandle().toByteArray(),
+          syncTaskProto.getOffset(),
+          syncTaskProto.getLength(),
+          syncTaskIdProto.getSyncMountId());
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException();
+    }
+  }
+
+  public static UUID convert(SyncTaskIdProto syncTaskIdProto) {
+    byte[] clientId = syncTaskIdProto.getSyncTaskId().toByteArray();
+    long syncTaskIdMsb = ClientId.getMsb(clientId);
+    long syncTaskIdLsb = ClientId.getLsb(clientId);
+    return new UUID(syncTaskIdMsb, syncTaskIdLsb);
+  }
+
+
   public static BalancerBandwidthCommandProto convert(
       BalancerBandwidthCommand bbCmd) {
     return BalancerBandwidthCommandProto.newBuilder()
@@ -603,6 +661,10 @@ public class PBHelper {
           .setBlkECReconstructionCmd(
               convert((BlockECReconstructionCommand) datanodeCommand));
       break;
+    case DatanodeProtocol.DNA_BACKUP:
+      builder.setCmdType(DatanodeCommandProto.Type.SyncCommand)
+          .setSyncCommand(convert((SyncCommand) datanodeCommand));
+      break;
     case DatanodeProtocol.DNA_UNKNOWN: //Not expected
     default:
       builder.setCmdType(DatanodeCommandProto.Type.NullDatanodeCommand);
@@ -1124,4 +1186,130 @@ public class PBHelper {
 
     return new FileRegion(block, providedStorageLocation);
   }
+
+  private static SyncCommandProto convert(SyncCommand syncCommand) {
+    SyncCommandProto.Builder builder = SyncCommandProto.newBuilder();
+
+    List<BlockSyncTaskProto> syncTaskProtos = syncCommand.getSyncTasks()
+        .stream()
+        .map(syncTask -> convert(syncTask))
+        .collect(Collectors.toList());
+
+    builder.addAllSyncTasks(syncTaskProtos);
+
+    return builder.build();
+  }
+
+  private static BlockSyncTaskProto convert(BlockSyncTask blockSyncTask) {
+    BlockSyncTaskProto.Builder builder = BlockSyncTaskProto.newBuilder();
+    builder.addAllLocatedBlocks(
+        PBHelperClient.convertLocatedBlocks2(blockSyncTask.getLocatedBlocks()));
+    builder.setUploadHandle(
+        ByteString.copyFrom(blockSyncTask.getUploadHandle()));
+    builder.setPartNumber(blockSyncTask.getPartNumber());
+    builder.setUri(blockSyncTask.getRemoteURI().toString());
+    builder.setOffset(blockSyncTask.getOffset());
+    builder.setLength(blockSyncTask.getLength());
+
+    return builder.build();
+  }
+
+  public static SyncTaskIdProto convert(UUID syncTaskId, String syncMountId) {
+    SyncTaskIdProto.Builder builder = SyncTaskIdProto.newBuilder();
+    ByteBuffer syncTaskIdBytes = ByteBuffer.wrap(new byte[16]);
+    syncTaskIdBytes.putLong(syncTaskId.getMostSignificantBits());
+    syncTaskIdBytes.putLong(syncTaskId.getLeastSignificantBits());
+    builder.setSyncTaskId(ByteString.copyFrom(syncTaskIdBytes.array()));
+    builder.setSyncMountId(syncMountId);
+    return builder.build();
+  }
+
+
+  public static BulkSyncTaskExecutionFeedbackProto convert(
+      BulkSyncTaskExecutionFeedback bulkFeedback) {
+    return BulkSyncTaskExecutionFeedbackProto.newBuilder()
+        .addAllFeedbacks(bulkFeedback.getFeedbacks().stream()
+            .map(f -> convert(f)).collect(Collectors.toList()))
+        .build();
+  }
+
+  public static SyncTaskExecutionFeedbackProto convert(
+      BlockSyncTaskExecutionFeedback feedback) {
+    SyncTaskExecutionFeedbackProto.Builder builder =
+        SyncTaskExecutionFeedbackProto.newBuilder()
+            .setSyncTaskId(
+                convert(feedback.getSyncTaskId(), feedback.getSyncMountId()))
+            .setOutcome(convert(feedback.getOutcome()));
+    if (feedback.getResult() != null) {
+      builder.setResult(convert(feedback.getResult()));
+    }
+    return builder.build();
+  }
+
+  public static SyncTaskExecutionOutcomeProto convert(
+      SyncTaskExecutionOutcome outcome) {
+    switch (outcome) {
+    case FINISHED_SUCCESSFULLY:
+      return SyncTaskExecutionOutcomeProto.FINISHED_SUCCESSFULLY;
+    case EXCEPTION:
+      return SyncTaskExecutionOutcomeProto.EXCEPTION;
+    default:
+      throw new IllegalArgumentException(
+          "Unknown SyncTaskExecutionOutcome: " + outcome);
+    }
+  }
+
+  public static SyncTaskExecutionResultProto convert(
+      SyncTaskExecutionResult result) {
+    SyncTaskExecutionResultProto.Builder builder =
+        SyncTaskExecutionResultProto.newBuilder();
+    if (result.getResult() != null) {
+      builder.setResult(ByteString.copyFrom(result.getResult()));
+    }
+    if (result.getNumberOfBytes() != null) {
+      builder.setNumberOfBytes(result.getNumberOfBytes());
+    }
+    return builder.build();
+  }
+
+  public static BulkSyncTaskExecutionFeedback convertBulkSyncTaskExecutionFeedback(
+      BulkSyncTaskExecutionFeedbackProto bulkSyncTaskExecutionFeedback) {
+    return new BulkSyncTaskExecutionFeedback(
+        bulkSyncTaskExecutionFeedback.getFeedbacksList().stream()
+            .map(feedback -> convert(feedback)).collect(Collectors.toList()));
+  }
+
+  public static BlockSyncTaskExecutionFeedback convert(
+      SyncTaskExecutionFeedbackProto feedback) {
+    return new BlockSyncTaskExecutionFeedback(convert(feedback.getSyncTaskId()),
+        convert(feedback.getOutcome()),
+        feedback.hasResult() ? convert(feedback.getResult()) : null,
+        feedback.getSyncTaskId().getSyncMountId());
+  }
+
+  public static SyncTaskExecutionOutcome convert(
+      SyncTaskExecutionOutcomeProto outcome) {
+    switch (outcome) {
+    case FINISHED_SUCCESSFULLY:
+      return SyncTaskExecutionOutcome.FINISHED_SUCCESSFULLY;
+    case EXCEPTION:
+      return SyncTaskExecutionOutcome.EXCEPTION;
+    default:
+      throw new IllegalArgumentException(
+          "Unknown SyncTaskExecutionOutcomeProto: " + outcome);
+    }
+  }
+
+  public static SyncTaskExecutionResult convert(
+      SyncTaskExecutionResultProto result) {
+    byte[] bytes = null;
+    if (result.getResult() != null) {
+      bytes = result.getResult().toByteArray();
+    }
+
+    ByteBuffer byteBuffer =
+        (bytes == null) ? null : ByteBuffer.wrap(bytes).asReadOnlyBuffer();
+    return new SyncTaskExecutionResult(byteBuffer, result.getNumberOfBytes());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 9ebc693..bea5fe0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -1631,7 +1631,9 @@ public class DatanodeManager {
       int maxTransfers, int failedVolumes,
       VolumeFailureSummary volumeFailureSummary,
       @Nonnull SlowPeerReports slowPeers,
-      @Nonnull SlowDiskReports slowDisks) throws IOException {
+      @Nonnull SlowDiskReports slowDisks,
+      BulkSyncTaskExecutionFeedback bulkSyncTaskExecutionFeedback)
+      throws IOException {
     final DatanodeDescriptor nodeinfo;
     try {
       nodeinfo = getDatanode(nodeReg);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index a94d2df..0e8144a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -511,6 +512,11 @@ class BPServiceActor implements Runnable {
         outliersReportDue && dn.getDiskMetrics() != null ?
             SlowDiskReports.create(dn.getDiskMetrics().getDiskOutliersStats()) :
             SlowDiskReports.EMPTY_REPORT;
+
+    // TODO - collect feedback from SyncTasks here.
+    BulkSyncTaskExecutionFeedback bulkSyncTaskExecutionFeedback =
+        new BulkSyncTaskExecutionFeedback(Collections.emptyList());
+
     HeartbeatResponse response = bpNamenode.sendHeartbeat(bpRegistration,
         reports,
         dn.getFSDataset().getCacheCapacity(),
@@ -521,7 +527,8 @@ class BPServiceActor implements Runnable {
         volumeFailureSummary,
         requestBlockReportLease,
         slowPeers,
-        slowDisks);
+        slowDisks,
+        bulkSyncTaskExecutionFeedback);
 
     if (outliersReportDue) {
       // If the report was due and successfully sent, schedule the next one.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 8c95f7d..268f27f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -101,6 +101,7 @@ import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.server.namenode.metrics.ReplicatedBlocksMBean;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.apache.hadoop.util.Time.now;
 import static org.apache.hadoop.util.Time.monotonicNow;
@@ -3860,7 +3861,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       VolumeFailureSummary volumeFailureSummary,
       boolean requestFullBlockReportLease,
       @Nonnull SlowPeerReports slowPeers,
-      @Nonnull SlowDiskReports slowDisks) throws IOException {
+      @Nonnull SlowDiskReports slowDisks,
+      BulkSyncTaskExecutionFeedback bulkSyncTaskExecutionFeedback)
+      throws IOException {
     readLock();
     try {
       //get datanode commands
@@ -3869,7 +3872,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
           nodeReg, reports, getBlockPoolId(), cacheCapacity, cacheUsed,
           xceiverCount, maxTransfer, failedVolumes, volumeFailureSummary,
-          slowPeers, slowDisks);
+          slowPeers, slowDisks, bulkSyncTaskExecutionFeedback);
       long blockReportLeaseId = 0;
       if (requestFullBlockReportLease) {
         blockReportLeaseId =  blockManager.requestBlockReportLeaseId(nodeReg);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index c5b9d5a..413a14a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -157,6 +157,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -1498,13 +1499,15 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       int failedVolumes, VolumeFailureSummary volumeFailureSummary,
       boolean requestFullBlockReportLease,
       @Nonnull SlowPeerReports slowPeers,
-      @Nonnull SlowDiskReports slowDisks) throws IOException {
+      @Nonnull SlowDiskReports slowDisks,
+      BulkSyncTaskExecutionFeedback bulkSyncTaskExecutionFeedback)
+      throws IOException {
     checkNNStartup();
     verifyRequest(nodeReg);
     return namesystem.handleHeartbeat(nodeReg, report,
         dnCacheCapacity, dnCacheUsed, xceiverCount, xmitsInProgress,
         failedVolumes, volumeFailureSummary, requestFullBlockReportLease,
-        slowPeers, slowDisks);
+        slowPeers, slowDisks, bulkSyncTaskExecutionFeedback);
   }
 
   @Override // DatanodeProtocol

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTask.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTask.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTask.java
new file mode 100644
index 0000000..875a409
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTask.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.protocol;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import java.net.URI;
+import java.util.List;
+import java.util.UUID;
+
+/**
+ * A BlockSyncTask is an operation that is sent to the datanodes to copy
+ * blocks to an external storage endpoint as a part of an orchestrated
+ * synchronization across multiple datanodes.
+ * BlockSyncTask is intended to be an immutable POJO.
+ */
+public class BlockSyncTask {
+  private final UUID syncTaskId;
+  private final URI remoteURI;
+  private final List<LocatedBlock> locatedBlocks;
+  private String syncMountId;
+  private final int partNumber;
+  private byte[] uploadHandle;
+  private final int offset;
+  private final long length;
+
+  public BlockSyncTask(UUID syncTaskId, URI remoteURI,
+      List<LocatedBlock> locatedBlocks, Integer partNumber, byte[] uploadHandle,
+      int offset, long length, String syncMountId) {
+    this.syncTaskId = syncTaskId;
+    this.remoteURI = remoteURI;
+    this.locatedBlocks = locatedBlocks;
+    this.syncMountId = syncMountId;
+    this.partNumber = partNumber;
+    this.uploadHandle = uploadHandle;
+    this.offset = offset;
+    this.length = length;
+  }
+
+  public int getPartNumber() {
+    return partNumber;
+  }
+
+  public byte[] getUploadHandle() {
+    return uploadHandle;
+  }
+
+  public int getOffset() {
+    return offset;
+  }
+
+  public long getLength() {
+    return length;
+  }
+
+  public UUID getSyncTaskId() {
+    return syncTaskId;
+  }
+
+  public URI getRemoteURI() {
+    return remoteURI;
+  }
+
+  public List<LocatedBlock> getLocatedBlocks() {
+    return locatedBlocks;
+  }
+
+  public String getSyncMountId() {
+    return syncMountId;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BulkSyncTaskExecutionFeedback.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BulkSyncTaskExecutionFeedback.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BulkSyncTaskExecutionFeedback.java
new file mode 100644
index 0000000..0d459e8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BulkSyncTaskExecutionFeedback.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.protocol;
+
+import java.util.Collection;
+
+/**
+ * Feedback for a collection of {@link BlockSyncTask}s.
+ */
+public class BulkSyncTaskExecutionFeedback {
+
+  private Collection<BlockSyncTaskExecutionFeedback> feedbacks;
+
+  public BulkSyncTaskExecutionFeedback(
+      Collection<BlockSyncTaskExecutionFeedback> feedbacks) {
+    this.feedbacks = feedbacks;
+  }
+
+  public Collection<BlockSyncTaskExecutionFeedback> getFeedbacks() {
+    return feedbacks;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
index 1f55100..d69dee7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
@@ -79,6 +79,7 @@ public interface DatanodeProtocol {
   final static int DNA_CACHE = 9;      // cache blocks
   final static int DNA_UNCACHE = 10;   // uncache blocks
   final static int DNA_ERASURE_CODING_RECONSTRUCTION = 11; // erasure coding reconstruction command
+  final static int DNA_BACKUP = 14; // back up data to PROVIDED stores.
 
   /** 
    * Register Datanode.
@@ -101,6 +102,8 @@ public interface DatanodeProtocol {
    * or to copy them to other DataNodes, etc.
    * @param registration datanode registration information
    * @param reports utilization report per storage
+   * @param cacheCapacity
+   * @param cacheUsed
    * @param xmitsInProgress number of transfers from this datanode to others
    * @param xceiverCount number of active transceiver threads
    * @param failedVolumes number of failed volumes
@@ -110,20 +113,19 @@ public interface DatanodeProtocol {
    * @param slowPeers Details of peer DataNodes that were detected as being
    *                  slow to respond to packet writes. Empty report if no
    *                  slow peers were detected by the DataNode.
+   * @param bulkSyncTaskExecutionFeedback Result of the execution of the
+   *                                      sync tasks.
    * @throws IOException on error
    */
   @Idempotent
   public HeartbeatResponse sendHeartbeat(DatanodeRegistration registration,
-                                       StorageReport[] reports,
-                                       long dnCacheCapacity,
-                                       long dnCacheUsed,
-                                       int xmitsInProgress,
-                                       int xceiverCount,
-                                       int failedVolumes,
-                                       VolumeFailureSummary volumeFailureSummary,
-                                       boolean requestFullBlockReportLease,
-                                       @Nonnull SlowPeerReports slowPeers,
-                                       @Nonnull SlowDiskReports slowDisks)
+      StorageReport[] reports, long cacheCapacity, long cacheUsed,
+      int xmitsInProgress, int xceiverCount, int failedVolumes,
+      VolumeFailureSummary volumeFailureSummary,
+      boolean requestFullBlockReportLease,
+      @Nonnull SlowPeerReports slowPeers,
+      @Nonnull SlowDiskReports slowDisks,
+      BulkSyncTaskExecutionFeedback bulkSyncTaskExecutionFeedback)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncCommand.java
new file mode 100644
index 0000000..7e2c242
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncCommand.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.protocol;
+
+import java.util.List;
+
+/**
+ * A SyncCommand is an instruction to a DataNode to move the
+ * give file to specified target DataNodes provided storage.
+ */
+public class SyncCommand extends DatanodeCommand {
+
+  private final List<BlockSyncTask> syncTasks;
+
+  public SyncCommand(int action, List<BlockSyncTask> syncTasks) {
+    super(action);
+    this.syncTasks = syncTasks;
+  }
+
+  public List<BlockSyncTask> getSyncTasks() {
+    return syncTasks;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index bf0df5b..5d6ef41 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -60,6 +60,7 @@ message DatanodeCommandProto {
     NullDatanodeCommand = 7;
     BlockIdCommand = 8;
     BlockECReconstructionCommand = 9;
+    SyncCommand = 10;
   }
 
   required Type cmdType = 1;    // Type of the command
@@ -74,6 +75,7 @@ message DatanodeCommandProto {
   optional RegisterCommandProto registerCmd = 7;
   optional BlockIdCommandProto blkIdCmd = 8;
   optional BlockECReconstructionCommandProto blkECReconstructionCmd = 9;
+  optional SyncCommandProto syncCommand = 10;
 }
 
 /**
@@ -154,6 +156,89 @@ message BlockECReconstructionCommandProto {
   repeated BlockECReconstructionInfoProto blockECReconstructioninfo = 1;
 }
 
+message SyncTaskIdProto {
+  required bytes syncTaskId = 1;
+  required string syncMountId = 2;
+}
+
+/**
+ * Instruct the datanode to perform a backup command
+ */
+message SyncCommandProto {
+  repeated BlockSyncTaskProto syncTasks = 1;
+}
+
+/**
+ * A block synchronization task as part of an orchestrated synchronization
+ * across potentially multiple datanodes (i.e. multipart put part).
+ */
+message BlockSyncTaskProto {
+  required SyncTaskIdProto syncTaskId = 1;
+
+  required bytes uploadHandle = 2;
+  required int32 partNumber = 3;
+  repeated LocatedBlockProto locatedBlocks = 4;
+  required string uri = 5;
+  required int32 offset = 6;
+  required int64 length = 7;
+}
+
+/**
+ * Block storage movement information
+ */
+message BlockMovingInfoProto {
+  required BlockProto block = 1;
+  required DatanodeInfoProto sourceDnInfo = 2;
+  required DatanodeInfoProto targetDnInfo = 3;
+  required StorageTypeProto sourceStorageType = 4;
+  required StorageTypeProto targetStorageType = 5;
+}
+
+/**
+ * Blocks for which storage movements has been attempted and finished
+ * with either success or failure.
+ */
+message BlocksStorageMoveAttemptFinishedProto {
+  repeated BlockProto blocks = 1;
+}
+
+/**
+ * A collection of feedbacks for a collection of sync tasks.
+ */
+message BulkSyncTaskExecutionFeedbackProto {
+  repeated SyncTaskExecutionFeedbackProto feedbacks = 1;
+}
+
+/**
+ * Feedback for a sync task that has been executed.
+ * syncTaskId - identifier for the task.
+ * outcome - success/error.
+ * operation - the type of operation.
+ * result - if the outcome is successful, the results of the sync task.
+ */
+message SyncTaskExecutionFeedbackProto {
+  required SyncTaskIdProto syncTaskId = 1;
+  required SyncTaskExecutionOutcomeProto outcome = 2;
+  optional SyncTaskExecutionResultProto result = 3;
+}
+
+/**
+ * Success of failure indication of a sync task.
+ */
+enum SyncTaskExecutionOutcomeProto {
+  FINISHED_SUCCESSFULLY = 0;
+  EXCEPTION = 1;
+}
+
+/**
+ * result - the opaque result data from the sync task.
+ * numberOfBytes - the number of bytes copied.
+ */
+message SyncTaskExecutionResultProto {
+  optional bytes result = 1;
+  optional int64 numberOfBytes = 2;
+}
+
 /**
  * registration - Information of the datanode registering with the namenode
  */
@@ -210,6 +295,7 @@ message HeartbeatRequestProto {
   optional bool requestFullBlockReportLease = 9 [ default = false ];
   repeated SlowPeerReportProto slowPeers = 10;
   repeated SlowDiskReportProto slowDisks = 11;
+  optional BulkSyncTaskExecutionFeedbackProto bulkSyncTaskExecutionFeedback = 12;
 }
 
 /**
@@ -275,7 +361,7 @@ message StorageBlockReportProto {
  */
 message BlockReportResponseProto {
   optional DatanodeCommandProto cmd = 1;
-} 
+}
 
 /**
  * registration - datanode registration information

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
index dd6f40a..8ba0396 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
@@ -589,7 +589,7 @@ public class TestDatanodeManager {
     Mockito.when(dm.getDatanode(dnReg)).thenReturn(nodeInfo);
     DatanodeCommand[] cmds = dm.handleHeartbeat(
         dnReg, new StorageReport[1], "bp-123", 0, 0, 10, maxTransfers, 0, null,
-        SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT);
+        SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT, null);
 
     long expectedNumCmds = Arrays.stream(
         new int[]{numReplicationTasks, numECTasks})

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
index 05b6d30..e81fb1f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
@@ -116,7 +116,7 @@ public class TestNameNodePrunesMissingStorages {
       cluster.stopDataNode(0);
       cluster.getNameNodeRpc().sendHeartbeat(dnReg, prunedReports, 0L, 0L, 0, 0,
           0, null, true, SlowPeerReports.EMPTY_REPORT,
-          SlowDiskReports.EMPTY_REPORT);
+          SlowDiskReports.EMPTY_REPORT, null);
 
       // Check that the missing storage was pruned.
       assertThat(dnDescriptor.getStorageInfos().length, is(expectedStoragesAfterTest));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
index b453991..2a6975f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
@@ -167,7 +167,8 @@ public class InternalDataNodeTestUtils {
             Mockito.anyInt(), Mockito.any(VolumeFailureSummary.class),
             Mockito.anyBoolean(),
             Mockito.any(SlowPeerReports.class),
-            Mockito.any(SlowDiskReports.class))).thenReturn(
+            Mockito.any(SlowDiskReports.class),
+            null)).thenReturn(
         new HeartbeatResponse(new DatanodeCommand[0], new NNHAStatusHeartbeat(
             HAServiceState.ACTIVE, 1), null, ThreadLocalRandom.current()
             .nextLong() | 1L));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
index 4863ca1..62b84d9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
+
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
@@ -157,7 +159,8 @@ public class TestBPOfferService {
           Mockito.any(VolumeFailureSummary.class),
           Mockito.anyBoolean(),
           Mockito.any(SlowPeerReports.class),
-          Mockito.any(SlowDiskReports.class));
+          Mockito.any(SlowDiskReports.class),
+          Mockito.any(BulkSyncTaskExecutionFeedback.class));
     mockHaStatuses[nnIdx] = new NNHAStatusHeartbeat(HAServiceState.STANDBY, 0);
     datanodeCommands[nnIdx] = new DatanodeCommand[0];
     return mock;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
index 07fd4ae..09fae14 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 import org.apache.hadoop.hdfs.AppendTestUtil;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
@@ -232,7 +233,8 @@ public class TestBlockRecovery {
             Mockito.any(VolumeFailureSummary.class),
             Mockito.anyBoolean(),
             Mockito.any(SlowPeerReports.class),
-            Mockito.any(SlowDiskReports.class)))
+            Mockito.any(SlowDiskReports.class),
+            Mockito.any(BulkSyncTaskExecutionFeedback.class)))
         .thenReturn(new HeartbeatResponse(
             new DatanodeCommand[0],
             new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
index 28427bc..6374540 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
@@ -24,13 +24,14 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHEC
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIFELINE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY;
 
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.any;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.anyBoolean;
 import static org.mockito.Mockito.anyInt;
 import static org.mockito.Mockito.anyLong;
@@ -172,7 +173,8 @@ public class TestDataNodeLifeline {
             any(VolumeFailureSummary.class),
             anyBoolean(),
             any(SlowPeerReports.class),
-            any(SlowDiskReports.class));
+            any(SlowDiskReports.class),
+            any(BulkSyncTaskExecutionFeedback.class));
 
     // Intercept lifeline to trigger latch count-down on each call.
     doAnswer(new LatchCountingAnswer<Void>(lifelinesSent))
@@ -237,7 +239,8 @@ public class TestDataNodeLifeline {
             any(VolumeFailureSummary.class),
             anyBoolean(),
             any(SlowPeerReports.class),
-            any(SlowDiskReports.class));
+            any(SlowDiskReports.class),
+            any(BulkSyncTaskExecutionFeedback.class));
 
     // While waiting on the latch for the expected number of heartbeat messages,
     // poll DataNode tracking information.  We expect that the DataNode always

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
index bb1d9ef..2d7dea9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdfs.server.datanode;
 
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -222,7 +223,8 @@ public class TestDatanodeProtocolRetryPolicy {
            Mockito.any(VolumeFailureSummary.class),
            Mockito.anyBoolean(),
            Mockito.any(SlowPeerReports.class),
-           Mockito.any(SlowDiskReports.class));
+           Mockito.any(SlowDiskReports.class),
+           Mockito.any(BulkSyncTaskExecutionFeedback.class));
 
     dn = new DataNode(conf, locations, null, null) {
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
index 2dbd5b9..2cf0135 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import net.jcip.annotations.NotThreadSafe;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
 import static org.junit.Assert.assertEquals;
@@ -208,7 +209,8 @@ public class TestFsDatasetCache {
           (StorageReport[]) any(), anyLong(), anyLong(),
           anyInt(), anyInt(), anyInt(), (VolumeFailureSummary) any(),
           anyBoolean(), any(SlowPeerReports.class),
-          any(SlowDiskReports.class));
+          any(SlowDiskReports.class),
+          any(BulkSyncTaskExecutionFeedback.class));
     } finally {
       lock.writeLock().unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
index 5f62ddb..0f0bc1d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
@@ -110,7 +111,8 @@ public class TestStorageReport {
         anyLong(), anyLong(), anyInt(), anyInt(), anyInt(),
         Mockito.any(VolumeFailureSummary.class), Mockito.anyBoolean(),
         Mockito.any(SlowPeerReports.class),
-        Mockito.any(SlowDiskReports.class));
+        Mockito.any(SlowDiskReports.class),
+        Mockito.any(BulkSyncTaskExecutionFeedback.class));
 
     StorageReport[] reports = captor.getValue();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
index 3a3c471..9940174 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
@@ -956,8 +956,8 @@ public class NNThroughputBenchmark implements Tool {
           DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0L) };
       DatanodeCommand[] cmds = dataNodeProto.sendHeartbeat(dnRegistration, rep,
           0L, 0L, 0, 0, 0, null, true,
-          SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT)
-          .getCommands();
+          SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
+          null).getCommands();
       if(cmds != null) {
         for (DatanodeCommand cmd : cmds ) {
           if(LOG.isDebugEnabled()) {
@@ -1007,8 +1007,8 @@ public class NNThroughputBenchmark implements Tool {
           false, DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0) };
       DatanodeCommand[] cmds = dataNodeProto.sendHeartbeat(dnRegistration,
           rep, 0L, 0L, 0, 0, 0, null, true,
-          SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT)
-          .getCommands();
+          SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
+          null).getCommands();
       if (cmds != null) {
         for (DatanodeCommand cmd : cmds) {
           if (cmd.getAction() == DatanodeProtocol.DNA_TRANSFER) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
index b85527a..8589f75 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
@@ -17,11 +17,13 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.mockito.Mockito.spy;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.commons.lang3.reflect.FieldUtils;
@@ -130,7 +132,8 @@ public class NameNodeAdapter {
     return namesystem.handleHeartbeat(nodeReg,
         BlockManagerTestUtil.getStorageReportsForDatanode(dd),
         dd.getCacheCapacity(), dd.getCacheRemaining(), 0, 0, 0, null, true,
-        SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT);
+        SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
+        new BulkSyncTaskExecutionFeedback(new ArrayList<>()));
   }
 
   public static boolean setReplication(final FSNamesystem ns,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d52a2afb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index 366f584..9f9a897 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@ -139,8 +139,8 @@ public class TestDeadDatanode {
         false, 0, 0, 0, 0, 0) };
     DatanodeCommand[] cmd =
         dnp.sendHeartbeat(reg, rep, 0L, 0L, 0, 0, 0, null, true,
-            SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT)
-            .getCommands();
+            SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
+            null).getCommands();
     assertEquals(1, cmd.length);
     assertEquals(cmd[0].getAction(), RegisterCommand.REGISTER
         .getAction());


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


[36/50] hadoop git commit: HADOOP-15636. Follow-up from HADOOP-14918; restoring test under new name. Contributed by Gabor Bota.

Posted by eh...@apache.org.
HADOOP-15636. Follow-up from HADOOP-14918; restoring test under new name. Contributed by Gabor Bota.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/59adeb8d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/59adeb8d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/59adeb8d

Branch: refs/heads/HDFS-12090
Commit: 59adeb8d7f2f04bc56d37b2a2e65596fee6e4894
Parents: ed9d60e
Author: Sean Mackrory <ma...@apache.org>
Authored: Thu Jul 26 10:25:47 2018 -0600
Committer: Sean Mackrory <ma...@apache.org>
Committed: Fri Jul 27 18:23:29 2018 -0600

----------------------------------------------------------------------
 .../s3a/s3guard/ITestDynamoDBMetadataStore.java | 649 +++++++++++++++++++
 1 file changed, 649 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/59adeb8d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java
new file mode 100644
index 0000000..a597858
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java
@@ -0,0 +1,649 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import com.amazonaws.services.dynamodbv2.document.DynamoDB;
+import com.amazonaws.services.dynamodbv2.document.Item;
+import com.amazonaws.services.dynamodbv2.document.PrimaryKey;
+import com.amazonaws.services.dynamodbv2.document.Table;
+import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription;
+import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
+import com.amazonaws.services.dynamodbv2.model.TableDescription;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.Tristate;
+
+import org.apache.hadoop.io.IOUtils;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*;
+import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.*;
+import static org.apache.hadoop.test.LambdaTestUtils.*;
+
+/**
+ * Test that {@link DynamoDBMetadataStore} implements {@link MetadataStore}.
+ *
+ * In this integration test, we use a real AWS DynamoDB. A
+ * {@link DynamoDBMetadataStore} object is created in the @BeforeClass method,
+ * and shared for all test in the @BeforeClass method. You will be charged
+ * bills for AWS S3 or DynamoDB when you run these tests.
+ *
+ * According to the base class, every test case will have independent contract
+ * to create a new {@link S3AFileSystem} instance and initializes it.
+ * A table will be created and shared between the tests,
+ */
+public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestDynamoDBMetadataStore.class);
+  public static final PrimaryKey
+      VERSION_MARKER_PRIMARY_KEY = createVersionMarkerPrimaryKey(
+      DynamoDBMetadataStore.VERSION_MARKER);
+
+  private S3AFileSystem fileSystem;
+  private S3AContract s3AContract;
+
+  private URI fsUri;
+
+  private String bucket;
+
+  private static DynamoDBMetadataStore ddbmsStatic;
+
+  private static String TEST_DYNAMODB_TABLE_NAME;
+
+  /**
+   * Create a path under the test path provided by
+   * the FS contract.
+   * @param filepath path string in
+   * @return a path qualified by the test filesystem
+   */
+  protected Path path(String filepath) {
+    return getFileSystem().makeQualified(
+        new Path(s3AContract.getTestPath(), filepath));
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    Configuration conf = prepareTestConfiguration(new Configuration());
+    assertThatDynamoMetadataStoreImpl(conf);
+    Assume.assumeTrue("Test DynamoDB table name should be set to run "
+            + "integration tests.", TEST_DYNAMODB_TABLE_NAME != null);
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, TEST_DYNAMODB_TABLE_NAME);
+
+    s3AContract = new S3AContract(conf);
+    s3AContract.init();
+
+    fileSystem = (S3AFileSystem) s3AContract.getTestFileSystem();
+    assume("No test filesystem", s3AContract.isEnabled());
+    assertNotNull("No test filesystem", fileSystem);
+    fsUri = fileSystem.getUri();
+    bucket = fileSystem.getBucket();
+
+    try{
+      super.setUp();
+    } catch (FileNotFoundException e){
+      LOG.warn("MetadataStoreTestBase setup failed. Waiting for table to be "
+          + "deleted before trying again.");
+      ddbmsStatic.getTable().waitForDelete();
+      super.setUp();
+    }
+  }
+
+
+  @BeforeClass
+  public static void beforeClassSetup() throws IOException {
+    Configuration conf = prepareTestConfiguration(new Configuration());
+    assertThatDynamoMetadataStoreImpl(conf);
+    TEST_DYNAMODB_TABLE_NAME = conf.get(S3GUARD_DDB_TEST_TABLE_NAME_KEY);
+    Assume.assumeTrue("Test DynamoDB table name should be set to run "
+        + "integration tests.", TEST_DYNAMODB_TABLE_NAME != null);
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, TEST_DYNAMODB_TABLE_NAME);
+
+    LOG.debug("Creating static ddbms which will be shared between tests.");
+    ddbmsStatic = new DynamoDBMetadataStore();
+    ddbmsStatic.initialize(conf);
+  }
+
+  @AfterClass
+  public static void afterClassTeardown() {
+    LOG.debug("Destroying static DynamoDBMetadataStore.");
+    if (ddbmsStatic != null) {
+      try {
+        ddbmsStatic.destroy();
+      } catch (Exception e) {
+        LOG.warn("Failed to destroy tables in teardown", e);
+      }
+      IOUtils.closeStream(ddbmsStatic);
+      ddbmsStatic = null;
+    }
+  }
+
+  private static void assertThatDynamoMetadataStoreImpl(Configuration conf){
+    Assume.assumeTrue("Test only applies when DynamoDB is used for S3Guard",
+        conf.get(Constants.S3_METADATA_STORE_IMPL).equals(
+            Constants.S3GUARD_METASTORE_DYNAMO));
+  }
+
+
+  @Override
+  public void tearDown() throws Exception {
+    LOG.info("Removing data from ddbms table in teardown.");
+    // The following is a way to be sure the table will be cleared and there
+    // will be no leftovers after the test.
+    PathMetadata meta = ddbmsStatic.get(strToPath("/"));
+    if (meta != null){
+      for (DescendantsIterator desc = new DescendantsIterator(ddbmsStatic, meta);
+           desc.hasNext();) {
+        ddbmsStatic.forgetMetadata(desc.next().getPath());
+      }
+    }
+
+    fileSystem.close();
+  }
+
+  /**
+   * Each contract has its own S3AFileSystem and DynamoDBMetadataStore objects.
+   */
+  private class DynamoDBMSContract extends AbstractMSContract {
+
+    DynamoDBMSContract(Configuration conf) {
+    }
+
+    DynamoDBMSContract() {
+      this(new Configuration());
+    }
+
+    @Override
+    public S3AFileSystem getFileSystem() {
+      return ITestDynamoDBMetadataStore.this.fileSystem;
+    }
+
+    @Override
+    public DynamoDBMetadataStore getMetadataStore() {
+      return ITestDynamoDBMetadataStore.ddbmsStatic;
+    }
+  }
+
+  @Override
+  public DynamoDBMSContract createContract() {
+    return new DynamoDBMSContract();
+  }
+
+  @Override
+  public DynamoDBMSContract createContract(Configuration conf) {
+    return new DynamoDBMSContract(conf);
+  }
+
+  @Override
+  FileStatus basicFileStatus(Path path, int size, boolean isDir)
+      throws IOException {
+    String owner = UserGroupInformation.getCurrentUser().getShortUserName();
+    return isDir
+        ? new S3AFileStatus(true, path, owner)
+        : new S3AFileStatus(size, getModTime(), path, BLOCK_SIZE, owner);
+  }
+
+  private DynamoDBMetadataStore getDynamoMetadataStore() throws IOException {
+    return (DynamoDBMetadataStore) getContract().getMetadataStore();
+  }
+
+  private S3AFileSystem getFileSystem() {
+    return this.fileSystem;
+  }
+
+  /**
+   * This tests that after initialize() using an S3AFileSystem object, the
+   * instance should have been initialized successfully, and tables are ACTIVE.
+   */
+  @Test
+  public void testInitialize() throws IOException {
+    final S3AFileSystem s3afs = this.fileSystem;
+    final String tableName = "testInitialize";
+    final Configuration conf = s3afs.getConf();
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(s3afs);
+      verifyTableInitialized(tableName, ddbms.getDynamoDB());
+      assertNotNull(ddbms.getTable());
+      assertEquals(tableName, ddbms.getTable().getTableName());
+      String expectedRegion = conf.get(S3GUARD_DDB_REGION_KEY,
+          s3afs.getBucketLocation(bucket));
+      assertEquals("DynamoDB table should be in configured region or the same" +
+              " region as S3 bucket",
+          expectedRegion,
+          ddbms.getRegion());
+      ddbms.destroy();
+    }
+  }
+
+  /**
+   * This tests that after initialize() using a Configuration object, the
+   * instance should have been initialized successfully, and tables are ACTIVE.
+   */
+  @Test
+  public void testInitializeWithConfiguration() throws IOException {
+    final String tableName = "testInitializeWithConfiguration";
+    final Configuration conf = getFileSystem().getConf();
+    conf.unset(S3GUARD_DDB_TABLE_NAME_KEY);
+    String savedRegion = conf.get(S3GUARD_DDB_REGION_KEY,
+        getFileSystem().getBucketLocation());
+    conf.unset(S3GUARD_DDB_REGION_KEY);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(conf);
+      fail("Should have failed because the table name is not set!");
+    } catch (IllegalArgumentException ignored) {
+    }
+    // config table name
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(conf);
+      fail("Should have failed because as the region is not set!");
+    } catch (IllegalArgumentException ignored) {
+    }
+    // config region
+    conf.set(S3GUARD_DDB_REGION_KEY, savedRegion);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(conf);
+      verifyTableInitialized(tableName, ddbms.getDynamoDB());
+      assertNotNull(ddbms.getTable());
+      assertEquals(tableName, ddbms.getTable().getTableName());
+      assertEquals("Unexpected key schema found!",
+          keySchema(),
+          ddbms.getTable().describe().getKeySchema());
+      ddbms.destroy();
+    }
+  }
+
+  /**
+   * Test that for a large batch write request, the limit is handled correctly.
+   */
+  @Test
+  public void testBatchWrite() throws IOException {
+    final int[] numMetasToDeleteOrPut = {
+        -1, // null
+        0, // empty collection
+        1, // one path
+        S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT, // exact limit of a batch request
+        S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT + 1 // limit + 1
+    };
+    DynamoDBMetadataStore ms = getDynamoMetadataStore();
+    for (int numOldMetas : numMetasToDeleteOrPut) {
+      for (int numNewMetas : numMetasToDeleteOrPut) {
+        doTestBatchWrite(numOldMetas, numNewMetas, ms);
+      }
+    }
+  }
+
+  private void doTestBatchWrite(int numDelete, int numPut,
+      DynamoDBMetadataStore ms) throws IOException {
+    Path path = new Path(
+        "/ITestDynamoDBMetadataStore_testBatchWrite_" + numDelete + '_'
+            + numPut);
+    final Path root = fileSystem.makeQualified(path);
+    final Path oldDir = new Path(root, "oldDir");
+    final Path newDir = new Path(root, "newDir");
+    LOG.info("doTestBatchWrite: oldDir={}, newDir={}", oldDir, newDir);
+
+    ms.put(new PathMetadata(basicFileStatus(oldDir, 0, true)));
+    ms.put(new PathMetadata(basicFileStatus(newDir, 0, true)));
+
+    final List<PathMetadata> oldMetas = numDelete < 0 ? null :
+        new ArrayList<>(numDelete);
+    for (int i = 0; i < numDelete; i++) {
+      oldMetas.add(new PathMetadata(
+          basicFileStatus(new Path(oldDir, "child" + i), i, true)));
+    }
+    final List<PathMetadata> newMetas = numPut < 0 ? null :
+        new ArrayList<>(numPut);
+    for (int i = 0; i < numPut; i++) {
+      newMetas.add(new PathMetadata(
+          basicFileStatus(new Path(newDir, "child" + i), i, false)));
+    }
+
+    Collection<Path> pathsToDelete = null;
+    if (oldMetas != null) {
+      // put all metadata of old paths and verify
+      ms.put(new DirListingMetadata(oldDir, oldMetas, false));
+      assertEquals(0, ms.listChildren(newDir).withoutTombstones().numEntries());
+      assertTrue(CollectionUtils
+          .isEqualCollection(oldMetas, ms.listChildren(oldDir).getListing()));
+
+      pathsToDelete = new ArrayList<>(oldMetas.size());
+      for (PathMetadata meta : oldMetas) {
+        pathsToDelete.add(meta.getFileStatus().getPath());
+      }
+    }
+
+    // move the old paths to new paths and verify
+    ms.move(pathsToDelete, newMetas);
+    assertEquals(0, ms.listChildren(oldDir).withoutTombstones().numEntries());
+    if (newMetas != null) {
+      assertTrue(CollectionUtils
+          .isEqualCollection(newMetas, ms.listChildren(newDir).getListing()));
+    }
+  }
+
+  @Test
+  public void testInitExistingTable() throws IOException {
+    final DynamoDBMetadataStore ddbms = getDynamoMetadataStore();
+    final String tableName = ddbms.getTable().getTableName();
+    verifyTableInitialized(tableName, ddbms.getDynamoDB());
+    // create existing table
+    ddbms.initTable();
+    verifyTableInitialized(tableName, ddbms.getDynamoDB());
+  }
+
+  /**
+   * Test the low level version check code.
+   */
+  @Test
+  public void testItemVersionCompatibility() throws Throwable {
+    verifyVersionCompatibility("table",
+        createVersionMarker(VERSION_MARKER, VERSION, 0));
+  }
+
+  /**
+   * Test that a version marker entry without the version number field
+   * is rejected as incompatible with a meaningful error message.
+   */
+  @Test
+  public void testItemLacksVersion() throws Throwable {
+    intercept(IOException.class, E_NOT_VERSION_MARKER,
+        () -> verifyVersionCompatibility("table",
+            new Item().withPrimaryKey(
+                createVersionMarkerPrimaryKey(VERSION_MARKER))));
+  }
+
+  /**
+   * Delete the version marker and verify that table init fails.
+   */
+  @Test
+  public void testTableVersionRequired() throws Exception {
+    String tableName = "testTableVersionRequired";
+    Configuration conf = getFileSystem().getConf();
+    int maxRetries = conf.getInt(S3GUARD_DDB_MAX_RETRIES,
+        S3GUARD_DDB_MAX_RETRIES_DEFAULT);
+    conf.setInt(S3GUARD_DDB_MAX_RETRIES, 3);
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+
+    try(DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(conf);
+      Table table = verifyTableInitialized(tableName, ddbms.getDynamoDB());
+      table.deleteItem(VERSION_MARKER_PRIMARY_KEY);
+
+      // create existing table
+      intercept(IOException.class, E_NO_VERSION_MARKER,
+          () -> ddbms.initTable());
+
+      conf.setInt(S3GUARD_DDB_MAX_RETRIES, maxRetries);
+      ddbms.destroy();
+    }
+  }
+
+  /**
+   * Set the version value to a different number and verify that
+   * table init fails.
+   */
+  @Test
+  public void testTableVersionMismatch() throws Exception {
+    String tableName = "testTableVersionMismatch";
+    Configuration conf = getFileSystem().getConf();
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+
+    try(DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(conf);
+      Table table = verifyTableInitialized(tableName, ddbms.getDynamoDB());
+      table.deleteItem(VERSION_MARKER_PRIMARY_KEY);
+      Item v200 = createVersionMarker(VERSION_MARKER, 200, 0);
+      table.putItem(v200);
+
+      // create existing table
+      intercept(IOException.class, E_INCOMPATIBLE_VERSION,
+          () -> ddbms.initTable());
+      ddbms.destroy();
+    }
+  }
+
+
+
+
+  /**
+   * Test that initTable fails with IOException when table does not exist and
+   * table auto-creation is disabled.
+   */
+  @Test
+  public void testFailNonexistentTable() throws IOException {
+    final String tableName = "testFailNonexistentTable";
+    final S3AFileSystem s3afs = getFileSystem();
+    final Configuration conf = s3afs.getConf();
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+    conf.unset(S3GUARD_DDB_TABLE_CREATE_KEY);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(s3afs);
+      fail("Should have failed as table does not exist and table auto-creation"
+          + " is disabled");
+    } catch (IOException ignored) {
+    }
+  }
+
+  /**
+   * Test cases about root directory as it is not in the DynamoDB table.
+   */
+  @Test
+  public void testRootDirectory() throws IOException {
+    final DynamoDBMetadataStore ddbms = getDynamoMetadataStore();
+    Path rootPath = new Path(new Path(fsUri), "/");
+    verifyRootDirectory(ddbms.get(rootPath), true);
+
+    ddbms.put(new PathMetadata(new S3AFileStatus(true,
+        new Path(rootPath, "foo"),
+        UserGroupInformation.getCurrentUser().getShortUserName())));
+    verifyRootDirectory(ddbms.get(rootPath), false);
+  }
+
+  private void verifyRootDirectory(PathMetadata rootMeta, boolean isEmpty) {
+    assertNotNull(rootMeta);
+    final FileStatus status = rootMeta.getFileStatus();
+    assertNotNull(status);
+    assertTrue(status.isDirectory());
+    // UNKNOWN is always a valid option, but true / false should not contradict
+    if (isEmpty) {
+      assertNotSame("Should not be marked non-empty",
+          Tristate.FALSE,
+          rootMeta.isEmptyDirectory());
+    } else {
+      assertNotSame("Should not be marked empty",
+          Tristate.TRUE,
+          rootMeta.isEmptyDirectory());
+    }
+  }
+
+  /**
+   * Test that when moving nested paths, all its ancestors up to destination
+   * root will also be created.
+   * Here is the directory tree before move:
+   * <pre>
+   * testMovePopulateAncestors
+   * ├── a
+   * │   └── b
+   * │       └── src
+   * │           ├── dir1
+   * │           │   └── dir2
+   * │           └── file1.txt
+   * └── c
+   *     └── d
+   *         └── dest
+   *</pre>
+   * As part of rename(a/b/src, d/c/dest), S3A will enumerate the subtree at
+   * a/b/src.  This test verifies that after the move, the new subtree at
+   * 'dest' is reachable from the root (i.e. c/ and c/d exist in the table.
+   * DynamoDBMetadataStore depends on this property to do recursive delete
+   * without a full table scan.
+   */
+  @Test
+  public void testMovePopulatesAncestors() throws IOException {
+    final DynamoDBMetadataStore ddbms = getDynamoMetadataStore();
+    final String testRoot = "/testMovePopulatesAncestors";
+    final String srcRoot = testRoot + "/a/b/src";
+    final String destRoot = testRoot + "/c/d/e/dest";
+
+    final Path nestedPath1 = strToPath(srcRoot + "/file1.txt");
+    ddbms.put(new PathMetadata(basicFileStatus(nestedPath1, 1024, false)));
+    final Path nestedPath2 = strToPath(srcRoot + "/dir1/dir2");
+    ddbms.put(new PathMetadata(basicFileStatus(nestedPath2, 0, true)));
+
+    // We don't put the destRoot path here, since put() would create ancestor
+    // entries, and we want to ensure that move() does it, instead.
+
+    // Build enumeration of src / dest paths and do the move()
+    final Collection<Path> fullSourcePaths = Lists.newArrayList(
+        strToPath(srcRoot),
+        strToPath(srcRoot + "/dir1"),
+        strToPath(srcRoot + "/dir1/dir2"),
+        strToPath(srcRoot + "/file1.txt")
+    );
+    final Collection<PathMetadata> pathsToCreate = Lists.newArrayList(
+        new PathMetadata(basicFileStatus(strToPath(destRoot),
+            0, true)),
+        new PathMetadata(basicFileStatus(strToPath(destRoot + "/dir1"),
+            0, true)),
+        new PathMetadata(basicFileStatus(strToPath(destRoot + "/dir1/dir2"),
+            0, true)),
+        new PathMetadata(basicFileStatus(strToPath(destRoot + "/file1.txt"),
+            1024, false))
+    );
+
+    ddbms.move(fullSourcePaths, pathsToCreate);
+
+    // assert that all the ancestors should have been populated automatically
+    assertCached(testRoot + "/c");
+    assertCached(testRoot + "/c/d");
+    assertCached(testRoot + "/c/d/e");
+    assertCached(destRoot /* /c/d/e/dest */);
+
+    // Also check moved files while we're at it
+    assertCached(destRoot + "/dir1");
+    assertCached(destRoot + "/dir1/dir2");
+    assertCached(destRoot + "/file1.txt");
+  }
+
+  @Test
+  public void testProvisionTable() throws IOException {
+    final String tableName = "testProvisionTable";
+    Configuration conf = getFileSystem().getConf();
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+
+    try(DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(conf);
+      DynamoDB dynamoDB = ddbms.getDynamoDB();
+      final ProvisionedThroughputDescription oldProvision =
+          dynamoDB.getTable(tableName).describe().getProvisionedThroughput();
+      ddbms.provisionTable(oldProvision.getReadCapacityUnits() * 2,
+          oldProvision.getWriteCapacityUnits() * 2);
+      ddbms.initTable();
+      final ProvisionedThroughputDescription newProvision =
+          dynamoDB.getTable(tableName).describe().getProvisionedThroughput();
+      LOG.info("Old provision = {}, new provision = {}", oldProvision,
+          newProvision);
+      assertEquals(oldProvision.getReadCapacityUnits() * 2,
+          newProvision.getReadCapacityUnits().longValue());
+      assertEquals(oldProvision.getWriteCapacityUnits() * 2,
+          newProvision.getWriteCapacityUnits().longValue());
+      ddbms.destroy();
+    }
+  }
+
+  @Test
+  public void testDeleteTable() throws Exception {
+    final String tableName = "testDeleteTable";
+    Path testPath = new Path(new Path(fsUri), "/" + tableName);
+    final S3AFileSystem s3afs = getFileSystem();
+    final Configuration conf = s3afs.getConf();
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
+    try (DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore()) {
+      ddbms.initialize(s3afs);
+      // we can list the empty table
+      ddbms.listChildren(testPath);
+      DynamoDB dynamoDB = ddbms.getDynamoDB();
+      ddbms.destroy();
+      verifyTableNotExist(tableName, dynamoDB);
+
+      // delete table once more; be ResourceNotFoundException swallowed silently
+      ddbms.destroy();
+      verifyTableNotExist(tableName, dynamoDB);
+      try {
+        // we can no longer list the destroyed table
+        ddbms.listChildren(testPath);
+        fail("Should have failed after the table is destroyed!");
+      } catch (IOException ignored) {
+      }
+      ddbms.destroy();
+    }
+  }
+
+  /**
+   * This validates the table is created and ACTIVE in DynamoDB.
+   *
+   * This should not rely on the {@link DynamoDBMetadataStore} implementation.
+   * Return the table
+   */
+  private Table verifyTableInitialized(String tableName, DynamoDB dynamoDB) {
+    final Table table = dynamoDB.getTable(tableName);
+    final TableDescription td = table.describe();
+    assertEquals(tableName, td.getTableName());
+    assertEquals("ACTIVE", td.getTableStatus());
+    return table;
+  }
+
+  /**
+   * This validates the table is not found in DynamoDB.
+   *
+   * This should not rely on the {@link DynamoDBMetadataStore} implementation.
+   */
+  private void verifyTableNotExist(String tableName, DynamoDB dynamoDB) throws
+      Exception{
+    intercept(ResourceNotFoundException.class,
+        () -> dynamoDB.getTable(tableName).describe());
+  }
+
+}


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


[21/50] hadoop git commit: HDDS-201. Add name for LeaseManager. Contributed by Sandeep Nemuri.

Posted by eh...@apache.org.
HDDS-201. Add name for LeaseManager. Contributed by Sandeep Nemuri.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a1922959
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a1922959
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a1922959

Branch: refs/heads/HDFS-12090
Commit: a19229594e48fad9f50dbdb1f0b2fcbf7443ce66
Parents: 9089790
Author: Nanda kumar <na...@apache.org>
Authored: Thu Jul 26 19:00:23 2018 +0530
Committer: Nanda kumar <na...@apache.org>
Committed: Thu Jul 26 19:00:23 2018 +0530

----------------------------------------------------------------------
 .../apache/hadoop/ozone/lease/LeaseManager.java | 14 ++++++++-----
 .../hadoop/ozone/lease/TestLeaseManager.java    | 21 ++++++++++----------
 .../hdds/server/events/TestEventWatcher.java    |  2 +-
 .../hdds/scm/container/ContainerMapping.java    |  4 ++--
 .../hdds/scm/pipelines/PipelineSelector.java    |  4 ++--
 .../scm/server/StorageContainerManager.java     |  3 ++-
 .../replication/TestReplicationManager.java     |  4 ++--
 7 files changed, 28 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1922959/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.java
index b8390dd..756a41a 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.java
@@ -42,6 +42,7 @@ public class LeaseManager<T> {
   private static final Logger LOG =
       LoggerFactory.getLogger(LeaseManager.class);
 
+  private final String name;
   private final long defaultTimeout;
   private Map<T, Lease<T>> activeLeases;
   private LeaseMonitor leaseMonitor;
@@ -51,10 +52,13 @@ public class LeaseManager<T> {
   /**
    * Creates an instance of lease manager.
    *
+   * @param name
+   *        Name for the LeaseManager instance.
    * @param defaultTimeout
    *        Default timeout in milliseconds to be used for lease creation.
    */
-  public LeaseManager(long defaultTimeout) {
+  public LeaseManager(String name, long defaultTimeout) {
+    this.name = name;
     this.defaultTimeout = defaultTimeout;
   }
 
@@ -62,11 +66,11 @@ public class LeaseManager<T> {
    * Starts the lease manager service.
    */
   public void start() {
-    LOG.debug("Starting LeaseManager service");
+    LOG.debug("Starting {} LeaseManager service", name);
     activeLeases = new ConcurrentHashMap<>();
     leaseMonitor = new LeaseMonitor();
     leaseMonitorThread = new Thread(leaseMonitor);
-    leaseMonitorThread.setName("LeaseManager#LeaseMonitor");
+    leaseMonitorThread.setName(name + "-LeaseManager#LeaseMonitor");
     leaseMonitorThread.setDaemon(true);
     leaseMonitorThread.setUncaughtExceptionHandler((thread, throwable) -> {
       // Let us just restart this thread after logging an error.
@@ -75,7 +79,7 @@ public class LeaseManager<T> {
           thread.toString(), throwable);
       leaseMonitorThread.start();
     });
-    LOG.debug("Starting LeaseManager#LeaseMonitor Thread");
+    LOG.debug("Starting {}-LeaseManager#LeaseMonitor Thread", name);
     leaseMonitorThread.start();
     isRunning = true;
   }
@@ -203,7 +207,7 @@ public class LeaseManager<T> {
     @Override
     public void run() {
       while(monitor) {
-        LOG.debug("LeaseMonitor: checking for lease expiry");
+        LOG.debug("{}-LeaseMonitor: checking for lease expiry", name);
         long sleepTime = Long.MAX_VALUE;
 
         for (T resource : activeLeases.keySet()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1922959/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/lease/TestLeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/lease/TestLeaseManager.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/lease/TestLeaseManager.java
index 517c1a7..bdc70fc 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/lease/TestLeaseManager.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/lease/TestLeaseManager.java
@@ -67,7 +67,7 @@ public class TestLeaseManager {
   public void testLeaseAcquireAndRelease() throws LeaseException {
     //It is assumed that the test case execution won't take more than 5 seconds,
     //if it takes more time increase the defaultTimeout value of LeaseManager.
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     DummyResource resourceTwo = new DummyResource("two");
@@ -93,7 +93,7 @@ public class TestLeaseManager {
 
   @Test
   public void testLeaseAlreadyExist() throws LeaseException {
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     DummyResource resourceTwo = new DummyResource("two");
@@ -113,7 +113,7 @@ public class TestLeaseManager {
 
   @Test
   public void testLeaseNotFound() throws LeaseException, InterruptedException {
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     DummyResource resourceTwo = new DummyResource("two");
@@ -154,7 +154,7 @@ public class TestLeaseManager {
 
   @Test
   public void testCustomLeaseTimeout() throws LeaseException {
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     DummyResource resourceTwo = new DummyResource("two");
@@ -179,7 +179,7 @@ public class TestLeaseManager {
   @Test
   public void testLeaseCallback() throws LeaseException, InterruptedException {
     Map<DummyResource, String> leaseStatus = new HashMap<>();
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     Lease<DummyResource> leaseOne = manager.acquire(resourceOne);
@@ -209,7 +209,7 @@ public class TestLeaseManager {
       throws LeaseException, InterruptedException {
     // Callbacks should not be executed in case of lease release
     Map<DummyResource, String> leaseStatus = new HashMap<>();
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     Lease<DummyResource> leaseOne = manager.acquire(resourceOne);
@@ -231,7 +231,7 @@ public class TestLeaseManager {
   public void testLeaseCallbackWithMultipleLeases()
       throws LeaseException, InterruptedException {
     Map<DummyResource, String> leaseStatus = new HashMap<>();
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     DummyResource resourceTwo = new DummyResource("two");
@@ -302,7 +302,7 @@ public class TestLeaseManager {
 
   @Test
   public void testReuseReleasedLease() throws LeaseException {
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     Lease<DummyResource> leaseOne = manager.acquire(resourceOne);
@@ -324,13 +324,12 @@ public class TestLeaseManager {
   @Test
   public void testReuseTimedOutLease()
       throws LeaseException, InterruptedException {
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     Lease<DummyResource> leaseOne = manager.acquire(resourceOne);
     Assert.assertEquals(leaseOne, manager.get(resourceOne));
     Assert.assertFalse(leaseOne.hasExpired());
-
     // wait for lease to expire
     long sleepTime = leaseOne.getRemainingTime() + 1000;
     try {
@@ -352,7 +351,7 @@ public class TestLeaseManager {
 
   @Test
   public void testRenewLease() throws LeaseException, InterruptedException {
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     Lease<DummyResource> leaseOne = manager.acquire(resourceOne);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1922959/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/events/TestEventWatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/events/TestEventWatcher.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/events/TestEventWatcher.java
index 786b7b8..b72d2ae 100644
--- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/events/TestEventWatcher.java
+++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/events/TestEventWatcher.java
@@ -46,7 +46,7 @@ public class TestEventWatcher {
   @Before
   public void startLeaseManager() {
     DefaultMetricsSystem.instance();
-    leaseManager = new LeaseManager<>(2000l);
+    leaseManager = new LeaseManager<>("Test", 2000L);
     leaseManager.start();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1922959/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
index f07d22b..e17fe3d 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
@@ -139,8 +139,8 @@ public class ContainerMapping implements Mapping {
         ScmConfigKeys.OZONE_SCM_CONTAINER_CREATION_LEASE_TIMEOUT,
         ScmConfigKeys.OZONE_SCM_CONTAINER_CREATION_LEASE_TIMEOUT_DEFAULT,
         TimeUnit.MILLISECONDS);
-    LOG.trace("Starting Container Lease Manager.");
-    containerLeaseManager = new LeaseManager<>(containerCreationLeaseTimeout);
+    containerLeaseManager = new LeaseManager<>("ContainerCreation",
+        containerCreationLeaseTimeout);
     containerLeaseManager.start();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1922959/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
index 08710e7..b1e1dd0 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
@@ -99,8 +99,8 @@ public class PipelineSelector {
         ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_LEASE_TIMEOUT,
         ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_LEASE_TIMEOUT_DEFAULT,
         TimeUnit.MILLISECONDS);
-    LOG.trace("Starting Pipeline Lease Manager.");
-    pipelineLeaseManager = new LeaseManager<>(pipelineCreationLeaseTimeout);
+    pipelineLeaseManager = new LeaseManager<>("PipelineCreation",
+        pipelineCreationLeaseTimeout);
     pipelineLeaseManager.start();
 
     // These are the steady states of a container.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1922959/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
index f4cd448..165805f 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
@@ -223,7 +223,8 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
         conf.getTimeDuration(ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT,
             HDDS_SCM_WATCHER_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS);
 
-    commandWatcherLeaseManager = new LeaseManager<>(watcherTimeout);
+    commandWatcherLeaseManager = new LeaseManager<>("CommandWatcher",
+        watcherTimeout);
 
     //TODO: support configurable containerPlacement policy
     ContainerPlacementPolicy containerPlacementPolicy =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1922959/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java
index 99ec59f..9aa4b64 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java
@@ -112,7 +112,7 @@ public class TestReplicationManager {
 
     //GIVEN
 
-    LeaseManager<Long> leaseManager = new LeaseManager<>(100000L);
+    LeaseManager<Long> leaseManager = new LeaseManager<>("Test", 100000L);
     try {
       leaseManager.start();
 
@@ -152,7 +152,7 @@ public class TestReplicationManager {
   public void testCommandWatcher() throws InterruptedException, IOException {
 
     Logger.getRootLogger().setLevel(Level.DEBUG);
-    LeaseManager<Long> leaseManager = new LeaseManager<>(1000L);
+    LeaseManager<Long> leaseManager = new LeaseManager<>("Test", 1000L);
 
     try {
       leaseManager.start();


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


[39/50] hadoop git commit: HDDS-273. DeleteLog entries should be purged only after corresponding DNs commit the transaction. Contributed by Lokesh Jain.

Posted by eh...@apache.org.
HDDS-273. DeleteLog entries should be purged only after corresponding DNs commit the transaction. Contributed by Lokesh Jain.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/feb795b5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/feb795b5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/feb795b5

Branch: refs/heads/HDFS-12090
Commit: feb795b58d2a3c20bdbddea1638a83f6637d3fc9
Parents: 6b038f8
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Sun Jul 29 01:02:24 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Sun Jul 29 01:02:24 2018 +0530

----------------------------------------------------------------------
 .../DeleteBlocksCommandHandler.java             |  12 +-
 .../StorageContainerDatanodeProtocol.proto      |   4 +-
 .../hadoop/hdds/scm/block/BlockManagerImpl.java |   2 +-
 .../block/DatanodeDeletedBlockTransactions.java |  47 ++--
 .../hadoop/hdds/scm/block/DeletedBlockLog.java  |  23 +-
 .../hdds/scm/block/DeletedBlockLogImpl.java     | 123 ++++++----
 .../scm/server/SCMDatanodeProtocolServer.java   |  19 +-
 .../hdds/scm/block/TestDeletedBlockLog.java     | 232 ++++++++++---------
 8 files changed, 256 insertions(+), 206 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/feb795b5/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
index 9640f93..b0d4cbc 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
@@ -113,8 +113,8 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
         DeleteBlockTransactionResult.Builder txResultBuilder =
             DeleteBlockTransactionResult.newBuilder();
         txResultBuilder.setTxID(entry.getTxID());
+        long containerId = entry.getContainerID();
         try {
-          long containerId = entry.getContainerID();
           Container cont = containerSet.getContainer(containerId);
           if (cont == null) {
             throw new StorageContainerException("Unable to find the container "
@@ -126,7 +126,8 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
             KeyValueContainerData containerData = (KeyValueContainerData)
                 cont.getContainerData();
             deleteKeyValueContainerBlocks(containerData, entry);
-            txResultBuilder.setSuccess(true);
+            txResultBuilder.setContainerID(containerId)
+                .setSuccess(true);
             break;
           default:
             LOG.error(
@@ -136,9 +137,12 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
         } catch (IOException e) {
           LOG.warn("Failed to delete blocks for container={}, TXID={}",
               entry.getContainerID(), entry.getTxID(), e);
-          txResultBuilder.setSuccess(false);
+          txResultBuilder.setContainerID(containerId)
+              .setSuccess(false);
         }
-        resultBuilder.addResults(txResultBuilder.build());
+        resultBuilder.addResults(txResultBuilder.build())
+            .setDnId(context.getParent().getDatanodeDetails()
+                .getUuid().toString());
       });
       ContainerBlocksDeletionACKProto blockDeletionACK = resultBuilder.build();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/feb795b5/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto b/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
index d89567b..0c52efb 100644
--- a/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
+++ b/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
@@ -229,9 +229,11 @@ message DeletedBlocksTransaction {
 message ContainerBlocksDeletionACKProto {
   message DeleteBlockTransactionResult {
     required int64 txID = 1;
-    required bool success = 2;
+    required int64 containerID = 2;
+    required bool success = 3;
   }
   repeated DeleteBlockTransactionResult results = 1;
+  required string dnId = 2;
 }
 
 // SendACK response returned by datanode to SCM, currently empty.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/feb795b5/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
index 6825ca4..8e1c2cc 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
@@ -112,7 +112,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
     mxBean = MBeans.register("BlockManager", "BlockManagerImpl", this);
 
     // SCM block deleting transaction log and deleting service.
-    deletedBlockLog = new DeletedBlockLogImpl(conf);
+    deletedBlockLog = new DeletedBlockLogImpl(conf, containerManager);
     long svcInterval =
         conf.getTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL,
             OZONE_BLOCK_DELETING_SERVICE_INTERVAL_DEFAULT,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/feb795b5/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DatanodeDeletedBlockTransactions.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DatanodeDeletedBlockTransactions.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DatanodeDeletedBlockTransactions.java
index d71e7b0..e33a700 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DatanodeDeletedBlockTransactions.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DatanodeDeletedBlockTransactions.java
@@ -53,7 +53,8 @@ public class DatanodeDeletedBlockTransactions {
     this.nodeNum = nodeNum;
   }
 
-  public void addTransaction(DeletedBlocksTransaction tx) throws IOException {
+  public void addTransaction(DeletedBlocksTransaction tx,
+      Set<UUID> dnsWithTransactionCommitted) throws IOException {
     Pipeline pipeline = null;
     try {
       pipeline = mappingService.getContainerWithPipeline(tx.getContainerID())
@@ -71,29 +72,37 @@ public class DatanodeDeletedBlockTransactions {
 
     for (DatanodeDetails dd : pipeline.getMachines()) {
       UUID dnID = dd.getUuid();
-      if (transactions.containsKey(dnID)) {
-        List<DeletedBlocksTransaction> txs = transactions.get(dnID);
-        if (txs != null && txs.size() < maximumAllowedTXNum) {
-          boolean hasContained = false;
-          for (DeletedBlocksTransaction t : txs) {
-            if (t.getContainerID() == tx.getContainerID()) {
-              hasContained = true;
-              break;
-            }
-          }
+      if (dnsWithTransactionCommitted == null ||
+          !dnsWithTransactionCommitted.contains(dnID)) {
+        // Transaction need not be sent to dns which have already committed it
+        addTransactionToDN(dnID, tx);
+      }
+    }
+  }
 
-          if (!hasContained) {
-            txs.add(tx);
-            currentTXNum++;
+  private void addTransactionToDN(UUID dnID, DeletedBlocksTransaction tx) {
+    if (transactions.containsKey(dnID)) {
+      List<DeletedBlocksTransaction> txs = transactions.get(dnID);
+      if (txs != null && txs.size() < maximumAllowedTXNum) {
+        boolean hasContained = false;
+        for (DeletedBlocksTransaction t : txs) {
+          if (t.getContainerID() == tx.getContainerID()) {
+            hasContained = true;
+            break;
           }
         }
-      } else {
-        currentTXNum++;
-        transactions.put(dnID, tx);
+
+        if (!hasContained) {
+          txs.add(tx);
+          currentTXNum++;
+        }
       }
-      SCMBlockDeletingService.LOG.debug("Transaction added: {} <- TX({})", dnID,
-          tx.getTxID());
+    } else {
+      currentTXNum++;
+      transactions.put(dnID, tx);
     }
+    SCMBlockDeletingService.LOG
+        .debug("Transaction added: {} <- TX({})", dnID, tx.getTxID());
   }
 
   Set<UUID> getDatanodeIDs() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/feb795b5/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java
index 28103be..2bb5686 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java
@@ -18,12 +18,16 @@
 package org.apache.hadoop.hdds.scm.block;
 
 import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto
+    .DeleteBlockTransactionResult;
+import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
 
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
+import java.util.UUID;
 
 /**
  * The DeletedBlockLog is a persisted log in SCM to keep tracking
@@ -34,18 +38,6 @@ import java.util.Map;
 public interface DeletedBlockLog extends Closeable {
 
   /**
-   *  A limit size list of transactions. Note count is the max number
-   *  of TXs to return, we might not be able to always return this
-   *  number. and the processCount of those transactions
-   *  should be [0, MAX_RETRY).
-   *
-   * @param count - number of transactions.
-   * @return a list of BlockDeletionTransaction.
-   */
-  List<DeletedBlocksTransaction> getTransactions(int count)
-      throws IOException;
-
-  /**
    * Scan entire log once and returns TXs to DatanodeDeletedBlockTransactions.
    * Once DatanodeDeletedBlockTransactions is full, the scan behavior will
    * stop.
@@ -81,10 +73,11 @@ public interface DeletedBlockLog extends Closeable {
    * Commits a transaction means to delete all footprints of a transaction
    * from the log. This method doesn't guarantee all transactions can be
    * successfully deleted, it tolerate failures and tries best efforts to.
-   *
-   * @param txIDs - transaction IDs.
+   *  @param transactionResults - delete block transaction results.
+   * @param dnID - ID of datanode which acknowledges the delete block command.
    */
-  void commitTransactions(List<Long> txIDs) throws IOException;
+  void commitTransactions(List<DeleteBlockTransactionResult> transactionResults,
+      UUID dnID);
 
   /**
    * Creates a block deletion transaction and adds that into the log.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/feb795b5/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java
index 48fa2eb..752c9c7 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java
@@ -21,27 +21,36 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import com.google.common.primitives.Longs;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto
+    .DeleteBlockTransactionResult;
+import org.apache.hadoop.hdds.scm.container.Mapping;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.utils.BatchOperation;
-import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
 import org.apache.hadoop.utils.MetadataStore;
 import org.apache.hadoop.utils.MetadataStoreBuilder;
+import org.eclipse.jetty.util.ConcurrentHashSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
 
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
     .OZONE_SCM_BLOCK_DELETION_MAX_RETRY;
@@ -74,12 +83,15 @@ public class DeletedBlockLogImpl implements DeletedBlockLog {
 
   private final int maxRetry;
   private final MetadataStore deletedStore;
+  private final Mapping containerManager;
   private final Lock lock;
   // The latest id of deleted blocks in the db.
   private long lastTxID;
-  private long lastReadTxID;
+  // Maps txId to set of DNs which are successful in committing the transaction
+  private Map<Long, Set<UUID>> transactionToDNsCommitMap;
 
-  public DeletedBlockLogImpl(Configuration conf) throws IOException {
+  public DeletedBlockLogImpl(Configuration conf, Mapping containerManager)
+      throws IOException {
     maxRetry = conf.getInt(OZONE_SCM_BLOCK_DELETION_MAX_RETRY,
         OZONE_SCM_BLOCK_DELETION_MAX_RETRY_DEFAULT);
 
@@ -95,11 +107,17 @@ public class DeletedBlockLogImpl implements DeletedBlockLog {
         .setDbFile(deletedLogDbPath)
         .setCacheSize(cacheSize * OzoneConsts.MB)
         .build();
+    this.containerManager = containerManager;
 
     this.lock = new ReentrantLock();
     // start from the head of deleted store.
-    lastReadTxID = 0;
     lastTxID = findLatestTxIDInStore();
+
+    // transactionToDNsCommitMap is updated only when
+    // transaction is added to the log and when it is removed.
+
+    // maps transaction to dns which have committed it.
+    transactionToDNsCommitMap = new ConcurrentHashMap<>();
   }
 
   @VisibleForTesting
@@ -124,39 +142,6 @@ public class DeletedBlockLogImpl implements DeletedBlockLog {
   }
 
   @Override
-  public List<DeletedBlocksTransaction> getTransactions(
-      int count) throws IOException {
-    List<DeletedBlocksTransaction> result = new ArrayList<>();
-    MetadataKeyFilter getNextTxID = (preKey, currentKey, nextKey)
-        -> Longs.fromByteArray(currentKey) > lastReadTxID;
-    MetadataKeyFilter avoidInvalidTxid = (preKey, currentKey, nextKey)
-        -> !Arrays.equals(LATEST_TXID, currentKey);
-    lock.lock();
-    try {
-      deletedStore.iterate(null, (key, value) -> {
-        if (getNextTxID.filterKey(null, key, null) &&
-            avoidInvalidTxid.filterKey(null, key, null)) {
-          DeletedBlocksTransaction block = DeletedBlocksTransaction
-              .parseFrom(value);
-          if (block.getCount() > -1 && block.getCount() <= maxRetry) {
-            result.add(block);
-          }
-        }
-        return result.size() < count;
-      });
-      // Scan the metadata from the beginning.
-      if (result.size() < count || result.size() < 1) {
-        lastReadTxID = 0;
-      } else {
-        lastReadTxID = result.get(result.size() - 1).getTxID();
-      }
-    } finally {
-      lock.unlock();
-    }
-    return result;
-  }
-
-  @Override
   public List<DeletedBlocksTransaction> getFailedTransactions()
       throws IOException {
     lock.lock();
@@ -235,18 +220,50 @@ public class DeletedBlockLogImpl implements DeletedBlockLog {
   /**
    * {@inheritDoc}
    *
-   * @param txIDs - transaction IDs.
+   * @param transactionResults - transaction IDs.
+   * @param dnID - Id of Datanode which has acknowledged a delete block command.
    * @throws IOException
    */
   @Override
-  public void commitTransactions(List<Long> txIDs) throws IOException {
+  public void commitTransactions(
+      List<DeleteBlockTransactionResult> transactionResults, UUID dnID) {
     lock.lock();
     try {
-      for (Long txID : txIDs) {
+      Set<UUID> dnsWithCommittedTxn;
+      for (DeleteBlockTransactionResult transactionResult : transactionResults) {
+        if (isTransactionFailed(transactionResult)) {
+          continue;
+        }
         try {
-          deletedStore.delete(Longs.toByteArray(txID));
-        } catch (IOException ex) {
-          LOG.warn("Cannot commit txID " + txID, ex);
+          long txID = transactionResult.getTxID();
+          // set of dns which have successfully committed transaction txId.
+          dnsWithCommittedTxn = transactionToDNsCommitMap.get(txID);
+          Long containerId = transactionResult.getContainerID();
+          if (dnsWithCommittedTxn == null || containerId == null) {
+            LOG.warn("Transaction txId={} commit by dnId={} failed."
+                + " Corresponding entry not found.", txID, dnID);
+            return;
+          }
+
+          dnsWithCommittedTxn.add(dnID);
+          Collection<DatanodeDetails> containerDnsDetails =
+              containerManager.getContainerWithPipeline(containerId)
+                  .getPipeline().getDatanodes().values();
+          // The delete entry can be safely removed from the log if all the
+          // corresponding nodes commit the txn.
+          if (dnsWithCommittedTxn.size() >= containerDnsDetails.size()) {
+            List<UUID> containerDns = containerDnsDetails.stream()
+                .map(dnDetails -> dnDetails.getUuid())
+                .collect(Collectors.toList());
+            if (dnsWithCommittedTxn.containsAll(containerDns)) {
+              transactionToDNsCommitMap.remove(txID);
+              LOG.debug("Purging txId={} from block deletion log", txID);
+              deletedStore.delete(Longs.toByteArray(txID));
+            }
+          }
+        } catch (IOException e) {
+          LOG.warn("Could not commit delete block transaction: " +
+              transactionResult.getTxID(), e);
         }
       }
     } finally {
@@ -254,6 +271,20 @@ public class DeletedBlockLogImpl implements DeletedBlockLog {
     }
   }
 
+  private boolean isTransactionFailed(DeleteBlockTransactionResult result) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+          "Got block deletion ACK from datanode, TXIDs={}, " + "success={}",
+          result.getTxID(), result.getSuccess());
+    }
+    if (!result.getSuccess()) {
+      LOG.warn("Got failed ACK for TXID={}, prepare to resend the "
+          + "TX in next interval", result.getTxID());
+      return true;
+    }
+    return false;
+  }
+
   /**
    * {@inheritDoc}
    *
@@ -355,7 +386,9 @@ public class DeletedBlockLogImpl implements DeletedBlockLog {
               .parseFrom(value);
 
           if (block.getCount() > -1 && block.getCount() <= maxRetry) {
-            transactions.addTransaction(block);
+            Set<UUID> dnsWithTransactionCommitted = transactionToDNsCommitMap
+                .putIfAbsent(block.getTxID(), new ConcurrentHashSet<>());
+            transactions.addTransaction(block, dnsWithTransactionCommitted);
           }
           return !transactions.isFull();
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/feb795b5/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
index aee64b9..0d34787 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
@@ -91,9 +91,9 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.UUID;
 import java.util.stream.Collectors;
 
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY;
@@ -230,21 +230,8 @@ public class SCMDatanodeProtocolServer implements
       ContainerBlocksDeletionACKProto acks) throws IOException {
     if (acks.getResultsCount() > 0) {
       List<DeleteBlockTransactionResult> resultList = acks.getResultsList();
-      for (DeleteBlockTransactionResult result : resultList) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Got block deletion ACK from datanode, TXIDs={}, "
-              + "success={}", result.getTxID(), result.getSuccess());
-        }
-        if (result.getSuccess()) {
-          LOG.debug("Purging TXID={} from block deletion log",
-              result.getTxID());
-          scm.getScmBlockManager().getDeletedBlockLog()
-              .commitTransactions(Collections.singletonList(result.getTxID()));
-        } else {
-          LOG.warn("Got failed ACK for TXID={}, prepare to resend the "
-              + "TX in next interval", result.getTxID());
-        }
-      }
+      scm.getScmBlockManager().getDeletedBlockLog()
+          .commitTransactions(resultList, UUID.fromString(acks.getDnId()));
     }
     return ContainerBlocksDeletionACKResponseProto.newBuilder()
         .getDefaultInstanceForType();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/feb795b5/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java
index 9255ec7..e86717b 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java
@@ -32,6 +32,9 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto
+    .DeleteBlockTransactionResult;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.utils.MetadataKeyFilters;
 import org.apache.hadoop.utils.MetadataStore;
@@ -45,6 +48,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -56,7 +60,8 @@ import java.util.stream.Collectors;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
     .OZONE_SCM_BLOCK_DELETION_MAX_RETRY;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_DIRS;
-import static org.mockito.Mockito.mock;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.when;
 
 /**
  * Tests for DeletedBlockLog.
@@ -66,6 +71,8 @@ public class TestDeletedBlockLog {
   private static DeletedBlockLogImpl deletedBlockLog;
   private OzoneConfiguration conf;
   private File testDir;
+  private Mapping containerManager;
+  private List<DatanodeDetails> dnList;
 
   @Before
   public void setup() throws Exception {
@@ -74,7 +81,36 @@ public class TestDeletedBlockLog {
     conf = new OzoneConfiguration();
     conf.setInt(OZONE_SCM_BLOCK_DELETION_MAX_RETRY, 20);
     conf.set(OZONE_METADATA_DIRS, testDir.getAbsolutePath());
-    deletedBlockLog = new DeletedBlockLogImpl(conf);
+    containerManager = Mockito.mock(ContainerMapping.class);
+    deletedBlockLog = new DeletedBlockLogImpl(conf, containerManager);
+    dnList = new ArrayList<>(3);
+    setupContainerManager();
+  }
+
+  private void setupContainerManager() throws IOException {
+    dnList.add(
+        DatanodeDetails.newBuilder().setUuid(UUID.randomUUID().toString())
+            .build());
+    dnList.add(
+        DatanodeDetails.newBuilder().setUuid(UUID.randomUUID().toString())
+            .build());
+    dnList.add(
+        DatanodeDetails.newBuilder().setUuid(UUID.randomUUID().toString())
+            .build());
+
+    ContainerInfo containerInfo =
+        new ContainerInfo.Builder().setContainerID(1).build();
+    Pipeline pipeline =
+        new Pipeline(null, LifeCycleState.CLOSED, ReplicationType.RATIS,
+            ReplicationFactor.THREE, null);
+    pipeline.addMember(dnList.get(0));
+    pipeline.addMember(dnList.get(1));
+    pipeline.addMember(dnList.get(2));
+    ContainerWithPipeline containerWithPipeline =
+        new ContainerWithPipeline(containerInfo, pipeline);
+    when(containerManager.getContainerWithPipeline(anyLong()))
+        .thenReturn(containerWithPipeline);
+    when(containerManager.getContainer(anyLong())).thenReturn(containerInfo);
   }
 
   @After
@@ -101,45 +137,50 @@ public class TestDeletedBlockLog {
     return blockMap;
   }
 
-  @Test
-  public void testGetTransactions() throws Exception {
-    List<DeletedBlocksTransaction> blocks =
-        deletedBlockLog.getTransactions(30);
-    Assert.assertEquals(0, blocks.size());
-
-    // Creates 40 TX in the log.
-    for (Map.Entry<Long, List<Long>> entry : generateData(40).entrySet()){
-      deletedBlockLog.addTransaction(entry.getKey(), entry.getValue());
+  private void commitTransactions(
+      List<DeleteBlockTransactionResult> transactionResults,
+      DatanodeDetails... dns) {
+    for (DatanodeDetails dnDetails : dns) {
+      deletedBlockLog
+          .commitTransactions(transactionResults, dnDetails.getUuid());
     }
+  }
 
-    // Get first 30 TXs.
-    blocks = deletedBlockLog.getTransactions(30);
-    Assert.assertEquals(30, blocks.size());
-    for (int i = 0; i < 30; i++) {
-      Assert.assertEquals(i + 1, blocks.get(i).getTxID());
-    }
+  private void commitTransactions(
+      List<DeleteBlockTransactionResult> transactionResults) {
+    commitTransactions(transactionResults,
+        dnList.toArray(new DatanodeDetails[3]));
+  }
 
-    // Get another 30 TXs.
-    // The log only 10 left, so this time it will only return 10 TXs.
-    blocks = deletedBlockLog.getTransactions(30);
-    Assert.assertEquals(10, blocks.size());
-    for (int i = 30; i < 40; i++) {
-      Assert.assertEquals(i + 1, blocks.get(i - 30).getTxID());
-    }
+  private void commitTransactions(
+      Collection<DeletedBlocksTransaction> deletedBlocksTransactions,
+      DatanodeDetails... dns) {
+    commitTransactions(deletedBlocksTransactions.stream()
+        .map(this::createDeleteBlockTransactionResult)
+        .collect(Collectors.toList()), dns);
+  }
 
-    // Get another 50 TXs.
-    // By now the position should have moved to the beginning,
-    // this call will return all 40 TXs.
-    blocks = deletedBlockLog.getTransactions(50);
-    Assert.assertEquals(40, blocks.size());
-    for (int i = 0; i < 40; i++) {
-      Assert.assertEquals(i + 1, blocks.get(i).getTxID());
-    }
-    List<Long> txIDs = new ArrayList<>();
-    for (DeletedBlocksTransaction block : blocks) {
-      txIDs.add(block.getTxID());
-    }
-    deletedBlockLog.commitTransactions(txIDs);
+  private void commitTransactions(
+      Collection<DeletedBlocksTransaction> deletedBlocksTransactions) {
+    commitTransactions(deletedBlocksTransactions.stream()
+        .map(this::createDeleteBlockTransactionResult)
+        .collect(Collectors.toList()));
+  }
+
+  private DeleteBlockTransactionResult createDeleteBlockTransactionResult(
+      DeletedBlocksTransaction transaction) {
+    return DeleteBlockTransactionResult.newBuilder()
+        .setContainerID(transaction.getContainerID()).setSuccess(true)
+        .setTxID(transaction.getTxID()).build();
+  }
+
+  private List<DeletedBlocksTransaction> getTransactions(
+      int maximumAllowedTXNum) throws IOException {
+    DatanodeDeletedBlockTransactions transactions =
+        new DatanodeDeletedBlockTransactions(containerManager,
+            maximumAllowedTXNum, 3);
+    deletedBlockLog.getTransactions(transactions);
+    return transactions.getDatanodeTransactions(dnList.get(0).getUuid());
   }
 
   @Test
@@ -153,7 +194,7 @@ public class TestDeletedBlockLog {
 
     // This will return all TXs, total num 30.
     List<DeletedBlocksTransaction> blocks =
-        deletedBlockLog.getTransactions(40);
+        getTransactions(40);
     List<Long> txIDs = blocks.stream().map(DeletedBlocksTransaction::getTxID)
         .collect(Collectors.toList());
 
@@ -164,13 +205,13 @@ public class TestDeletedBlockLog {
     // Increment another time so it exceed the maxRetry.
     // On this call, count will be set to -1 which means TX eventually fails.
     deletedBlockLog.incrementCount(txIDs);
-    blocks = deletedBlockLog.getTransactions(40);
+    blocks = getTransactions(40);
     for (DeletedBlocksTransaction block : blocks) {
       Assert.assertEquals(-1, block.getCount());
     }
 
     // If all TXs are failed, getTransactions call will always return nothing.
-    blocks = deletedBlockLog.getTransactions(40);
+    blocks = getTransactions(40);
     Assert.assertEquals(blocks.size(), 0);
   }
 
@@ -180,16 +221,26 @@ public class TestDeletedBlockLog {
       deletedBlockLog.addTransaction(entry.getKey(), entry.getValue());
     }
     List<DeletedBlocksTransaction> blocks =
-        deletedBlockLog.getTransactions(20);
-    List<Long> txIDs = new ArrayList<>();
-    for (DeletedBlocksTransaction block : blocks) {
-      txIDs.add(block.getTxID());
-    }
-    // Add an invalid txID.
-    txIDs.add(70L);
-    deletedBlockLog.commitTransactions(txIDs);
-    blocks = deletedBlockLog.getTransactions(50);
+        getTransactions(20);
+    // Add an invalid txn.
+    blocks.add(
+        DeletedBlocksTransaction.newBuilder().setContainerID(1).setTxID(70)
+            .setCount(0).addLocalID(0).build());
+    commitTransactions(blocks);
+    blocks.remove(blocks.size() - 1);
+
+    blocks = getTransactions(50);
+    Assert.assertEquals(30, blocks.size());
+    commitTransactions(blocks, dnList.get(1), dnList.get(2),
+        DatanodeDetails.newBuilder().setUuid(UUID.randomUUID().toString())
+            .build());
+
+    blocks = getTransactions(50);
     Assert.assertEquals(30, blocks.size());
+    commitTransactions(blocks, dnList.get(0));
+
+    blocks = getTransactions(50);
+    Assert.assertEquals(0, blocks.size());
   }
 
   @Test
@@ -213,20 +264,16 @@ public class TestDeletedBlockLog {
         }
         added += 10;
       } else if (state == 1) {
-        blocks = deletedBlockLog.getTransactions(20);
+        blocks = getTransactions(20);
         txIDs = new ArrayList<>();
         for (DeletedBlocksTransaction block : blocks) {
           txIDs.add(block.getTxID());
         }
         deletedBlockLog.incrementCount(txIDs);
       } else if (state == 2) {
-        txIDs = new ArrayList<>();
-        for (DeletedBlocksTransaction block : blocks) {
-          txIDs.add(block.getTxID());
-        }
+        commitTransactions(blocks);
+        committed += blocks.size();
         blocks = new ArrayList<>();
-        committed += txIDs.size();
-        deletedBlockLog.commitTransactions(txIDs);
       } else {
         // verify the number of added and committed.
         List<Map.Entry<byte[], byte[]>> result =
@@ -234,6 +281,8 @@ public class TestDeletedBlockLog {
         Assert.assertEquals(added, result.size() + committed);
       }
     }
+    blocks = getTransactions(1000);
+    commitTransactions(blocks);
   }
 
   @Test
@@ -244,16 +293,13 @@ public class TestDeletedBlockLog {
     // close db and reopen it again to make sure
     // transactions are stored persistently.
     deletedBlockLog.close();
-    deletedBlockLog = new DeletedBlockLogImpl(conf);
+    deletedBlockLog = new DeletedBlockLogImpl(conf, containerManager);
     List<DeletedBlocksTransaction> blocks =
-        deletedBlockLog.getTransactions(10);
-    List<Long> txIDs = new ArrayList<>();
-    for (DeletedBlocksTransaction block : blocks) {
-      txIDs.add(block.getTxID());
-    }
-    deletedBlockLog.commitTransactions(txIDs);
-    blocks = deletedBlockLog.getTransactions(10);
-    Assert.assertEquals(10, blocks.size());
+        getTransactions(10);
+    commitTransactions(blocks);
+    blocks = getTransactions(100);
+    Assert.assertEquals(40, blocks.size());
+    commitTransactions(blocks);
   }
 
   @Test
@@ -262,32 +308,11 @@ public class TestDeletedBlockLog {
     int maximumAllowedTXNum = 5;
     List<DeletedBlocksTransaction> blocks = null;
     List<Long> containerIDs = new LinkedList<>();
+    DatanodeDetails dnId1 = dnList.get(0), dnId2 = dnList.get(1);
 
     int count = 0;
     long containerID = 0L;
-    DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
-        DatanodeDetails.Port.Name.STANDALONE, 0);
-    DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
-        DatanodeDetails.Port.Name.RATIS, 0);
-    DatanodeDetails.Port restPort = DatanodeDetails.newPort(
-        DatanodeDetails.Port.Name.REST, 0);
-    DatanodeDetails dnId1 = DatanodeDetails.newBuilder()
-        .setUuid(UUID.randomUUID().toString())
-        .setIpAddress("127.0.0.1")
-        .setHostName("localhost")
-        .addPort(containerPort)
-        .addPort(ratisPort)
-        .addPort(restPort)
-        .build();
-    DatanodeDetails dnId2 = DatanodeDetails.newBuilder()
-        .setUuid(UUID.randomUUID().toString())
-        .setIpAddress("127.0.0.1")
-        .setHostName("localhost")
-        .addPort(containerPort)
-        .addPort(ratisPort)
-        .addPort(restPort)
-        .build();
-    Mapping mappingService = mock(ContainerMapping.class);
+
     // Creates {TXNum} TX in the log.
     for (Map.Entry<Long, List<Long>> entry : generateData(txNum)
         .entrySet()) {
@@ -298,29 +323,25 @@ public class TestDeletedBlockLog {
 
       // make TX[1-6] for datanode1; TX[7-10] for datanode2
       if (count <= (maximumAllowedTXNum + 1)) {
-        mockContainerInfo(mappingService, containerID, dnId1);
+        mockContainerInfo(containerID, dnId1);
       } else {
-        mockContainerInfo(mappingService, containerID, dnId2);
+        mockContainerInfo(containerID, dnId2);
       }
     }
 
     DatanodeDeletedBlockTransactions transactions =
-        new DatanodeDeletedBlockTransactions(mappingService,
+        new DatanodeDeletedBlockTransactions(containerManager,
             maximumAllowedTXNum, 2);
     deletedBlockLog.getTransactions(transactions);
 
-    List<Long> txIDs = new LinkedList<>();
     for (UUID id : transactions.getDatanodeIDs()) {
       List<DeletedBlocksTransaction> txs = transactions
           .getDatanodeTransactions(id);
-      for (DeletedBlocksTransaction tx : txs) {
-        txIDs.add(tx.getTxID());
-      }
+      // delete TX ID
+      commitTransactions(txs);
     }
 
-    // delete TX ID
-    deletedBlockLog.commitTransactions(txIDs);
-    blocks = deletedBlockLog.getTransactions(txNum);
+    blocks = getTransactions(txNum);
     // There should be one block remained since dnID1 reaches
     // the maximum value (5).
     Assert.assertEquals(1, blocks.size());
@@ -337,7 +358,8 @@ public class TestDeletedBlockLog {
     builder.setTxID(11);
     builder.setContainerID(containerID);
     builder.setCount(0);
-    transactions.addTransaction(builder.build());
+    transactions.addTransaction(builder.build(),
+        null);
 
     // The number of TX in dnID2 should not be changed.
     Assert.assertEquals(size,
@@ -349,14 +371,14 @@ public class TestDeletedBlockLog {
     builder.setTxID(12);
     builder.setContainerID(containerID);
     builder.setCount(0);
-    mockContainerInfo(mappingService, containerID, dnId2);
-    transactions.addTransaction(builder.build());
+    mockContainerInfo(containerID, dnId2);
+    transactions.addTransaction(builder.build(),
+        null);
     // Since all node are full, then transactions is full.
     Assert.assertTrue(transactions.isFull());
   }
 
-  private void mockContainerInfo(Mapping mappingService, long containerID,
-      DatanodeDetails dd) throws IOException {
+  private void mockContainerInfo(long containerID, DatanodeDetails dd) throws IOException {
     Pipeline pipeline =
         new Pipeline("fake", LifeCycleState.OPEN,
             ReplicationType.STAND_ALONE, ReplicationFactor.ONE, "fake");
@@ -370,9 +392,9 @@ public class TestDeletedBlockLog {
     ContainerInfo containerInfo = builder.build();
     ContainerWithPipeline containerWithPipeline = new ContainerWithPipeline(
         containerInfo, pipeline);
-    Mockito.doReturn(containerInfo).when(mappingService)
+    Mockito.doReturn(containerInfo).when(containerManager)
         .getContainer(containerID);
-    Mockito.doReturn(containerWithPipeline).when(mappingService)
+    Mockito.doReturn(containerWithPipeline).when(containerManager)
         .getContainerWithPipeline(containerID);
   }
 }


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


[31/50] hadoop git commit: YARN-8571. Validate service principal format prior to launching yarn service. Contributed by Eric Yang

Posted by eh...@apache.org.
YARN-8571. Validate service principal format prior to launching yarn service. Contributed by Eric Yang


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b429f19d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b429f19d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b429f19d

Branch: refs/heads/HDFS-12090
Commit: b429f19d32d8f60a3535e047ef10cfb3edeb54c8
Parents: 1c40bc2
Author: Billie Rinaldi <bi...@apache.org>
Authored: Fri Jul 27 11:30:19 2018 -0700
Committer: Billie Rinaldi <bi...@apache.org>
Committed: Fri Jul 27 11:30:19 2018 -0700

----------------------------------------------------------------------
 .../exceptions/RestApiErrorMessages.java        |  4 ++++
 .../yarn/service/utils/ServiceApiUtil.java      | 10 ++++++++
 .../hadoop/yarn/service/TestServiceApiUtil.java | 25 ++++++++++++++++++++
 3 files changed, 39 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b429f19d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java
index f10d884..8f831ee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java
@@ -125,4 +125,8 @@ public interface RestApiErrorMessages {
 
   String ERROR_COMP_DOES_NOT_NEED_UPGRADE = "The component (%s) does not need" +
       " an upgrade.";
+  String ERROR_KERBEROS_PRINCIPAL_NAME_FORMAT = "Kerberos principal (%s) does " +
+      " not contain a hostname.";
+  String ERROR_KERBEROS_PRINCIPAL_MISSING = "Kerberos principal or keytab is" +
+      " missing.";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b429f19d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
index bebf52c..9219569 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
@@ -243,6 +243,16 @@ public class ServiceApiUtil {
 
   public static void validateKerberosPrincipal(
       KerberosPrincipal kerberosPrincipal) throws IOException {
+    try {
+      if (!kerberosPrincipal.getPrincipalName().contains("/")) {
+        throw new IllegalArgumentException(String.format(
+            RestApiErrorMessages.ERROR_KERBEROS_PRINCIPAL_NAME_FORMAT,
+            kerberosPrincipal.getPrincipalName()));
+      }
+    } catch (NullPointerException e) {
+      throw new IllegalArgumentException(
+          RestApiErrorMessages.ERROR_KERBEROS_PRINCIPAL_MISSING);
+    }
     if (!StringUtils.isEmpty(kerberosPrincipal.getKeytab())) {
       try {
         // validate URI format

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b429f19d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java
index 47b2803..c2a80e7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java
@@ -625,4 +625,29 @@ public class TestServiceApiUtil {
       Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
     }
   }
+
+  @Test
+  public void testKerberosPrincipalNameFormat() throws IOException {
+    Service app = createValidApplication("comp-a");
+    KerberosPrincipal kp = new KerberosPrincipal();
+    kp.setPrincipalName("user@domain.com");
+    app.setKerberosPrincipal(kp);
+
+    try {
+      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
+      Assert.fail(EXCEPTION_PREFIX + "service with invalid principal name format.");
+    } catch (IllegalArgumentException e) {
+      assertEquals(
+          String.format(RestApiErrorMessages.ERROR_KERBEROS_PRINCIPAL_NAME_FORMAT,
+              kp.getPrincipalName()),
+          e.getMessage());
+    }
+
+    kp.setPrincipalName("user/_HOST@domain.com");
+    try {
+      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+  }
 }


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


[38/50] hadoop git commit: HDDS-246. Datanode should throw BlockNotCommittedException for uncommitted blocks to Ozone Client. Contributed by Shashikant Banerjee.

Posted by eh...@apache.org.
HDDS-246. Datanode should throw BlockNotCommittedException for uncommitted blocks to Ozone Client. Contributed by Shashikant Banerjee.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6b038f82
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6b038f82
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6b038f82

Branch: refs/heads/HDFS-12090
Commit: 6b038f82da8fa8c1c4f1e1bf448eacc6dd523044
Parents: 3d58684
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Sat Jul 28 22:04:11 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Sat Jul 28 22:04:11 2018 +0530

----------------------------------------------------------------------
 .../main/proto/DatanodeContainerProtocol.proto  |  1 +
 .../common/impl/OpenContainerBlockMap.java      | 12 ++++++
 .../container/keyvalue/KeyValueHandler.java     | 12 ++++--
 .../ozone/scm/TestCommittedBlockLengthAPI.java  | 45 +++++++++++++++-----
 4 files changed, 57 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b038f82/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
index a3c4467..6969fa6 100644
--- a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
+++ b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
@@ -138,6 +138,7 @@ enum Result {
   CONTAINER_FILES_CREATE_ERROR = 32;
   CONTAINER_CHECKSUM_ERROR = 33;
   UNKNOWN_CONTAINER_TYPE = 34;
+  BLOCK_NOT_COMMITTED = 35;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b038f82/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
index 6a93c9d..8e2667d 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
@@ -129,6 +129,18 @@ public class OpenContainerBlockMap {
         -> blocks.removeAndGetSize(blockID.getLocalID()) == 0? null: blocks);
   }
 
+  /**
+   * Returns true if the block exists in the map, false otherwise
+   *
+   * @param blockID
+   * @return True, if it exists, false otherwise
+   */
+  public boolean checkIfBlockExists(BlockID blockID) {
+    KeyDataMap keyDataMap = containers.get(blockID.getContainerID());
+    return keyDataMap == null ? false :
+        keyDataMap.get(blockID.getLocalID()) != null;
+  }
+
   @VisibleForTesting
   KeyDataMap getKeyDataMap(long containerId) {
     return containers.get(containerId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b038f82/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index b08e128..0b26a14 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -91,6 +91,8 @@ import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.GET_SMALL_FILE_ERROR;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.PUT_SMALL_FILE_ERROR;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.BLOCK_NOT_COMMITTED;
 
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Stage;
@@ -494,10 +496,14 @@ public class KeyValueHandler extends Handler {
 
     long blockLength;
     try {
-      BlockID blockID = BlockID.getFromProtobuf(
-          request.getGetCommittedBlockLength().getBlockID());
+      BlockID blockID = BlockID
+          .getFromProtobuf(request.getGetCommittedBlockLength().getBlockID());
+      // Check if it really exists in the openContainerBlockMap
+      if (openContainerBlockMap.checkIfBlockExists(blockID)) {
+        String msg = "Block " + blockID + " is not committed yet.";
+        throw new StorageContainerException(msg, BLOCK_NOT_COMMITTED);
+      }
       blockLength = keyManager.getCommittedBlockLength(kvContainer, blockID);
-
     } catch (StorageContainerException ex) {
       return ContainerUtils.logAndReturnError(LOG, ex, request);
     } catch (IOException ex) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b038f82/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java
index 7e8aa5f..3c6479f 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java
@@ -104,16 +104,6 @@ public class TestCommittedBlockLengthAPI {
             .getWriteChunkRequest(container.getPipeline(), blockID,
                 data.length);
     client.sendCommand(writeChunkRequest);
-    try {
-      // since there is neither explicit putKey request made for the block,
-      // nor the container is closed, GetCommittedBlockLength request
-      // should fail here.
-      response = ContainerProtocolCalls
-          .getCommittedBlockLength(client, blockID, traceID);
-      Assert.fail("Expected exception not thrown");
-    } catch (StorageContainerException sce) {
-      Assert.assertTrue(sce.getMessage().contains("Unable to find the key"));
-    }
     // Now, explicitly make a putKey request for the block.
     ContainerProtos.ContainerCommandRequestProto putKeyRequest =
         ContainerTestHelper
@@ -188,4 +178,39 @@ public class TestCommittedBlockLengthAPI {
     }
     xceiverClientManager.releaseClient(client);
   }
+
+  @Test
+  public void testGetCommittedBlockLengthForOpenBlock() throws Exception {
+    String traceID = UUID.randomUUID().toString();
+    ContainerWithPipeline container = storageContainerLocationClient
+        .allocateContainer(xceiverClientManager.getType(),
+            HddsProtos.ReplicationFactor.ONE, containerOwner);
+    long containerID = container.getContainerInfo().getContainerID();
+    XceiverClientSpi client = xceiverClientManager
+        .acquireClient(container.getPipeline(), containerID);
+    ContainerProtocolCalls
+        .createContainer(client, containerID, traceID);
+
+    BlockID blockID =
+        ContainerTestHelper.getTestBlockID(containerID);
+    ContainerProtos.ContainerCommandRequestProto requestProto =
+        ContainerTestHelper
+            .getWriteChunkRequest(container.getPipeline(), blockID, 1024);
+    client.sendCommand(requestProto);
+    try {
+      ContainerProtocolCalls.getCommittedBlockLength(client, blockID, traceID);
+      Assert.fail("Expected Exception not thrown");
+    } catch (StorageContainerException sce) {
+      Assert.assertEquals(ContainerProtos.Result.BLOCK_NOT_COMMITTED,
+          sce.getResult());
+    }
+    // now close the container, it should auto commit pending open blocks
+    ContainerProtocolCalls
+        .closeContainer(client, containerID, traceID);
+    ContainerProtos.GetCommittedBlockLengthResponseProto response =
+        ContainerProtocolCalls
+            .getCommittedBlockLength(client, blockID, traceID);
+    Assert.assertTrue(response.getBlockLength() == 1024);
+    xceiverClientManager.releaseClient(client);
+  }
 }
\ No newline at end of file


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


[07/50] hadoop git commit: HDFS-13448. HDFS Block Placement - Ignore Locality for First Block Replica (Contributed by BELUGA BEHR via Daniel Templeton)

Posted by eh...@apache.org.
HDFS-13448. HDFS Block Placement - Ignore Locality for First Block Replica
(Contributed by BELUGA BEHR via Daniel Templeton)

Change-Id: I965d1cfa642ad24296038b83e3d5c9983545267d


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/849c45db
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/849c45db
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/849c45db

Branch: refs/heads/HDFS-12090
Commit: 849c45db187224095b13fe297a4d7377fbb9d2cd
Parents: 6bec03c
Author: Daniel Templeton <te...@apache.org>
Authored: Tue Jul 24 15:34:19 2018 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Tue Jul 24 16:05:27 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/CreateFlag.java   |  9 ++-
 .../org/apache/hadoop/hdfs/AddBlockFlag.java    | 11 ++-
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  3 +
 .../hadoop/hdfs/DistributedFileSystem.java      | 11 +++
 .../src/main/proto/ClientNamenodeProtocol.proto |  1 +
 .../BlockPlacementPolicyDefault.java            |  4 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  | 30 +++++---
 .../server/namenode/TestFSDirWriteFileOp.java   | 79 ++++++++++++++++++++
 8 files changed, 134 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java
index 383d65a..c3e088b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java
@@ -116,7 +116,14 @@ public enum CreateFlag {
    * Enforce the file to be a replicated file, no matter what its parent
    * directory's replication or erasure coding policy is.
    */
-  SHOULD_REPLICATE((short) 0x80);
+  SHOULD_REPLICATE((short) 0x80),
+
+  /**
+   * Advise that the first block replica NOT take into account DataNode
+   * locality. The first block replica should be placed randomly within the
+   * cluster. Subsequent block replicas should follow DataNode locality rules.
+   */
+  IGNORE_CLIENT_LOCALITY((short) 0x100);
 
   private final short mode;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AddBlockFlag.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AddBlockFlag.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AddBlockFlag.java
index 6a0805b..b0686d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AddBlockFlag.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AddBlockFlag.java
@@ -36,7 +36,16 @@ public enum AddBlockFlag {
    *
    * @see CreateFlag#NO_LOCAL_WRITE
    */
-  NO_LOCAL_WRITE((short) 0x01);
+  NO_LOCAL_WRITE((short) 0x01),
+
+  /**
+   * Advise that the first block replica NOT take into account DataNode
+   * locality. The first block replica should be placed randomly within the
+   * cluster. Subsequent block replicas should follow DataNode locality rules.
+   *
+   * @see CreateFlag#IGNORE_CLIENT_LOCALITY
+   */
+  IGNORE_CLIENT_LOCALITY((short) 0x02);
 
   private final short mode;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 9734752..e977054 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -201,6 +201,9 @@ public class DFSOutputStream extends FSOutputSummer
     if (flag.contains(CreateFlag.NO_LOCAL_WRITE)) {
       this.addBlockFlags.add(AddBlockFlag.NO_LOCAL_WRITE);
     }
+    if (flag.contains(CreateFlag.IGNORE_CLIENT_LOCALITY)) {
+      this.addBlockFlags.add(AddBlockFlag.IGNORE_CLIENT_LOCALITY);
+    }
     if (progress != null) {
       DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
           +"{}", src);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 82cdd8c..3519c60 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -3205,6 +3205,17 @@ public class DistributedFileSystem extends FileSystem
       return this;
     }
 
+    /**
+     * Advise that the first block replica be written without regard to the
+     * client locality.
+     *
+     * @see CreateFlag for the details.
+     */
+    public HdfsDataOutputStreamBuilder ignoreClientLocality() {
+      getFlags().add(CreateFlag.IGNORE_CLIENT_LOCALITY);
+      return this;
+    }
+
     @VisibleForTesting
     @Override
     protected EnumSet<CreateFlag> getFlags() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index 0f5ce94..e51aeda 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -167,6 +167,7 @@ message AbandonBlockResponseProto { // void response
 
 enum AddBlockFlagProto {
   NO_LOCAL_WRITE = 1; // avoid writing to local node.
+  IGNORE_CLIENT_LOCALITY = 2; // write to a random node
 }
 
 message AddBlockRequestProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
index c94232f..6985f55 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
@@ -280,7 +280,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     if (avoidLocalNode) {
       results = new ArrayList<>(chosenStorage);
       Set<Node> excludedNodeCopy = new HashSet<>(excludedNodes);
-      excludedNodeCopy.add(writer);
+      if (writer != null) {
+        excludedNodeCopy.add(writer);
+      }
       localNode = chooseTarget(numOfReplicas, writer,
           excludedNodeCopy, blocksize, maxNodesPerRack, results,
           avoidStaleNodes, storagePolicy,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 03c349c..2875708 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -269,19 +269,27 @@ class FSDirWriteFileOp {
       BlockManager bm, String src, DatanodeInfo[] excludedNodes,
       String[] favoredNodes, EnumSet<AddBlockFlag> flags,
       ValidateAddBlockResult r) throws IOException {
-    Node clientNode = bm.getDatanodeManager()
-        .getDatanodeByHost(r.clientMachine);
-    if (clientNode == null) {
-      clientNode = getClientNode(bm, r.clientMachine);
-    }
+    Node clientNode = null;
 
-    Set<Node> excludedNodesSet = null;
-    if (excludedNodes != null) {
-      excludedNodesSet = new HashSet<>(excludedNodes.length);
-      Collections.addAll(excludedNodesSet, excludedNodes);
+    boolean ignoreClientLocality = (flags != null
+            && flags.contains(AddBlockFlag.IGNORE_CLIENT_LOCALITY));
+
+    // If client locality is ignored, clientNode remains 'null' to indicate
+    if (!ignoreClientLocality) {
+      clientNode = bm.getDatanodeManager().getDatanodeByHost(r.clientMachine);
+      if (clientNode == null) {
+        clientNode = getClientNode(bm, r.clientMachine);
+      }
     }
-    List<String> favoredNodesList = (favoredNodes == null) ? null
-        : Arrays.asList(favoredNodes);
+
+    Set<Node> excludedNodesSet =
+        (excludedNodes == null) ? new HashSet<>()
+            : new HashSet<>(Arrays.asList(excludedNodes));
+
+    List<String> favoredNodesList =
+        (favoredNodes == null) ? Collections.emptyList()
+            : Arrays.asList(favoredNodes);
+
     // choose targets for the new block to be allocated.
     return bm.chooseTarget4NewBlock(src, r.numTargets, clientNode,
                                     excludedNodesSet, r.blockSize,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirWriteFileOp.java
new file mode 100644
index 0000000..762fa61
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirWriteFileOp.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.junit.Assert.assertNull;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyByte;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyList;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anySet;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.EnumSet;
+
+import org.apache.hadoop.hdfs.AddBlockFlag;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.namenode.FSDirWriteFileOp.ValidateAddBlockResult;
+import org.apache.hadoop.net.Node;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+public class TestFSDirWriteFileOp {
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testIgnoreClientLocality() throws IOException {
+    ValidateAddBlockResult addBlockResult =
+        new ValidateAddBlockResult(1024L, 3, (byte) 0x01, null, null, null);
+
+    EnumSet<AddBlockFlag> addBlockFlags =
+        EnumSet.of(AddBlockFlag.IGNORE_CLIENT_LOCALITY);
+
+    BlockManager bmMock = mock(BlockManager.class);
+
+    ArgumentCaptor<Node> nodeCaptor = ArgumentCaptor.forClass(Node.class);
+
+    when(bmMock.chooseTarget4NewBlock(anyString(), anyInt(), any(), anySet(),
+        anyLong(), anyList(), anyByte(), any(), any(), any())).thenReturn(null);
+
+    FSDirWriteFileOp.chooseTargetForNewBlock(bmMock, "localhost", null, null,
+        addBlockFlags, addBlockResult);
+
+    // There should be no other interactions with the block manager when the
+    // IGNORE_CLIENT_LOCALITY is passed in because there is no need to discover
+    // the local node requesting the new block
+    verify(bmMock, times(1)).chooseTarget4NewBlock(anyString(), anyInt(),
+        nodeCaptor.capture(), anySet(), anyLong(), anyList(), anyByte(), any(),
+        any(), any());
+
+    verifyNoMoreInteractions(bmMock);
+
+    assertNull(
+        "Source node was assigned a value. Expected 'null' value because "
+            + "chooseTarget was flagged to ignore source node locality",
+        nodeCaptor.getValue());
+  }
+}


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


[44/50] hadoop git commit: HDDS-302. Fix javadoc and add implementation details in ContainerStateMachine. Contributed by Shashikant Banerjee.

Posted by eh...@apache.org.
HDDS-302. Fix javadoc and add implementation details in ContainerStateMachine. Contributed by Shashikant Banerjee.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/952dc2fd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/952dc2fd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/952dc2fd

Branch: refs/heads/HDFS-12090
Commit: 952dc2fd557f9aaf0f144ee32d0b7731a84bad73
Parents: 3108d27
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Mon Jul 30 18:45:58 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Mon Jul 30 18:45:58 2018 +0530

----------------------------------------------------------------------
 .../hadoop/hdds/scm/XceiverClientRatis.java     | 30 ++----------------
 .../java/org/apache/hadoop/hdds/HddsUtils.java  | 33 ++++++++++++++++++++
 .../server/ratis/ContainerStateMachine.java     | 14 ++++++++-
 3 files changed, 49 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/952dc2fd/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java
index 0effa8f..2541415 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdds.scm;
 
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.HddsUtils;
 import org.apache.ratis.shaded.com.google.protobuf
     .InvalidProtocolBufferException;
 import org.apache.hadoop.conf.Configuration;
@@ -183,34 +184,9 @@ public final class XceiverClientRatis extends XceiverClientSpi {
     return Objects.requireNonNull(client.get(), "client is null");
   }
 
-  private boolean isReadOnly(ContainerCommandRequestProto proto) {
-    switch (proto.getCmdType()) {
-    case ReadContainer:
-    case ReadChunk:
-    case ListKey:
-    case GetKey:
-    case GetSmallFile:
-    case ListContainer:
-    case ListChunk:
-      return true;
-    case CloseContainer:
-    case WriteChunk:
-    case UpdateContainer:
-    case CompactChunk:
-    case CreateContainer:
-    case DeleteChunk:
-    case DeleteContainer:
-    case DeleteKey:
-    case PutKey:
-    case PutSmallFile:
-    default:
-      return false;
-    }
-  }
-
   private RaftClientReply sendRequest(ContainerCommandRequestProto request)
       throws IOException {
-    boolean isReadOnlyRequest = isReadOnly(request);
+    boolean isReadOnlyRequest = HddsUtils.isReadOnly(request);
     ByteString byteString = request.toByteString();
     LOG.debug("sendCommand {} {}", isReadOnlyRequest, request);
     final RaftClientReply reply =  isReadOnlyRequest ?
@@ -222,7 +198,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
 
   private CompletableFuture<RaftClientReply> sendRequestAsync(
       ContainerCommandRequestProto request) throws IOException {
-    boolean isReadOnlyRequest = isReadOnly(request);
+    boolean isReadOnlyRequest = HddsUtils.isReadOnly(request);
     ByteString byteString = request.toByteString();
     LOG.debug("sendCommandAsync {} {}", isReadOnlyRequest, request);
     return isReadOnlyRequest ? getClient().sendReadOnlyAsync(() -> byteString) :

http://git-wip-us.apache.org/repos/asf/hadoop/blob/952dc2fd/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
index 48c6dce..33bf90c 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
@@ -24,6 +24,7 @@ import com.google.common.net.HostAndPort;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetUtils;
@@ -315,4 +316,36 @@ public final class HddsUtils {
     return name;
   }
 
+  /**
+   * Checks if the container command is read only or not.
+   * @param proto ContainerCommand Request proto
+   * @return True if its readOnly , false otherwise.
+   */
+  public static boolean isReadOnly(
+      ContainerProtos.ContainerCommandRequestProto proto) {
+    switch (proto.getCmdType()) {
+    case ReadContainer:
+    case ReadChunk:
+    case ListKey:
+    case GetKey:
+    case GetSmallFile:
+    case ListContainer:
+    case ListChunk:
+    case GetCommittedBlockLength:
+      return true;
+    case CloseContainer:
+    case WriteChunk:
+    case UpdateContainer:
+    case CompactChunk:
+    case CreateContainer:
+    case DeleteChunk:
+    case DeleteContainer:
+    case DeleteKey:
+    case PutKey:
+    case PutSmallFile:
+    default:
+      return false;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/952dc2fd/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
index ac7aa57..c0dd0ba 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
@@ -57,7 +57,7 @@ import java.util.concurrent.ThreadPoolExecutor;
  * requests.
  *
  * Read only requests are classified in
- * {@link org.apache.hadoop.hdds.scm.XceiverClientRatis#isReadOnly}
+ * {@link org.apache.hadoop.hdds.HddsUtils#isReadOnly}
  * and these readonly requests are replied from the {@link #query(Message)}.
  *
  * The write requests can be divided into requests with user data
@@ -84,6 +84,11 @@ import java.util.concurrent.ThreadPoolExecutor;
  * 2) Write chunk commit operation is executed after write chunk state machine
  * operation. This will ensure that commit operation is sync'd with the state
  * machine operation.
+ *
+ * Synchronization between {@link #writeStateMachineData} and
+ * {@link #applyTransaction} need to be enforced in the StateMachine
+ * implementation. For example, synchronization between writeChunk and
+ * createContainer in {@link ContainerStateMachine}.
  * */
 public class ContainerStateMachine extends BaseStateMachine {
   static final Logger LOG = LoggerFactory.getLogger(
@@ -213,6 +218,10 @@ public class ContainerStateMachine extends BaseStateMachine {
     return CompletableFuture.completedFuture(() -> ByteString.EMPTY);
   }
 
+  /*
+   * writeStateMachineData calls are not synchronized with each other
+   * and also with applyTransaction.
+   */
   @Override
   public CompletableFuture<Message> writeStateMachineData(LogEntryProto entry) {
     try {
@@ -244,6 +253,9 @@ public class ContainerStateMachine extends BaseStateMachine {
     }
   }
 
+  /*
+   * ApplyTransaction calls in Ratis are sequential.
+   */
   @Override
   public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
     try {


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


[14/50] hadoop git commit: YARN-8577. Fix the broken anchor in SLS site-doc. Contributed by Weiwei Yang.

Posted by eh...@apache.org.
YARN-8577. Fix the broken anchor in SLS site-doc. Contributed by Weiwei Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3d3158ce
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3d3158ce
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3d3158ce

Branch: refs/heads/HDFS-12090
Commit: 3d3158cea4580eb2e3b2af635c3a7d30f4dbb873
Parents: b507f83e
Author: bibinchundatt <bi...@apache.org>
Authored: Wed Jul 25 16:19:14 2018 +0530
Committer: bibinchundatt <bi...@apache.org>
Committed: Wed Jul 25 18:59:05 2018 +0530

----------------------------------------------------------------------
 .../hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md         | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d3158ce/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
index 9df4998..e487267 100644
--- a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
+++ b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
@@ -27,7 +27,7 @@ YARN Scheduler Load Simulator (SLS)
     * [Metrics](#Metrics)
         * [Real-time Tracking](#Real-time_Tracking)
         * [Offline Analysis](#Offline_Analysis)
-    * [Synthetic Load Generator](#SynthGen)
+    * [Synthetic Load Generator](#Synthetic_Load_Generator)
     * [Appendix](#Appendix)
         * [Resources](#Resources)
         * [SLS JSON input file format](#SLS_JSON_input_file_format)


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


[03/50] hadoop git commit: HDDS-282. Consolidate logging in scm/container-service. Contributed by Elek Marton.

Posted by eh...@apache.org.
HDDS-282. Consolidate logging in scm/container-service. Contributed by Elek Marton.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/cd0b9f13
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cd0b9f13
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cd0b9f13

Branch: refs/heads/HDFS-12090
Commit: cd0b9f13805affcc91a2cba42b176bb9031378eb
Parents: 35ce6eb
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Jul 24 10:16:53 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Tue Jul 24 10:17:03 2018 -0700

----------------------------------------------------------------------
 .../container/common/statemachine/EndpointStateMachine.java  | 4 ++--
 .../common/states/endpoint/RegisterEndpointTask.java         | 3 +--
 .../statemachine/background/BlockDeletingService.java        | 8 +++++---
 .../org/apache/hadoop/hdds/server/events/EventQueue.java     | 3 +--
 .../org/apache/hadoop/hdds/server/events/TypedEvent.java     | 6 ++++--
 5 files changed, 13 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd0b9f13/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
index 7e85923..fb32a05 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
@@ -203,11 +203,11 @@ public class EndpointStateMachine
     this.incMissed();
     if (this.getMissedCount() % getLogWarnInterval(conf) ==
         0) {
-      LOG.warn("Unable to communicate to SCM server at {}. We have not been " +
+      LOG.error("Unable to communicate to SCM server at {}. We have not been " +
               "able to communicate to this SCM server for past {} seconds.",
           this.getAddress().getHostString() + ":" + this.getAddress().getPort(),
           this.getMissedCount() * getScmHeartbeatInterval(
-              this.conf));
+              this.conf), ex);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd0b9f13/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java
index b3d2b62..25af4a1 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java
@@ -125,8 +125,7 @@ public final class RegisterEndpointTask implements
       rpcEndPoint.setState(nextState);
       rpcEndPoint.zeroMissedCount();
     } catch (IOException ex) {
-      rpcEndPoint.logIfNeeded(ex
-      );
+      rpcEndPoint.logIfNeeded(ex);
     } finally {
       rpcEndPoint.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd0b9f13/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
index 4a572ca..51eed7f 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
@@ -119,9 +119,11 @@ public class BlockDeletingService extends BackgroundService{
       // configured.
       containers = containerSet.chooseContainerForBlockDeletion(
           containerLimitPerInterval, containerDeletionPolicy);
-      LOG.info("Plan to choose {} containers for block deletion, "
-          + "actually returns {} valid containers.",
-          containerLimitPerInterval, containers.size());
+      if (containers.size() > 0) {
+        LOG.info("Plan to choose {} containers for block deletion, "
+                + "actually returns {} valid containers.",
+            containerLimitPerInterval, containers.size());
+      }
 
       for(ContainerData container : containers) {
         BlockDeletingTask containerTask =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd0b9f13/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java
index 7e29223..f93c54b 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java
@@ -155,8 +155,7 @@ public class EventQueue implements EventPublisher, AutoCloseable {
       }
 
     } else {
-      throw new IllegalArgumentException(
-          "No event handler registered for event " + event);
+      LOG.warn("No event handler registered for event " + event);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd0b9f13/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/TypedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/TypedEvent.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/TypedEvent.java
index 62e2419..27bba3a 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/TypedEvent.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/TypedEvent.java
@@ -50,7 +50,9 @@ public class TypedEvent<T> implements Event<T> {
 
   @Override
   public String toString() {
-    return "TypedEvent{" + "payloadType=" + payloadType + ", name='" + name
-        + '\'' + '}';
+    return "TypedEvent{" +
+        "payloadType=" + payloadType.getSimpleName() +
+        ", name='" + name + '\'' +
+        '}';
   }
 }


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


[37/50] hadoop git commit: YARN-8558. NM recovery level db not cleaned up properly on container finish. Contributed by Bibin A Chundatt.

Posted by eh...@apache.org.
YARN-8558. NM recovery level db not cleaned up properly on container finish. Contributed by Bibin A Chundatt.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3d586841
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3d586841
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3d586841

Branch: refs/heads/HDFS-12090
Commit: 3d586841aba99c7df98b2b4d3e48ec0144bad086
Parents: 59adeb8
Author: bibinchundatt <bi...@apache.org>
Authored: Sat Jul 28 20:52:39 2018 +0530
Committer: bibinchundatt <bi...@apache.org>
Committed: Sat Jul 28 20:52:39 2018 +0530

----------------------------------------------------------------------
 .../recovery/NMLeveldbStateStoreService.java          | 14 ++++++++++----
 .../recovery/TestNMLeveldbStateStoreService.java      |  7 +++++++
 2 files changed, 17 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d586841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
index 44f5e18..67f642d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
@@ -143,9 +143,9 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
       NM_TOKENS_KEY_PREFIX + PREV_MASTER_KEY_SUFFIX;
   private static final String CONTAINER_TOKENS_KEY_PREFIX =
       "ContainerTokens/";
-  private static final String CONTAINER_TOKENS_CURRENT_MASTER_KEY =
+  private static final String CONTAINER_TOKEN_SECRETMANAGER_CURRENT_MASTER_KEY =
       CONTAINER_TOKENS_KEY_PREFIX + CURRENT_MASTER_KEY_SUFFIX;
-  private static final String CONTAINER_TOKENS_PREV_MASTER_KEY =
+  private static final String CONTAINER_TOKEN_SECRETMANAGER_PREV_MASTER_KEY =
       CONTAINER_TOKENS_KEY_PREFIX + PREV_MASTER_KEY_SUFFIX;
 
   private static final String LOG_DELETER_KEY_PREFIX = "LogDeleters/";
@@ -658,6 +658,12 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         batch.delete(bytes(keyPrefix + CONTAINER_KILLED_KEY_SUFFIX));
         batch.delete(bytes(keyPrefix + CONTAINER_EXIT_CODE_KEY_SUFFIX));
         batch.delete(bytes(keyPrefix + CONTAINER_UPDATE_TOKEN_SUFFIX));
+        batch.delete(bytes(keyPrefix + CONTAINER_START_TIME_KEY_SUFFIX));
+        batch.delete(bytes(keyPrefix + CONTAINER_LOG_DIR_KEY_SUFFIX));
+        batch.delete(bytes(keyPrefix + CONTAINER_VERSION_KEY_SUFFIX));
+        batch.delete(bytes(keyPrefix + CONTAINER_REMAIN_RETRIES_KEY_SUFFIX));
+        batch.delete(bytes(keyPrefix + CONTAINER_RESTART_TIMES_SUFFIX));
+        batch.delete(bytes(keyPrefix + CONTAINER_WORK_DIR_KEY_SUFFIX));
         List<String> unknownKeysForContainer = containerUnknownKeySuffixes
             .removeAll(containerId);
         for (String unknownKeySuffix : unknownKeysForContainer) {
@@ -1169,13 +1175,13 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
   @Override
   public void storeContainerTokenCurrentMasterKey(MasterKey key)
       throws IOException {
-    storeMasterKey(CONTAINER_TOKENS_CURRENT_MASTER_KEY, key);
+    storeMasterKey(CONTAINER_TOKEN_SECRETMANAGER_CURRENT_MASTER_KEY, key);
   }
 
   @Override
   public void storeContainerTokenPreviousMasterKey(MasterKey key)
       throws IOException {
-    storeMasterKey(CONTAINER_TOKENS_PREV_MASTER_KEY, key);
+    storeMasterKey(CONTAINER_TOKEN_SECRETMANAGER_PREV_MASTER_KEY, key);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d586841/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
index c8c07d1..8a8cfa2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
@@ -28,7 +28,9 @@ import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.isNull;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.timeout;
+import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
@@ -382,6 +384,11 @@ public class TestNMLeveldbStateStoreService {
     restartStateStore();
     recoveredContainers = stateStore.loadContainersState();
     assertTrue(recoveredContainers.isEmpty());
+    // recover again to check remove clears all containers
+    restartStateStore();
+    NMStateStoreService nmStoreSpy = spy(stateStore);
+    nmStoreSpy.loadContainersState();
+    verify(nmStoreSpy,times(0)).removeContainer(any(ContainerId.class));
   }
 
   private void validateRetryAttempts(ContainerId containerId)


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


[06/50] hadoop git commit: HADOOP-15612. Improve exception when tfile fails to load LzoCodec. (gera)

Posted by eh...@apache.org.
HADOOP-15612. Improve exception when tfile fails to load LzoCodec. (gera)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6bec03cf
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6bec03cf
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6bec03cf

Branch: refs/heads/HDFS-12090
Commit: 6bec03cfc8bdcf6aa3df9c22231ab959ba31f2f5
Parents: ea2c6c8
Author: Gera Shegalov <ge...@apache.org>
Authored: Tue Jul 17 00:05:39 2018 -0700
Committer: Gera Shegalov <ge...@apache.org>
Committed: Tue Jul 24 14:32:30 2018 -0700

----------------------------------------------------------------------
 .../hadoop/io/file/tfile/Compression.java       | 31 +++++++++++-------
 .../hadoop/io/file/tfile/TestCompression.java   | 34 +++++++++++++++++++-
 2 files changed, 53 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6bec03cf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java
index fa85ed7..c4347e0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java
@@ -5,9 +5,9 @@
  * licenses this file to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance with the License.
  * You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
  * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
@@ -24,6 +24,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.ArrayList;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.compress.CodecPool;
 import org.apache.hadoop.io.compress.CompressionCodec;
@@ -78,25 +79,33 @@ public final class Compression {
   public enum Algorithm {
     LZO(TFile.COMPRESSION_LZO) {
       private transient boolean checked = false;
+      private transient ClassNotFoundException cnf;
+      private transient boolean reinitCodecInTests;
       private static final String defaultClazz =
           "org.apache.hadoop.io.compress.LzoCodec";
+      private transient String clazz;
       private transient CompressionCodec codec = null;
 
+      private String getLzoCodecClass() {
+        String extClazzConf = conf.get(CONF_LZO_CLASS);
+        String extClazz = (extClazzConf != null) ?
+            extClazzConf : System.getProperty(CONF_LZO_CLASS);
+        return (extClazz != null) ? extClazz : defaultClazz;
+      }
+
       @Override
       public synchronized boolean isSupported() {
-        if (!checked) {
+        if (!checked || reinitCodecInTests) {
           checked = true;
-          String extClazzConf = conf.get(CONF_LZO_CLASS);
-          String extClazz = (extClazzConf != null) ?
-              extClazzConf : System.getProperty(CONF_LZO_CLASS);
-          String clazz = (extClazz != null) ? extClazz : defaultClazz;
+          reinitCodecInTests = conf.getBoolean("test.reload.lzo.codec", false);
+          clazz = getLzoCodecClass();
           try {
             LOG.info("Trying to load Lzo codec class: " + clazz);
             codec =
                 (CompressionCodec) ReflectionUtils.newInstance(Class
                     .forName(clazz), conf);
           } catch (ClassNotFoundException e) {
-            // that is okay
+            cnf = e;
           }
         }
         return codec != null;
@@ -105,9 +114,9 @@ public final class Compression {
       @Override
       CompressionCodec getCodec() throws IOException {
         if (!isSupported()) {
-          throw new IOException(
-              "LZO codec class not specified. Did you forget to set property "
-                  + CONF_LZO_CLASS + "?");
+          throw new IOException(String.format(
+              "LZO codec %s=%s could not be loaded", CONF_LZO_CLASS, clazz),
+                  cnf);
         }
 
         return codec;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6bec03cf/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestCompression.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestCompression.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestCompression.java
index ff6c72a..b1bf077 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestCompression.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestCompression.java
@@ -17,14 +17,28 @@
  */
 package org.apache.hadoop.io.file.tfile;
 
-import org.junit.Test;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.*;
 
 import java.io.IOException;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public class TestCompression {
 
+  @BeforeClass
+  public static void resetConfigBeforeAll() {
+    Compression.Algorithm.LZO.conf.setBoolean("test.reload.lzo.codec", true);
+  }
+
+  @AfterClass
+  public static void resetConfigAfterAll() {
+    Compression.Algorithm.LZO.conf.setBoolean("test.reload.lzo.codec", false);
+  }
+
   /**
    * Regression test for HADOOP-11418.
    * Verify we can set a LZO codec different from default LZO codec.
@@ -38,4 +52,22 @@ public class TestCompression {
     assertEquals(defaultCodec,
         Compression.Algorithm.LZO.getCodec().getClass().getName());
   }
+
+
+  @Test
+  public void testMisconfiguredLZOCodec() throws Exception {
+    // Dummy codec
+    String defaultCodec = "org.apache.hadoop.io.compress.InvalidLzoCodec";
+    Compression.Algorithm.conf.set(
+        Compression.Algorithm.CONF_LZO_CLASS, defaultCodec);
+    IOException ioEx = LambdaTestUtils.intercept(
+        IOException.class,
+        defaultCodec,
+        () -> Compression.Algorithm.LZO.getCodec());
+
+    if (!(ioEx.getCause() instanceof ClassNotFoundException)) {
+      throw ioEx;
+    }
+  }
+
 }


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


[10/50] hadoop git commit: HDDS-203. Add getCommittedBlockLength API in datanode. Contributed by Shashikant Banerjee.

Posted by eh...@apache.org.
HDDS-203. Add getCommittedBlockLength API in datanode. Contributed by Shashikant Banerjee.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/955f7951
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/955f7951
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/955f7951

Branch: refs/heads/HDFS-12090
Commit: 955f795101c1a0ae176b4cebda3ffbe9850dd687
Parents: 81d5950
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Wed Jul 25 14:15:54 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Wed Jul 25 14:15:54 2018 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hdds/client/BlockID.java  |  20 +-
 .../scm/storage/ContainerProtocolCalls.java     |  28 +++
 .../ozone/container/common/helpers/KeyData.java |  27 ++-
 .../main/proto/DatanodeContainerProtocol.proto  |  17 +-
 .../container/common/impl/HddsDispatcher.java   |   2 +
 .../container/keyvalue/KeyValueHandler.java     |  34 ++++
 .../container/keyvalue/helpers/KeyUtils.java    |  20 ++
 .../container/keyvalue/impl/KeyManagerImpl.java |  27 +++
 .../keyvalue/interfaces/KeyManager.java         |   7 +
 .../ozone/scm/TestCommittedBlockLengthAPI.java  | 191 +++++++++++++++++++
 10 files changed, 368 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
index 62b12e3..74e90e9 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
@@ -20,8 +20,10 @@ import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 
+import java.util.Objects;
+
 /**
- * BlockID of ozone (containerID + localID)
+ * BlockID of ozone (containerID  localID)
  */
 public class BlockID {
   private long containerID;
@@ -68,4 +70,20 @@ public class BlockID {
         blockID.getLocalID());
   }
 
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    BlockID blockID = (BlockID) o;
+    return containerID == blockID.containerID && localID == blockID.localID;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(containerID, localID);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
index f4f14ef..36cdfc9 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
@@ -99,6 +99,34 @@ public final class ContainerProtocolCalls  {
   }
 
   /**
+   * Calls the container protocol to get the length of a committed block.
+   *
+   * @param xceiverClient client to perform call
+   * @param blockID blockId for the Block
+   * @param traceID container protocol call args
+   * @return container protocol getLastCommittedBlockLength response
+   * @throws IOException if there is an I/O error while performing the call
+   */
+  public static ContainerProtos.GetCommittedBlockLengthResponseProto
+  getCommittedBlockLength(
+      XceiverClientSpi xceiverClient, BlockID blockID, String traceID)
+      throws IOException {
+    ContainerProtos.GetCommittedBlockLengthRequestProto.Builder
+        getBlockLengthRequestBuilder =
+        ContainerProtos.GetCommittedBlockLengthRequestProto.newBuilder().
+            setBlockID(blockID.getDatanodeBlockIDProtobuf());
+    String id = xceiverClient.getPipeline().getLeader().getUuidString();
+    ContainerCommandRequestProto request =
+        ContainerCommandRequestProto.newBuilder()
+            .setCmdType(Type.GetCommittedBlockLength).setTraceID(traceID)
+            .setDatanodeUuid(id)
+            .setGetCommittedBlockLength(getBlockLengthRequestBuilder).build();
+    ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
+    validateContainerResponse(response);
+    return response.getGetCommittedBlockLength();
+  }
+
+  /**
    * Calls the container protocol to put a container key.
    *
    * @param xceiverClient client to perform call

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
index b63332f..1919ed9 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
@@ -42,6 +42,11 @@ public class KeyData {
   private List<ContainerProtos.ChunkInfo> chunks;
 
   /**
+   * total size of the key.
+   */
+  private long size;
+
+  /**
    * Constructs a KeyData Object.
    *
    * @param blockID
@@ -49,6 +54,7 @@ public class KeyData {
   public KeyData(BlockID blockID) {
     this.blockID = blockID;
     this.metadata = new TreeMap<>();
+    this.size = 0;
   }
 
   /**
@@ -66,6 +72,9 @@ public class KeyData {
           data.getMetadata(x).getValue());
     }
     keyData.setChunks(data.getChunksList());
+    if (data.hasSize()) {
+      keyData.setSize(data.getSize());
+    }
     return keyData;
   }
 
@@ -84,6 +93,7 @@ public class KeyData {
       builder.addMetadata(keyValBuilder.setKey(entry.getKey())
           .setValue(entry.getValue()).build());
     }
+    builder.setSize(size);
     return builder.build();
   }
 
@@ -183,10 +193,25 @@ public class KeyData {
   }
 
   /**
+   * sets the total size of the block
+   * @param size size of the block
+   */
+  public void setSize(long size) {
+    this.size = size;
+  }
+
+  /**
    * Get the total size of chunks allocated for the key.
    * @return total size of the key.
    */
   public long getSize() {
-    return chunks.parallelStream().mapToLong(e->e.getLen()).sum();
+    return size;
+  }
+
+  /**
+   * computes the total size of chunks allocated for the key.
+   */
+  public void computeSize() {
+    setSize(chunks.parallelStream().mapToLong(e -> e.getLen()).sum());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
index ff1582e..a3c4467 100644
--- a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
+++ b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
@@ -99,7 +99,7 @@ enum Type {
   PutSmallFile = 15;
   GetSmallFile = 16;
   CloseContainer = 17;
-
+  GetCommittedBlockLength = 18;
 }
 
 
@@ -193,8 +193,8 @@ message ContainerCommandRequestProto {
   optional   PutSmallFileRequestProto putSmallFile = 16;
   optional   GetSmallFileRequestProto getSmallFile = 17;
   optional   CloseContainerRequestProto closeContainer = 18;
-
-  required   string datanodeUuid = 19;
+  optional   GetCommittedBlockLengthRequestProto getCommittedBlockLength = 19;
+  required   string datanodeUuid = 20;
 }
 
 message ContainerCommandResponseProto {
@@ -223,6 +223,7 @@ message ContainerCommandResponseProto {
   optional PutSmallFileResponseProto putSmallFile = 19;
   optional GetSmallFileResponseProto getSmallFile = 20;
   optional CloseContainerResponseProto closeContainer = 21;
+  optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 22;
 
 }
 
@@ -302,6 +303,7 @@ message KeyData {
   optional int64 flags = 2; // for future use.
   repeated KeyValue metadata = 3;
   repeated ChunkInfo chunks = 4;
+  optional int64 size = 5;
 }
 
 // Key Messages.
@@ -325,6 +327,15 @@ message  DeleteKeyRequestProto {
   required DatanodeBlockID blockID = 1;
 }
 
+message  GetCommittedBlockLengthRequestProto {
+  required DatanodeBlockID blockID = 1;
+}
+
+message  GetCommittedBlockLengthResponseProto {
+  required DatanodeBlockID blockID = 1;
+  required int64 blockLength = 2;
+}
+
 message   DeleteKeyResponseProto {
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
index bee8417..6d11abb 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
@@ -181,6 +181,8 @@ public class HddsDispatcher implements ContainerDispatcher {
           .getContainerID();
     case GetSmallFile:
       return request.getGetSmallFile().getKey().getBlockID().getContainerID();
+    case GetCommittedBlockLength:
+      return request.getGetCommittedBlockLength().getBlockID().getContainerID();
     }
 
     throw new StorageContainerException(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index d3a1ca4..4123dc8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -198,6 +198,8 @@ public class KeyValueHandler extends Handler {
       return handlePutSmallFile(request, kvContainer);
     case GetSmallFile:
       return handleGetSmallFile(request, kvContainer);
+    case GetCommittedBlockLength:
+      return handleGetCommittedBlockLength(request, kvContainer);
     }
     return null;
   }
@@ -443,6 +445,8 @@ public class KeyValueHandler extends Handler {
   private void commitKey(KeyData keyData, KeyValueContainer kvContainer)
       throws IOException {
     Preconditions.checkNotNull(keyData);
+    //sets the total size of the key before committing
+    keyData.computeSize();
     keyManager.putKey(kvContainer, keyData);
     //update the open key Map in containerManager
     this.openContainerBlockMap.removeFromKeyMap(keyData.getBlockID());
@@ -479,6 +483,35 @@ public class KeyValueHandler extends Handler {
   }
 
   /**
+   * Handles GetCommittedBlockLength operation.
+   * Calls KeyManager to process the request.
+   */
+  ContainerCommandResponseProto handleGetCommittedBlockLength(
+      ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+    if (!request.hasGetCommittedBlockLength()) {
+      LOG.debug("Malformed Get Key request. trace ID: {}",
+          request.getTraceID());
+      return ContainerUtils.malformedRequest(request);
+    }
+
+    long blockLength;
+    try {
+      BlockID blockID = BlockID.getFromProtobuf(
+          request.getGetCommittedBlockLength().getBlockID());
+      blockLength = keyManager.getCommittedBlockLength(kvContainer, blockID);
+
+    } catch (StorageContainerException ex) {
+      return ContainerUtils.logAndReturnError(LOG, ex, request);
+    } catch (IOException ex) {
+      return ContainerUtils.logAndReturnError(LOG,
+          new StorageContainerException("GetCommittedBlockLength failed", ex,
+              IO_EXCEPTION), request);
+    }
+
+    return KeyUtils.getBlockLengthResponse(request, blockLength);
+  }
+
+  /**
    * Handle Delete Key operation. Calls KeyManager to process the request.
    */
   ContainerCommandResponseProto handleDeleteKey(
@@ -665,6 +698,7 @@ public class KeyValueHandler extends Handler {
       List<ContainerProtos.ChunkInfo> chunks = new LinkedList<>();
       chunks.add(chunkInfo.getProtoBufMessage());
       keyData.setChunks(chunks);
+      keyData.computeSize();
       keyManager.putKey(kvContainer, keyData);
       metrics.incContainerBytesStats(Type.PutSmallFile, data.length);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
index 5845fae..2be966d 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
@@ -144,4 +144,24 @@ public final class KeyUtils {
     builder.setGetKey(getKey);
     return  builder.build();
   }
+
+  /**
+   * Returns successful getCommittedBlockLength Response.
+   * @param msg - Request.
+   * @return Response.
+   */
+  public static ContainerProtos.ContainerCommandResponseProto
+  getBlockLengthResponse(ContainerProtos.
+      ContainerCommandRequestProto msg, long blockLength) {
+    ContainerProtos.GetCommittedBlockLengthResponseProto.Builder
+        getCommittedBlockLengthResponseBuilder = ContainerProtos.
+        GetCommittedBlockLengthResponseProto.newBuilder();
+    getCommittedBlockLengthResponseBuilder.setBlockLength(blockLength);
+    getCommittedBlockLengthResponseBuilder
+        .setBlockID(msg.getGetCommittedBlockLength().getBlockID());
+    ContainerProtos.ContainerCommandResponseProto.Builder builder =
+        ContainerUtils.getSuccessResponseBuilder(msg);
+    builder.setGetCommittedBlockLength(getCommittedBlockLengthResponseBuilder);
+    return  builder.build();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
index 6a8897a..58bf1f8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
@@ -120,6 +120,32 @@ public class KeyManagerImpl implements KeyManager {
   }
 
   /**
+   * Returns the length of the committed block.
+   *
+   * @param container - Container from which key need to be get.
+   * @param blockID - BlockID of the key.
+   * @return length of the block.
+   * @throws IOException in case, the block key does not exist in db.
+   */
+  @Override
+  public long getCommittedBlockLength(Container container, BlockID blockID)
+      throws IOException {
+    KeyValueContainerData containerData = (KeyValueContainerData) container
+        .getContainerData();
+    MetadataStore db = KeyUtils.getDB(containerData, config);
+    // This is a post condition that acts as a hint to the user.
+    // Should never fail.
+    Preconditions.checkNotNull(db, "DB cannot be null here");
+    byte[] kData = db.get(Longs.toByteArray(blockID.getLocalID()));
+    if (kData == null) {
+      throw new StorageContainerException("Unable to find the key.",
+          NO_SUCH_KEY);
+    }
+    ContainerProtos.KeyData keyData = ContainerProtos.KeyData.parseFrom(kData);
+    return keyData.getSize();
+  }
+
+  /**
    * Deletes an existing Key.
    *
    * @param container - Container from which key need to be deleted.
@@ -164,6 +190,7 @@ public class KeyManagerImpl implements KeyManager {
    * @param count    - Number of keys to return.
    * @return List of Keys that match the criteria.
    */
+  @Override
   public List<KeyData> listKey(Container container, long startLocalID, int
       count) throws IOException {
     Preconditions.checkNotNull(container, "container cannot be null");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
index 7a5d48b..dad688e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
@@ -70,6 +70,13 @@ public interface KeyManager {
       IOException;
 
   /**
+   * Returns the last committed block length for the block.
+   * @param blockID blockId
+   */
+  long getCommittedBlockLength(Container container, BlockID blockID)
+      throws IOException;
+
+  /**
    * Shutdown ContainerManager.
    */
   void shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java
new file mode 100644
index 0000000..7e8aa5f
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java
@@ -0,0 +1,191 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.scm;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.XceiverClientManager;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import org.apache.hadoop.hdds.scm.container.common.helpers.
+    ContainerWithPipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers.
+    StorageContainerException;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.
+    ContainerPlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.
+    SCMContainerPlacementCapacity;
+import org.apache.hadoop.hdds.scm.protocolPB.
+    StorageContainerLocationProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+
+import java.util.UUID;
+
+/**
+ * Test Container calls.
+ */
+public class TestCommittedBlockLengthAPI {
+
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration ozoneConfig;
+  private static StorageContainerLocationProtocolClientSideTranslatorPB
+      storageContainerLocationClient;
+  private static XceiverClientManager xceiverClientManager;
+  private static String containerOwner = "OZONE";
+
+  @BeforeClass
+  public static void init() throws Exception {
+    ozoneConfig = new OzoneConfiguration();
+    ozoneConfig.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
+        SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class);
+    cluster =
+        MiniOzoneCluster.newBuilder(ozoneConfig).setNumDatanodes(1).build();
+    cluster.waitForClusterToBeReady();
+    storageContainerLocationClient =
+        cluster.getStorageContainerLocationClient();
+    xceiverClientManager = new XceiverClientManager(ozoneConfig);
+  }
+
+  @AfterClass
+  public static void shutdown() throws InterruptedException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+    IOUtils.cleanupWithLogger(null, storageContainerLocationClient);
+  }
+
+  @Test
+  public void tesGetCommittedBlockLength() throws Exception {
+    ContainerProtos.GetCommittedBlockLengthResponseProto response;
+    String traceID = UUID.randomUUID().toString();
+    ContainerWithPipeline container = storageContainerLocationClient
+        .allocateContainer(xceiverClientManager.getType(),
+            HddsProtos.ReplicationFactor.ONE, containerOwner);
+    long containerID = container.getContainerInfo().getContainerID();
+    Pipeline pipeline = container.getPipeline();
+    XceiverClientSpi client =
+        xceiverClientManager.acquireClient(pipeline, containerID);
+    //create the container
+    ContainerProtocolCalls.createContainer(client, containerID, traceID);
+
+    BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
+    byte[] data =
+        RandomStringUtils.random(RandomUtils.nextInt(0, 1024)).getBytes();
+    ContainerProtos.ContainerCommandRequestProto writeChunkRequest =
+        ContainerTestHelper
+            .getWriteChunkRequest(container.getPipeline(), blockID,
+                data.length);
+    client.sendCommand(writeChunkRequest);
+    try {
+      // since there is neither explicit putKey request made for the block,
+      // nor the container is closed, GetCommittedBlockLength request
+      // should fail here.
+      response = ContainerProtocolCalls
+          .getCommittedBlockLength(client, blockID, traceID);
+      Assert.fail("Expected exception not thrown");
+    } catch (StorageContainerException sce) {
+      Assert.assertTrue(sce.getMessage().contains("Unable to find the key"));
+    }
+    // Now, explicitly make a putKey request for the block.
+    ContainerProtos.ContainerCommandRequestProto putKeyRequest =
+        ContainerTestHelper
+            .getPutKeyRequest(pipeline, writeChunkRequest.getWriteChunk());
+    client.sendCommand(putKeyRequest);
+    response = ContainerProtocolCalls
+        .getCommittedBlockLength(client, blockID, traceID);
+    // make sure the block ids in the request and response are same.
+    Assert.assertTrue(
+        BlockID.getFromProtobuf(response.getBlockID()).equals(blockID));
+    Assert.assertTrue(response.getBlockLength() == data.length);
+    xceiverClientManager.releaseClient(client);
+  }
+
+  @Test
+  public void tesGetCommittedBlockLengthWithClosedContainer()
+      throws Exception {
+    String traceID = UUID.randomUUID().toString();
+    ContainerWithPipeline container = storageContainerLocationClient
+        .allocateContainer(xceiverClientManager.getType(),
+            HddsProtos.ReplicationFactor.ONE, containerOwner);
+    long containerID = container.getContainerInfo().getContainerID();
+    Pipeline pipeline = container.getPipeline();
+    XceiverClientSpi client =
+        xceiverClientManager.acquireClient(pipeline, containerID);
+    // create the container
+    ContainerProtocolCalls.createContainer(client, containerID, traceID);
+
+    byte[] data =
+        RandomStringUtils.random(RandomUtils.nextInt(0, 1024)).getBytes();
+    BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
+    ContainerProtos.ContainerCommandRequestProto writeChunkRequest =
+        ContainerTestHelper
+            .getWriteChunkRequest(container.getPipeline(), blockID,
+                data.length);
+    client.sendCommand(writeChunkRequest);
+    // close the container
+    ContainerProtocolCalls.closeContainer(client, containerID, traceID);
+    ContainerProtos.GetCommittedBlockLengthResponseProto response =
+        ContainerProtocolCalls
+            .getCommittedBlockLength(client, blockID, traceID);
+    // make sure the block ids in the request and response are same.
+    // This will also ensure that closing the container committed the block
+    // on the Datanodes.
+    Assert.assertTrue(
+        BlockID.getFromProtobuf(response.getBlockID()).equals(blockID));
+    Assert.assertTrue(response.getBlockLength() == data.length);
+    xceiverClientManager.releaseClient(client);
+  }
+
+  @Test
+  public void tesGetCommittedBlockLengthForInvalidBlock() throws Exception {
+    String traceID = UUID.randomUUID().toString();
+    ContainerWithPipeline container = storageContainerLocationClient
+        .allocateContainer(xceiverClientManager.getType(),
+            HddsProtos.ReplicationFactor.ONE, containerOwner);
+    long containerID = container.getContainerInfo().getContainerID();
+    XceiverClientSpi client = xceiverClientManager
+        .acquireClient(container.getPipeline(), containerID);
+    ContainerProtocolCalls.createContainer(client, containerID, traceID);
+
+    BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
+    // move the container to closed state
+    ContainerProtocolCalls.closeContainer(client, containerID, traceID);
+    try {
+      // There is no block written inside the container. The request should
+      // fail.
+      ContainerProtocolCalls.getCommittedBlockLength(client, blockID, traceID);
+      Assert.fail("Expected exception not thrown");
+    } catch (StorageContainerException sce) {
+      Assert.assertTrue(sce.getMessage().contains("Unable to find the key"));
+    }
+    xceiverClientManager.releaseClient(client);
+  }
+}
\ No newline at end of file


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


[22/50] hadoop git commit: HDFS-13622. mkdir should print the parent directory in the error message when parent directories do not exist. Contributed by Shweta.

Posted by eh...@apache.org.
HDFS-13622. mkdir should print the parent directory in the error message when parent directories do not exist. Contributed by Shweta.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/be150a17
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/be150a17
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/be150a17

Branch: refs/heads/HDFS-12090
Commit: be150a17b15d15f5de6d4839d5e805e8d6c57850
Parents: a192295
Author: Xiao Chen <xi...@apache.org>
Authored: Thu Jul 26 10:23:30 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Thu Jul 26 10:24:32 2018 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/fs/shell/Mkdir.java    | 13 ++++++++-----
 .../test/java/org/apache/hadoop/hdfs/TestDFSShell.java |  8 ++++++++
 .../hadoop-hdfs/src/test/resources/testHDFSConf.xml    |  4 ++--
 3 files changed, 18 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/be150a17/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Mkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Mkdir.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Mkdir.java
index 9f39da2..5828b0b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Mkdir.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Mkdir.java
@@ -68,11 +68,14 @@ class Mkdir extends FsCommand {
 
   @Override
   protected void processNonexistentPath(PathData item) throws IOException {
-    // check if parent exists. this is complicated because getParent(a/b/c/) returns a/b/c, but
-    // we want a/b
-    if (!createParents &&
-        !item.fs.exists(new Path(item.path.toString()).getParent())) {
-      throw new PathNotFoundException(item.toString());
+    if (!createParents) {
+      // check if parent exists. this is complicated because getParent(a/b/c/) returns a/b/c, but
+      // we want a/b
+      final Path itemPath = new Path(item.path.toString());
+      final Path itemParentPath = itemPath.getParent();
+      if (!item.fs.exists(itemParentPath)) {
+        throw new PathNotFoundException(itemParentPath.toString());
+      }
     }
     if (!item.fs.mkdirs(item.path)) {
       throw new PathIOException(item.toString());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/be150a17/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
index b19bdea..1d2042e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
@@ -721,6 +721,14 @@ public class TestDFSShell {
       assertTrue(" -mkdir returned this is a file ",
           (returned.lastIndexOf("not a directory") != -1));
       out.reset();
+      argv[0] = "-mkdir";
+      argv[1] = "/testParent/testChild";
+      ret = ToolRunner.run(shell, argv);
+      returned = out.toString();
+      assertEquals(" -mkdir returned 1", 1, ret);
+      assertTrue(" -mkdir returned there is No file or directory but has testChild in the path",
+          (returned.lastIndexOf("testChild") == -1));
+      out.reset();
       argv = new String[3];
       argv[0] = "-mv";
       argv[1] = "/testfile";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/be150a17/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
index a13c441..4ab093b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
@@ -6183,11 +6183,11 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>mkdir: `dir0/dir1': No such file or directory</expected-output>
+          <expected-output>.*mkdir:.*dir0': No such file or directory$</expected-output>
         </comparator>
       </comparators>
     </test>
-    
+
     <test> <!-- TESTED -->
       <description>mkdir: Test recreate of existing directory fails</description>
       <test-commands>


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


[12/50] hadoop git commit: HDDS-285. Create a generic Metadata Iterator. Contributed by Bharat Viswanadham.

Posted by eh...@apache.org.
HDDS-285. Create a generic Metadata Iterator. Contributed by Bharat Viswanadham.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/43db0cb5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/43db0cb5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/43db0cb5

Branch: refs/heads/HDFS-12090
Commit: 43db0cb518375eb767401fa525ea6b5026ed9a8a
Parents: 5be9f4a
Author: Nanda kumar <na...@apache.org>
Authored: Wed Jul 25 18:11:35 2018 +0530
Committer: Nanda kumar <na...@apache.org>
Committed: Wed Jul 25 18:13:49 2018 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/utils/LevelDBStore.java   |   5 +
 .../hadoop/utils/LevelDBStoreIterator.java      |  64 ++++++
 .../apache/hadoop/utils/MetaStoreIterator.java  |  39 ++++
 .../org/apache/hadoop/utils/MetadataStore.java  |  55 +++++
 .../org/apache/hadoop/utils/RocksDBStore.java   |   5 +
 .../hadoop/utils/RocksDBStoreIterator.java      |  66 ++++++
 .../apache/hadoop/utils/TestMetadataStore.java  | 206 ++++++++++++-------
 7 files changed, 366 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/43db0cb5/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java
index 13b9180..ed116a3 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java
@@ -379,4 +379,9 @@ public class LevelDBStore implements MetadataStore {
     }
     return result;
   }
+
+  @Override
+  public MetaStoreIterator<KeyValue> iterator() {
+    return new LevelDBStoreIterator(db.iterator());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43db0cb5/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStoreIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStoreIterator.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStoreIterator.java
new file mode 100644
index 0000000..7b62f7a
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStoreIterator.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.utils;
+
+import org.iq80.leveldb.DBIterator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.utils.MetadataStore.KeyValue;
+
+
+/**
+ * LevelDB store iterator.
+ */
+public class LevelDBStoreIterator implements MetaStoreIterator<KeyValue> {
+
+
+  private DBIterator levelDBIterator;
+
+  public LevelDBStoreIterator(DBIterator iterator) {
+    this.levelDBIterator = iterator;
+    levelDBIterator.seekToFirst();
+  }
+
+  @Override
+  public boolean hasNext() {
+    return levelDBIterator.hasNext();
+  }
+
+  @Override
+  public KeyValue next() {
+    if(levelDBIterator.hasNext()) {
+      Map.Entry<byte[], byte[]> entry = levelDBIterator.next();
+      return KeyValue.create(entry.getKey(), entry.getValue());
+    }
+    throw new NoSuchElementException("LevelDB Store has no more elements");
+  }
+
+  @Override
+  public void seekToFirst() {
+    levelDBIterator.seekToFirst();
+  }
+
+  @Override
+  public void seekToLast() {
+    levelDBIterator.seekToLast();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43db0cb5/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetaStoreIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetaStoreIterator.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetaStoreIterator.java
new file mode 100644
index 0000000..758d194
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetaStoreIterator.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.utils;
+
+import java.util.Iterator;
+
+/**
+ * Iterator for MetaDataStore DB.
+ * @param <T>
+ */
+interface MetaStoreIterator<T> extends Iterator<T> {
+
+  /**
+   * seek to first entry.
+   */
+  void seekToFirst();
+
+  /**
+   * seek to last entry.
+   */
+  void seekToLast();
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43db0cb5/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStore.java
index b90b08f..7d3bc6b 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStore.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStore.java
@@ -169,4 +169,59 @@ public interface MetadataStore extends Closeable{
    */
   void iterate(byte[] from, EntryConsumer consumer)
       throws IOException;
+
+  /**
+   * Returns the iterator for this metadata store.
+   * @return MetaStoreIterator
+   */
+  MetaStoreIterator<KeyValue> iterator();
+
+  /**
+   * Class used to represent the key and value pair of a db entry.
+   */
+  class KeyValue {
+
+    private final byte[] key;
+    private final byte[] value;
+
+    /**
+     * KeyValue Constructor, used to represent a key and value of a db entry.
+     * @param key
+     * @param value
+     */
+    private KeyValue(byte[] key, byte[] value) {
+      this.key = key;
+      this.value = value;
+    }
+
+    /**
+     * Return key.
+     * @return byte[]
+     */
+    public byte[] getKey() {
+      byte[] result = new byte[key.length];
+      System.arraycopy(key, 0, result, 0, key.length);
+      return result;
+    }
+
+    /**
+     * Return value.
+     * @return byte[]
+     */
+    public byte[] getValue() {
+      byte[] result = new byte[value.length];
+      System.arraycopy(value, 0, result, 0, value.length);
+      return result;
+    }
+
+    /**
+     * Create a KeyValue pair.
+     * @param key
+     * @param value
+     * @return KeyValue object.
+     */
+    public static KeyValue create(byte[] key, byte[] value) {
+      return new KeyValue(key, value);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43db0cb5/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java
index 0dfca20..f5f070d 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java
@@ -380,4 +380,9 @@ public class RocksDBStore implements MetadataStore {
     return statMBeanName;
   }
 
+  @Override
+  public MetaStoreIterator<KeyValue> iterator() {
+    return new RocksDBStoreIterator(db.newIterator());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43db0cb5/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStoreIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStoreIterator.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStoreIterator.java
new file mode 100644
index 0000000..6e9b695
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStoreIterator.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+
+package org.apache.hadoop.utils;
+
+import org.rocksdb.RocksIterator;
+
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.utils.MetadataStore.KeyValue;
+
+/**
+ * RocksDB store iterator.
+ */
+public class RocksDBStoreIterator implements MetaStoreIterator<KeyValue> {
+
+  private RocksIterator rocksDBIterator;
+
+  public RocksDBStoreIterator(RocksIterator iterator) {
+    this.rocksDBIterator = iterator;
+    rocksDBIterator.seekToFirst();
+  }
+
+  @Override
+  public boolean hasNext() {
+    return rocksDBIterator.isValid();
+  }
+
+  @Override
+  public KeyValue next() {
+    if (rocksDBIterator.isValid()) {
+      KeyValue value = KeyValue.create(rocksDBIterator.key(), rocksDBIterator
+          .value());
+      rocksDBIterator.next();
+      return value;
+    }
+    throw new NoSuchElementException("RocksDB Store has no more elements");
+  }
+
+  @Override
+  public void seekToFirst() {
+    rocksDBIterator.seekToFirst();
+  }
+
+  @Override
+  public void seekToLast() {
+    rocksDBIterator.seekToLast();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43db0cb5/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
index d697bbf..1bce022 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
@@ -28,10 +28,10 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.utils.MetadataStore.KeyValue;
 import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
 import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -48,10 +48,17 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.NoSuchElementException;
 import java.util.UUID;
+
 import java.util.concurrent.atomic.AtomicInteger;
 
-import static junit.framework.TestCase.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import static org.junit.runners.Parameterized.Parameters;
 
 /**
@@ -110,6 +117,58 @@ public class TestMetadataStore {
   }
 
   @Test
+  public void testIterator() throws Exception {
+    Configuration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
+    File dbDir = GenericTestUtils.getRandomizedTestDir();
+    MetadataStore dbStore = MetadataStoreBuilder.newBuilder()
+        .setConf(conf)
+        .setCreateIfMissing(true)
+        .setDbFile(dbDir)
+        .build();
+
+    //As database is empty, check whether iterator is working as expected or
+    // not.
+    MetaStoreIterator<KeyValue> metaStoreIterator = dbStore.iterator();
+    assertFalse(metaStoreIterator.hasNext());
+    try {
+      metaStoreIterator.next();
+      fail("testIterator failed");
+    } catch (NoSuchElementException ex) {
+      GenericTestUtils.assertExceptionContains("Store has no more elements",
+          ex);
+    }
+
+    for (int i = 0; i < 10; i++) {
+      store.put(getBytes("a" + i), getBytes("a-value" + i));
+    }
+
+    metaStoreIterator = dbStore.iterator();
+
+    int i = 0;
+    while (metaStoreIterator.hasNext()) {
+      KeyValue val = metaStoreIterator.next();
+      assertEquals("a" + i, getString(val.getKey()));
+      assertEquals("a-value" + i, getString(val.getValue()));
+      i++;
+    }
+
+    // As we have iterated all the keys in database, hasNext should return
+    // false and next() should throw NoSuchElement exception.
+
+    assertFalse(metaStoreIterator.hasNext());
+    try {
+      metaStoreIterator.next();
+      fail("testIterator failed");
+    } catch (NoSuchElementException ex) {
+      GenericTestUtils.assertExceptionContains("Store has no more elements",
+          ex);
+    }
+    FileUtils.deleteDirectory(dbDir);
+
+  }
+
+  @Test
   public void testMetaStoreConfigDifferentFromType() throws IOException {
 
     Configuration conf = new OzoneConfiguration();
@@ -183,17 +242,17 @@ public class TestMetadataStore {
   public void testGetDelete() throws IOException {
     for (int i=0; i<10; i++) {
       byte[] va = store.get(getBytes("a" + i));
-      Assert.assertEquals("a-value" + i, getString(va));
+      assertEquals("a-value" + i, getString(va));
 
       byte[] vb = store.get(getBytes("b" + i));
-      Assert.assertEquals("b-value" + i, getString(vb));
+      assertEquals("b-value" + i, getString(vb));
     }
 
     String keyToDel = "del-" + UUID.randomUUID().toString();
     store.put(getBytes(keyToDel), getBytes(keyToDel));
-    Assert.assertEquals(keyToDel, getString(store.get(getBytes(keyToDel))));
+    assertEquals(keyToDel, getString(store.get(getBytes(keyToDel))));
     store.delete(getBytes(keyToDel));
-    Assert.assertEquals(null, store.get(getBytes(keyToDel)));
+    assertEquals(null, store.get(getBytes(keyToDel)));
   }
 
   @Test
@@ -228,8 +287,8 @@ public class TestMetadataStore {
       k = getString(current.getKey());
       v = getString(current.getValue());
     }
-    Assert.assertEquals(peekKey, k);
-    Assert.assertEquals(v, getExpectedValue(peekKey));
+    assertEquals(peekKey, k);
+    assertEquals(v, getExpectedValue(peekKey));
 
     // Look for prev
     k = null;
@@ -240,8 +299,8 @@ public class TestMetadataStore {
       k = getString(prev.getKey());
       v = getString(prev.getValue());
     }
-    Assert.assertEquals(prevKey, k);
-    Assert.assertEquals(v, getExpectedValue(prevKey));
+    assertEquals(prevKey, k);
+    assertEquals(v, getExpectedValue(prevKey));
 
     // Look for next
     k = null;
@@ -252,8 +311,8 @@ public class TestMetadataStore {
       k = getString(next.getKey());
       v = getString(next.getValue());
     }
-    Assert.assertEquals(nextKey, k);
-    Assert.assertEquals(v, getExpectedValue(nextKey));
+    assertEquals(nextKey, k);
+    assertEquals(v, getExpectedValue(nextKey));
   }
 
   @Test
@@ -271,9 +330,9 @@ public class TestMetadataStore {
       return true;
     });
 
-    Assert.assertFalse(result.isEmpty());
+    assertFalse(result.isEmpty());
     for (int i=0; i<result.size(); i++) {
-      Assert.assertEquals("b-value" + (i+1), result.get(i));
+      assertEquals("b-value" + (i+1), result.get(i));
     }
 
     // iterate from a non exist key
@@ -282,7 +341,7 @@ public class TestMetadataStore {
       result.add(getString(v));
       return true;
     });
-    Assert.assertTrue(result.isEmpty());
+    assertTrue(result.isEmpty());
 
     // iterate from the beginning
     result.clear();
@@ -290,7 +349,7 @@ public class TestMetadataStore {
       result.add(getString(v));
       return true;
     });
-    Assert.assertEquals(20, result.size());
+    assertEquals(20, result.size());
   }
 
   @Test
@@ -299,66 +358,66 @@ public class TestMetadataStore {
 
     // Set empty startKey will return values from beginning.
     result = store.getRangeKVs(null, 5);
-    Assert.assertEquals(5, result.size());
-    Assert.assertEquals("a-value2", getString(result.get(2).getValue()));
+    assertEquals(5, result.size());
+    assertEquals("a-value2", getString(result.get(2).getValue()));
 
     // Empty list if startKey doesn't exist.
     result = store.getRangeKVs(getBytes("a12"), 5);
-    Assert.assertEquals(0, result.size());
+    assertEquals(0, result.size());
 
     // Returns max available entries after a valid startKey.
     result = store.getRangeKVs(getBytes("b0"), MAX_GETRANGE_LENGTH);
-    Assert.assertEquals(10, result.size());
-    Assert.assertEquals("b0", getString(result.get(0).getKey()));
-    Assert.assertEquals("b-value0", getString(result.get(0).getValue()));
+    assertEquals(10, result.size());
+    assertEquals("b0", getString(result.get(0).getKey()));
+    assertEquals("b-value0", getString(result.get(0).getValue()));
     result = store.getRangeKVs(getBytes("b0"), 5);
-    Assert.assertEquals(5, result.size());
+    assertEquals(5, result.size());
 
     // Both startKey and count are honored.
     result = store.getRangeKVs(getBytes("a9"), 2);
-    Assert.assertEquals(2, result.size());
-    Assert.assertEquals("a9", getString(result.get(0).getKey()));
-    Assert.assertEquals("a-value9", getString(result.get(0).getValue()));
-    Assert.assertEquals("b0", getString(result.get(1).getKey()));
-    Assert.assertEquals("b-value0", getString(result.get(1).getValue()));
+    assertEquals(2, result.size());
+    assertEquals("a9", getString(result.get(0).getKey()));
+    assertEquals("a-value9", getString(result.get(0).getValue()));
+    assertEquals("b0", getString(result.get(1).getKey()));
+    assertEquals("b-value0", getString(result.get(1).getValue()));
 
     // Filter keys by prefix.
     // It should returns all "b*" entries.
     MetadataKeyFilter filter1 = new KeyPrefixFilter().addFilter("b");
     result = store.getRangeKVs(null, 100, filter1);
-    Assert.assertEquals(10, result.size());
-    Assert.assertTrue(result.stream().allMatch(entry ->
+    assertEquals(10, result.size());
+    assertTrue(result.stream().allMatch(entry ->
         new String(entry.getKey()).startsWith("b")
     ));
-    Assert.assertEquals(20, filter1.getKeysScannedNum());
-    Assert.assertEquals(10, filter1.getKeysHintedNum());
+    assertEquals(20, filter1.getKeysScannedNum());
+    assertEquals(10, filter1.getKeysHintedNum());
     result = store.getRangeKVs(null, 3, filter1);
-    Assert.assertEquals(3, result.size());
+    assertEquals(3, result.size());
     result = store.getRangeKVs(getBytes("b3"), 1, filter1);
-    Assert.assertEquals("b-value3", getString(result.get(0).getValue()));
+    assertEquals("b-value3", getString(result.get(0).getValue()));
 
     // Define a customized filter that filters keys by suffix.
     // Returns all "*2" entries.
     MetadataKeyFilter filter2 = (preKey, currentKey, nextKey)
         -> getString(currentKey).endsWith("2");
     result = store.getRangeKVs(null, MAX_GETRANGE_LENGTH, filter2);
-    Assert.assertEquals(2, result.size());
-    Assert.assertEquals("a2", getString(result.get(0).getKey()));
-    Assert.assertEquals("b2", getString(result.get(1).getKey()));
+    assertEquals(2, result.size());
+    assertEquals("a2", getString(result.get(0).getKey()));
+    assertEquals("b2", getString(result.get(1).getKey()));
     result = store.getRangeKVs(null, 1, filter2);
-    Assert.assertEquals(1, result.size());
-    Assert.assertEquals("a2", getString(result.get(0).getKey()));
+    assertEquals(1, result.size());
+    assertEquals("a2", getString(result.get(0).getKey()));
 
     // Apply multiple filters.
     result = store.getRangeKVs(null, MAX_GETRANGE_LENGTH, filter1, filter2);
-    Assert.assertEquals(1, result.size());
-    Assert.assertEquals("b2", getString(result.get(0).getKey()));
-    Assert.assertEquals("b-value2", getString(result.get(0).getValue()));
+    assertEquals(1, result.size());
+    assertEquals("b2", getString(result.get(0).getKey()));
+    assertEquals("b-value2", getString(result.get(0).getValue()));
 
     // If filter is null, no effect.
     result = store.getRangeKVs(null, 1, null);
-    Assert.assertEquals(1, result.size());
-    Assert.assertEquals("a0", getString(result.get(0).getKey()));
+    assertEquals(1, result.size());
+    assertEquals("a0", getString(result.get(0).getKey()));
   }
 
   @Test
@@ -368,16 +427,16 @@ public class TestMetadataStore {
     // Suppose to return a2 and b2
     List<Map.Entry<byte[], byte[]>> result =
         store.getRangeKVs(null, MAX_GETRANGE_LENGTH, suffixFilter);
-    Assert.assertEquals(2, result.size());
-    Assert.assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
-    Assert.assertEquals("b2", DFSUtil.bytes2String(result.get(1).getKey()));
+    assertEquals(2, result.size());
+    assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
+    assertEquals("b2", DFSUtil.bytes2String(result.get(1).getKey()));
 
     // Suppose to return just a2, because when it iterates to a3,
     // the filter no long matches and it should stop from there.
     result = store.getSequentialRangeKVs(null,
         MAX_GETRANGE_LENGTH, suffixFilter);
-    Assert.assertEquals(1, result.size());
-    Assert.assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
+    assertEquals(1, result.size());
+    assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
   }
 
   @Test
@@ -385,10 +444,10 @@ public class TestMetadataStore {
     List<Map.Entry<byte[], byte[]>> result = null;
 
     result = store.getRangeKVs(null, 0);
-    Assert.assertEquals(0, result.size());
+    assertEquals(0, result.size());
 
     result = store.getRangeKVs(null, 1);
-    Assert.assertEquals(1, result.size());
+    assertEquals(1, result.size());
 
     // Count less than zero is invalid.
     expectedException.expect(IllegalArgumentException.class);
@@ -401,7 +460,7 @@ public class TestMetadataStore {
     // If startKey is invalid, the returned list should be empty.
     List<Map.Entry<byte[], byte[]>> kvs =
         store.getRangeKVs(getBytes("unknownKey"), MAX_GETRANGE_LENGTH);
-    Assert.assertEquals(kvs.size(), 0);
+    assertEquals(kvs.size(), 0);
   }
 
   @Test
@@ -421,13 +480,13 @@ public class TestMetadataStore {
     dbStore.put(getBytes("key1"), getBytes("value1"));
     dbStore.put(getBytes("key2"), getBytes("value2"));
 
-    Assert.assertFalse(dbStore.isEmpty());
-    Assert.assertTrue(dbDir.exists());
-    Assert.assertTrue(dbDir.listFiles().length > 0);
+    assertFalse(dbStore.isEmpty());
+    assertTrue(dbDir.exists());
+    assertTrue(dbDir.listFiles().length > 0);
 
     dbStore.destroy();
 
-    Assert.assertFalse(dbDir.exists());
+    assertFalse(dbDir.exists());
   }
 
   @Test
@@ -469,7 +528,7 @@ public class TestMetadataStore {
       return it.hasNext() && it.next().equals(getString(key));
     });
 
-    Assert.assertEquals(8, count.get());
+    assertEquals(8, count.get());
   }
 
   @Test
@@ -482,52 +541,51 @@ public class TestMetadataStore {
     } catch (IllegalArgumentException e) {
       exception = e;
     }
-    Assert.assertTrue(
-        exception.getMessage().contains("KeyPrefix: b already rejected"));
+    assertTrue(exception.getMessage().contains("KeyPrefix: b already " +
+        "rejected"));
 
     try {
       new KeyPrefixFilter().addFilter("b0").addFilter("b", true);
     } catch (IllegalArgumentException e) {
       exception = e;
     }
-    Assert.assertTrue(
-        exception.getMessage().contains("KeyPrefix: b already accepted"));
+    assertTrue(exception.getMessage().contains("KeyPrefix: b already " +
+        "accepted"));
 
     try {
       new KeyPrefixFilter().addFilter("b", true).addFilter("b0");
     } catch (IllegalArgumentException e) {
       exception = e;
     }
-    Assert.assertTrue(
-        exception.getMessage().contains("KeyPrefix: b0 already rejected"));
+    assertTrue(exception.getMessage().contains("KeyPrefix: b0 already " +
+        "rejected"));
 
     try {
       new KeyPrefixFilter().addFilter("b").addFilter("b0", true);
     } catch (IllegalArgumentException e) {
       exception = e;
     }
-    Assert.assertTrue(
-        exception.getMessage().contains("KeyPrefix: b0 already accepted"));
+    assertTrue(exception.getMessage().contains("KeyPrefix: b0 already " +
+        "accepted"));
 
     MetadataKeyFilter filter1 = new KeyPrefixFilter(true)
             .addFilter("a0")
             .addFilter("a1")
             .addFilter("b", true);
     result = store.getRangeKVs(null, 100, filter1);
-    Assert.assertEquals(2, result.size());
-    Assert.assertTrue(result.stream()
-        .anyMatch(entry -> new String(entry.getKey()).startsWith("a0"))
-        && result.stream()
-        .anyMatch(entry -> new String(entry.getKey()).startsWith("a1")));
+    assertEquals(2, result.size());
+    assertTrue(result.stream().anyMatch(entry -> new String(entry.getKey())
+        .startsWith("a0")) && result.stream().anyMatch(entry -> new String(
+            entry.getKey()).startsWith("a1")));
 
     filter1 = new KeyPrefixFilter(true).addFilter("b", true);
     result = store.getRangeKVs(null, 100, filter1);
-    Assert.assertEquals(0, result.size());
+    assertEquals(0, result.size());
 
     filter1 = new KeyPrefixFilter().addFilter("b", true);
     result = store.getRangeKVs(null, 100, filter1);
-    Assert.assertEquals(10, result.size());
-    Assert.assertTrue(result.stream()
-        .allMatch(entry -> new String(entry.getKey()).startsWith("a")));
+    assertEquals(10, result.size());
+    assertTrue(result.stream().allMatch(entry -> new String(entry.getKey())
+        .startsWith("a")));
   }
 }


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


[41/50] hadoop git commit: HADOOP-15607. AliyunOSS: fix duplicated partNumber issue in AliyunOSSBlockOutputStream. Contributed by Jinhu Wu.

Posted by eh...@apache.org.
HADOOP-15607. AliyunOSS: fix duplicated partNumber issue in AliyunOSSBlockOutputStream. Contributed by Jinhu Wu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0857f116
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0857f116
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0857f116

Branch: refs/heads/HDFS-12090
Commit: 0857f116b754d83d3c540cd6f989087af24fef27
Parents: 007e6f5
Author: Sammi Chen <sa...@intel.com>
Authored: Mon Jul 30 10:53:44 2018 +0800
Committer: Sammi Chen <sa...@intel.com>
Committed: Mon Jul 30 10:53:44 2018 +0800

----------------------------------------------------------------------
 .../aliyun/oss/AliyunOSSBlockOutputStream.java  | 59 ++++++++++++--------
 .../fs/aliyun/oss/AliyunOSSFileSystemStore.java |  2 +
 .../oss/TestAliyunOSSBlockOutputStream.java     | 12 +++-
 3 files changed, 49 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0857f116/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSBlockOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSBlockOutputStream.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSBlockOutputStream.java
index 12d551b..0a833b2 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSBlockOutputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSBlockOutputStream.java
@@ -33,7 +33,9 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 
@@ -50,7 +52,7 @@ public class AliyunOSSBlockOutputStream extends OutputStream {
   private boolean closed;
   private String key;
   private File blockFile;
-  private List<File> blockFiles = new ArrayList<>();
+  private Map<Integer, File> blockFiles = new HashMap<>();
   private long blockSize;
   private int blockId = 0;
   private long blockWritten = 0L;
@@ -94,8 +96,9 @@ public class AliyunOSSBlockOutputStream extends OutputStream {
 
     blockStream.flush();
     blockStream.close();
-    if (!blockFiles.contains(blockFile)) {
-      blockFiles.add(blockFile);
+    if (!blockFiles.values().contains(blockFile)) {
+      blockId++;
+      blockFiles.put(blockId, blockFile);
     }
 
     try {
@@ -107,7 +110,7 @@ public class AliyunOSSBlockOutputStream extends OutputStream {
           ListenableFuture<PartETag> partETagFuture =
               executorService.submit(() -> {
                 PartETag partETag = store.uploadPart(blockFile, key, uploadId,
-                    blockId + 1);
+                    blockId);
                 return partETag;
               });
           partETagsFutures.add(partETagFuture);
@@ -120,11 +123,7 @@ public class AliyunOSSBlockOutputStream extends OutputStream {
         store.completeMultipartUpload(key, uploadId, partETags);
       }
     } finally {
-      for (File tFile: blockFiles) {
-        if (tFile.exists() && !tFile.delete()) {
-          LOG.warn("Failed to delete temporary file {}", tFile);
-        }
-      }
+      removePartFiles();
       closed = true;
     }
   }
@@ -141,38 +140,52 @@ public class AliyunOSSBlockOutputStream extends OutputStream {
     if (closed) {
       throw new IOException("Stream closed.");
     }
-    try {
-      blockStream.write(b, off, len);
-      blockWritten += len;
-      if (blockWritten >= blockSize) {
-        uploadCurrentPart();
-        blockWritten = 0L;
+    blockStream.write(b, off, len);
+    blockWritten += len;
+    if (blockWritten >= blockSize) {
+      uploadCurrentPart();
+      blockWritten = 0L;
+    }
+  }
+
+  private void removePartFiles() throws IOException {
+    for (ListenableFuture<PartETag> partETagFuture : partETagsFutures) {
+      if (!partETagFuture.isDone()) {
+        continue;
       }
-    } finally {
-      for (File tFile: blockFiles) {
-        if (tFile.exists() && !tFile.delete()) {
-          LOG.warn("Failed to delete temporary file {}", tFile);
+
+      try {
+        File blockFile = blockFiles.get(partETagFuture.get().getPartNumber());
+        if (blockFile != null && blockFile.exists() && !blockFile.delete()) {
+          LOG.warn("Failed to delete temporary file {}", blockFile);
         }
+      } catch (InterruptedException | ExecutionException e) {
+        throw new IOException(e);
       }
     }
   }
 
   private void uploadCurrentPart() throws IOException {
-    blockFiles.add(blockFile);
     blockStream.flush();
     blockStream.close();
     if (blockId == 0) {
       uploadId = store.getUploadId(key);
     }
+
+    blockId++;
+    blockFiles.put(blockId, blockFile);
+
+    File currentFile = blockFile;
+    int currentBlockId = blockId;
     ListenableFuture<PartETag> partETagFuture =
         executorService.submit(() -> {
-          PartETag partETag = store.uploadPart(blockFile, key, uploadId,
-              blockId + 1);
+          PartETag partETag = store.uploadPart(currentFile, key, uploadId,
+              currentBlockId);
           return partETag;
         });
     partETagsFutures.add(partETagFuture);
+    removePartFiles();
     blockFile = newBlockFile();
-    blockId++;
     blockStream = new BufferedOutputStream(new FileOutputStream(blockFile));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0857f116/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
index 5e21759..dc5f99ee 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
@@ -450,6 +450,8 @@ public class AliyunOSSFileSystemStore {
       request.setRange(byteStart, byteEnd);
       return ossClient.getObject(request).getObjectContent();
     } catch (OSSException | ClientException e) {
+      LOG.error("Exception thrown when store retrieves key: "
+              + key + ", exception: " + e);
       return null;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0857f116/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSBlockOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSBlockOutputStream.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSBlockOutputStream.java
index 365d931..6fe6f03 100644
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSBlockOutputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSBlockOutputStream.java
@@ -31,6 +31,7 @@ import org.junit.rules.Timeout;
 import java.io.IOException;
 
 import static org.apache.hadoop.fs.aliyun.oss.Constants.MULTIPART_UPLOAD_PART_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.IO_CHUNK_BUFFER_SIZE;
 
 /**
  * Tests regular and multi-part upload functionality for
@@ -48,7 +49,10 @@ public class TestAliyunOSSBlockOutputStream {
   public void setUp() throws Exception {
     Configuration conf = new Configuration();
     conf.setLong(Constants.MIN_MULTIPART_UPLOAD_THRESHOLD_KEY, 5 * 1024 * 1024);
-    conf.setInt(Constants.MULTIPART_UPLOAD_PART_SIZE_KEY, 5 * 1024 * 1024);
+    conf.setInt(Constants.MULTIPART_UPLOAD_PART_SIZE_KEY, 1024 * 1024);
+    conf.setInt(IO_CHUNK_BUFFER_SIZE,
+        conf.getInt(Constants.MULTIPART_UPLOAD_PART_SIZE_KEY, 0));
+    conf.setInt(Constants.UPLOAD_ACTIVE_BLOCKS_KEY, 20);
     fs = AliyunOSSTestUtils.createTestFileSystem(conf);
   }
 
@@ -85,6 +89,12 @@ public class TestAliyunOSSBlockOutputStream {
   }
 
   @Test
+  public void testMultiPartUploadConcurrent() throws IOException {
+    ContractTestUtils.createAndVerifyFile(fs, getTestPath(),
+        50 * 1024 * 1024 - 1);
+  }
+
+  @Test
   public void testHugeUpload() throws IOException {
     ContractTestUtils.createAndVerifyFile(fs, getTestPath(),
         MULTIPART_UPLOAD_PART_SIZE_DEFAULT - 1);


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


[25/50] hadoop git commit: YARN-8545. Return allocated resource to RM for failed container. Contributed by Chandni Singh

Posted by eh...@apache.org.
YARN-8545.  Return allocated resource to RM for failed container.
            Contributed by Chandni Singh


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/40fad328
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/40fad328
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/40fad328

Branch: refs/heads/HDFS-12090
Commit: 40fad32824d2f8f960c779d78357e62103453da0
Parents: d70d845
Author: Eric Yang <ey...@apache.org>
Authored: Thu Jul 26 18:22:57 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Thu Jul 26 18:22:57 2018 -0400

----------------------------------------------------------------------
 .../hadoop/yarn/service/ServiceScheduler.java   |  3 +-
 .../yarn/service/component/Component.java       | 42 +++++++++++---------
 .../component/instance/ComponentInstance.java   | 21 +++++++---
 .../instance/ComponentInstanceEvent.java        |  2 +
 .../containerlaunch/ContainerLaunchService.java | 12 ++++--
 .../hadoop/yarn/service/MockServiceAM.java      | 34 +++++++++++++++-
 .../hadoop/yarn/service/TestServiceAM.java      | 35 ++++++++++++++++
 .../yarn/service/component/TestComponent.java   |  3 +-
 .../instance/TestComponentInstance.java         | 26 ++++++------
 9 files changed, 135 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
index d3e8e4f..cfaf356 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
@@ -687,7 +687,8 @@ public class ServiceScheduler extends CompositeService {
         }
         ComponentEvent event =
             new ComponentEvent(instance.getCompName(), CONTAINER_COMPLETED)
-                .setStatus(status).setInstance(instance);
+                .setStatus(status).setInstance(instance)
+                .setContainerId(containerId);
         dispatcher.getEventHandler().handle(event);
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
index a1ee796..aaa23da 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.service.component;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
@@ -518,10 +519,10 @@ public class Component implements EventHandler<ComponentEvent> {
   private static class ContainerCompletedTransition extends BaseTransition {
     @Override
     public void transition(Component component, ComponentEvent event) {
-
+      Preconditions.checkNotNull(event.getContainerId());
       component.updateMetrics(event.getStatus());
       component.dispatcher.getEventHandler().handle(
-          new ComponentInstanceEvent(event.getStatus().getContainerId(), STOP)
+          new ComponentInstanceEvent(event.getContainerId(), STOP)
               .setStatus(event.getStatus()));
 
       ComponentRestartPolicy restartPolicy =
@@ -784,28 +785,33 @@ public class Component implements EventHandler<ComponentEvent> {
   }
 
   private void updateMetrics(ContainerStatus status) {
-    switch (status.getExitStatus()) {
-    case SUCCESS:
-      componentMetrics.containersSucceeded.incr();
-      scheduler.getServiceMetrics().containersSucceeded.incr();
-      return;
-    case PREEMPTED:
-      componentMetrics.containersPreempted.incr();
-      scheduler.getServiceMetrics().containersPreempted.incr();
-      break;
-    case DISKS_FAILED:
-      componentMetrics.containersDiskFailure.incr();
-      scheduler.getServiceMetrics().containersDiskFailure.incr();
-      break;
-    default:
-      break;
+    //when a container preparation fails while building launch context, then
+    //the container status may not exist.
+    if (status != null) {
+      switch (status.getExitStatus()) {
+        case SUCCESS:
+          componentMetrics.containersSucceeded.incr();
+          scheduler.getServiceMetrics().containersSucceeded.incr();
+          return;
+        case PREEMPTED:
+          componentMetrics.containersPreempted.incr();
+          scheduler.getServiceMetrics().containersPreempted.incr();
+          break;
+        case DISKS_FAILED:
+          componentMetrics.containersDiskFailure.incr();
+          scheduler.getServiceMetrics().containersDiskFailure.incr();
+          break;
+        default:
+          break;
+      }
     }
 
     // containersFailed include preempted, disks_failed etc.
     componentMetrics.containersFailed.incr();
     scheduler.getServiceMetrics().containersFailed.incr();
 
-    if (Apps.shouldCountTowardsNodeBlacklisting(status.getExitStatus())) {
+    if (status != null && Apps.shouldCountTowardsNodeBlacklisting(
+        status.getExitStatus())) {
       String host = scheduler.getLiveInstances().get(status.getContainerId())
           .getNodeId().getHost();
       failureTracker.incNodeFailure(host);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
index 64f35d3..3499d92 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
@@ -76,6 +76,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
     Comparable<ComponentInstance> {
   private static final Logger LOG =
       LoggerFactory.getLogger(ComponentInstance.class);
+  private static final String FAILED_BEFORE_LAUNCH_DIAG =
+      "failed before launch";
 
   private  StateMachine<ComponentInstanceState, ComponentInstanceEventType,
       ComponentInstanceEvent> stateMachine;
@@ -241,7 +243,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
 
   @VisibleForTesting
   static void handleComponentInstanceRelaunch(
-      ComponentInstance compInstance, ComponentInstanceEvent event) {
+      ComponentInstance compInstance, ComponentInstanceEvent event,
+      boolean failureBeforeLaunch) {
     Component comp = compInstance.getComponent();
 
     // Do we need to relaunch the service?
@@ -257,8 +260,10 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
               + ": {} completed. Reinsert back to pending list and requested " +
               "a new container." + System.lineSeparator() +
               " exitStatus={}, diagnostics={}.",
-          event.getContainerId(), event.getStatus().getExitStatus(),
-          event.getStatus().getDiagnostics());
+          event.getContainerId(), failureBeforeLaunch ? null :
+              event.getStatus().getExitStatus(),
+          failureBeforeLaunch ? FAILED_BEFORE_LAUNCH_DIAG :
+              event.getStatus().getDiagnostics());
     } else {
       // When no relaunch, update component's #succeeded/#failed
       // instances.
@@ -297,8 +302,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
 
       Component comp = compInstance.component;
       String containerDiag =
-          compInstance.getCompInstanceId() + ": " + event.getStatus()
-              .getDiagnostics();
+          compInstance.getCompInstanceId() + ": " + (failedBeforeLaunching ?
+              FAILED_BEFORE_LAUNCH_DIAG : event.getStatus().getDiagnostics());
       compInstance.diagnostics.append(containerDiag + System.lineSeparator());
       compInstance.cancelContainerStatusRetriever();
       if (compInstance.getState().equals(ComponentInstanceState.UPGRADING)) {
@@ -312,6 +317,9 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
       boolean shouldFailService = false;
 
       final ServiceScheduler scheduler = comp.getScheduler();
+      scheduler.getAmRMClient().releaseAssignedContainer(
+          event.getContainerId());
+
       // Check if it exceeds the failure threshold, but only if health threshold
       // monitor is not enabled
       if (!comp.isHealthThresholdMonitorEnabled()
@@ -352,7 +360,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
 
       // According to component restart policy, handle container restart
       // or finish the service (if all components finished)
-      handleComponentInstanceRelaunch(compInstance, event);
+      handleComponentInstanceRelaunch(compInstance, event,
+          failedBeforeLaunching);
 
       if (shouldFailService) {
         scheduler.getTerminationHandler().terminate(-1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstanceEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstanceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstanceEvent.java
index 707b034..889da6e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstanceEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstanceEvent.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.service.component.instance;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.event.AbstractEvent;
@@ -32,6 +33,7 @@ public class ComponentInstanceEvent
   public ComponentInstanceEvent(ContainerId containerId,
       ComponentInstanceEventType componentInstanceEventType) {
     super(componentInstanceEventType);
+    Preconditions.checkNotNull(containerId);
     this.id = containerId;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
index 084c721..f674e0d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
@@ -22,8 +22,11 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.service.ServiceContext;
 import org.apache.hadoop.yarn.service.api.records.Artifact;
+import org.apache.hadoop.yarn.service.component.ComponentEvent;
+import org.apache.hadoop.yarn.service.component.ComponentEventType;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
 import org.apache.hadoop.yarn.service.provider.ProviderService;
 import org.apache.hadoop.yarn.service.provider.ProviderFactory;
@@ -116,9 +119,12 @@ public class ContainerLaunchService extends AbstractService{
                   launcher.completeContainerLaunch(), true);
         }
       } catch (Exception e) {
-        LOG.error(instance.getCompInstanceId()
-            + ": Failed to launch container. ", e);
-
+        LOG.error("{}: Failed to launch container.",
+            instance.getCompInstanceId(), e);
+        ComponentEvent event = new ComponentEvent(instance.getCompName(),
+            ComponentEventType.CONTAINER_COMPLETED)
+            .setInstance(instance).setContainerId(container.getId());
+        context.scheduler.getDispatcher().getEventHandler().handle(event);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java
index 4a75aef..729287c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java
@@ -68,6 +68,7 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeoutException;
 
@@ -99,6 +100,8 @@ public class MockServiceAM extends ServiceMaster {
   private Map<ContainerId, ContainerStatus> containerStatuses =
       new ConcurrentHashMap<>();
 
+  private Set<ContainerId> releasedContainers = ConcurrentHashMap.newKeySet();
+
   private Credentials amCreds;
 
   public MockServiceAM(Service service) {
@@ -223,6 +226,13 @@ public class MockServiceAM extends ServiceMaster {
             return response;
           }
 
+          @Override
+          public synchronized void releaseAssignedContainer(
+              ContainerId containerId) {
+            releasedContainers.add(containerId);
+            super.releaseAssignedContainer(containerId);
+          }
+
           @Override public void unregisterApplicationMaster(
               FinalApplicationStatus appStatus, String appMessage,
               String appTrackingUrl) {
@@ -288,7 +298,7 @@ public class MockServiceAM extends ServiceMaster {
   }
 
   /**
-   *
+   * Creates a mock container and container ID and feeds to the component.
    * @param service The service for the component
    * @param id The id for the container
    * @param compName The component to which the container is fed
@@ -297,6 +307,18 @@ public class MockServiceAM extends ServiceMaster {
   public Container feedContainerToComp(Service service, int id,
       String compName) {
     ContainerId containerId = createContainerId(id);
+    return feedContainerToComp(service, containerId, compName);
+  }
+
+  /**
+   * Feeds the container to the component.
+   * @param service The service for the component
+   * @param containerId container id
+   * @param compName The component to which the container is fed
+   * @return
+   */
+  public Container feedContainerToComp(Service service, ContainerId containerId,
+      String compName) {
     Container container = createContainer(containerId, compName);
     synchronized (feedContainers) {
       feedContainers.add(container);
@@ -423,4 +445,14 @@ public class MockServiceAM extends ServiceMaster {
     }
     return ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
   }
+
+  /**
+   * Waits for the container to get released
+   * @param containerId           ContainerId
+   */
+  public void waitForContainerToRelease(ContainerId containerId)
+      throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(() -> releasedContainers.contains(containerId),
+        1000, 9990000);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java
index e9478f0..21e93fa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.service;
 
+import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableMap;
 import org.apache.commons.io.FileUtils;
 import org.apache.curator.test.TestingCluster;
@@ -391,4 +392,38 @@ public class TestServiceAM extends ServiceTestUtils{
         .equals("newer.host"), 2000, 200000);
     am.stop();
   }
+
+  // Test to verify that the containers are released and the
+  // component instance is added to the pending queue when building the launch
+  // context fails.
+  @Test(timeout = 9990000)
+  public void testContainersReleasedWhenPreLaunchFails()
+      throws Exception {
+    ApplicationId applicationId = ApplicationId.newInstance(
+        System.currentTimeMillis(), 1);
+    Service exampleApp = new Service();
+    exampleApp.setId(applicationId.toString());
+    exampleApp.setVersion("v1");
+    exampleApp.setName("testContainersReleasedWhenPreLaunchFails");
+
+    Component compA = createComponent("compa", 1, "pwd");
+    Artifact artifact = new Artifact();
+    artifact.setType(Artifact.TypeEnum.TARBALL);
+    compA.artifact(artifact);
+    exampleApp.addComponent(compA);
+
+    MockServiceAM am = new MockServiceAM(exampleApp);
+    am.init(conf);
+    am.start();
+
+    ContainerId containerId = am.createContainerId(1);
+
+    // allocate a container
+    am.feedContainerToComp(exampleApp, containerId, "compa");
+    am.waitForContainerToRelease(containerId);
+
+    Assert.assertEquals(1,
+        am.getComponent("compa").getPendingInstances().size());
+    am.stop();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponent.java
index d5fb941..2e17c7f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponent.java
@@ -178,7 +178,8 @@ public class TestComponent {
           org.apache.hadoop.yarn.api.records.ContainerState.COMPLETE,
           "successful", 0);
       comp.handle(new ComponentEvent(comp.getName(),
-          ComponentEventType.CONTAINER_COMPLETED).setStatus(containerStatus));
+          ComponentEventType.CONTAINER_COMPLETED).setStatus(containerStatus)
+          .setContainerId(instanceContainer.getId()));
       componentInstance.handle(
           new ComponentInstanceEvent(componentInstance.getContainer().getId(),
               ComponentInstanceEventType.STOP).setStatus(containerStatus));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/instance/TestComponentInstance.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/instance/TestComponentInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/instance/TestComponentInstance.java
index 0e7816c..bb480ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/instance/TestComponentInstance.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/instance/TestComponentInstance.java
@@ -245,7 +245,7 @@ public class TestComponentInstance {
         comp.getAllComponentInstances().iterator().next();
 
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent);
+        componentInstanceEvent, false);
 
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, never()).markAsFailed(any(ComponentInstance.class));
@@ -262,7 +262,7 @@ public class TestComponentInstance {
     componentInstance = comp.getAllComponentInstances().iterator().next();
     containerStatus.setExitStatus(1);
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent);
+        componentInstanceEvent, false);
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, never()).markAsFailed(any(ComponentInstance.class));
     verify(comp, times(1)).reInsertPendingInstance(
@@ -286,7 +286,7 @@ public class TestComponentInstance {
     when(comp.getNumSucceededInstances()).thenReturn(new Long(1));
 
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent);
+        componentInstanceEvent, false);
     verify(comp, times(1)).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, never()).markAsFailed(any(ComponentInstance.class));
     verify(comp, times(0)).reInsertPendingInstance(
@@ -304,7 +304,7 @@ public class TestComponentInstance {
 
     when(comp.getNumFailedInstances()).thenReturn(new Long(1));
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent);
+        componentInstanceEvent, false);
 
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, times(1)).markAsFailed(any(ComponentInstance.class));
@@ -323,7 +323,7 @@ public class TestComponentInstance {
     componentInstance = comp.getAllComponentInstances().iterator().next();
     containerStatus.setExitStatus(1);
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent);
+        componentInstanceEvent, false);
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, never()).markAsFailed(any(ComponentInstance.class));
     verify(comp, times(1)).reInsertPendingInstance(
@@ -340,7 +340,7 @@ public class TestComponentInstance {
     componentInstance = comp.getAllComponentInstances().iterator().next();
     containerStatus.setExitStatus(1);
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent);
+        componentInstanceEvent, false);
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, times(1)).markAsFailed(any(ComponentInstance.class));
     verify(comp, times(0)).reInsertPendingInstance(
@@ -363,7 +363,7 @@ public class TestComponentInstance {
     containerStatus.setExitStatus(1);
     ComponentInstance commponentInstance = iter.next();
     ComponentInstance.handleComponentInstanceRelaunch(commponentInstance,
-        componentInstanceEvent);
+        componentInstanceEvent, false);
 
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, never()).markAsFailed(any(ComponentInstance.class));
@@ -404,7 +404,7 @@ public class TestComponentInstance {
       when(component2Instance.getComponent().getNumFailedInstances())
           .thenReturn(new Long(failed2Instances.size()));
       ComponentInstance.handleComponentInstanceRelaunch(component2Instance,
-          componentInstanceEvent);
+          componentInstanceEvent, false);
     }
 
     Map<String, ComponentInstance> failed1Instances = new HashMap<>();
@@ -418,7 +418,7 @@ public class TestComponentInstance {
       when(component1Instance.getComponent().getNumFailedInstances())
           .thenReturn(new Long(failed1Instances.size()));
       ComponentInstance.handleComponentInstanceRelaunch(component1Instance,
-          componentInstanceEvent);
+          componentInstanceEvent, false);
     }
 
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
@@ -458,7 +458,7 @@ public class TestComponentInstance {
       when(component2Instance.getComponent().getNumSucceededInstances())
           .thenReturn(new Long(succeeded2Instances.size()));
       ComponentInstance.handleComponentInstanceRelaunch(component2Instance,
-          componentInstanceEvent);
+          componentInstanceEvent, false);
     }
 
     Map<String, ComponentInstance> succeeded1Instances = new HashMap<>();
@@ -471,7 +471,7 @@ public class TestComponentInstance {
       when(component1Instance.getComponent().getNumSucceededInstances())
           .thenReturn(new Long(succeeded1Instances.size()));
       ComponentInstance.handleComponentInstanceRelaunch(component1Instance,
-          componentInstanceEvent);
+          componentInstanceEvent, false);
     }
 
     verify(comp, times(2)).markAsSucceeded(any(ComponentInstance.class));
@@ -500,7 +500,7 @@ public class TestComponentInstance {
 
     for (ComponentInstance component2Instance : component2Instances) {
       ComponentInstance.handleComponentInstanceRelaunch(component2Instance,
-          componentInstanceEvent);
+          componentInstanceEvent, false);
     }
 
     succeeded1Instances = new HashMap<>();
@@ -511,7 +511,7 @@ public class TestComponentInstance {
       when(component1Instance.getComponent().getSucceededInstances())
           .thenReturn(succeeded1Instances.values());
       ComponentInstance.handleComponentInstanceRelaunch(component1Instance,
-          componentInstanceEvent);
+          componentInstanceEvent, false);
     }
 
     verify(comp, times(2)).markAsSucceeded(any(ComponentInstance.class));


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


[16/50] hadoop git commit: HADOOP-15395. DefaultImpersonationProvider fails to parse proxy user config if username has . in it. Contributed by Ajay Kumar.

Posted by eh...@apache.org.
HADOOP-15395. DefaultImpersonationProvider fails to parse proxy user config if username has . in it. Contributed by Ajay Kumar.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5f0b9243
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5f0b9243
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5f0b9243

Branch: refs/heads/HDFS-12090
Commit: 5f0b924360b345f491c2d6693882f1069c7f3508
Parents: 3c4fbc6
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Wed Jul 25 21:09:11 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Wed Jul 25 21:09:11 2018 +0530

----------------------------------------------------------------------
 .../authorize/DefaultImpersonationProvider.java |   4 +-
 .../TestDefaultImpersonationProvider.java       | 100 +++++++++++++++++++
 2 files changed, 102 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f0b9243/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/DefaultImpersonationProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/DefaultImpersonationProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/DefaultImpersonationProvider.java
index 26cd7ab..b766d5c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/DefaultImpersonationProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/DefaultImpersonationProvider.java
@@ -75,9 +75,9 @@ public class DefaultImpersonationProvider implements ImpersonationProvider {
     //   $configPrefix.[ANY].hosts
     //
     String prefixRegEx = configPrefix.replace(".", "\\.");
-    String usersGroupsRegEx = prefixRegEx + "[^.]*(" +
+    String usersGroupsRegEx = prefixRegEx + "[\\S]*(" +
         Pattern.quote(CONF_USERS) + "|" + Pattern.quote(CONF_GROUPS) + ")";
-    String hostsRegEx = prefixRegEx + "[^.]*" + Pattern.quote(CONF_HOSTS);
+    String hostsRegEx = prefixRegEx + "[\\S]*" + Pattern.quote(CONF_HOSTS);
 
   // get list of users and groups per proxyuser
     Map<String,String> allMatchKeys = 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f0b9243/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestDefaultImpersonationProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestDefaultImpersonationProvider.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestDefaultImpersonationProvider.java
new file mode 100644
index 0000000..ef86697
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestDefaultImpersonationProvider.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.security.authorize;
+
+import static org.mockito.Mockito.when;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.mockito.Mockito;
+
+/**
+ * Test class for @DefaultImpersonationProvider
+ */
+public class TestDefaultImpersonationProvider {
+
+  private String proxyUser;
+  private String user;
+  private DefaultImpersonationProvider provider;
+  private UserGroupInformation userGroupInformation = Mockito
+      .mock(UserGroupInformation.class);
+  private UserGroupInformation realUserUGI = Mockito
+      .mock(UserGroupInformation.class);
+  private Configuration conf;
+  @Rule
+  public Timeout globalTimeout = new Timeout(10000);
+
+  @Before
+  public void setup() {
+    conf = new Configuration();
+    provider = new DefaultImpersonationProvider();
+
+    // Setup 3 proxy users
+    conf.set("hadoop.proxyuser.fakeuser.groups", "*");
+    conf.set("hadoop.proxyuser.fakeuser.hosts", "*");
+    conf.set("hadoop.proxyuser.test.user.groups", "*");
+    conf.set("hadoop.proxyuser.test.user.hosts", "*");
+    conf.set("hadoop.proxyuser.test user2.groups", "*");
+    conf.set("hadoop.proxyuser.test user2.hosts", "*");
+    provider.setConf(conf);
+    provider.init(ProxyUsers.CONF_HADOOP_PROXYUSER);
+  }
+
+  @Test
+  public void testAuthorizationSuccess() throws AuthorizationException {
+    proxyUser = "fakeuser";
+    user = "dummyUser";
+    when(realUserUGI.getShortUserName()).thenReturn(proxyUser);
+    when(userGroupInformation.getRealUser()).thenReturn(realUserUGI);
+    provider.authorize(userGroupInformation, "2.2.2.2");
+
+    user = "somerandomuser";
+    proxyUser = "test.user";
+    when(realUserUGI.getShortUserName()).thenReturn(proxyUser);
+    when(userGroupInformation.getRealUser()).thenReturn(realUserUGI);
+    provider.authorize(userGroupInformation, "2.2.2.2");
+  }
+
+  @Test
+  public void testAuthorizationFailure() throws Exception {
+    user = "dummyUser";
+    proxyUser = "test user2";
+    when(realUserUGI.getShortUserName()).thenReturn(proxyUser);
+    when(realUserUGI.getUserName()).thenReturn(proxyUser);
+    when(userGroupInformation.getUserName()).thenReturn(user);
+    when(userGroupInformation.getRealUser()).thenReturn(realUserUGI);
+    LambdaTestUtils.intercept(AuthorizationException.class, "User: "
+        + proxyUser + " is not allowed to impersonate " + user, () ->
+        provider.authorize(userGroupInformation, "2.2.2.2"));
+  }
+
+  @After
+  public void clear() {
+    provider = null;
+    conf = null;
+    userGroupInformation = null;
+    realUserUGI = null;
+  }
+
+}


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


[49/50] hadoop git commit: YARN-7974. Allow updating application tracking url after registration. Contributed by Jonathan Hung

Posted by eh...@apache.org.
YARN-7974. Allow updating application tracking url after registration. Contributed by Jonathan Hung


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3e06a5dc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3e06a5dc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3e06a5dc

Branch: refs/heads/HDFS-12090
Commit: 3e06a5dcea8224ba71aec284df23b47d536bb06d
Parents: ee53602
Author: Jonathan Hung <jh...@linkedin.com>
Authored: Mon Jul 30 17:41:01 2018 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Mon Jul 30 17:44:18 2018 -0700

----------------------------------------------------------------------
 .../api/protocolrecords/AllocateRequest.java    | 47 +++++++++++-
 .../src/main/proto/yarn_service_protos.proto    |  1 +
 .../hadoop/yarn/client/api/AMRMClient.java      | 11 +++
 .../yarn/client/api/async/AMRMClientAsync.java  | 11 +++
 .../api/async/impl/AMRMClientAsyncImpl.java     |  5 ++
 .../yarn/client/api/impl/AMRMClientImpl.java    | 11 +++
 .../yarn/client/api/impl/TestAMRMClient.java    | 77 ++++++++++++++++++++
 .../impl/pb/AllocateRequestPBImpl.java          | 27 ++++++-
 .../resourcemanager/DefaultAMSProcessor.java    |  2 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         | 20 +++++
 .../event/RMAppAttemptStatusupdateEvent.java    | 11 +++
 .../TestApplicationMasterService.java           | 34 +++++++++
 .../server/resourcemanager/TestRMRestart.java   | 45 ++++++++++++
 13 files changed, 298 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e06a5dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
index eee50e3..799088b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
@@ -73,7 +73,21 @@ public abstract class AllocateRequest {
         .releaseList(containersToBeReleased)
         .resourceBlacklistRequest(resourceBlacklistRequest).build();
   }
-  
+
+  @Public
+  @Unstable
+  public static AllocateRequest newInstance(int responseID, float appProgress,
+      List<ResourceRequest> resourceAsk,
+      List<ContainerId> containersToBeReleased,
+      ResourceBlacklistRequest resourceBlacklistRequest,
+      String trackingUrl) {
+    return AllocateRequest.newBuilder().responseId(responseID)
+        .progress(appProgress).askList(resourceAsk)
+        .releaseList(containersToBeReleased)
+        .resourceBlacklistRequest(resourceBlacklistRequest)
+        .trackingUrl(trackingUrl).build();
+  }
+
   @Public
   @Unstable
   public static AllocateRequest newInstance(int responseID, float appProgress,
@@ -240,6 +254,22 @@ public abstract class AllocateRequest {
       List<SchedulingRequest> schedulingRequests) {
   }
 
+  /**
+   * Get the tracking url update for this heartbeat.
+   * @return tracking url to update this application with
+   */
+  @Public
+  @Unstable
+  public abstract String getTrackingUrl();
+
+  /**
+   * Set the new tracking url for this application.
+   * @param trackingUrl the new tracking url
+   */
+  @Public
+  @Unstable
+  public abstract void setTrackingUrl(String trackingUrl);
+
   @Public
   @Unstable
   public static AllocateRequestBuilder newBuilder() {
@@ -356,6 +386,19 @@ public abstract class AllocateRequest {
     }
 
     /**
+     * Set the <code>trackingUrl</code> of the request.
+     * @see AllocateRequest#setTrackingUrl(String)
+     * @param trackingUrl new tracking url
+     * @return {@link AllocateRequestBuilder}
+     */
+    @Public
+    @Unstable
+    public AllocateRequestBuilder trackingUrl(String trackingUrl) {
+      allocateRequest.setTrackingUrl(trackingUrl);
+      return this;
+    }
+
+    /**
      * Return generated {@link AllocateRequest} object.
      * @return {@link AllocateRequest}
      */
@@ -365,4 +408,4 @@ public abstract class AllocateRequest {
       return allocateRequest;
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e06a5dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index 92a65ad..acd452d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -92,6 +92,7 @@ message AllocateRequestProto {
   optional float progress = 5;
   repeated UpdateContainerRequestProto update_requests = 7;
   repeated SchedulingRequestProto scheduling_requests = 10;
+  optional string tracking_url = 11;
 }
 
 message NMTokenProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e06a5dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
index 32aa21d..59b3353 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
@@ -805,6 +805,17 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
   }
 
   /**
+   * Update application's tracking url on next heartbeat.
+   *
+   * @param trackingUrl new tracking url for this application
+   */
+  @Public
+  @InterfaceStability.Unstable
+  public void updateTrackingUrl(String trackingUrl) {
+    // Unimplemented.
+  }
+
+  /**
    * Wait for <code>check</code> to return true for each 1000 ms.
    * See also {@link #waitFor(java.util.function.Supplier, int)}
    * and {@link #waitFor(java.util.function.Supplier, int, int)}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e06a5dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
index 0af687b..3dd2f71 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
@@ -413,6 +413,17 @@ extends AbstractService {
                                        List<String> blacklistRemovals);
 
   /**
+   * Update application's tracking url on next heartbeat.
+   *
+   * @param trackingUrl new tracking url for this application
+   */
+  @Public
+  @Unstable
+  public void updateTrackingUrl(String trackingUrl) {
+    // Unimplemented.
+  }
+
+  /**
    * Wait for <code>check</code> to return true for each 1000 ms.
    * See also {@link #waitFor(java.util.function.Supplier, int)}
    * and {@link #waitFor(java.util.function.Supplier, int, int)}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e06a5dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
index 4f04b66..3cf2c34 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
@@ -286,6 +286,11 @@ extends AMRMClientAsync<T> {
                               List<String> blacklistRemovals) {
     client.updateBlacklist(blacklistAdditions, blacklistRemovals);
   }
+
+  @Override
+  public void updateTrackingUrl(String trackingUrl) {
+    client.updateTrackingUrl(trackingUrl);
+  }
   
   private class HeartbeatThread extends Thread {
     public HeartbeatThread() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e06a5dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
index 7265d24..6dcecde 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
@@ -99,6 +99,7 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
   protected String appHostName;
   protected int appHostPort;
   protected String appTrackingUrl;
+  protected String newTrackingUrl;
 
   protected ApplicationMasterProtocol rmClient;
   protected Resource clusterAvailableResources;
@@ -308,6 +309,11 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
             .releaseList(releaseList).updateRequests(updateList)
             .schedulingRequests(schedulingRequestList).build();
 
+        if (this.newTrackingUrl != null) {
+          allocateRequest.setTrackingUrl(this.newTrackingUrl);
+          this.appTrackingUrl = this.newTrackingUrl;
+          this.newTrackingUrl = null;
+        }
         // clear blacklistAdditions and blacklistRemovals before
         // unsynchronized part
         blacklistAdditions.clear();
@@ -1008,6 +1014,11 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
     }
   }
 
+  @Override
+  public synchronized void updateTrackingUrl(String trackingUrl) {
+    this.newTrackingUrl = trackingUrl;
+  }
+
   private void updateAMRMToken(Token token) throws IOException {
     org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> amrmToken =
         new org.apache.hadoop.security.token.Token<AMRMTokenIdentifier>(token

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e06a5dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
index 8dda8b4..cf83779 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
@@ -20,10 +20,12 @@ package org.apache.hadoop.yarn.client.api.impl;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
@@ -79,6 +81,7 @@ import org.junit.Assume;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+import org.mockito.ArgumentCaptor;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.eclipse.jetty.util.log.Log;
@@ -1994,4 +1997,78 @@ public class TestAMRMClient extends BaseAMRMClientTest{
       }
     }
   }
+
+  @Test(timeout = 60000)
+  public void testNoUpdateTrackingUrl()  {
+    try {
+      AMRMClientImpl<ContainerRequest> amClient = null;
+      amClient = new AMRMClientImpl<>();
+      amClient.init(conf);
+      amClient.start();
+      amClient.registerApplicationMaster("Host", 10000, "");
+
+      assertEquals("", amClient.appTrackingUrl);
+
+      ApplicationMasterProtocol mockRM = mock(ApplicationMasterProtocol.class);
+      AllocateResponse mockResponse = mock(AllocateResponse.class);
+      when(mockRM.allocate(any(AllocateRequest.class)))
+          .thenReturn(mockResponse);
+      ApplicationMasterProtocol realRM = amClient.rmClient;
+      amClient.rmClient = mockRM;
+      // Do allocate without updated tracking url
+      amClient.allocate(0.1f);
+      ArgumentCaptor<AllocateRequest> argument =
+          ArgumentCaptor.forClass(AllocateRequest.class);
+      verify(mockRM).allocate(argument.capture());
+      assertNull(argument.getValue().getTrackingUrl());
+
+      amClient.rmClient = realRM;
+      amClient
+          .unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, null,
+              null);
+    } catch (IOException | YarnException e) {
+      throw new AssertionError(
+          "testNoUpdateTrackingUrl unexpectedly threw exception: " + e);
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testUpdateTrackingUrl() {
+    try {
+      AMRMClientImpl<ContainerRequest> amClient = null;
+      amClient = new AMRMClientImpl<>();
+      amClient.init(conf);
+      amClient.start();
+      amClient.registerApplicationMaster("Host", 10000, "");
+
+      String trackingUrl = "hadoop.apache.org";
+      assertEquals("", amClient.appTrackingUrl);
+
+      ApplicationMasterProtocol mockRM = mock(ApplicationMasterProtocol.class);
+      AllocateResponse mockResponse = mock(AllocateResponse.class);
+      when(mockRM.allocate(any(AllocateRequest.class)))
+          .thenReturn(mockResponse);
+      ApplicationMasterProtocol realRM = amClient.rmClient;
+      amClient.rmClient = mockRM;
+      // Do allocate with updated tracking url
+      amClient.updateTrackingUrl(trackingUrl);
+      assertEquals(trackingUrl, amClient.newTrackingUrl);
+      assertEquals("", amClient.appTrackingUrl);
+      amClient.allocate(0.1f);
+      assertNull(amClient.newTrackingUrl);
+      assertEquals(trackingUrl, amClient.appTrackingUrl);
+      ArgumentCaptor<AllocateRequest> argument
+          = ArgumentCaptor.forClass(AllocateRequest.class);
+      verify(mockRM).allocate(argument.capture());
+      assertEquals(trackingUrl, argument.getValue().getTrackingUrl());
+
+      amClient.rmClient = realRM;
+      amClient
+          .unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, null,
+              null);
+    } catch (IOException | YarnException e) {
+      throw new AssertionError(
+          "testUpdateTrackingUrl unexpectedly threw exception: " + e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e06a5dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
index 50672a3..b5360a5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
@@ -58,6 +58,7 @@ public class AllocateRequestPBImpl extends AllocateRequest {
   private List<UpdateContainerRequest> updateRequests = null;
   private List<SchedulingRequest> schedulingRequests = null;
   private ResourceBlacklistRequest blacklistRequest = null;
+  private String trackingUrl = null;
   
   public AllocateRequestPBImpl() {
     builder = AllocateRequestProto.newBuilder();
@@ -111,6 +112,9 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     if (this.blacklistRequest != null) {
       builder.setBlacklistRequest(convertToProtoFormat(this.blacklistRequest));
     }
+    if (this.trackingUrl != null) {
+      builder.setTrackingUrl(this.trackingUrl);
+    }
   }
 
   private void mergeLocalToProto() {
@@ -398,7 +402,28 @@ public class AllocateRequestPBImpl extends AllocateRequest {
       this.release.add(convertFromProtoFormat(c));
     }
   }
-  
+
+  @Override
+  public String getTrackingUrl() {
+    AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.trackingUrl != null) {
+      return this.trackingUrl;
+    }
+    if (p.hasTrackingUrl()) {
+      this.trackingUrl = p.getTrackingUrl();
+    }
+    return this.trackingUrl;
+  }
+
+  @Override
+  public void setTrackingUrl(String trackingUrl) {
+    maybeInitBuilder();
+    if (trackingUrl == null) {
+      builder.clearTrackingUrl();
+    }
+    this.trackingUrl = trackingUrl;
+  }
+
   private void addReleasesToProto() {
     maybeInitBuilder();
     builder.clearRelease();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e06a5dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java
index 43f73e4..4cd5925 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java
@@ -401,7 +401,7 @@ final class DefaultAMSProcessor implements ApplicationMasterServiceProcessor {
     // Send the status update to the appAttempt.
     getRmContext().getDispatcher().getEventHandler().handle(
         new RMAppAttemptStatusupdateEvent(appAttemptId, request
-            .getProgress()));
+            .getProgress(), request.getTrackingUrl()));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e06a5dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index 32f275f..3ec9c49 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -1823,6 +1823,26 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       // Update progress
       appAttempt.progress = statusUpdateEvent.getProgress();
 
+      // Update tracking url if changed and save it to state store
+      String newTrackingUrl = statusUpdateEvent.getTrackingUrl();
+      if (newTrackingUrl != null &&
+          !newTrackingUrl.equals(appAttempt.originalTrackingUrl)) {
+        appAttempt.originalTrackingUrl = newTrackingUrl;
+        ApplicationAttemptStateData attemptState = ApplicationAttemptStateData
+            .newInstance(appAttempt.applicationAttemptId,
+                appAttempt.getMasterContainer(),
+                appAttempt.rmContext.getStateStore()
+                    .getCredentialsFromAppAttempt(appAttempt),
+                appAttempt.startTime, appAttempt.recoveredFinalState,
+                newTrackingUrl, appAttempt.getDiagnostics(), null,
+                ContainerExitStatus.INVALID, appAttempt.getFinishTime(),
+                appAttempt.attemptMetrics.getAggregateAppResourceUsage()
+                    .getResourceUsageSecondsMap(),
+                appAttempt.attemptMetrics.getPreemptedResourceSecondsMap());
+        appAttempt.rmContext.getStateStore()
+            .updateApplicationAttemptState(attemptState);
+      }
+
       // Ping to AMLivelinessMonitor
       appAttempt.rmContext.getAMLivelinessMonitor().receivedPing(
           statusUpdateEvent.getApplicationAttemptId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e06a5dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptStatusupdateEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptStatusupdateEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptStatusupdateEvent.java
index b1b63b1..1b7442d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptStatusupdateEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptStatusupdateEvent.java
@@ -25,15 +25,26 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
 public class RMAppAttemptStatusupdateEvent extends RMAppAttemptEvent {
 
   private final float progress;
+  private final String trackingUrl;
 
   public RMAppAttemptStatusupdateEvent(ApplicationAttemptId appAttemptId,
       float progress) {
+    this(appAttemptId, progress, null);
+  }
+
+  public RMAppAttemptStatusupdateEvent(ApplicationAttemptId appAttemptId,
+                                       float progress, String trackingUrl) {
     super(appAttemptId, RMAppAttemptEventType.STATUS_UPDATE);
     this.progress = progress;
+    this.trackingUrl = trackingUrl;
   }
 
   public float getProgress() {
     return this.progress;
   }
 
+  public String getTrackingUrl() {
+    return this.trackingUrl;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e06a5dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java
index 9696741..562ba5d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java
@@ -956,4 +956,38 @@ public class TestApplicationMasterService {
       fail("Cannot find RMContainer");
     }
   }
+
+  @Test(timeout = 300000)
+  public void testUpdateTrackingUrl() throws Exception {
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    MockRM rm = new MockRM(conf);
+    rm.start();
+
+    // Register node1
+    MockNM nm1 = rm.registerNode("127.0.0.1:1234", 6 * GB);
+
+    RMApp app1 = rm.submitApp(2048);
+
+    nm1.nodeHeartbeat(true);
+    RMAppAttempt attempt1 = app1.getCurrentAppAttempt();
+    MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId());
+    am1.registerAppAttempt();
+    Assert.assertEquals("N/A", rm.getRMContext().getRMApps().get(
+        app1.getApplicationId()).getOriginalTrackingUrl());
+
+    AllocateRequestPBImpl allocateRequest = new AllocateRequestPBImpl();
+    String newTrackingUrl = "hadoop.apache.org";
+    allocateRequest.setTrackingUrl(newTrackingUrl);
+
+    am1.allocate(allocateRequest);
+    Assert.assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get(
+        app1.getApplicationId()).getOriginalTrackingUrl());
+
+    // Send it again
+    am1.allocate(allocateRequest);
+    Assert.assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get(
+        app1.getApplicationId()).getOriginalTrackingUrl());
+    rm.stop();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e06a5dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index 07c5268..9aa5c53 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -2698,6 +2698,51 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     rm2.stop();
   }
 
+  @Test(timeout = 20000)
+  public void testRMRestartAfterUpdateTrackingUrl() throws Exception {
+    MockRM rm = new MockRM(conf);
+    rm.start();
+
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm.getRMStateStore();
+
+    // Register node1
+    MockNM nm1 = rm.registerNode("127.0.0.1:1234", 6 * 1024);
+
+    RMApp app1 = rm.submitApp(2048);
+
+    nm1.nodeHeartbeat(true);
+    RMAppAttempt attempt1 = app1.getCurrentAppAttempt();
+    MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId());
+    am1.registerAppAttempt();
+
+    AllocateRequestPBImpl allocateRequest = new AllocateRequestPBImpl();
+    String newTrackingUrl = "hadoop.apache.org";
+    allocateRequest.setTrackingUrl(newTrackingUrl);
+
+    am1.allocate(allocateRequest);
+    // Check in-memory and stored tracking url
+    Assert.assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get(
+        app1.getApplicationId()).getOriginalTrackingUrl());
+    Assert.assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get(
+        app1.getApplicationId()).getCurrentAppAttempt()
+        .getOriginalTrackingUrl());
+    Assert.assertEquals(newTrackingUrl, memStore.getState()
+        .getApplicationState().get(app1.getApplicationId())
+        .getAttempt(attempt1.getAppAttemptId()).getFinalTrackingUrl());
+
+    // Start new RM, should recover updated tracking url
+    MockRM rm2 = new MockRM(conf, memStore);
+    rm2.start();
+    Assert.assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get(
+        app1.getApplicationId()).getOriginalTrackingUrl());
+    Assert.assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get(
+        app1.getApplicationId()).getCurrentAppAttempt()
+        .getOriginalTrackingUrl());
+
+    rm.stop();
+    rm2.stop();
+  }
+
   private Credentials getCreds() throws IOException {
     Credentials ts = new Credentials();
     DataOutputBuffer dob = new DataOutputBuffer();


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


[35/50] hadoop git commit: YARN-8508. Release GPU resource for killed container. Contributed by Chandni Singh

Posted by eh...@apache.org.
YARN-8508.  Release GPU resource for killed container.
            Contributed by Chandni Singh


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ed9d60e8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ed9d60e8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ed9d60e8

Branch: refs/heads/HDFS-12090
Commit: ed9d60e888d0acfd748fda7f66249f5b79a3ed6d
Parents: 79091cf
Author: Eric Yang <ey...@apache.org>
Authored: Fri Jul 27 19:33:58 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Fri Jul 27 19:33:58 2018 -0400

----------------------------------------------------------------------
 .../nodemanager/LinuxContainerExecutor.java     | 34 ++++++++++----------
 .../nodemanager/TestLinuxContainerExecutor.java |  9 +++++-
 2 files changed, 25 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed9d60e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
index 03b88a4..4253f2f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
@@ -573,15 +573,7 @@ public class LinuxContainerExecutor extends ContainerExecutor {
       return handleExitCode(e, container, containerId);
     } finally {
       resourcesHandler.postExecute(containerId);
-
-      try {
-        if (resourceHandlerChain != null) {
-          resourceHandlerChain.postComplete(containerId);
-        }
-      } catch (ResourceHandlerException e) {
-        LOG.warn("ResourceHandlerChain.postComplete failed for " +
-            "containerId: " + containerId + ". Exception: " + e);
-      }
+      postComplete(containerId);
     }
 
     return 0;
@@ -721,14 +713,7 @@ public class LinuxContainerExecutor extends ContainerExecutor {
       return super.reacquireContainer(ctx);
     } finally {
       resourcesHandler.postExecute(containerId);
-      if (resourceHandlerChain != null) {
-        try {
-          resourceHandlerChain.postComplete(containerId);
-        } catch (ResourceHandlerException e) {
-          LOG.warn("ResourceHandlerChain.postComplete failed for " +
-              "containerId: " + containerId + " Exception: " + e);
-        }
-      }
+      postComplete(containerId);
     }
   }
 
@@ -798,6 +783,8 @@ public class LinuxContainerExecutor extends ContainerExecutor {
       logOutput(e.getOutput());
       throw new IOException("Error in reaping container "
           + container.getContainerId().toString() + " exit = " + retCode, e);
+    } finally {
+      postComplete(container.getContainerId());
     }
     return true;
   }
@@ -968,4 +955,17 @@ public class LinuxContainerExecutor extends ContainerExecutor {
       LOG.warn("Unable to remove docker container: " + containerId);
     }
   }
+
+  @VisibleForTesting
+  void postComplete(final ContainerId containerId) {
+    try {
+      if (resourceHandlerChain != null) {
+        LOG.debug("{} post complete", containerId);
+        resourceHandlerChain.postComplete(containerId);
+      }
+    } catch (ResourceHandlerException e) {
+      LOG.warn("ResourceHandlerChain.postComplete failed for " +
+          "containerId: {}. Exception: ", containerId, e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed9d60e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.java
index ddbf3b9..6d77fc4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.java
@@ -25,11 +25,14 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.anyObject;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntime;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerReapContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -40,6 +43,7 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -667,12 +671,15 @@ public class TestLinuxContainerExecutor {
   @Test
   public void testReapContainer() throws Exception {
     Container container = mock(Container.class);
-    LinuxContainerExecutor lce = mock(LinuxContainerExecutor.class);
+    LinuxContainerRuntime containerRuntime = mock(LinuxContainerRuntime.class);
+    LinuxContainerExecutor lce = spy(new LinuxContainerExecutor(
+        containerRuntime));
     ContainerReapContext.Builder builder =  new ContainerReapContext.Builder();
     builder.setContainer(container).setUser("foo");
     ContainerReapContext ctx = builder.build();
     lce.reapContainer(ctx);
     verify(lce, times(1)).reapContainer(ctx);
+    verify(lce, times(1)).postComplete(anyObject());
   }
 
   @Test


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


[05/50] hadoop git commit: YARN-7133. Clean up lock-try order in fair scheduler. (Szilard Nemeth via Haibo Chen)

Posted by eh...@apache.org.
YARN-7133. Clean up lock-try order in fair scheduler. (Szilard Nemeth via Haibo Chen)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ea2c6c8c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ea2c6c8c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ea2c6c8c

Branch: refs/heads/HDFS-12090
Commit: ea2c6c8c9a55813a19b3dbd0d29747d6a7739030
Parents: e673dd1
Author: Haibo Chen <ha...@apache.org>
Authored: Tue Jul 24 12:46:15 2018 -0700
Committer: Haibo Chen <ha...@apache.org>
Committed: Tue Jul 24 12:46:59 2018 -0700

----------------------------------------------------------------------
 .../scheduler/fair/FairScheduler.java           | 36 ++++++++++----------
 1 file changed, 18 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2c6c8c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index eb9f6af..20d1afe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -467,8 +467,8 @@ public class FairScheduler extends
       return;
     }
 
+    writeLock.lock();
     try {
-      writeLock.lock();
       RMApp rmApp = rmContext.getRMApps().get(applicationId);
       FSLeafQueue queue = assignToQueue(rmApp, queueName, user);
       if (queue == null) {
@@ -550,8 +550,8 @@ public class FairScheduler extends
       ApplicationAttemptId applicationAttemptId,
       boolean transferStateFromPreviousAttempt,
       boolean isAttemptRecovering) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       SchedulerApplication<FSAppAttempt> application = applications.get(
           applicationAttemptId.getApplicationId());
       String user = application.getUser();
@@ -653,8 +653,8 @@ public class FairScheduler extends
   private void removeApplicationAttempt(
       ApplicationAttemptId applicationAttemptId,
       RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       LOG.info("Application " + applicationAttemptId + " is done. finalState="
               + rmAppAttemptFinalState);
       FSAppAttempt attempt = getApplicationAttempt(applicationAttemptId);
@@ -720,8 +720,8 @@ public class FairScheduler extends
   protected void completedContainerInternal(
       RMContainer rmContainer, ContainerStatus containerStatus,
       RMContainerEventType event) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       Container container = rmContainer.getContainer();
 
       // Get the application for the finished container
@@ -768,8 +768,8 @@ public class FairScheduler extends
 
   private void addNode(List<NMContainerStatus> containerReports,
       RMNode node) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       FSSchedulerNode schedulerNode = new FSSchedulerNode(node,
           usePortForNodeName);
       nodeTracker.addNode(schedulerNode);
@@ -790,8 +790,8 @@ public class FairScheduler extends
   }
 
   private void removeNode(RMNode rmNode) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       NodeId nodeId = rmNode.getNodeID();
       FSSchedulerNode node = nodeTracker.getNode(nodeId);
       if (node == null) {
@@ -988,8 +988,8 @@ public class FairScheduler extends
 
   @Override
   protected void nodeUpdate(RMNode nm) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       long start = getClock().getTime();
       super.nodeUpdate(nm);
 
@@ -1089,8 +1089,8 @@ public class FairScheduler extends
 
   @VisibleForTesting
   void attemptScheduling(FSSchedulerNode node) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       if (rmContext.isWorkPreservingRecoveryEnabled() && !rmContext
           .isSchedulerReadyForAllocatingContainers()) {
         return;
@@ -1305,8 +1305,8 @@ public class FairScheduler extends
   private String resolveReservationQueueName(String queueName,
       ApplicationId applicationId, ReservationId reservationID,
       boolean isRecovering) {
+    readLock.lock();
     try {
-      readLock.lock();
       FSQueue queue = queueMgr.getQueue(queueName);
       if ((queue == null) || !allocConf.isReservable(queue.getQueueName())) {
         return queueName;
@@ -1372,8 +1372,8 @@ public class FairScheduler extends
 
   @SuppressWarnings("deprecation")
   private void initScheduler(Configuration conf) throws IOException {
+    writeLock.lock();
     try {
-      writeLock.lock();
       this.conf = new FairSchedulerConfiguration(conf);
       validateConf(this.conf);
       authorizer = YarnAuthorizationProvider.getInstance(conf);
@@ -1464,8 +1464,8 @@ public class FairScheduler extends
   }
 
   private void startSchedulerThreads() {
+    writeLock.lock();
     try {
-      writeLock.lock();
       Preconditions.checkNotNull(allocsLoader, "allocsLoader is null");
       if (continuousSchedulingEnabled) {
         Preconditions.checkNotNull(schedulingThread,
@@ -1499,8 +1499,8 @@ public class FairScheduler extends
   @SuppressWarnings("deprecation")
   @Override
   public void serviceStop() throws Exception {
+    writeLock.lock();
     try {
-      writeLock.lock();
       if (continuousSchedulingEnabled) {
         if (schedulingThread != null) {
           schedulingThread.interrupt();
@@ -1562,8 +1562,8 @@ public class FairScheduler extends
   @Override
   public boolean checkAccess(UserGroupInformation callerUGI,
       QueueACL acl, String queueName) {
+    readLock.lock();
     try {
-      readLock.lock();
       FSQueue queue = getQueueManager().getQueue(queueName);
       if (queue == null) {
         if (LOG.isDebugEnabled()) {
@@ -1691,8 +1691,8 @@ public class FairScheduler extends
   @Override
   public String moveApplication(ApplicationId appId,
       String queueName) throws YarnException {
+    writeLock.lock();
     try {
-      writeLock.lock();
       SchedulerApplication<FSAppAttempt> app = applications.get(appId);
       if (app == null) {
         throw new YarnException("App to be moved " + appId + " not found.");
@@ -1700,8 +1700,8 @@ public class FairScheduler extends
       FSAppAttempt attempt = (FSAppAttempt) app.getCurrentAppAttempt();
       // To serialize with FairScheduler#allocate, synchronize on app attempt
 
+      attempt.getWriteLock().lock();
       try {
-        attempt.getWriteLock().lock();
         FSLeafQueue oldQueue = (FSLeafQueue) app.getQueue();
         // Check if the attempt is already stopped: don't move stopped app
         // attempt. The attempt has already been removed from all queues.
@@ -1737,8 +1737,8 @@ public class FairScheduler extends
   @Override
   public void preValidateMoveApplication(ApplicationId appId, String newQueue)
       throws YarnException {
+    writeLock.lock();
     try {
-      writeLock.lock();
       SchedulerApplication<FSAppAttempt> app = applications.get(appId);
       if (app == null) {
         throw new YarnException("App to be moved " + appId + " not found.");
@@ -1747,8 +1747,8 @@ public class FairScheduler extends
       FSAppAttempt attempt = app.getCurrentAppAttempt();
       // To serialize with FairScheduler#allocate, synchronize on app attempt
 
+      attempt.getWriteLock().lock();
       try {
-        attempt.getWriteLock().lock();
         FSLeafQueue oldQueue = (FSLeafQueue) app.getQueue();
         String destQueueName = handleMoveToPlanQueue(newQueue);
         FSLeafQueue targetQueue = queueMgr.getLeafQueue(destQueueName, false);
@@ -1869,8 +1869,8 @@ public class FairScheduler extends
   @Override
   public void updateNodeResource(RMNode nm,
       ResourceOption resourceOption) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       super.updateNodeResource(nm, resourceOption);
       updateRootQueueMetrics();
       queueMgr.getRootQueue().setSteadyFairShare(getClusterResource());


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


[48/50] hadoop git commit: HDDS-293. Reduce memory usage and object creation in KeyData.

Posted by eh...@apache.org.
HDDS-293. Reduce memory usage and object creation in KeyData.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ee53602a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ee53602a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ee53602a

Branch: refs/heads/HDFS-12090
Commit: ee53602a8179e76f4102d9062d0bebe8bb09d875
Parents: 2b39ad2
Author: Tsz Wo Nicholas Sze <sz...@apache.org>
Authored: Mon Jul 30 15:00:29 2018 -0700
Committer: Tsz Wo Nicholas Sze <sz...@apache.org>
Committed: Mon Jul 30 15:00:29 2018 -0700

----------------------------------------------------------------------
 .../ozone/container/common/helpers/KeyData.java |  84 +++++++++----
 .../common/impl/OpenContainerBlockMap.java      |   2 +-
 .../container/keyvalue/KeyValueHandler.java     |   3 -
 .../container/common/helpers/TestKeyData.java   | 119 +++++++++++++++++++
 4 files changed, 179 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee53602a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
index 1919ed9..84a6f71 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.container.common.helpers;
 
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.client.BlockID;
+import com.google.common.base.Preconditions;
 
 import java.io.IOException;
 import java.util.Collections;
@@ -35,11 +36,17 @@ public class KeyData {
   private final Map<String, String> metadata;
 
   /**
+   * Represent a list of chunks.
+   * In order to reduce memory usage, chunkList is declared as an {@link Object}.
+   * When #elements == 0, chunkList is null.
+   * When #elements == 1, chunkList refers to the only element.
+   * When #elements > 1, chunkList refers to the list.
+   *
    * Please note : when we are working with keys, we don't care what they point
    * to. So we We don't read chunkinfo nor validate them. It is responsibility
    * of higher layer like ozone. We just read and write data from network.
    */
-  private List<ContainerProtos.ChunkInfo> chunks;
+  private Object chunkList;
 
   /**
    * total size of the key.
@@ -73,7 +80,7 @@ public class KeyData {
     }
     keyData.setChunks(data.getChunksList());
     if (data.hasSize()) {
-      keyData.setSize(data.getSize());
+      Preconditions.checkArgument(data.getSize() == keyData.getSize());
     }
     return keyData;
   }
@@ -86,13 +93,13 @@ public class KeyData {
     ContainerProtos.KeyData.Builder builder =
         ContainerProtos.KeyData.newBuilder();
     builder.setBlockID(this.blockID.getDatanodeBlockIDProtobuf());
-    builder.addAllChunks(this.chunks);
     for (Map.Entry<String, String> entry : metadata.entrySet()) {
       ContainerProtos.KeyValue.Builder keyValBuilder =
           ContainerProtos.KeyValue.newBuilder();
       builder.addMetadata(keyValBuilder.setKey(entry.getKey())
           .setValue(entry.getValue()).build());
     }
+    builder.addAllChunks(getChunks());
     builder.setSize(size);
     return builder.build();
   }
@@ -132,30 +139,65 @@ public class KeyData {
     metadata.remove(key);
   }
 
+  @SuppressWarnings("unchecked")
+  private List<ContainerProtos.ChunkInfo> castChunkList() {
+    return (List<ContainerProtos.ChunkInfo>)chunkList;
+  }
+
   /**
    * Returns chunks list.
    *
    * @return list of chunkinfo.
    */
   public List<ContainerProtos.ChunkInfo> getChunks() {
-    return chunks;
+    return chunkList == null? Collections.emptyList()
+        : chunkList instanceof ContainerProtos.ChunkInfo?
+            Collections.singletonList((ContainerProtos.ChunkInfo)chunkList)
+        : Collections.unmodifiableList(castChunkList());
   }
 
   /**
    * Adds chinkInfo to the list
    */
   public void addChunk(ContainerProtos.ChunkInfo chunkInfo) {
-    if (chunks == null) {
-      chunks = new ArrayList<>();
+    if (chunkList == null) {
+      chunkList = chunkInfo;
+    } else {
+      final List<ContainerProtos.ChunkInfo> list;
+      if (chunkList instanceof ContainerProtos.ChunkInfo) {
+        list = new ArrayList<>(2);
+        list.add((ContainerProtos.ChunkInfo)chunkList);
+        chunkList = list;
+      } else {
+        list = castChunkList();
+      }
+      list.add(chunkInfo);
     }
-    chunks.add(chunkInfo);
+    size += chunkInfo.getLen();
   }
 
   /**
    * removes the chunk.
    */
-  public void removeChunk(ContainerProtos.ChunkInfo chunkInfo) {
-    chunks.remove(chunkInfo);
+  public boolean removeChunk(ContainerProtos.ChunkInfo chunkInfo) {
+    final boolean removed;
+    if (chunkList instanceof List) {
+      final List<ContainerProtos.ChunkInfo> list = castChunkList();
+      removed = list.remove(chunkInfo);
+      if (list.size() == 1) {
+        chunkList = list.get(0);
+      }
+    } else if (chunkInfo.equals(chunkList)) {
+      chunkList = null;
+      removed = true;
+    } else {
+      removed = false;
+    }
+
+    if (removed) {
+      size -= chunkInfo.getLen();
+    }
+    return removed;
   }
 
   /**
@@ -189,15 +231,14 @@ public class KeyData {
    * @param chunks - List of chunks.
    */
   public void setChunks(List<ContainerProtos.ChunkInfo> chunks) {
-    this.chunks = chunks;
-  }
-
-  /**
-   * sets the total size of the block
-   * @param size size of the block
-   */
-  public void setSize(long size) {
-    this.size = size;
+    if (chunks == null) {
+      chunkList = null;
+      size = 0L;
+    } else {
+      final int n = chunks.size();
+      chunkList = n == 0? null: n == 1? chunks.get(0): chunks;
+      size = chunks.parallelStream().mapToLong(ContainerProtos.ChunkInfo::getLen).sum();
+    }
   }
 
   /**
@@ -207,11 +248,4 @@ public class KeyData {
   public long getSize() {
     return size;
   }
-
-  /**
-   * computes the total size of chunks allocated for the key.
-   */
-  public void computeSize() {
-    setSize(chunks.parallelStream().mapToLong(e -> e.getLen()).sum());
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee53602a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
index 8e2667d..ab7789b 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
@@ -33,7 +33,7 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.function.Function;
 
 /**
- * Map: containerId -> (localId -> KeyData).
+ * Map: containerId -> (localId -> {@link KeyData}).
  * The outer container map does not entail locking for a better performance.
  * The inner {@link KeyDataMap} is synchronized.
  *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee53602a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index a4e124b..fac3f3c 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -439,8 +439,6 @@ public class KeyValueHandler extends Handler {
   private void commitKey(KeyData keyData, KeyValueContainer kvContainer)
       throws IOException {
     Preconditions.checkNotNull(keyData);
-    //sets the total size of the key before committing
-    keyData.computeSize();
     keyManager.putKey(kvContainer, keyData);
     //update the open key Map in containerManager
     this.openContainerBlockMap.removeFromKeyMap(keyData.getBlockID());
@@ -696,7 +694,6 @@ public class KeyValueHandler extends Handler {
       List<ContainerProtos.ChunkInfo> chunks = new LinkedList<>();
       chunks.add(chunkInfo.getProtoBufMessage());
       keyData.setChunks(chunks);
-      keyData.computeSize();
       keyManager.putKey(kvContainer, keyData);
       metrics.incContainerBytesStats(Type.PutSmallFile, data.length);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee53602a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestKeyData.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestKeyData.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestKeyData.java
new file mode 100644
index 0000000..f57fe99
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestKeyData.java
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.helpers;
+
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * Tests to test block deleting service.
+ */
+public class TestKeyData {
+  static final Logger LOG = LoggerFactory.getLogger(TestKeyData.class);
+  @Rule
+  public TestRule timeout = new Timeout(10000);
+
+  static ContainerProtos.ChunkInfo buildChunkInfo(String name, long offset, long len) {
+    return ContainerProtos.ChunkInfo.newBuilder()
+        .setChunkName(name).setOffset(offset).setLen(len).build();
+  }
+
+  @Test
+  public void testAddAndRemove() {
+    final KeyData computed = new KeyData(null);
+    final List<ContainerProtos.ChunkInfo> expected = new ArrayList<>();
+
+    assertChunks(expected, computed);
+    long offset = 0;
+    int n = 5;
+    for(int i = 0; i < n; i++) {
+      offset += assertAddChunk(expected, computed, offset);
+    }
+
+    for(; !expected.isEmpty(); ) {
+      removeChunk(expected, computed);
+    }
+  }
+
+  private static int chunkCount = 0;
+  static ContainerProtos.ChunkInfo addChunk(List<ContainerProtos.ChunkInfo> expected, long offset) {
+    final long length = ThreadLocalRandom.current().nextLong(1000);
+    final ContainerProtos.ChunkInfo info = buildChunkInfo("c" + ++chunkCount, offset, length);
+    expected.add(info);
+    return info;
+  }
+
+  static long assertAddChunk(List<ContainerProtos.ChunkInfo> expected, KeyData computed, long offset) {
+    final ContainerProtos.ChunkInfo info = addChunk(expected, offset);
+    LOG.info("addChunk: " + toString(info));
+    computed.addChunk(info);
+    assertChunks(expected, computed);
+    return info.getLen();
+  }
+
+
+  static void removeChunk(List<ContainerProtos.ChunkInfo> expected, KeyData computed) {
+    final int i = ThreadLocalRandom.current().nextInt(expected.size());
+    final ContainerProtos.ChunkInfo info = expected.remove(i);
+    LOG.info("removeChunk: " + toString(info));
+    computed.removeChunk(info);
+    assertChunks(expected, computed);
+  }
+
+  static void assertChunks(List<ContainerProtos.ChunkInfo> expected, KeyData computed) {
+    final List<ContainerProtos.ChunkInfo> computedChunks = computed.getChunks();
+    Assert.assertEquals("expected=" + expected + "\ncomputed=" + computedChunks, expected, computedChunks);
+    Assert.assertEquals(expected.stream().mapToLong(i -> i.getLen()).sum(), computed.getSize());
+  }
+
+  static String toString(ContainerProtos.ChunkInfo info) {
+    return info.getChunkName() + ":" + info.getOffset() + "," + info.getLen();
+  }
+
+  static String toString(List<ContainerProtos.ChunkInfo> infos) {
+    return infos.stream().map(TestKeyData::toString)
+        .reduce((left, right) -> left + ", " + right)
+        .orElse("");
+  }
+
+  @Test
+  public void testSetChunks() {
+    final KeyData computed = new KeyData(null);
+    final List<ContainerProtos.ChunkInfo> expected = new ArrayList<>();
+
+    assertChunks(expected, computed);
+    long offset = 0;
+    int n = 5;
+    for(int i = 0; i < n; i++) {
+      offset += addChunk(expected, offset).getLen();
+      LOG.info("setChunk: " + toString(expected));
+      computed.setChunks(expected);
+      assertChunks(expected, computed);
+    }
+  }
+}


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


[19/50] hadoop git commit: YARN-8330. Improved publishing ALLOCATED events to ATS. Contributed by Suma Shivaprasad

Posted by eh...@apache.org.
YARN-8330.  Improved publishing ALLOCATED events to ATS.
            Contributed by Suma Shivaprasad


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f93ecf5c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f93ecf5c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f93ecf5c

Branch: refs/heads/HDFS-12090
Commit: f93ecf5c1e0b3db27424963814fc01ec43eb76e0
Parents: e95c5e9
Author: Eric Yang <ey...@apache.org>
Authored: Wed Jul 25 18:49:30 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Wed Jul 25 18:49:30 2018 -0400

----------------------------------------------------------------------
 .../rmcontainer/RMContainerImpl.java            | 64 +++++++++++---------
 .../rmcontainer/TestRMContainerImpl.java        | 11 +++-
 2 files changed, 43 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f93ecf5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index efac666..945e7cb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -244,23 +244,13 @@ public class RMContainerImpl implements RMContainer {
     this.readLock = lock.readLock();
     this.writeLock = lock.writeLock();
 
-    saveNonAMContainerMetaInfo = rmContext.getYarnConfiguration().getBoolean(
-       YarnConfiguration.APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
-       YarnConfiguration
-                 .DEFAULT_APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO);
+    saveNonAMContainerMetaInfo =
+        shouldPublishNonAMContainerEventstoATS(rmContext);
 
     if (container.getId() != null) {
       rmContext.getRMApplicationHistoryWriter().containerStarted(this);
     }
 
-    // If saveNonAMContainerMetaInfo is true, store system metrics for all
-    // containers. If false, and if this container is marked as the AM, metrics
-    // will still be published for this container, but that calculation happens
-    // later.
-    if (saveNonAMContainerMetaInfo && null != container.getId()) {
-      rmContext.getSystemMetricsPublisher().containerCreated(
-          this, this.creationTime);
-    }
     if (this.container != null) {
       this.allocationTags = this.container.getAllocationTags();
     }
@@ -590,8 +580,12 @@ public class RMContainerImpl implements RMContainer {
           container.getNodeId(), container.getContainerId(),
           container.getAllocationTags());
 
-      container.eventHandler.handle(new RMAppAttemptEvent(
-          container.appAttemptId, RMAppAttemptEventType.CONTAINER_ALLOCATED));
+      container.eventHandler.handle(
+          new RMAppAttemptEvent(container.appAttemptId,
+              RMAppAttemptEventType.CONTAINER_ALLOCATED));
+
+      publishNonAMContainerEventstoATS(container);
+
     }
   }
 
@@ -610,9 +604,11 @@ public class RMContainerImpl implements RMContainer {
       // Tell the app
       container.eventHandler.handle(new RMAppRunningOnNodeEvent(container
           .getApplicationAttemptId().getApplicationId(), container.nodeId));
+
+      publishNonAMContainerEventstoATS(container);
     }
   }
-  
+
   private static final class ContainerAcquiredWhileRunningTransition extends
       BaseTransition {
 
@@ -718,17 +714,12 @@ public class RMContainerImpl implements RMContainer {
         container);
 
       boolean saveNonAMContainerMetaInfo =
-          container.rmContext.getYarnConfiguration().getBoolean(
-              YarnConfiguration
-                .APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
-              YarnConfiguration
-                .DEFAULT_APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO);
+          shouldPublishNonAMContainerEventstoATS(container.rmContext);
 
       if (saveNonAMContainerMetaInfo || container.isAMContainer()) {
         container.rmContext.getSystemMetricsPublisher().containerFinished(
             container, container.finishTime);
       }
-
     }
 
     private static void updateAttemptMetrics(RMContainerImpl container) {
@@ -754,6 +745,29 @@ public class RMContainerImpl implements RMContainer {
     }
   }
 
+  private static boolean shouldPublishNonAMContainerEventstoATS(
+      RMContext rmContext) {
+    return rmContext.getYarnConfiguration().getBoolean(
+        YarnConfiguration.APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
+        YarnConfiguration
+            .DEFAULT_APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO);
+  }
+
+  private static void publishNonAMContainerEventstoATS(
+      RMContainerImpl rmContainer) {
+    boolean saveNonAMContainerMetaInfo = shouldPublishNonAMContainerEventstoATS(
+        rmContainer.rmContext);
+
+    // If saveNonAMContainerMetaInfo is true, store system metrics for all
+    // containers. If false, and if this container is marked as the AM, metrics
+    // will still be published for this container, but that calculation happens
+    // later.
+    if (saveNonAMContainerMetaInfo && null != rmContainer.container.getId()) {
+      rmContainer.rmContext.getSystemMetricsPublisher().containerCreated(
+          rmContainer, rmContainer.creationTime);
+    }
+  }
+
   private static final class KillTransition extends FinishedTransition {
 
     @Override
@@ -884,13 +898,5 @@ public class RMContainerImpl implements RMContainer {
     if (containerId != null) {
       rmContext.getRMApplicationHistoryWriter().containerStarted(this);
     }
-    // If saveNonAMContainerMetaInfo is true, store system metrics for all
-    // containers. If false, and if this container is marked as the AM, metrics
-    // will still be published for this container, but that calculation happens
-    // later.
-    if (saveNonAMContainerMetaInfo && null != container.getId()) {
-      rmContext.getSystemMetricsPublisher().containerCreated(
-          this, this.creationTime);
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f93ecf5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
index 1115e8c..bb6591b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
@@ -135,7 +135,6 @@ public class TestRMContainerImpl {
     assertEquals(priority,
         rmContainer.getAllocatedSchedulerKey().getPriority());
     verify(writer).containerStarted(any(RMContainer.class));
-    verify(publisher).containerCreated(any(RMContainer.class), anyLong());
 
     rmContainer.handle(new RMContainerEvent(containerId,
         RMContainerEventType.START));
@@ -150,6 +149,8 @@ public class TestRMContainerImpl {
         RMContainerEventType.LAUNCHED));
     drainDispatcher.await();
     assertEquals(RMContainerState.RUNNING, rmContainer.getState());
+    verify(publisher, times(2)).containerCreated(any(RMContainer.class),
+        anyLong());
     assertEquals("http://host:3465/node/containerlogs/container_1_0001_01_000001/user",
         rmContainer.getLogURL());
 
@@ -240,22 +241,25 @@ public class TestRMContainerImpl {
     assertEquals(priority,
         rmContainer.getAllocatedSchedulerKey().getPriority());
     verify(writer).containerStarted(any(RMContainer.class));
-    verify(publisher).containerCreated(any(RMContainer.class), anyLong());
 
     rmContainer.handle(new RMContainerEvent(containerId,
         RMContainerEventType.START));
     drainDispatcher.await();
     assertEquals(RMContainerState.ALLOCATED, rmContainer.getState());
+    verify(publisher).containerCreated(any(RMContainer.class), anyLong());
 
     rmContainer.handle(new RMContainerEvent(containerId,
         RMContainerEventType.ACQUIRED));
     drainDispatcher.await();
     assertEquals(RMContainerState.ACQUIRED, rmContainer.getState());
+    verify(publisher, times(2)).containerCreated(any(RMContainer.class),
+        anyLong());
 
     rmContainer.handle(new RMContainerEvent(containerId,
         RMContainerEventType.LAUNCHED));
     drainDispatcher.await();
     assertEquals(RMContainerState.RUNNING, rmContainer.getState());
+
     assertEquals("http://host:3465/node/containerlogs/container_1_0001_01_000001/user",
         rmContainer.getLogURL());
 
@@ -340,7 +344,8 @@ public class TestRMContainerImpl {
     // RMContainer should be publishing system metrics for all containers.
     // Since there is 1 AM container and 1 non-AM container, there should be 2
     // container created events and 2 container finished events.
-    verify(publisher, times(2)).containerCreated(any(RMContainer.class), anyLong());
+    verify(publisher, times(4)).containerCreated(any(RMContainer.class),
+        anyLong());
     verify(publisher, times(2)).containerFinished(any(RMContainer.class), anyLong());
   }
 


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


[28/50] hadoop git commit: HDFS-13727. Log full stack trace if DiskBalancer exits with an unhandled exception. Contributed by Gabor Bota.

Posted by eh...@apache.org.
HDFS-13727. Log full stack trace if DiskBalancer exits with an unhandled exception.
Contributed by Gabor Bota.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/64e739e3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/64e739e3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/64e739e3

Branch: refs/heads/HDFS-12090
Commit: 64e739e344ac474046d4f4ecf0865dd92be13762
Parents: 8d3c068
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Jul 27 06:11:56 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri Jul 27 06:11:56 2018 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hdfs/tools/DiskBalancerCLI.java  | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/64e739e3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancerCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancerCLI.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancerCLI.java
index 00e6f04..34bd68b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancerCLI.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancerCLI.java
@@ -172,7 +172,9 @@ public class DiskBalancerCLI extends Configured implements Tool {
     try {
       res = ToolRunner.run(shell, argv);
     } catch (Exception ex) {
-      LOG.error(ex.toString());
+      String msg = String.format("Exception thrown while running %s.",
+          DiskBalancerCLI.class.getSimpleName());
+      LOG.error(msg, ex);
       res = 1;
     }
     System.exit(res);


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


[04/50] hadoop git commit: YARN-8541. RM startup failure on recovery after user deletion. Contributed by Bibin A Chundatt.

Posted by eh...@apache.org.
YARN-8541. RM startup failure on recovery after user deletion. Contributed by Bibin A Chundatt.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e673dd1d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e673dd1d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e673dd1d

Branch: refs/heads/HDFS-12090
Commit: e673dd1d4d78b66e7b6705ec6dc3679d2347d704
Parents: cd0b9f1
Author: bibinchundatt <bi...@apache.org>
Authored: Tue Jul 24 18:36:49 2018 +0530
Committer: bibinchundatt <bi...@apache.org>
Committed: Tue Jul 24 23:26:59 2018 +0530

----------------------------------------------------------------------
 .../server/resourcemanager/RMAppManager.java    | 48 ++++++++++----------
 .../placement/PlacementManager.java             |  9 ----
 .../TestWorkPreservingRMRestart.java            | 48 ++++++++++++++++++++
 .../placement/TestPlacementManager.java         | 20 ++++----
 4 files changed, 80 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e673dd1d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
index 3e64cfc..7011aaa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
@@ -364,17 +364,9 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
       ApplicationSubmissionContext submissionContext, long submitTime,
       String user, boolean isRecovery, long startTime) throws YarnException {
 
-    ApplicationPlacementContext placementContext = null;
-    try {
-      placementContext = placeApplication(rmContext, submissionContext, user);
-    } catch (YarnException e) {
-      String msg =
-          "Failed to place application " + submissionContext.getApplicationId()
-              + " to queue and specified " + "queue is invalid : "
-              + submissionContext.getQueue();
-      LOG.error(msg, e);
-      throw e;
-    }
+    ApplicationPlacementContext placementContext =
+        placeApplication(rmContext.getQueuePlacementManager(),
+            submissionContext, user, isRecovery);
 
     // We only replace the queue when it's a new application
     if (!isRecovery) {
@@ -789,23 +781,31 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
   }
 
   @VisibleForTesting
-  ApplicationPlacementContext placeApplication(RMContext rmContext,
-      ApplicationSubmissionContext context, String user) throws YarnException {
+  ApplicationPlacementContext placeApplication(
+      PlacementManager placementManager, ApplicationSubmissionContext context,
+      String user, boolean isRecovery) throws YarnException {
     ApplicationPlacementContext placementContext = null;
-    PlacementManager placementManager = rmContext.getQueuePlacementManager();
-
     if (placementManager != null) {
-      placementContext = placementManager.placeApplication(context, user);
-    } else{
-      if ( context.getQueue() == null || context.getQueue().isEmpty()) {
-        final String msg = "Queue Placement Manager is not set. Cannot place "
-            + "application : " + context.getApplicationId() + " to queue and "
-            + "specified queue is invalid " + context.getQueue();
-        LOG.error(msg);
-        throw new YarnException(msg);
+      try {
+        placementContext = placementManager.placeApplication(context, user);
+      } catch (YarnException e) {
+        // Placement could also fail if the user doesn't exist in system
+        // skip if the user is not found during recovery.
+        if (isRecovery) {
+          LOG.warn("PlaceApplication failed,skipping on recovery of rm");
+          return placementContext;
+        }
+        throw e;
       }
     }
-
+    if (placementContext == null && (context.getQueue() == null) || context
+        .getQueue().isEmpty()) {
+      String msg = "Failed to place application " + context.getApplicationId()
+          + " to queue and specified " + "queue is invalid : " + context
+          .getQueue();
+      LOG.error(msg);
+      throw new YarnException(msg);
+    }
     return placementContext;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e673dd1d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementManager.java
index 5fa7723..74cf7ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementManager.java
@@ -70,15 +70,6 @@ public class PlacementManager {
         }
       }
 
-      // Failed to get where to place application
-      if (null == placement && null == asc.getQueue()) {
-        String msg = "Failed to place application " +
-            asc.getApplicationId() + " to queue and specified "
-            + "queue is invalid : " + asc.getQueue();
-        LOG.error(msg);
-        throw new YarnException(msg);
-      }
-
       return placement;
     } finally {
       readLock.unlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e673dd1d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
index 88c19a1..a821b0a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
@@ -39,8 +39,12 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.TestRMRestart.TestSecurityMockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.placement
+    .ApplicationPlacementContext;
+import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationAttemptStateData;
@@ -105,6 +109,8 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
 import static org.apache.hadoop.yarn.server.resourcemanager.webapp
     .RMWebServices.DEFAULT_QUEUE;
 import static org.junit.Assert.*;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -1555,6 +1561,48 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   }
 
   @Test(timeout = 30000)
+  public void testUnknownUserOnRecovery() throws Exception {
+
+    MockRM rm1 = new MockRM(conf);
+    rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
+    MockNM nm1 =
+        new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
+    nm1.registerNode();
+
+    // create app and launch the UAM
+    RMApp app0 = rm1.submitApp(200, true);
+    MockAM am0 = MockRM.launchUAM(app0, rm1, nm1);
+    am0.registerAppAttempt();
+    rm1.killApp(app0.getApplicationId());
+    PlacementManager placementMgr = mock(PlacementManager.class);
+    doThrow(new YarnException("No groups for user")).when(placementMgr)
+        .placeApplication(any(ApplicationSubmissionContext.class),
+            any(String.class));
+    MockRM rm2 = new MockRM(conf, memStore) {
+      @Override
+      protected RMAppManager createRMAppManager() {
+        return new RMAppManager(this.rmContext, this.scheduler,
+            this.masterService, this.applicationACLsManager, conf) {
+          @Override
+          ApplicationPlacementContext placeApplication(
+              PlacementManager placementManager,
+              ApplicationSubmissionContext context, String user,
+              boolean isRecovery) throws YarnException {
+            return super
+                .placeApplication(placementMgr, context, user, isRecovery);
+          }
+        };
+      }
+    };
+    rm2.start();
+    RMApp recoveredApp =
+        rm2.getRMContext().getRMApps().get(app0.getApplicationId());
+    Assert.assertEquals(RMAppState.KILLED, recoveredApp.getState());
+  }
+
+  @Test(timeout = 30000)
   public void testDynamicAutoCreatedQueueRecoveryWithDefaultQueue()
       throws Exception {
     //if queue name is not specified, it should submit to 'default' queue

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e673dd1d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestPlacementManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestPlacementManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestPlacementManager.java
index 13111be..db5cd60 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestPlacementManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestPlacementManager.java
@@ -83,16 +83,11 @@ public class TestPlacementManager {
 
     ApplicationSubmissionContext asc = Records.newRecord(
         ApplicationSubmissionContext.class);
+    asc.setQueue(YarnConfiguration.DEFAULT_QUEUE_NAME);
     asc.setApplicationName(APP_NAME);
 
-    boolean caughtException = false;
-    try{
-      pm.placeApplication(asc, USER2);
-    } catch (Exception e) {
-      caughtException = true;
-    }
-    Assert.assertTrue(caughtException);
-
+    Assert.assertNull("Placement should be null",
+        pm.placeApplication(asc, USER2));
     QueueMappingEntity queueMappingEntity = new QueueMappingEntity(APP_NAME,
         USER1, PARENT_QUEUE);
 
@@ -100,12 +95,13 @@ public class TestPlacementManager {
         Arrays.asList(queueMappingEntity));
     queuePlacementRules.add(anRule);
     pm.updateRules(queuePlacementRules);
-    try{
-      pm.placeApplication(asc, USER2);
+    try {
+      ApplicationPlacementContext pc = pm.placeApplication(asc, USER2);
+      Assert.assertNotNull(pc);
     } catch (Exception e) {
-      caughtException = false;
+      e.printStackTrace();
+      Assert.fail("Exception not expected");
     }
-    Assert.assertFalse(caughtException);
   }
 
 }
\ No newline at end of file


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


[26/50] hadoop git commit: HADOOP-15593. Fixed NPE in UGI spawnAutoRenewalThreadForUserCreds. Contributed by Gabor Bota

Posted by eh...@apache.org.
HADOOP-15593.  Fixed NPE in UGI spawnAutoRenewalThreadForUserCreds.
               Contributed by Gabor Bota


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/77721f39
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/77721f39
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/77721f39

Branch: refs/heads/HDFS-12090
Commit: 77721f39e26b630352a1f4087524a3fbd21ff06e
Parents: 40fad32
Author: Eric Yang <ey...@apache.org>
Authored: Thu Jul 26 18:35:36 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Thu Jul 26 18:35:36 2018 -0400

----------------------------------------------------------------------
 .../hadoop/security/UserGroupInformation.java   | 179 ++++++++++++-------
 .../security/TestUserGroupInformation.java      |  38 ++++
 2 files changed, 148 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/77721f39/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
index 29b9fea..6ce72edb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
@@ -40,6 +40,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Date;
 import java.util.EnumMap;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -851,81 +852,121 @@ public class UserGroupInformation {
     }
 
     //spawn thread only if we have kerb credentials
-    Thread t = new Thread(new Runnable() {
+    KerberosTicket tgt = getTGT();
+    if (tgt == null) {
+      return;
+    }
+    String cmd = conf.get("hadoop.kerberos.kinit.command", "kinit");
+    long nextRefresh = getRefreshTime(tgt);
+    Thread t =
+        new Thread(new AutoRenewalForUserCredsRunnable(tgt, cmd, nextRefresh));
+    t.setDaemon(true);
+    t.setName("TGT Renewer for " + getUserName());
+    t.start();
+  }
+
+  @VisibleForTesting
+  class AutoRenewalForUserCredsRunnable implements Runnable {
+    private KerberosTicket tgt;
+    private RetryPolicy rp;
+    private String kinitCmd;
+    private long nextRefresh;
+    private boolean runRenewalLoop = true;
+
+    AutoRenewalForUserCredsRunnable(KerberosTicket tgt, String kinitCmd,
+        long nextRefresh){
+      this.tgt = tgt;
+      this.kinitCmd = kinitCmd;
+      this.nextRefresh = nextRefresh;
+      this.rp = null;
+    }
+
+    public void setRunRenewalLoop(boolean runRenewalLoop) {
+      this.runRenewalLoop = runRenewalLoop;
+    }
 
-      @Override
-      public void run() {
-        String cmd = conf.get("hadoop.kerberos.kinit.command", "kinit");
-        KerberosTicket tgt = getTGT();
-        if (tgt == null) {
+    @Override
+    public void run() {
+      do {
+        try {
+          long now = Time.now();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Current time is " + now);
+            LOG.debug("Next refresh is " + nextRefresh);
+          }
+          if (now < nextRefresh) {
+            Thread.sleep(nextRefresh - now);
+          }
+          String output = Shell.execCommand(kinitCmd, "-R");
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Renewed ticket. kinit output: {}", output);
+          }
+          reloginFromTicketCache();
+          tgt = getTGT();
+          if (tgt == null) {
+            LOG.warn("No TGT after renewal. Aborting renew thread for " +
+                getUserName());
+            return;
+          }
+          nextRefresh = Math.max(getRefreshTime(tgt),
+              now + kerberosMinSecondsBeforeRelogin);
+          metrics.renewalFailures.set(0);
+          rp = null;
+        } catch (InterruptedException ie) {
+          LOG.warn("Terminating renewal thread");
           return;
-        }
-        long nextRefresh = getRefreshTime(tgt);
-        RetryPolicy rp = null;
-        while (true) {
+        } catch (IOException ie) {
+          metrics.renewalFailuresTotal.incr();
+          final long now = Time.now();
+
+          if (tgt.isDestroyed()) {
+            LOG.error("TGT is destroyed. Aborting renew thread for {}.",
+                getUserName());
+            return;
+          }
+
+          long tgtEndTime;
+          // As described in HADOOP-15593 we need to handle the case when
+          // tgt.getEndTime() throws NPE because of JDK issue JDK-8147772
+          // NPE is only possible if this issue is not fixed in the JDK
+          // currently used
           try {
-            long now = Time.now();
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Current time is " + now);
-              LOG.debug("Next refresh is " + nextRefresh);
-            }
-            if (now < nextRefresh) {
-              Thread.sleep(nextRefresh - now);
-            }
-            String output = Shell.execCommand(cmd, "-R");
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Renewed ticket. kinit output: {}", output);
-            }
-            reloginFromTicketCache();
-            tgt = getTGT();
-            if (tgt == null) {
-              LOG.warn("No TGT after renewal. Aborting renew thread for " +
-                  getUserName());
-              return;
-            }
-            nextRefresh = Math.max(getRefreshTime(tgt),
-              now + kerberosMinSecondsBeforeRelogin);
-            metrics.renewalFailures.set(0);
-            rp = null;
-          } catch (InterruptedException ie) {
-            LOG.warn("Terminating renewal thread");
+            tgtEndTime = tgt.getEndTime().getTime();
+          } catch (NullPointerException npe) {
+            LOG.error("NPE thrown while getting KerberosTicket endTime. "
+                + "Aborting renew thread for {}.", getUserName());
+            return;
+          }
+
+          LOG.warn("Exception encountered while running the renewal "
+                  + "command for {}. (TGT end time:{}, renewalFailures: {},"
+                  + "renewalFailuresTotal: {})", getUserName(), tgtEndTime,
+              metrics.renewalFailures.value(),
+              metrics.renewalFailuresTotal.value(), ie);
+          if (rp == null) {
+            // Use a dummy maxRetries to create the policy. The policy will
+            // only be used to get next retry time with exponential back-off.
+            // The final retry time will be later limited within the
+            // tgt endTime in getNextTgtRenewalTime.
+            rp = RetryPolicies.exponentialBackoffRetry(Long.SIZE - 2,
+                kerberosMinSecondsBeforeRelogin, TimeUnit.MILLISECONDS);
+          }
+          try {
+            nextRefresh = getNextTgtRenewalTime(tgtEndTime, now, rp);
+          } catch (Exception e) {
+            LOG.error("Exception when calculating next tgt renewal time", e);
+            return;
+          }
+          metrics.renewalFailures.incr();
+          // retry until close enough to tgt endTime.
+          if (now > nextRefresh) {
+            LOG.error("TGT is expired. Aborting renew thread for {}.",
+                getUserName());
             return;
-          } catch (IOException ie) {
-            metrics.renewalFailuresTotal.incr();
-            final long tgtEndTime = tgt.getEndTime().getTime();
-            LOG.warn("Exception encountered while running the renewal "
-                    + "command for {}. (TGT end time:{}, renewalFailures: {},"
-                    + "renewalFailuresTotal: {})", getUserName(), tgtEndTime,
-                metrics.renewalFailures, metrics.renewalFailuresTotal, ie);
-            final long now = Time.now();
-            if (rp == null) {
-              // Use a dummy maxRetries to create the policy. The policy will
-              // only be used to get next retry time with exponential back-off.
-              // The final retry time will be later limited within the
-              // tgt endTime in getNextTgtRenewalTime.
-              rp = RetryPolicies.exponentialBackoffRetry(Long.SIZE - 2,
-                  kerberosMinSecondsBeforeRelogin, TimeUnit.MILLISECONDS);
-            }
-            try {
-              nextRefresh = getNextTgtRenewalTime(tgtEndTime, now, rp);
-            } catch (Exception e) {
-              LOG.error("Exception when calculating next tgt renewal time", e);
-              return;
-            }
-            metrics.renewalFailures.incr();
-            // retry until close enough to tgt endTime.
-            if (now > nextRefresh) {
-              LOG.error("TGT is expired. Aborting renew thread for {}.",
-                  getUserName());
-              return;
-            }
           }
         }
-      }
-    });
-    t.setDaemon(true);
-    t.setName("TGT Renewer for " + getUserName());
-    t.start();
+      } while (runRenewalLoop);
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77721f39/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
index 9477990..011e930 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
@@ -47,6 +47,7 @@ import org.slf4j.event.Level;
 
 import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.kerberos.KerberosTicket;
 import javax.security.auth.kerberos.KeyTab;
 import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.LoginContext;
@@ -61,6 +62,7 @@ import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.ConcurrentModificationException;
 import java.util.Date;
+import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.Set;
 import java.util.concurrent.Callable;
@@ -88,7 +90,10 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 
 public class TestUserGroupInformation {
@@ -1211,4 +1216,37 @@ public class TestUserGroupInformation {
     barrier.await();
     assertSame(testUgi1.getSubject(), blockingLookup.get().getSubject());
   }
+
+  @Test
+  public void testKerberosTicketIsDestroyedChecked() throws Exception {
+    // Create UserGroupInformation
+    GenericTestUtils.setLogLevel(UserGroupInformation.LOG, Level.DEBUG);
+    Set<User> users = new HashSet<>();
+    users.add(new User("Foo"));
+    Subject subject =
+        new Subject(true, users, new HashSet<>(), new HashSet<>());
+    UserGroupInformation ugi = spy(new UserGroupInformation(subject));
+
+    // throw IOException in the middle of the autoRenewalForUserCreds
+    doThrow(new IOException()).when(ugi).reloginFromTicketCache();
+
+    // Create and destroy the KerberosTicket, so endTime will be null
+    Date d = new Date();
+    KerberosPrincipal kp = new KerberosPrincipal("Foo");
+    KerberosTicket tgt = spy(new KerberosTicket(new byte[]{}, kp, kp, new
+        byte[]{}, 0, null, d, d, d, d, null));
+    tgt.destroy();
+
+    // run AutoRenewalForUserCredsRunnable with this
+    UserGroupInformation.AutoRenewalForUserCredsRunnable userCredsRunnable =
+        ugi.new AutoRenewalForUserCredsRunnable(tgt,
+            Boolean.toString(Boolean.TRUE), 100);
+
+    // Set the runnable to not to run in a loop
+    userCredsRunnable.setRunRenewalLoop(false);
+    // there should be no exception when calling this
+    userCredsRunnable.run();
+    // isDestroyed should be called at least once
+    Mockito.verify(tgt, atLeastOnce()).isDestroyed();
+  }
 }


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


[11/50] hadoop git commit: YARN-8546. Resource leak caused by a reserved container being released more than once under async scheduling. Contributed by Tao Yang.

Posted by eh...@apache.org.
YARN-8546. Resource leak caused by a reserved container being released more than once under async scheduling. Contributed by Tao Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5be9f4a5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5be9f4a5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5be9f4a5

Branch: refs/heads/HDFS-12090
Commit: 5be9f4a5d05c9cb99348719fe35626b1de3055db
Parents: 955f795
Author: Weiwei Yang <ww...@apache.org>
Authored: Wed Jul 25 17:35:27 2018 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Wed Jul 25 17:35:27 2018 +0800

----------------------------------------------------------------------
 .../scheduler/common/fica/FiCaSchedulerApp.java | 15 ++++
 .../TestCapacitySchedulerAsyncScheduling.java   | 89 ++++++++++++++++++++
 2 files changed, 104 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5be9f4a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
index 3b1b82c..9810e98 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
@@ -361,6 +361,21 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
         .isEmpty()) {
       for (SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode>
           releaseContainer : allocation.getToRelease()) {
+        // Make sure to-release reserved containers are not outdated
+        if (releaseContainer.getRmContainer().getState()
+            == RMContainerState.RESERVED
+            && releaseContainer.getRmContainer() != releaseContainer
+            .getSchedulerNode().getReservedContainer()) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Failed to accept this proposal because "
+                + "it tries to release an outdated reserved container "
+                + releaseContainer.getRmContainer().getContainerId()
+                + " on node " + releaseContainer.getSchedulerNode().getNodeID()
+                + " whose reserved container is "
+                + releaseContainer.getSchedulerNode().getReservedContainer());
+          }
+          return false;
+        }
         // Only consider non-reserved container (reserved container will
         // not affect available resource of node) on the same node
         if (releaseContainer.getRmContainer().getState()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5be9f4a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java
index 338b9f9..c2c1519 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -41,6 +42,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -685,6 +687,93 @@ public class TestCapacitySchedulerAsyncScheduling {
     rm.stop();
   }
 
+
+  @Test(timeout = 60000)
+  public void testReleaseOutdatedReservedContainer() throws Exception {
+    /*
+     * Submit a application, reserved container_02 on nm1,
+     * submit two allocate proposals which contain the same reserved
+     * container_02 as to-released container.
+     * First proposal should be accepted, second proposal should be rejected
+     * because it try to release an outdated reserved container
+     */
+    MockRM rm1 = new MockRM();
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 8 * GB);
+    MockNM nm3 = rm1.registerNode("h3:1234", 8 * GB);
+    rm1.drainEvents();
+
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    LeafQueue defaultQueue = (LeafQueue) cs.getQueue("default");
+    SchedulerNode sn1 = cs.getSchedulerNode(nm1.getNodeId());
+    SchedulerNode sn2 = cs.getSchedulerNode(nm2.getNodeId());
+    SchedulerNode sn3 = cs.getSchedulerNode(nm3.getNodeId());
+
+    // launch another app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(4 * GB, "app", "user", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+    Resource allocateResource = Resources.createResource(5 * GB);
+    am1.allocate("*", (int) allocateResource.getMemorySize(), 3, 0,
+        new ArrayList<ContainerId>(), "");
+    FiCaSchedulerApp schedulerApp1 =
+        cs.getApplicationAttempt(am1.getApplicationAttemptId());
+
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    Assert.assertEquals(1, schedulerApp1.getReservedContainers().size());
+    Assert.assertEquals(9 * GB,
+        defaultQueue.getQueueResourceUsage().getUsed().getMemorySize());
+
+    RMContainer reservedContainer =
+        schedulerApp1.getReservedContainers().get(0);
+    ResourceCommitRequest allocateFromSameReservedContainerProposal1 =
+        createAllocateFromReservedProposal(3, allocateResource, schedulerApp1,
+            sn2, sn1, cs.getRMContext(), reservedContainer);
+    boolean tryCommitResult = cs.tryCommit(cs.getClusterResource(),
+        allocateFromSameReservedContainerProposal1, true);
+    Assert.assertTrue(tryCommitResult);
+    ResourceCommitRequest allocateFromSameReservedContainerProposal2 =
+        createAllocateFromReservedProposal(4, allocateResource, schedulerApp1,
+            sn3, sn1, cs.getRMContext(), reservedContainer);
+    tryCommitResult = cs.tryCommit(cs.getClusterResource(),
+        allocateFromSameReservedContainerProposal2, true);
+    Assert.assertFalse("This proposal should be rejected because "
+        + "it try to release an outdated reserved container", tryCommitResult);
+
+    rm1.close();
+  }
+
+  private ResourceCommitRequest createAllocateFromReservedProposal(
+      int containerId, Resource allocateResource, FiCaSchedulerApp schedulerApp,
+      SchedulerNode allocateNode, SchedulerNode reservedNode,
+      RMContext rmContext, RMContainer reservedContainer) {
+    Container container = Container.newInstance(
+        ContainerId.newContainerId(schedulerApp.getApplicationAttemptId(), containerId),
+        allocateNode.getNodeID(), allocateNode.getHttpAddress(), allocateResource,
+        Priority.newInstance(0), null);
+    RMContainer rmContainer = new RMContainerImpl(container, SchedulerRequestKey
+        .create(ResourceRequest
+            .newInstance(Priority.newInstance(0), "*", allocateResource, 1)),
+        schedulerApp.getApplicationAttemptId(), allocateNode.getNodeID(), "user",
+        rmContext);
+    SchedulerContainer allocateContainer =
+        new SchedulerContainer(schedulerApp, allocateNode, rmContainer, "", true);
+    SchedulerContainer reservedSchedulerContainer =
+        new SchedulerContainer(schedulerApp, reservedNode, reservedContainer, "",
+            false);
+    List<SchedulerContainer> toRelease = new ArrayList<>();
+    toRelease.add(reservedSchedulerContainer);
+    ContainerAllocationProposal allocateFromReservedProposal =
+        new ContainerAllocationProposal(allocateContainer, toRelease, null,
+            NodeType.OFF_SWITCH, NodeType.OFF_SWITCH,
+            SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY, allocateResource);
+    List<ContainerAllocationProposal> allocateProposals = new ArrayList<>();
+    allocateProposals.add(allocateFromReservedProposal);
+    return new ResourceCommitRequest(allocateProposals, null, null);
+  }
+
   private void keepNMHeartbeat(List<MockNM> mockNMs, int interval) {
     if (nmHeartbeatThread != null) {
       nmHeartbeatThread.setShouldStop();


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


[43/50] hadoop git commit: HDFS-12716. 'dfs.datanode.failed.volumes.tolerated' to support minimum number of volumes to be available. Contributed by Ranith Sardar and usharani

Posted by eh...@apache.org.
HDFS-12716. 'dfs.datanode.failed.volumes.tolerated' to support minimum number of volumes to be available. Contributed by Ranith Sardar and usharani


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3108d27e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3108d27e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3108d27e

Branch: refs/heads/HDFS-12090
Commit: 3108d27edde941d153a58f71fb1096cce2995531
Parents: 63e08ec
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Mon Jul 30 15:50:04 2018 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Mon Jul 30 15:50:04 2018 +0530

----------------------------------------------------------------------
 .../hadoop/hdfs/server/datanode/DataNode.java   |  7 +++-
 .../datanode/checker/DatasetVolumeChecker.java  |  6 ++-
 .../checker/StorageLocationChecker.java         | 28 ++++++++++----
 .../datanode/fsdataset/impl/FsDatasetImpl.java  | 40 ++++++++++++++++----
 .../src/main/resources/hdfs-default.xml         |  2 +
 .../TestDataNodeVolumeFailureToleration.java    |  6 ++-
 6 files changed, 68 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3108d27e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 7df92f6..1e9c57a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -293,6 +293,8 @@ public class DataNode extends ReconfigurableBase
       "  and rolling upgrades.";
 
   static final int CURRENT_BLOCK_FORMAT_VERSION = 1;
+  public static final int MAX_VOLUME_FAILURE_TOLERATED_LIMIT = -1;
+  public static final String MAX_VOLUME_FAILURES_TOLERATED_MSG = "should be greater than -1";
 
   /** A list of property that are reconfigurable at runtime. */
   private static final List<String> RECONFIGURABLE_PROPERTIES =
@@ -1389,10 +1391,11 @@ public class DataNode extends ReconfigurableBase
 
     int volFailuresTolerated = dnConf.getVolFailuresTolerated();
     int volsConfigured = dnConf.getVolsConfigured();
-    if (volFailuresTolerated < 0 || volFailuresTolerated >= volsConfigured) {
+    if (volFailuresTolerated < MAX_VOLUME_FAILURE_TOLERATED_LIMIT
+        || volFailuresTolerated >= volsConfigured) {
       throw new DiskErrorException("Invalid value configured for "
           + "dfs.datanode.failed.volumes.tolerated - " + volFailuresTolerated
-          + ". Value configured is either less than 0 or >= "
+          + ". Value configured is either greater than -1 or >= "
           + "to the number of configured volumes (" + volsConfigured + ").");
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3108d27e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java
index 3889e23..30602c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java
@@ -28,6 +28,7 @@ import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -153,10 +154,11 @@ public class DatasetVolumeChecker {
 
     lastAllVolumesCheck = timer.monotonicNow() - minDiskCheckGapMs;
 
-    if (maxVolumeFailuresTolerated < 0) {
+    if (maxVolumeFailuresTolerated < DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
       throw new DiskErrorException("Invalid value configured for "
           + DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY + " - "
-          + maxVolumeFailuresTolerated + " (should be non-negative)");
+          + maxVolumeFailuresTolerated + " "
+          + DataNode.MAX_VOLUME_FAILURES_TOLERATED_MSG);
     }
 
     delegateChecker = new ThrottledAsyncChecker<>(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3108d27e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/StorageLocationChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/StorageLocationChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/StorageLocationChecker.java
index 81575e2..dabaa83 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/StorageLocationChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/StorageLocationChecker.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation.CheckContext;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
@@ -105,10 +106,11 @@ public class StorageLocationChecker {
         DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
         DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
 
-    if (maxVolumeFailuresTolerated < 0) {
+    if (maxVolumeFailuresTolerated < DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
       throw new DiskErrorException("Invalid value configured for "
           + DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY + " - "
-          + maxVolumeFailuresTolerated + " (should be non-negative)");
+          + maxVolumeFailuresTolerated + " "
+          + DataNode.MAX_VOLUME_FAILURES_TOLERATED_MSG);
     }
 
     this.timer = timer;
@@ -213,12 +215,22 @@ public class StorageLocationChecker {
       }
     }
 
-    if (failedLocations.size() > maxVolumeFailuresTolerated) {
-      throw new DiskErrorException("Too many failed volumes - "
-          + "current valid volumes: " + goodLocations.size()
-          + ", volumes configured: " + dataDirs.size()
-          + ", volumes failed: " + failedLocations.size()
-          + ", volume failures tolerated: " + maxVolumeFailuresTolerated);
+    if (maxVolumeFailuresTolerated == DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
+      if (dataDirs.size() == failedLocations.size()) {
+        throw new DiskErrorException("Too many failed volumes - "
+            + "current valid volumes: " + goodLocations.size()
+            + ", volumes configured: " + dataDirs.size()
+            + ", volumes failed: " + failedLocations.size()
+            + ", volume failures tolerated: " + maxVolumeFailuresTolerated);
+      }
+    } else {
+      if (failedLocations.size() > maxVolumeFailuresTolerated) {
+        throw new DiskErrorException("Too many failed volumes - "
+            + "current valid volumes: " + goodLocations.size()
+            + ", volumes configured: " + dataDirs.size()
+            + ", volumes failed: " + failedLocations.size()
+            + ", volume failures tolerated: " + maxVolumeFailuresTolerated);
+      }
     }
 
     if (goodLocations.size() == 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3108d27e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 89c278a..d7f133e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -237,6 +237,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   final FsDatasetCache cacheManager;
   private final Configuration conf;
   private final int volFailuresTolerated;
+  private final int volsConfigured;
   private volatile boolean fsRunning;
 
   final ReplicaMap volumeMap;
@@ -285,15 +286,32 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     List<VolumeFailureInfo> volumeFailureInfos = getInitialVolumeFailureInfos(
         dataLocations, storage);
 
-    int volsConfigured = datanode.getDnConf().getVolsConfigured();
+    volsConfigured = datanode.getDnConf().getVolsConfigured();
     int volsFailed = volumeFailureInfos.size();
 
-    if (volsFailed > volFailuresTolerated) {
-      throw new DiskErrorException("Too many failed volumes - "
-          + "current valid volumes: " + storage.getNumStorageDirs() 
-          + ", volumes configured: " + volsConfigured 
-          + ", volumes failed: " + volsFailed
-          + ", volume failures tolerated: " + volFailuresTolerated);
+    if (volFailuresTolerated < DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT
+        || volFailuresTolerated >= volsConfigured) {
+      throw new DiskErrorException("Invalid value configured for "
+          + "dfs.datanode.failed.volumes.tolerated - " + volFailuresTolerated
+          + ". Value configured is either less than maxVolumeFailureLimit or greater than "
+          + "to the number of configured volumes (" + volsConfigured + ").");
+    }
+    if (volFailuresTolerated == DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
+      if (volsConfigured == volsFailed) {
+        throw new DiskErrorException(
+            "Too many failed volumes - " + "current valid volumes: "
+                + storage.getNumStorageDirs() + ", volumes configured: "
+                + volsConfigured + ", volumes failed: " + volsFailed
+                + ", volume failures tolerated: " + volFailuresTolerated);
+      }
+    } else {
+      if (volsFailed > volFailuresTolerated) {
+        throw new DiskErrorException(
+            "Too many failed volumes - " + "current valid volumes: "
+                + storage.getNumStorageDirs() + ", volumes configured: "
+                + volsConfigured + ", volumes failed: " + volsFailed
+                + ", volume failures tolerated: " + volFailuresTolerated);
+      }
     }
 
     storageMap = new ConcurrentHashMap<String, DatanodeStorage>();
@@ -597,7 +615,13 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
    */
   @Override // FsDatasetSpi
   public boolean hasEnoughResource() {
-    return getNumFailedVolumes() <= volFailuresTolerated;
+    if (volFailuresTolerated == DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
+      // If volFailuresTolerated configured maxVolumeFailureLimit then minimum
+      // one volume is required.
+      return volumes.getVolumes().size() >= 1;
+    } else {
+      return getNumFailedVolumes() <= volFailuresTolerated;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3108d27e/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index a10be27..9e73197 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -1276,6 +1276,8 @@
   <description>The number of volumes that are allowed to
   fail before a datanode stops offering service. By default
   any volume failure will cause a datanode to shutdown.
+  The range of the value is -1 now, -1 represents the minimum
+  of volume valids is 1.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3108d27e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
index f83609a..825887c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
@@ -201,7 +201,11 @@ public class TestDataNodeVolumeFailureToleration {
   @Test
   public void testVolumeAndTolerableConfiguration() throws Exception {
     // Check if Block Pool Service exit for an invalid conf value.
-    testVolumeConfig(-1, 0, false, true);
+    testVolumeConfig(-2, 0, false, true);
+    // Test for one good volume at least
+    testVolumeConfig(-1, 0, true, true);
+    testVolumeConfig(-1, 1, true, true);
+    testVolumeConfig(-1, 2, false, true);
 
     // Ditto if the value is too big.
     testVolumeConfig(100, 0, false, true);


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


[20/50] hadoop git commit: HADOOP-15611. Log more details for FairCallQueue. Contributed by Ryan Wu.

Posted by eh...@apache.org.
HADOOP-15611. Log more details for FairCallQueue. Contributed by Ryan Wu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9089790c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9089790c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9089790c

Branch: refs/heads/HDFS-12090
Commit: 9089790cabb4771198be0fe64c1317a3ff1c80f1
Parents: f93ecf5
Author: Yiqun Lin <yq...@apache.org>
Authored: Thu Jul 26 18:08:28 2018 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Thu Jul 26 18:08:28 2018 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java   | 8 ++++++++
 .../org/apache/hadoop/ipc/WeightedRoundRobinMultiplexer.java | 3 +++
 2 files changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9089790c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java
index f12ecb6..8bb0ce4 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java
@@ -391,6 +391,7 @@ public class DecayRpcScheduler implements RpcScheduler,
    * counts current.
    */
   private void decayCurrentCounts() {
+    LOG.debug("Start to decay current counts.");
     try {
       long totalDecayedCount = 0;
       long totalRawCount = 0;
@@ -410,7 +411,12 @@ public class DecayRpcScheduler implements RpcScheduler,
         totalDecayedCount += nextValue;
         decayedCount.set(nextValue);
 
+        LOG.debug("Decaying counts for the user: {}, " +
+            "its decayedCount: {}, rawCount: {}", entry.getKey(),
+            nextValue, rawCount.get());
         if (nextValue == 0) {
+          LOG.debug("The decayed count for the user {} is zero " +
+              "and being cleaned.", entry.getKey());
           // We will clean up unused keys here. An interesting optimization
           // might be to have an upper bound on keyspace in callCounts and only
           // clean once we pass it.
@@ -422,6 +428,8 @@ public class DecayRpcScheduler implements RpcScheduler,
       totalDecayedCallCount.set(totalDecayedCount);
       totalRawCallCount.set(totalRawCount);
 
+      LOG.debug("After decaying the stored counts, totalDecayedCount: {}, " +
+          "totalRawCallCount: {}.", totalDecayedCount, totalRawCount);
       // Now refresh the cache of scheduling decisions
       recomputeScheduleCache();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9089790c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WeightedRoundRobinMultiplexer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WeightedRoundRobinMultiplexer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WeightedRoundRobinMultiplexer.java
index d308725..096cc1a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WeightedRoundRobinMultiplexer.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WeightedRoundRobinMultiplexer.java
@@ -109,6 +109,9 @@ public class WeightedRoundRobinMultiplexer implements RpcMultiplexer {
     // Finally, reset requestsLeft. This will enable moveToNextQueue to be
     // called again, for the new currentQueueIndex
     this.requestsLeft.set(this.queueWeights[nextIdx]);
+    LOG.debug("Moving to next queue from queue index {} to index {}, " +
+        "number of requests left for current queue: {}.",
+        thisIdx, nextIdx, requestsLeft);
   }
 
   /**


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


[23/50] hadoop git commit: HDDS-277. PipelineStateMachine should handle closure of pipelines in SCM. Contributed by Mukul Kumar Singh.

Posted by eh...@apache.org.
HDDS-277. PipelineStateMachine should handle closure of pipelines in SCM. Contributed by Mukul Kumar Singh.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fd31cb6c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fd31cb6c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fd31cb6c

Branch: refs/heads/HDFS-12090
Commit: fd31cb6cfeef0c7e9bb0a054cb0f78853df8976f
Parents: be150a1
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Thu Jul 26 13:15:27 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Thu Jul 26 13:15:55 2018 -0700

----------------------------------------------------------------------
 .../container/common/helpers/ContainerInfo.java |   7 +-
 .../container/CloseContainerEventHandler.java   |  28 ++--
 .../hdds/scm/container/ContainerMapping.java    |  16 +-
 .../scm/container/ContainerStateManager.java    |  11 ++
 .../scm/container/states/ContainerStateMap.java |   2 +-
 .../hdds/scm/pipelines/Node2PipelineMap.java    |  33 ++--
 .../hdds/scm/pipelines/PipelineManager.java     |  31 ++--
 .../hdds/scm/pipelines/PipelineSelector.java    |  70 +++++++--
 .../scm/pipelines/ratis/RatisManagerImpl.java   |  14 +-
 .../standalone/StandaloneManagerImpl.java       |  13 +-
 .../scm/server/StorageContainerManager.java     |   2 +-
 .../hadoop/hdds/scm/block/TestBlockManager.java |   4 +-
 .../TestCloseContainerEventHandler.java         |  13 +-
 .../scm/container/TestContainerMapping.java     |   4 +-
 .../container/closer/TestContainerCloser.java   |   4 +-
 .../hdds/scm/node/TestContainerPlacement.java   |   3 +-
 .../hdds/scm/pipeline/TestPipelineClose.java    | 152 +++++++++++++++++++
 .../hadoop/ozone/scm/TestContainerSQLCli.java   |   4 +-
 18 files changed, 331 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
index 4074b21..b194c14 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
@@ -459,12 +459,13 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
 
   /**
    * Check if a container is in open state, this will check if the
-   * container is either open or allocated or creating. Any containers in
-   * these states is managed as an open container by SCM.
+   * container is either open, allocated, creating or creating.
+   * Any containers in these states is managed as an open container by SCM.
    */
   public boolean isContainerOpen() {
     return state == HddsProtos.LifeCycleState.ALLOCATED ||
         state == HddsProtos.LifeCycleState.CREATING ||
-        state == HddsProtos.LifeCycleState.OPEN;
+        state == HddsProtos.LifeCycleState.OPEN ||
+        state == HddsProtos.LifeCycleState.CLOSING;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java
index 859e5d5..949eb13 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java
@@ -21,7 +21,6 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
-import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.server.events.EventHandler;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
 import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
@@ -63,13 +62,13 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
           containerManager.getContainerWithPipeline(containerID.getId());
       info = containerWithPipeline.getContainerInfo();
       if (info == null) {
-        LOG.info("Failed to update the container state. Container with id : {} "
+        LOG.error("Failed to update the container state. Container with id : {} "
             + "does not exist", containerID.getId());
         return;
       }
     } catch (IOException e) {
-      LOG.info("Failed to update the container state. Container with id : {} "
-          + "does not exist", containerID.getId());
+      LOG.error("Failed to update the container state. Container with id : {} "
+          + "does not exist", containerID.getId(), e);
       return;
     }
 
@@ -85,11 +84,22 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
       try {
         // Finalize event will make sure the state of the container transitions
         // from OPEN to CLOSING in containerStateManager.
-        containerManager.getStateManager()
-            .updateContainerState(info, HddsProtos.LifeCycleEvent.FINALIZE);
-      } catch (SCMException ex) {
-        LOG.error("Failed to update the container state for container : {}"
-            + containerID);
+        containerManager.updateContainerState(containerID.getId(),
+            HddsProtos.LifeCycleEvent.FINALIZE);
+      } catch (IOException ex) {
+        LOG.error("Failed to update the container state to FINALIZE for"
+            + "container : {}" + containerID, ex);
+      }
+    } else if (info.getState() == HddsProtos.LifeCycleState.ALLOCATED) {
+      try {
+        // Create event will make sure the state of the container transitions
+        // from OPEN to CREATING in containerStateManager, this will move
+        // the container out of active allocation path.
+        containerManager.updateContainerState(containerID.getId(),
+            HddsProtos.LifeCycleEvent.CREATE);
+      } catch (IOException ex) {
+        LOG.error("Failed to update the container state to CREATE for"
+            + "container:{}" + containerID, ex);
       }
     } else {
       LOG.info("container with id : {} is in {} state and need not be closed.",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
index e17fe3d..d84551a 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
+import org.apache.hadoop.hdds.server.events.EventPublisher;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.lease.Lease;
 import org.apache.hadoop.ozone.lease.LeaseException;
@@ -104,7 +105,7 @@ public class ContainerMapping implements Mapping {
   @SuppressWarnings("unchecked")
   public ContainerMapping(
       final Configuration conf, final NodeManager nodeManager, final int
-      cacheSizeMB) throws IOException {
+      cacheSizeMB, EventPublisher eventPublisher) throws IOException {
     this.nodeManager = nodeManager;
     this.cacheSize = cacheSizeMB;
     this.closer = new ContainerCloser(nodeManager, conf);
@@ -122,14 +123,15 @@ public class ContainerMapping implements Mapping {
 
     this.lock = new ReentrantLock();
 
-    this.pipelineSelector = new PipelineSelector(nodeManager, conf);
-
     // To be replaced with code getStorageSize once it is committed.
     size = conf.getLong(OZONE_SCM_CONTAINER_SIZE_GB,
         OZONE_SCM_CONTAINER_SIZE_DEFAULT) * 1024 * 1024 * 1024;
     this.containerStateManager =
         new ContainerStateManager(conf, this);
 
+    this.pipelineSelector = new PipelineSelector(nodeManager,
+        containerStateManager, conf, eventPublisher);
+
     this.containerCloseThreshold = conf.getFloat(
         ScmConfigKeys.OZONE_SCM_CONTAINER_CLOSE_THRESHOLD,
         ScmConfigKeys.OZONE_SCM_CONTAINER_CLOSE_THRESHOLD_DEFAULT);
@@ -372,6 +374,12 @@ public class ContainerMapping implements Mapping {
       // Like releasing the lease in case of BEGIN_CREATE.
       ContainerInfo updatedContainer = containerStateManager
           .updateContainerState(containerInfo, event);
+      if (!updatedContainer.isContainerOpen()) {
+        Pipeline pipeline = pipelineSelector
+            .getPipeline(containerInfo.getPipelineName(),
+                containerInfo.getReplicationType());
+        pipelineSelector.closePipelineIfNoOpenContainers(pipeline);
+      }
       containerStore.put(dbKey, updatedContainer.getProtobuf().toByteArray());
       return updatedContainer.getState();
     } catch (LeaseException e) {
@@ -446,7 +454,7 @@ public class ContainerMapping implements Mapping {
         .getPipeline(containerInfo.getPipelineName(),
             containerInfo.getReplicationType());
     if (pipeline == null) {
-      pipelineSelector
+      pipeline = pipelineSelector
           .getReplicationPipeline(containerInfo.getReplicationType(),
               containerInfo.getReplicationFactor());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
index b2431dc..f0ab213 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
@@ -463,6 +463,17 @@ public class ContainerStateManager implements Closeable {
   }
 
   /**
+   * Returns a set of open ContainerIDs that reside on a pipeline.
+   *
+   * @param pipeline Pipeline of the Containers.
+   * @return Set of containers that match the specific query parameters.
+   */
+  public NavigableSet<ContainerID> getMatchingContainerIDsByPipeline(String
+      pipeline) {
+    return containers.getOpenContainerIDsByPipeline(pipeline);
+  }
+
+  /**
    * Returns the containerInfo with pipeline for the given container id.
    * @param selector -- Pipeline selector class.
    * @param containerID id of the container

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java
index 46fe2ab..b358b7c 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java
@@ -346,7 +346,7 @@ public class ContainerStateMap {
     }
     // In case the container is set to closed state, it needs to be removed from
     // the pipeline Map.
-    if (newState == LifeCycleState.CLOSED) {
+    if (!info.isContainerOpen()) {
       openPipelineMap.remove(info.getPipelineName(), id);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/Node2PipelineMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/Node2PipelineMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/Node2PipelineMap.java
index 2e89616..b860082 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/Node2PipelineMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/Node2PipelineMap.java
@@ -102,18 +102,27 @@ public class Node2PipelineMap {
         Collections.unmodifiableSet(v));
   }
 
-/**
- * Adds a pipeline entry to a given dataNode in the map.
- * @param pipeline Pipeline to be added
- */
- public synchronized void addPipeline(Pipeline pipeline) throws SCMException {
-   for (DatanodeDetails details : pipeline.getDatanodes().values()) {
-     UUID dnId = details.getUuid();
-     dn2PipelineMap
-         .computeIfAbsent(dnId,k->Collections.synchronizedSet(new HashSet<>()))
-         .add(pipeline);
-   }
- }
+  /**
+   * Adds a pipeline entry to a given dataNode in the map.
+   * @param pipeline Pipeline to be added
+   */
+  public synchronized void addPipeline(Pipeline pipeline) {
+    for (DatanodeDetails details : pipeline.getDatanodes().values()) {
+      UUID dnId = details.getUuid();
+      dn2PipelineMap
+          .computeIfAbsent(dnId,
+              k -> Collections.synchronizedSet(new HashSet<>()))
+          .add(pipeline);
+    }
+  }
+
+  public synchronized void removePipeline(Pipeline pipeline) {
+    for (DatanodeDetails details : pipeline.getDatanodes().values()) {
+      UUID dnId = details.getUuid();
+      dn2PipelineMap.computeIfPresent(dnId,
+          (k, v) -> {v.remove(pipeline); return v;});
+    }
+  }
 
   public Map<UUID, Set<Pipeline>> getDn2PipelineMap() {
     return Collections.unmodifiableMap(dn2PipelineMap);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java
index 77d8211..266b1f3 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java
@@ -38,14 +38,14 @@ public abstract class PipelineManager {
   private static final Logger LOG =
       LoggerFactory.getLogger(PipelineManager.class);
   private final List<Pipeline> activePipelines;
-  private final Map<String, Pipeline> activePipelineMap;
+  private final Map<String, Pipeline> pipelineMap;
   private final AtomicInteger pipelineIndex;
   private final Node2PipelineMap node2PipelineMap;
 
   public PipelineManager(Node2PipelineMap map) {
     activePipelines = new LinkedList<>();
     pipelineIndex = new AtomicInteger(0);
-    activePipelineMap = new WeakHashMap<>();
+    pipelineMap = new WeakHashMap<>();
     node2PipelineMap = map;
   }
 
@@ -85,8 +85,8 @@ public abstract class PipelineManager {
     Pipeline pipeline = null;
 
     // 1. Check if pipeline already exists
-    if (activePipelineMap.containsKey(pipelineName)) {
-      pipeline = activePipelineMap.get(pipelineName);
+    if (pipelineMap.containsKey(pipelineName)) {
+      pipeline = pipelineMap.get(pipelineName);
       LOG.debug("Returning pipeline for pipelineName:{}", pipelineName);
       return pipeline;
     } else {
@@ -115,11 +115,6 @@ public abstract class PipelineManager {
    */
   public abstract void initializePipeline(Pipeline pipeline) throws IOException;
 
-  public void removePipeline(Pipeline pipeline) {
-    activePipelines.remove(pipeline);
-    activePipelineMap.remove(pipeline.getPipelineName());
-  }
-
   /**
    * Find a Pipeline that is operational.
    *
@@ -172,16 +167,28 @@ public abstract class PipelineManager {
               + "replicationType:{} replicationFactor:{}",
           pipeline.getPipelineName(), replicationType, replicationFactor);
       activePipelines.add(pipeline);
-      activePipelineMap.put(pipeline.getPipelineName(), pipeline);
+      pipelineMap.put(pipeline.getPipelineName(), pipeline);
       node2PipelineMap.addPipeline(pipeline);
     }
     return pipeline;
   }
 
   /**
-   * Close the  pipeline with the given clusterId.
+   * Remove the pipeline from active allocation
+   * @param pipeline pipeline to be finalized
+   */
+  public synchronized void finalizePipeline(Pipeline pipeline) {
+    activePipelines.remove(pipeline);
+  }
+
+  /**
+   *
+   * @param pipeline
    */
-  public abstract void closePipeline(String pipelineID) throws IOException;
+  public void closePipeline(Pipeline pipeline) {
+    pipelineMap.remove(pipeline.getPipelineName());
+    node2PipelineMap.removePipeline(pipeline);
+  }
 
   /**
    * list members in the pipeline .

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
index b1e1dd0..ebe39d0 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
@@ -19,11 +19,14 @@ package org.apache.hadoop.hdds.scm.pipelines;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.container.placement.algorithms
     .ContainerPlacementPolicy;
 import org.apache.hadoop.hdds.scm.container.placement.algorithms
     .SCMContainerPlacementRandom;
+import org.apache.hadoop.hdds.scm.events.SCMEvents;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.hdds.scm.pipelines.ratis.RatisManagerImpl;
@@ -33,6 +36,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
+import org.apache.hadoop.hdds.server.events.EventPublisher;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.statemachine
     .InvalidStateTransitionException;
@@ -48,6 +52,7 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.HashSet;
 import java.util.List;
+import java.util.NavigableSet;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
@@ -65,6 +70,8 @@ public class PipelineSelector {
   private final ContainerPlacementPolicy placementPolicy;
   private final NodeManager nodeManager;
   private final Configuration conf;
+  private final ContainerStateManager containerStateManager;
+  private final EventPublisher eventPublisher;
   private final RatisManagerImpl ratisManager;
   private final StandaloneManagerImpl standaloneManager;
   private final long containerSize;
@@ -79,9 +86,12 @@ public class PipelineSelector {
    * @param nodeManager - node manager
    * @param conf - Ozone Config
    */
-  public PipelineSelector(NodeManager nodeManager, Configuration conf) {
+  public PipelineSelector(NodeManager nodeManager,
+      ContainerStateManager containerStateManager, Configuration conf,
+      EventPublisher eventPublisher) {
     this.nodeManager = nodeManager;
     this.conf = conf;
+    this.eventPublisher = eventPublisher;
     this.placementPolicy = createContainerPlacementPolicy(nodeManager, conf);
     this.containerSize = OzoneConsts.GB * this.conf.getInt(
         ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
@@ -99,6 +109,7 @@ public class PipelineSelector {
         ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_LEASE_TIMEOUT,
         ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_LEASE_TIMEOUT_DEFAULT,
         TimeUnit.MILLISECONDS);
+    this.containerStateManager = containerStateManager;
     pipelineLeaseManager = new LeaseManager<>("PipelineCreation",
         pipelineCreationLeaseTimeout);
     pipelineLeaseManager.start();
@@ -306,15 +317,54 @@ public class PipelineSelector {
   }
 
   /**
-   * Close the  pipeline with the given clusterId.
+   * Finalize a given pipeline.
    */
+  public void finalizePipeline(Pipeline pipeline) throws IOException {
+    PipelineManager manager = getPipelineManager(pipeline.getType());
+    Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
+    LOG.debug("Finalizing pipeline. pipelineID: {}", pipeline.getPipelineName());
+    // Remove the pipeline from active allocation
+    manager.finalizePipeline(pipeline);
+    updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.FINALIZE);
+    closePipelineIfNoOpenContainers(pipeline);
+  }
 
-  public void closePipeline(ReplicationType replicationType, String
-      pipelineID) throws IOException {
-    PipelineManager manager = getPipelineManager(replicationType);
+  /**
+   * Close a given pipeline.
+   */
+  public void closePipelineIfNoOpenContainers(Pipeline pipeline) throws IOException {
+    if (pipeline.getLifeCycleState() != LifeCycleState.CLOSING) {
+      return;
+    }
+    NavigableSet<ContainerID> containerIDS = containerStateManager
+        .getMatchingContainerIDsByPipeline(pipeline.getPipelineName());
+    if (containerIDS.size() == 0) {
+      updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.CLOSE);
+      LOG.info("Closing pipeline. pipelineID: {}", pipeline.getPipelineName());
+    }
+  }
+
+  /**
+   * Close a given pipeline.
+   */
+  private void closePipeline(Pipeline pipeline) {
+    PipelineManager manager = getPipelineManager(pipeline.getType());
     Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
-    LOG.debug("Closing pipeline. pipelineID: {}", pipelineID);
-    manager.closePipeline(pipelineID);
+    LOG.debug("Closing pipeline. pipelineID: {}", pipeline.getPipelineName());
+    NavigableSet<ContainerID> containers =
+        containerStateManager
+            .getMatchingContainerIDsByPipeline(pipeline.getPipelineName());
+    Preconditions.checkArgument(containers.size() == 0);
+    manager.closePipeline(pipeline);
+  }
+
+  private void closeContainersByPipeline(Pipeline pipeline) {
+    NavigableSet<ContainerID> containers =
+        containerStateManager
+            .getMatchingContainerIDsByPipeline(pipeline.getPipelineName());
+    for (ContainerID id : containers) {
+      eventPublisher.fireEvent(SCMEvents.CLOSE_CONTAINER, id);
+    }
   }
 
   /**
@@ -352,7 +402,7 @@ public class PipelineSelector {
         node2PipelineMap.getPipelines(dnId);
     for (Pipeline pipeline : pipelineSet) {
       getPipelineManager(pipeline.getType())
-          .removePipeline(pipeline);
+          .closePipeline(pipeline);
     }
     node2PipelineMap.removeDatanode(dnId);
   }
@@ -398,12 +448,12 @@ public class PipelineSelector {
         break;
 
       case FINALIZE:
-        //TODO: cleanup pipeline by closing all the containers on the pipeline
+        closeContainersByPipeline(pipeline);
         break;
 
       case CLOSE:
       case TIMEOUT:
-        // TODO: Release the nodes here when pipelines are destroyed
+        closePipeline(pipeline);
         break;
       default:
         throw new SCMException("Unsupported pipeline LifeCycleEvent.",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java
index c726ef6..fdd0605 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java
@@ -108,13 +108,15 @@ public class RatisManagerImpl extends PipelineManager {
   }
 
   /**
-   * Close the  pipeline with the given clusterId.
-   *
-   * @param pipelineID
+   * Close the pipeline.
    */
-  @Override
-  public void closePipeline(String pipelineID) throws IOException {
-
+  public void closePipeline(Pipeline pipeline) {
+    super.closePipeline(pipeline);
+    for (DatanodeDetails node : pipeline.getMachines()) {
+      // A node should always be the in ratis members list.
+      Preconditions.checkArgument(ratisMembers.remove(node));
+    }
+    //TODO: should the raft ring also be destroyed as well?
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/standalone/StandaloneManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/standalone/StandaloneManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/standalone/StandaloneManagerImpl.java
index bb4951f..0506e59 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/standalone/StandaloneManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/standalone/StandaloneManagerImpl.java
@@ -101,13 +101,14 @@ public class StandaloneManagerImpl extends PipelineManager {
   }
 
   /**
-   * Close the  pipeline with the given clusterId.
-   *
-   * @param pipelineID
+   * Close the pipeline.
    */
-  @Override
-  public void closePipeline(String pipelineID) throws IOException {
-
+  public void closePipeline(Pipeline pipeline) {
+    super.closePipeline(pipeline);
+    for (DatanodeDetails node : pipeline.getMachines()) {
+      // A node should always be the in standalone members list.
+      Preconditions.checkArgument(standAloneMembers.remove(node));
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
index 165805f..be8fb43 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
@@ -192,7 +192,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
     scmNodeManager = new SCMNodeManager(
         conf, scmStorage.getClusterID(), this, eventQueue);
     scmContainerManager = new ContainerMapping(
-        conf, getScmNodeManager(), cacheSize);
+        conf, getScmNodeManager(), cacheSize, eventQueue);
     scmBlockManager = new BlockManagerImpl(
         conf, getScmNodeManager(), scmContainerManager, eventQueue);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
index 06e7420..7049029 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.scm.container.ContainerMapping;
 import org.apache.hadoop.hdds.scm.container.MockNodeManager;
 import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.events.EventQueue;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.common.SCMTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -73,7 +74,8 @@ public class TestBlockManager {
       throw new IOException("Unable to create test directory path");
     }
     nodeManager = new MockNodeManager(true, 10);
-    mapping = new ContainerMapping(conf, nodeManager, 128);
+    mapping =
+        new ContainerMapping(conf, nodeManager, 128, new EventQueue());
     blockManager = new BlockManagerImpl(conf, nodeManager, mapping, null);
     if(conf.getBoolean(ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY,
         ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT)){

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
index 0764b12..543cad3 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
@@ -66,7 +66,8 @@ public class TestCloseContainerEventHandler {
     configuration
         .set(OzoneConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
     nodeManager = new MockNodeManager(true, 10);
-    mapping = new ContainerMapping(configuration, nodeManager, 128);
+    mapping = new ContainerMapping(configuration, nodeManager, 128,
+        new EventQueue());
     eventQueue = new EventQueue();
     eventQueue.addHandler(CLOSE_CONTAINER,
         new CloseContainerEventHandler(mapping));
@@ -122,12 +123,7 @@ public class TestCloseContainerEventHandler {
     // state, so firing close container event should not queue CLOSE
     // command in the Datanode
     Assert.assertEquals(0, nodeManager.getCommandCount(datanode));
-    // Make sure the information is logged
-    Assert.assertTrue(logCapturer.getOutput().contains(
-        "container with id : " + id.getId()
-            + " is in ALLOCATED state and need not be closed"));
     //Execute these state transitions so that we can close the container.
-    mapping.updateContainerState(id.getId(), CREATE);
     mapping.updateContainerState(id.getId(), CREATED);
     eventQueue.fireEvent(CLOSE_CONTAINER,
         new ContainerID(
@@ -164,12 +160,7 @@ public class TestCloseContainerEventHandler {
       Assert.assertEquals(closeCount[i], nodeManager.getCommandCount(details));
       i++;
     }
-    // Make sure the information is logged
-    Assert.assertTrue(logCapturer.getOutput().contains(
-        "container with id : " + id.getId()
-            + " is in ALLOCATED state and need not be closed"));
     //Execute these state transitions so that we can close the container.
-    mapping.updateContainerState(id.getId(), CREATE);
     mapping.updateContainerState(id.getId(), CREATED);
     eventQueue.fireEvent(CLOSE_CONTAINER, id);
     eventQueue.processAll(1000);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
index 79ac9cf..6269514 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
+import org.apache.hadoop.hdds.server.events.EventQueue;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.common.SCMTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -84,7 +85,8 @@ public class TestContainerMapping {
       throw new IOException("Unable to create test directory path");
     }
     nodeManager = new MockNodeManager(true, 10);
-    mapping = new ContainerMapping(conf, nodeManager, 128);
+    mapping = new ContainerMapping(conf, nodeManager, 128,
+        new EventQueue());
     xceiverClientManager = new XceiverClientManager(conf);
     random = new Random();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
index cc25544..0c0f25d 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
+import org.apache.hadoop.hdds.server.events.EventQueue;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.common.SCMTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -79,7 +80,8 @@ public class TestContainerCloser {
     configuration.set(OzoneConfigKeys.OZONE_METADATA_DIRS,
         testDir.getAbsolutePath());
     nodeManager = new MockNodeManager(true, 10);
-    mapping = new ContainerMapping(configuration, nodeManager, 128);
+    mapping = new ContainerMapping(configuration, nodeManager, 128,
+        new EventQueue());
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
index 2fef620..52963c0 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
@@ -105,9 +105,10 @@ public class TestContainerPlacement {
 
   ContainerMapping createContainerManager(Configuration config,
       NodeManager scmNodeManager) throws IOException {
+    EventQueue eventQueue = new EventQueue();
     final int cacheSize = config.getInt(OZONE_SCM_DB_CACHE_SIZE_MB,
         OZONE_SCM_DB_CACHE_SIZE_DEFAULT);
-    return new ContainerMapping(config, scmNodeManager, cacheSize);
+    return new ContainerMapping(config, scmNodeManager, cacheSize, eventQueue);
 
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java
new file mode 100644
index 0000000..24e25ab
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ *
+ */
+package org.apache.hadoop.hdds.scm.pipeline;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerMapping;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .ContainerWithPipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
+import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+import org.apache.hadoop.ozone.HddsDatanodeService;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.NavigableSet;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos
+    .ReplicationFactor.THREE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos
+    .ReplicationType.RATIS;
+
+public class TestPipelineClose {
+
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration conf;
+  private static StorageContainerManager scm;
+  private static ContainerWithPipeline ratisContainer1;
+  private static ContainerWithPipeline ratisContainer2;
+  private static ContainerStateMap stateMap;
+  private static ContainerMapping mapping;
+  private static PipelineSelector pipelineSelector;
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   *
+   * @throws IOException
+   */
+  @BeforeClass
+  public static void init() throws Exception {
+    conf = new OzoneConfiguration();
+    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(6).build();
+    cluster.waitForClusterToBeReady();
+    scm = cluster.getStorageContainerManager();
+    mapping = (ContainerMapping)scm.getScmContainerManager();
+    stateMap = mapping.getStateManager().getContainerStateMap();
+    ratisContainer1 = mapping.allocateContainer(RATIS, THREE, "testOwner");
+    ratisContainer2 = mapping.allocateContainer(RATIS, THREE, "testOwner");
+    pipelineSelector = mapping.getPipelineSelector();
+    // At this stage, there should be 2 pipeline one with 1 open container each.
+    // Try closing the both the pipelines, one with a closed container and
+    // the other with an open container.
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterClass
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+
+  @Test
+  public void testPipelineCloseWithClosedContainer() throws IOException {
+    NavigableSet<ContainerID> set = stateMap.getOpenContainerIDsByPipeline(
+        ratisContainer1.getPipeline().getPipelineName());
+
+    long cId = ratisContainer1.getContainerInfo().getContainerID();
+    Assert.assertEquals(1, set.size());
+    Assert.assertEquals(cId, set.first().getId());
+
+    // Now close the container and it should not show up while fetching
+    // containers by pipeline
+    mapping
+        .updateContainerState(cId, HddsProtos.LifeCycleEvent.CREATE);
+    mapping
+        .updateContainerState(cId, HddsProtos.LifeCycleEvent.CREATED);
+    mapping
+        .updateContainerState(cId, HddsProtos.LifeCycleEvent.FINALIZE);
+    mapping
+        .updateContainerState(cId, HddsProtos.LifeCycleEvent.CLOSE);
+
+    NavigableSet<ContainerID> setClosed = stateMap.getOpenContainerIDsByPipeline(
+        ratisContainer1.getPipeline().getPipelineName());
+    Assert.assertEquals(0, setClosed.size());
+
+    pipelineSelector.finalizePipeline(ratisContainer1.getPipeline());
+    Pipeline pipeline1 = pipelineSelector
+        .getPipeline(ratisContainer1.getPipeline().getPipelineName(),
+            ratisContainer1.getContainerInfo().getReplicationType());
+    Assert.assertNull(pipeline1);
+    Assert.assertEquals(ratisContainer1.getPipeline().getLifeCycleState(),
+        HddsProtos.LifeCycleState.CLOSED);
+    for (DatanodeDetails dn : ratisContainer1.getPipeline().getMachines()) {
+      // Assert that the pipeline has been removed from Node2PipelineMap as well
+      Assert.assertEquals(pipelineSelector.getNode2PipelineMap()
+          .getPipelines(dn.getUuid()).size(), 0);
+    }
+  }
+
+  @Test
+  public void testPipelineCloseWithOpenContainer() throws IOException,
+      TimeoutException, InterruptedException {
+    NavigableSet<ContainerID> setOpen = stateMap.getOpenContainerIDsByPipeline(
+        ratisContainer2.getPipeline().getPipelineName());
+    Assert.assertEquals(1, setOpen.size());
+
+    long cId2 = ratisContainer2.getContainerInfo().getContainerID();
+    mapping
+        .updateContainerState(cId2, HddsProtos.LifeCycleEvent.CREATE);
+    mapping
+        .updateContainerState(cId2, HddsProtos.LifeCycleEvent.CREATED);
+    pipelineSelector.finalizePipeline(ratisContainer2.getPipeline());
+    Assert.assertEquals(ratisContainer2.getPipeline().getLifeCycleState(),
+        HddsProtos.LifeCycleState.CLOSING);
+    Pipeline pipeline2 = pipelineSelector
+        .getPipeline(ratisContainer2.getPipeline().getPipelineName(),
+            ratisContainer2.getContainerInfo().getReplicationType());
+    Assert.assertEquals(pipeline2.getLifeCycleState(),
+        HddsProtos.LifeCycleState.CLOSING);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java
index a878627..65bd036 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.ozone.scm;
 
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.server.events.EventQueue;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -115,7 +116,8 @@ public class TestContainerSQLCli {
     cluster.getStorageContainerManager().stop();
 
     nodeManager = cluster.getStorageContainerManager().getScmNodeManager();
-    mapping = new ContainerMapping(conf, nodeManager, 128);
+    mapping = new ContainerMapping(conf, nodeManager, 128,
+        new EventQueue());
     blockManager = new BlockManagerImpl(conf, nodeManager, mapping, null);
 
     // blockManager.allocateBlock() will create containers if there is none


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


[29/50] hadoop git commit: HDDS-270. Move generic container util functions to ContianerUtils. Contributed by Hanisha Koneru.

Posted by eh...@apache.org.
HDDS-270. Move generic container util functions to ContianerUtils.
Contributed by Hanisha Koneru.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3cc7ce81
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3cc7ce81
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3cc7ce81

Branch: refs/heads/HDFS-12090
Commit: 3cc7ce816e4ffb7287aa05cc5e00b2e058b4a2a4
Parents: 64e739e
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Jul 27 07:12:21 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri Jul 27 07:12:21 2018 -0700

----------------------------------------------------------------------
 .../common/helpers/ContainerUtils.java          | 34 ++++++++++++--------
 .../container/common/interfaces/Container.java  |  6 ++++
 .../container/keyvalue/KeyValueContainer.java   | 22 +++++++++----
 .../helpers/KeyValueContainerLocationUtil.java  | 17 ----------
 .../container/ozoneimpl/ContainerReader.java    | 27 ++++++----------
 .../keyvalue/TestKeyValueContainer.java         | 24 ++++----------
 .../common/impl/TestContainerPersistence.java   | 13 +++-----
 7 files changed, 62 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3cc7ce81/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
index 77a891a..469c969 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
@@ -199,20 +199,6 @@ public final class ContainerUtils {
   }
 
   /**
-   * Returns container file location.
-   *
-   * @param containerData - Data
-   * @param location - Root path
-   * @return Path
-   */
-  public static File getContainerFile(ContainerData containerData,
-      Path location) {
-    return location.resolve(Long.toString(containerData
-        .getContainerID()).concat(CONTAINER_EXTENSION))
-        .toFile();
-  }
-
-  /**
    * Persistent a {@link DatanodeDetails} to a local file.
    *
    * @throws IOException when read/write error occurs
@@ -300,4 +286,24 @@ public final class ContainerUtils {
     }
   }
 
+  /**
+   * Get the .container file from the containerBaseDir
+   * @param containerBaseDir container base directory. The name of this
+   *                         directory is same as the containerID
+   * @return the .container file
+   */
+  public static File getContainerFile(File containerBaseDir) {
+    // Container file layout is
+    // .../<<containerID>>/metadata/<<containerID>>.container
+    String containerFilePath = OzoneConsts.CONTAINER_META_PATH + File.separator
+        + getContainerID(containerBaseDir) + OzoneConsts.CONTAINER_EXTENSION;
+    return new File(containerBaseDir, containerFilePath);
+  }
+
+  /**
+   * ContainerID can be decoded from the container base directory name
+   */
+  public static long getContainerID(File containerBaseDir) {
+    return Long.parseLong(containerBaseDir.getName());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3cc7ce81/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
index fe35e1d..fc91920 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 
+import java.io.File;
 import java.util.Map;
 
 
@@ -92,6 +93,11 @@ public interface Container extends RwLock {
   ContainerProtos.ContainerType getContainerType();
 
   /**
+   * Returns containerFile.
+   */
+  File getContainerFile();
+
+  /**
    * updates the DeleteTransactionId.
    * @param deleteTransactionId
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3cc7ce81/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index 14f731a..d0e77d2 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -114,18 +114,16 @@ public class KeyValueContainer implements Container {
 
       containerMetaDataPath = KeyValueContainerLocationUtil
           .getContainerMetaDataPath(hddsVolumeDir, scmId, containerID);
+      containerData.setMetadataPath(containerMetaDataPath.getPath());
+
       File chunksPath = KeyValueContainerLocationUtil.getChunksLocationPath(
           hddsVolumeDir, scmId, containerID);
 
-      File containerFile = KeyValueContainerLocationUtil.getContainerFile(
-          containerMetaDataPath, containerID);
-      File dbFile = KeyValueContainerLocationUtil.getContainerDBFile(
-          containerMetaDataPath, containerID);
-
       // Check if it is new Container.
       ContainerUtils.verifyIsNewContainer(containerMetaDataPath);
 
       //Create Metadata path chunks path and metadata db
+      File dbFile = getContainerDBFile();
       KeyValueContainerUtil.createContainerMetaData(containerMetaDataPath,
           chunksPath, dbFile, config);
 
@@ -133,13 +131,13 @@ public class KeyValueContainer implements Container {
           OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_DEFAULT);
 
       //Set containerData for the KeyValueContainer.
-      containerData.setMetadataPath(containerMetaDataPath.getPath());
       containerData.setChunksPath(chunksPath.getPath());
       containerData.setContainerDBType(impl);
       containerData.setDbFile(dbFile);
       containerData.setVolume(containerVolume);
 
       // Create .container file
+      File containerFile = getContainerFile();
       writeToContainerFile(containerFile, true);
 
     } catch (StorageContainerException ex) {
@@ -415,12 +413,22 @@ public class KeyValueContainer implements Container {
    * Returns containerFile.
    * @return .container File name
    */
-  private File getContainerFile() {
+  @Override
+  public File getContainerFile() {
     return new File(containerData.getMetadataPath(), containerData
         .getContainerID() + OzoneConsts.CONTAINER_EXTENSION);
   }
 
   /**
+   * Returns container DB file
+   * @return
+   */
+  public File getContainerDBFile() {
+    return new File(containerData.getMetadataPath(), containerData
+        .getContainerID() + OzoneConsts.DN_CONTAINER_DB);
+  }
+
+  /**
    * Creates a temporary file.
    * @param file
    * @return

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3cc7ce81/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
index 02a8e73..0a81ed8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
@@ -99,27 +99,10 @@ public final class KeyValueContainerLocationUtil {
   }
 
   /**
-   * Returns containerFile.
-   * @param containerMetaDataPath
-   * @param containerID
-   * @return .container File name
-   */
-  public static File getContainerFile(File containerMetaDataPath,
-      long containerID) {
-    Preconditions.checkNotNull(containerMetaDataPath);
-    return new File(containerMetaDataPath, containerID +
-        OzoneConsts.CONTAINER_EXTENSION);
-  }
-
-  /**
    * Return containerDB File.
-   * @param containerMetaDataPath
-   * @param containerID
-   * @return containerDB File name
    */
   public static File getContainerDBFile(File containerMetaDataPath,
       long containerID) {
-    Preconditions.checkNotNull(containerMetaDataPath);
     return new File(containerMetaDataPath, containerID + OzoneConsts
         .DN_CONTAINER_DB);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3cc7ce81/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
index dc33f2e..7c986f0 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
@@ -23,8 +23,8 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.Storage;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
@@ -32,7 +32,6 @@ import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
-import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -119,8 +118,7 @@ public class ContainerReader implements Runnable {
     }
 
     for (File scmLoc : scmDir) {
-      File currentDir = null;
-      currentDir = new File(scmLoc, Storage.STORAGE_DIR_CURRENT);
+      File currentDir = new File(scmLoc, Storage.STORAGE_DIR_CURRENT);
       File[] containerTopDirs = currentDir.listFiles();
       if (containerTopDirs != null) {
         for (File containerTopDir : containerTopDirs) {
@@ -128,21 +126,14 @@ public class ContainerReader implements Runnable {
             File[] containerDirs = containerTopDir.listFiles();
             if (containerDirs != null) {
               for (File containerDir : containerDirs) {
-                File metadataPath = new File(containerDir + File.separator +
-                    OzoneConsts.CONTAINER_META_PATH);
-                long containerID = Long.parseLong(containerDir.getName());
-                if (metadataPath.exists()) {
-                  File containerFile = KeyValueContainerLocationUtil
-                      .getContainerFile(metadataPath, containerID);
-                  if (containerFile.exists()) {
-                    verifyContainerFile(containerID, containerFile);
-                  } else {
-                    LOG.error("Missing .container file for ContainerID: {}",
-                        containerID);
-                  }
+                File containerFile = ContainerUtils.getContainerFile(
+                    containerDir);
+                long containerID = ContainerUtils.getContainerID(containerDir);
+                if (containerFile.exists()) {
+                  verifyContainerFile(containerID, containerFile);
                 } else {
-                  LOG.error("Missing container metadata directory for " +
-                      "ContainerID: {}", containerID);
+                  LOG.error("Missing .container file for ContainerID: {}",
+                      containerDir.getName());
                 }
               }
             }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3cc7ce81/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
index 2bf41e5..35772ff 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
@@ -111,11 +111,9 @@ public class TestKeyValueContainer {
     File containerMetaDataLoc = new File(containerMetaDataPath);
 
     //Check whether container file and container db file exists or not.
-    assertTrue(KeyValueContainerLocationUtil.getContainerFile(
-        containerMetaDataLoc, containerID).exists(), ".Container File does" +
-        " not exist");
-    assertTrue(KeyValueContainerLocationUtil.getContainerDBFile(
-        containerMetaDataLoc, containerID).exists(), "Container DB does " +
+    assertTrue(keyValueContainer.getContainerFile().exists(),
+        ".Container File does not exist");
+    assertTrue(keyValueContainer.getContainerDBFile().exists(), "Container DB does " +
         "not exist");
   }
 
@@ -166,11 +164,9 @@ public class TestKeyValueContainer {
         .getParentFile().exists());
 
     assertFalse("Container File still exists",
-        KeyValueContainerLocationUtil.getContainerFile(containerMetaDataLoc,
-            containerID).exists());
+        keyValueContainer.getContainerFile().exists());
     assertFalse("Container DB file still exists",
-        KeyValueContainerLocationUtil.getContainerDBFile(containerMetaDataLoc,
-            containerID).exists());
+        keyValueContainer.getContainerDBFile().exists());
   }
 
 
@@ -188,9 +184,7 @@ public class TestKeyValueContainer {
     //Check state in the .container file
     String containerMetaDataPath = keyValueContainerData
         .getMetadataPath();
-    File containerMetaDataLoc = new File(containerMetaDataPath);
-    File containerFile = KeyValueContainerLocationUtil.getContainerFile(
-        containerMetaDataLoc, containerID);
+    File containerFile = keyValueContainer.getContainerFile();
 
     keyValueContainerData = (KeyValueContainerData) ContainerDataYaml
         .readContainerFile(containerFile);
@@ -227,11 +221,7 @@ public class TestKeyValueContainer {
     assertEquals(2, keyValueContainerData.getMetadata().size());
 
     //Check metadata in the .container file
-    String containerMetaDataPath = keyValueContainerData
-        .getMetadataPath();
-    File containerMetaDataLoc = new File(containerMetaDataPath);
-    File containerFile = KeyValueContainerLocationUtil.getContainerFile(
-        containerMetaDataLoc, containerID);
+    File containerFile = keyValueContainer.getContainerFile();
 
     keyValueContainerData = (KeyValueContainerData) ContainerDataYaml
         .readContainerFile(containerFile);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3cc7ce81/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
index c2e1645..5322c8e 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.volume
@@ -40,8 +41,6 @@ import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
-import org.apache.hadoop.ozone.container.keyvalue.helpers
-    .KeyValueContainerLocationUtil;
 import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerImpl;
 import org.apache.hadoop.ozone.container.keyvalue.impl.KeyManagerImpl;
 import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
@@ -714,9 +713,7 @@ public class TestContainerPersistence {
     KeyValueContainer container =
         (KeyValueContainer) addContainer(containerSet, testContainerID);
 
-    File orgContainerFile = KeyValueContainerLocationUtil.getContainerFile(
-        new File(container.getContainerData().getMetadataPath()),
-        testContainerID);
+    File orgContainerFile = container.getContainerFile();
     Assert.assertTrue(orgContainerFile.exists());
 
     Map<String, String> newMetadata = Maps.newHashMap();
@@ -738,9 +735,9 @@ public class TestContainerPersistence {
         actualNewData.getMetadata().get("owner"));
 
     // Verify container data on disk
-    File newContainerFile = KeyValueContainerLocationUtil.getContainerFile(
-        new File(actualNewData.getMetadataPath()),
-        testContainerID);
+    File containerBaseDir = new File(actualNewData.getMetadataPath())
+        .getParentFile();
+    File newContainerFile = ContainerUtils.getContainerFile(containerBaseDir);
     Assert.assertTrue("Container file should exist.",
         newContainerFile.exists());
     Assert.assertEquals("Container file should be in same location.",


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


[42/50] hadoop git commit: YARN-8591. [ATSv2] NPE while checking for entity acl in non-secure cluster. Contributed by Rohith Sharma K S.

Posted by eh...@apache.org.
YARN-8591. [ATSv2] NPE while checking for entity acl in non-secure cluster. Contributed by Rohith Sharma K S.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/63e08ec0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/63e08ec0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/63e08ec0

Branch: refs/heads/HDFS-12090
Commit: 63e08ec071852640babea9e39780327a0907712a
Parents: 0857f11
Author: Sunil G <su...@apache.org>
Authored: Mon Jul 30 14:48:04 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Mon Jul 30 14:48:04 2018 +0530

----------------------------------------------------------------------
 .../server/timelineservice/reader/TimelineReaderWebServices.java | 3 ++-
 .../reader/TestTimelineReaderWebServicesBasicAcl.java            | 4 ++++
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/63e08ec0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
index 7f96bfb..b10b705 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
@@ -3532,7 +3532,8 @@ public class TimelineReaderWebServices {
   static boolean checkAccess(TimelineReaderManager readerManager,
       UserGroupInformation ugi, String entityUser) {
     if (isDisplayEntityPerUserFilterEnabled(readerManager.getConfig())) {
-      if (!validateAuthUserWithEntityUser(readerManager, ugi, entityUser)) {
+      if (ugi != null && !validateAuthUserWithEntityUser(readerManager, ugi,
+          entityUser)) {
         String userName = ugi.getShortUserName();
         String msg = "User " + userName
             + " is not allowed to read TimelineService V2 data.";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63e08ec0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesBasicAcl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesBasicAcl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesBasicAcl.java
index 4239bf0..6651457 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesBasicAcl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesBasicAcl.java
@@ -88,6 +88,10 @@ public class TestTimelineReaderWebServicesBasicAcl {
     Assert.assertFalse(TimelineReaderWebServices
         .validateAuthUserWithEntityUser(manager, null, user1));
 
+    // true because ugi is null
+    Assert.assertTrue(
+        TimelineReaderWebServices.checkAccess(manager, null, user1));
+
     // incoming ugi is admin asking for entity owner user1
     Assert.assertTrue(
         TimelineReaderWebServices.checkAccess(manager, adminUgi, user1));


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


[34/50] hadoop git commit: YARN-8596. Allow SQLFederationStateStore to submit the same app in the same subcluster. Contributed by Giovanni Matteo Fumarola.

Posted by eh...@apache.org.
YARN-8596. Allow SQLFederationStateStore to submit the same app in the same subcluster. Contributed by Giovanni Matteo Fumarola.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/79091cf7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/79091cf7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/79091cf7

Branch: refs/heads/HDFS-12090
Commit: 79091cf76f6e966f64ac1d65e43e95782695e678
Parents: 2cccf40
Author: Inigo Goiri <in...@apache.org>
Authored: Fri Jul 27 15:23:57 2018 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Fri Jul 27 15:23:57 2018 -0700

----------------------------------------------------------------------
 .../store/impl/SQLFederationStateStore.java      | 14 +++++++-------
 .../store/impl/FederationStateStoreBaseTest.java | 19 +++++++++++++++++++
 2 files changed, 26 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/79091cf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java
index e62dcaf..273118a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java
@@ -564,13 +564,13 @@ public class SQLFederationStateStore implements FederationStateStore {
         // Check the ROWCOUNT value, if it is equal to 0 it means the call
         // did not add a new application into FederationStateStore
         if (cstmt.getInt(4) == 0) {
-          String errMsg = "The application " + appId
-              + " was not insert into the StateStore";
-          FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
-        }
-        // Check the ROWCOUNT value, if it is different from 1 it means the call
-        // had a wrong behavior. Maybe the database is not set correctly.
-        if (cstmt.getInt(4) != 1) {
+          LOG.info(
+              "The application {} was not inserted in the StateStore because it"
+                  + " was already present in SubCluster {}",
+              appId, subClusterHome);
+        } else if (cstmt.getInt(4) != 1) {
+          // Check the ROWCOUNT value, if it is different from 1 it means the
+          // call had a wrong behavior. Maybe the database is not set correctly.
           String errMsg = "Wrong behavior during the insertion of SubCluster "
               + subClusterId;
           FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79091cf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java
index 15cc0f0..b17f870 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java
@@ -282,6 +282,25 @@ public abstract class FederationStateStoreBaseTest {
   }
 
   @Test
+  public void testAddApplicationHomeSubClusterAppAlreadyExistsInTheSameSC()
+      throws Exception {
+    ApplicationId appId = ApplicationId.newInstance(1, 1);
+    SubClusterId subClusterId1 = SubClusterId.newInstance("SC1");
+    addApplicationHomeSC(appId, subClusterId1);
+
+    ApplicationHomeSubCluster ahsc2 =
+        ApplicationHomeSubCluster.newInstance(appId, subClusterId1);
+
+    AddApplicationHomeSubClusterResponse response =
+        stateStore.addApplicationHomeSubCluster(
+            AddApplicationHomeSubClusterRequest.newInstance(ahsc2));
+
+    Assert.assertEquals(subClusterId1, response.getHomeSubCluster());
+    Assert.assertEquals(subClusterId1, queryApplicationHomeSC(appId));
+
+  }
+
+  @Test
   public void testDeleteApplicationHomeSubCluster() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(1, 1);
     SubClusterId subClusterId = SubClusterId.newInstance("SC");


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


[27/50] hadoop git commit: YARN-8429. Improve diagnostic message when artifact is not set properly. Contributed by Gour Saha

Posted by eh...@apache.org.
YARN-8429. Improve diagnostic message when artifact is not set properly.
           Contributed by Gour Saha


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8d3c068e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8d3c068e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8d3c068e

Branch: refs/heads/HDFS-12090
Commit: 8d3c068e59fdddd18e3f8260713fee83c458aa1d
Parents: 77721f3
Author: Eric Yang <ey...@apache.org>
Authored: Thu Jul 26 20:02:13 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Thu Jul 26 20:02:13 2018 -0400

----------------------------------------------------------------------
 .../exceptions/RestApiErrorMessages.java        |  6 +-
 .../provider/AbstractClientProvider.java        | 14 ++---
 .../defaultImpl/DefaultClientProvider.java      | 22 ++++---
 .../provider/docker/DockerClientProvider.java   | 15 ++---
 .../provider/tarball/TarballClientProvider.java | 27 ++++----
 .../yarn/service/utils/ServiceApiUtil.java      |  4 +-
 .../hadoop/yarn/service/TestServiceApiUtil.java |  9 ++-
 .../providers/TestAbstractClientProvider.java   | 29 ++++-----
 .../providers/TestDefaultClientProvider.java    | 66 ++++++++++++++++++++
 9 files changed, 138 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java
index 5b3c72c..f10d884 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java
@@ -50,6 +50,10 @@ public interface RestApiErrorMessages {
       "Artifact id (like docker image name) is either empty or not provided";
   String ERROR_ARTIFACT_ID_FOR_COMP_INVALID =
       ERROR_ARTIFACT_ID_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+  String ERROR_ARTIFACT_PATH_FOR_COMP_INVALID = "For component %s with %s "
+      + "artifact, path does not exist: %s";
+  String ERROR_CONFIGFILE_DEST_FILE_FOR_COMP_NOT_ABSOLUTE = "For component %s "
+      + "with %s artifact, dest_file must be a relative path: %s";
 
   String ERROR_RESOURCE_INVALID = "Resource is not provided";
   String ERROR_RESOURCE_FOR_COMP_INVALID =
@@ -89,7 +93,7 @@ public interface RestApiErrorMessages {
   String ERROR_ABSENT_NUM_OF_INSTANCE =
       "Num of instances should appear either globally or per component";
   String ERROR_ABSENT_LAUNCH_COMMAND =
-      "Launch_command is required when type is not DOCKER";
+      "launch_command is required when type is not DOCKER";
 
   String ERROR_QUICKLINKS_FOR_COMP_INVALID = "Quicklinks specified at"
       + " component level, needs corresponding values set at service level";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java
index 672c435..ae79619 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java
@@ -68,18 +68,18 @@ public abstract class AbstractClientProvider {
    * Validate the artifact.
    * @param artifact
    */
-  public abstract void validateArtifact(Artifact artifact, FileSystem
-      fileSystem) throws IOException;
+  public abstract void validateArtifact(Artifact artifact, String compName,
+      FileSystem fileSystem) throws IOException;
 
-  protected abstract void validateConfigFile(ConfigFile configFile, FileSystem
-      fileSystem) throws IOException;
+  protected abstract void validateConfigFile(ConfigFile configFile,
+      String compName, FileSystem fileSystem) throws IOException;
 
   /**
    * Validate the config files.
    * @param configFiles config file list
    * @param fs file system
    */
-  public void validateConfigFiles(List<ConfigFile> configFiles,
+  public void validateConfigFiles(List<ConfigFile> configFiles, String compName,
       FileSystem fs) throws IOException {
     Set<String> destFileSet = new HashSet<>();
 
@@ -128,7 +128,7 @@ public abstract class AbstractClientProvider {
       }
 
       if (StringUtils.isEmpty(file.getDestFile())) {
-        throw new IllegalArgumentException("Dest_file is empty.");
+        throw new IllegalArgumentException("dest_file is empty.");
       }
 
       if (destFileSet.contains(file.getDestFile())) {
@@ -144,7 +144,7 @@ public abstract class AbstractClientProvider {
       }
 
       // provider-specific validation
-      validateConfigFile(file, fs);
+      validateConfigFile(file, compName, fs);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultClientProvider.java
index 0920a9c..999a8dc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultClientProvider.java
@@ -17,13 +17,16 @@
  */
 package org.apache.hadoop.yarn.service.provider.defaultImpl;
 
+import java.io.IOException;
+import java.nio.file.Paths;
+
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.yarn.service.provider.AbstractClientProvider;
 import org.apache.hadoop.yarn.service.api.records.Artifact;
 import org.apache.hadoop.yarn.service.api.records.ConfigFile;
+import org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages;
+import org.apache.hadoop.yarn.service.provider.AbstractClientProvider;
 
-import java.io.IOException;
-import java.nio.file.Paths;
+import com.google.common.annotations.VisibleForTesting;
 
 public class DefaultClientProvider extends AbstractClientProvider {
 
@@ -31,16 +34,19 @@ public class DefaultClientProvider extends AbstractClientProvider {
   }
 
   @Override
-  public void validateArtifact(Artifact artifact, FileSystem fileSystem) {
+  public void validateArtifact(Artifact artifact, String compName,
+      FileSystem fileSystem) {
   }
 
   @Override
-  protected void validateConfigFile(ConfigFile configFile, FileSystem
-      fileSystem) throws IOException {
+  @VisibleForTesting
+  public void validateConfigFile(ConfigFile configFile, String compName,
+      FileSystem fileSystem) throws IOException {
     // validate dest_file is not absolute
     if (Paths.get(configFile.getDestFile()).isAbsolute()) {
-      throw new IllegalArgumentException(
-          "Dest_file must not be absolute path: " + configFile.getDestFile());
+      throw new IllegalArgumentException(String.format(
+          RestApiErrorMessages.ERROR_CONFIGFILE_DEST_FILE_FOR_COMP_NOT_ABSOLUTE,
+          compName, "no", configFile.getDestFile()));
     }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerClientProvider.java
index f91742e..901d779 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerClientProvider.java
@@ -35,19 +35,20 @@ public class DockerClientProvider extends AbstractClientProvider
   }
 
   @Override
-  public void validateArtifact(Artifact artifact, FileSystem fileSystem) {
+  public void validateArtifact(Artifact artifact, String compName,
+      FileSystem fileSystem) {
     if (artifact == null) {
-      throw new IllegalArgumentException(
-          RestApiErrorMessages.ERROR_ARTIFACT_INVALID);
+      throw new IllegalArgumentException(String.format(
+          RestApiErrorMessages.ERROR_ARTIFACT_FOR_COMP_INVALID, compName));
     }
     if (StringUtils.isEmpty(artifact.getId())) {
-      throw new IllegalArgumentException(
-          RestApiErrorMessages.ERROR_ARTIFACT_ID_INVALID);
+      throw new IllegalArgumentException(String.format(
+          RestApiErrorMessages.ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName));
     }
   }
 
   @Override
-  protected void validateConfigFile(ConfigFile configFile, FileSystem
-      fileSystem) throws IOException {
+  protected void validateConfigFile(ConfigFile configFile, String compName,
+      FileSystem fileSystem) throws IOException {
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballClientProvider.java
index 3b890fd..b801e0c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballClientProvider.java
@@ -36,30 +36,33 @@ public class TarballClientProvider extends AbstractClientProvider
   }
 
   @Override
-  public void validateArtifact(Artifact artifact, FileSystem fs)
-      throws IOException {
+  public void validateArtifact(Artifact artifact, String compName,
+      FileSystem fs) throws IOException {
     if (artifact == null) {
-      throw new IllegalArgumentException(
-          RestApiErrorMessages.ERROR_ARTIFACT_INVALID);
+      throw new IllegalArgumentException(String.format(
+          RestApiErrorMessages.ERROR_ARTIFACT_FOR_COMP_INVALID, compName));
     }
     if (StringUtils.isEmpty(artifact.getId())) {
-      throw new IllegalArgumentException(
-          RestApiErrorMessages.ERROR_ARTIFACT_ID_INVALID);
+      throw new IllegalArgumentException(String.format(
+          RestApiErrorMessages.ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName));
     }
     Path p = new Path(artifact.getId());
     if (!fs.exists(p)) {
-      throw new IllegalArgumentException( "Artifact tarball does not exist "
-          + artifact.getId());
+      throw new IllegalArgumentException(String.format(
+          RestApiErrorMessages.ERROR_ARTIFACT_PATH_FOR_COMP_INVALID, compName,
+          Artifact.TypeEnum.TARBALL.name(), artifact.getId()));
     }
   }
 
   @Override
-  protected void validateConfigFile(ConfigFile configFile, FileSystem
-      fileSystem) throws IOException {
+  protected void validateConfigFile(ConfigFile configFile, String compName,
+      FileSystem fileSystem) throws IOException {
     // validate dest_file is not absolute
     if (Paths.get(configFile.getDestFile()).isAbsolute()) {
-      throw new IllegalArgumentException(
-          "Dest_file must not be absolute path: " + configFile.getDestFile());
+      throw new IllegalArgumentException(String.format(
+          RestApiErrorMessages.ERROR_CONFIGFILE_DEST_FILE_FOR_COMP_NOT_ABSOLUTE,
+          compName, Artifact.TypeEnum.TARBALL.name(),
+          configFile.getDestFile()));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
index 447250f..bebf52c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
@@ -282,7 +282,7 @@ public class ServiceApiUtil {
 
     AbstractClientProvider compClientProvider = ProviderFactory
         .getClientProvider(comp.getArtifact());
-    compClientProvider.validateArtifact(comp.getArtifact(), fs);
+    compClientProvider.validateArtifact(comp.getArtifact(), comp.getName(), fs);
 
     if (comp.getLaunchCommand() == null && (comp.getArtifact() == null || comp
         .getArtifact().getType() != Artifact.TypeEnum.DOCKER)) {
@@ -299,7 +299,7 @@ public class ServiceApiUtil {
               + ": " + comp.getNumberOfContainers(), comp.getName()));
     }
     compClientProvider.validateConfigFiles(comp.getConfiguration()
-        .getFiles(), fs);
+        .getFiles(), comp.getName(), fs);
 
     MonitorUtils.getProbe(comp.getReadinessCheck());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java
index ae031d4..47b2803 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java
@@ -227,14 +227,16 @@ public class TestServiceApiUtil {
     // no artifact id fails with default type
     Artifact artifact = new Artifact();
     app.setArtifact(artifact);
-    Component comp = ServiceTestUtils.createComponent("comp1");
+    String compName = "comp1";
+    Component comp = ServiceTestUtils.createComponent(compName);
 
     app.setComponents(Collections.singletonList(comp));
     try {
       ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
       Assert.fail(EXCEPTION_PREFIX + "service with no artifact id");
     } catch (IllegalArgumentException e) {
-      assertEquals(ERROR_ARTIFACT_ID_INVALID, e.getMessage());
+      assertEquals(String.format(ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName),
+          e.getMessage());
     }
 
     // no artifact id fails with SERVICE type
@@ -252,7 +254,8 @@ public class TestServiceApiUtil {
       ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
       Assert.fail(EXCEPTION_PREFIX + "service with no artifact id");
     } catch (IllegalArgumentException e) {
-      assertEquals(ERROR_ARTIFACT_ID_INVALID, e.getMessage());
+      assertEquals(String.format(ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName),
+          e.getMessage());
     }
 
     // everything valid here

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java
index 1d6be91..ae62608 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java
@@ -45,12 +45,12 @@ public class TestAbstractClientProvider {
 
   private static class ClientProvider extends AbstractClientProvider {
     @Override
-    public void validateArtifact(Artifact artifact, FileSystem fileSystem)
-        throws IOException {
+    public void validateArtifact(Artifact artifact, String compName,
+        FileSystem fileSystem) throws IOException {
     }
 
     @Override
-    protected void validateConfigFile(ConfigFile configFile,
+    protected void validateConfigFile(ConfigFile configFile, String compName,
         FileSystem fileSystem) throws IOException {
     }
   }
@@ -62,33 +62,34 @@ public class TestAbstractClientProvider {
     FileStatus mockFileStatus = mock(FileStatus.class);
     when(mockFs.exists(anyObject())).thenReturn(true);
 
+    String compName = "sleeper";
     ConfigFile configFile = new ConfigFile();
     List<ConfigFile> configFiles = new ArrayList<>();
     configFiles.add(configFile);
 
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "null file type");
     } catch (IllegalArgumentException e) {
     }
 
     configFile.setType(ConfigFile.TypeEnum.TEMPLATE);
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "empty src_file for type template");
     } catch (IllegalArgumentException e) {
     }
 
     configFile.setSrcFile("srcfile");
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "empty dest file");
     } catch (IllegalArgumentException e) {
     }
 
     configFile.setDestFile("destfile");
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
     } catch (IllegalArgumentException e) {
       Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
     }
@@ -99,21 +100,21 @@ public class TestAbstractClientProvider {
     configFile.setDestFile("path/destfile2");
     configFiles.add(configFile);
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "dest file with multiple path elements");
     } catch (IllegalArgumentException e) {
     }
 
     configFile.setDestFile("/path/destfile2");
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
     } catch (IllegalArgumentException e) {
       Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
     }
 
     configFile.setDestFile("destfile");
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "duplicate dest file");
     } catch (IllegalArgumentException e) {
     }
@@ -125,14 +126,14 @@ public class TestAbstractClientProvider {
     configFile.setDestFile("path/destfile3");
     configFiles.add(configFile);
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "dest file with multiple path elements");
     } catch (IllegalArgumentException e) {
     }
 
     configFile.setDestFile("/path/destfile3");
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "src file should be specified");
     } catch (IllegalArgumentException e) {
     }
@@ -140,7 +141,7 @@ public class TestAbstractClientProvider {
     //should succeed
     configFile.setSrcFile("srcFile");
     configFile.setDestFile("destfile3");
-    clientProvider.validateConfigFiles(configFiles, mockFs);
+    clientProvider.validateConfigFiles(configFiles, compName, mockFs);
 
     when(mockFileStatus.isDirectory()).thenReturn(true);
     when(mockFs.getFileStatus(new Path("srcFile")))
@@ -154,7 +155,7 @@ public class TestAbstractClientProvider {
     configFiles.add(configFile);
 
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "src file is a directory");
     } catch (IllegalArgumentException e) {
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestDefaultClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestDefaultClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestDefaultClientProvider.java
new file mode 100644
index 0000000..366515c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestDefaultClientProvider.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.service.providers;
+
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.yarn.service.api.records.ConfigFile;
+import org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages;
+import org.apache.hadoop.yarn.service.provider.defaultImpl.DefaultClientProvider;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestDefaultClientProvider {
+  private static final String EXCEPTION_PREFIX = "Should have thrown "
+      + "exception: ";
+  private static final String NO_EXCEPTION_PREFIX = "Should not have thrown "
+      + "exception: ";
+
+  @Test
+  public void testConfigFile() throws IOException {
+    DefaultClientProvider defaultClientProvider = new DefaultClientProvider();
+    FileSystem mockFs = mock(FileSystem.class);
+    when(mockFs.exists(anyObject())).thenReturn(true);
+
+    String compName = "sleeper";
+    ConfigFile configFile = new ConfigFile();
+    configFile.setDestFile("/var/tmp/a.txt");
+
+    try {
+      defaultClientProvider.validateConfigFile(configFile, compName, mockFs);
+      Assert.fail(EXCEPTION_PREFIX + " dest_file must be relative");
+    } catch (IllegalArgumentException e) {
+      String actualMsg = String.format(
+          RestApiErrorMessages.ERROR_CONFIGFILE_DEST_FILE_FOR_COMP_NOT_ABSOLUTE,
+          compName, "no", configFile.getDestFile());
+      Assert.assertEquals(actualMsg, e.getLocalizedMessage());
+    }
+
+    configFile.setDestFile("../a.txt");
+    try {
+      defaultClientProvider.validateConfigFile(configFile, compName, mockFs);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getLocalizedMessage());
+    }
+  }
+}


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


[17/50] hadoop git commit: YARN-4606. CapacityScheduler: applications could get starved because computation of #activeUsers considers pending apps. Contributed by Manikandan R

Posted by eh...@apache.org.
YARN-4606. CapacityScheduler: applications could get starved because computation of #activeUsers considers pending apps. Contributed by Manikandan R


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9485c9ae
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9485c9ae
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9485c9ae

Branch: refs/heads/HDFS-12090
Commit: 9485c9aee6e9bb935c3e6ae4da81d70b621781de
Parents: 5f0b924
Author: Eric E Payne <er...@oath.com>
Authored: Wed Jul 25 16:22:04 2018 +0000
Committer: Eric E Payne <er...@oath.com>
Committed: Wed Jul 25 16:22:04 2018 +0000

----------------------------------------------------------------------
 .../scheduler/capacity/UsersManager.java        |  27 +++-
 .../capacity/TestCapacityScheduler.java         | 128 +++++++++++++++++++
 .../capacity/TestContainerAllocation.java       |  43 +++++++
 3 files changed, 197 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9485c9ae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java
index 747a488..83ee6c0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java
@@ -85,6 +85,7 @@ public class UsersManager implements AbstractUsersManager {
 
   private final QueueMetrics metrics;
   private AtomicInteger activeUsers = new AtomicInteger(0);
+  private AtomicInteger activeUsersWithOnlyPendingApps = new AtomicInteger(0);
   private Map<String, Set<ApplicationId>> usersApplications =
       new HashMap<String, Set<ApplicationId>>();
 
@@ -671,9 +672,23 @@ public class UsersManager implements AbstractUsersManager {
     // update in local storage
     userLimitPerSchedulingMode.put(schedulingMode, computedUserLimit);
 
+    computeNumActiveUsersWithOnlyPendingApps();
+
     return userLimitPerSchedulingMode;
   }
 
+  // This method is called within the lock.
+  private void computeNumActiveUsersWithOnlyPendingApps() {
+    int numPendingUsers = 0;
+    for (User user : users.values()) {
+      if ((user.getPendingApplications() > 0)
+          && (user.getActiveApplications() <= 0)) {
+        numPendingUsers++;
+      }
+    }
+    activeUsersWithOnlyPendingApps = new AtomicInteger(numPendingUsers);
+  }
+
   private Resource computeUserLimit(String userName, Resource clusterResource,
       String nodePartition, SchedulingMode schedulingMode, boolean activeUser) {
     Resource partitionResource = labelManager.getResourceByLabel(nodePartition,
@@ -839,6 +854,11 @@ public class UsersManager implements AbstractUsersManager {
     try {
       this.writeLock.lock();
 
+      User userDesc = getUser(user);
+      if (userDesc != null && userDesc.getActiveApplications() <= 0) {
+        return;
+      }
+
       Set<ApplicationId> userApps = usersApplications.get(user);
       if (userApps == null) {
         userApps = new HashSet<ApplicationId>();
@@ -893,7 +913,7 @@ public class UsersManager implements AbstractUsersManager {
 
   @Override
   public int getNumActiveUsers() {
-    return activeUsers.get();
+    return activeUsers.get() + activeUsersWithOnlyPendingApps.get();
   }
 
   float sumActiveUsersTimesWeights() {
@@ -1090,4 +1110,9 @@ public class UsersManager implements AbstractUsersManager {
       this.writeLock.unlock();
     }
   }
+
+  @VisibleForTesting
+  public int getNumActiveUsersWithOnlyPendingApps() {
+    return activeUsersWithOnlyPendingApps.get();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9485c9ae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index 79cdcfe..8d948b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -4978,4 +4978,132 @@ public class TestCapacityScheduler extends CapacitySchedulerTestBase {
     Assert.assertEquals(AllocationState.QUEUE_SKIPPED,
         ContainerAllocation.QUEUE_SKIPPED.getAllocationState());
   }
+
+  @Test
+  public void testMoveAppWithActiveUsersWithOnlyPendingApps() throws Exception {
+
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+      ResourceScheduler.class);
+
+    CapacitySchedulerConfiguration newConf =
+        new CapacitySchedulerConfiguration(conf);
+
+    // Define top-level queues
+    newConf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[] { "a", "b" });
+
+    newConf.setCapacity(A, 50);
+    newConf.setCapacity(B, 50);
+
+    // Define 2nd-level queues
+    newConf.setQueues(A, new String[] { "a1" });
+    newConf.setCapacity(A1, 100);
+    newConf.setUserLimitFactor(A1, 2.0f);
+    newConf.setMaximumAMResourcePercentPerPartition(A1, "", 0.1f);
+
+    newConf.setQueues(B, new String[] { "b1" });
+    newConf.setCapacity(B1, 100);
+    newConf.setUserLimitFactor(B1, 2.0f);
+
+    LOG.info("Setup top-level queues a and b");
+
+    MockRM rm = new MockRM(newConf);
+    rm.start();
+
+    CapacityScheduler scheduler =
+        (CapacityScheduler) rm.getResourceScheduler();
+
+    MockNM nm1 = rm.registerNode("h1:1234", 16 * GB);
+
+    // submit an app
+    RMApp app = rm.submitApp(GB, "test-move-1", "u1", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app, rm, nm1);
+
+    ApplicationAttemptId appAttemptId =
+        rm.getApplicationReport(app.getApplicationId())
+            .getCurrentApplicationAttemptId();
+
+    RMApp app2 = rm.submitApp(1 * GB, "app", "u2", null, "a1");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm1);
+
+    RMApp app3 = rm.submitApp(1 * GB, "app", "u3", null, "a1");
+
+    RMApp app4 = rm.submitApp(1 * GB, "app", "u4", null, "a1");
+
+    // Each application asks 50 * 1GB containers
+    am1.allocate("*", 1 * GB, 50, null);
+    am2.allocate("*", 1 * GB, 50, null);
+
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    RMNode rmNode1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    // check preconditions
+    List<ApplicationAttemptId> appsInA1 = scheduler.getAppsInQueue("a1");
+    assertEquals(4, appsInA1.size());
+    String queue =
+        scheduler.getApplicationAttempt(appsInA1.get(0)).getQueue()
+            .getQueueName();
+    Assert.assertEquals("a1", queue);
+
+    List<ApplicationAttemptId> appsInA = scheduler.getAppsInQueue("a");
+    assertTrue(appsInA.contains(appAttemptId));
+    assertEquals(4, appsInA.size());
+
+    List<ApplicationAttemptId> appsInRoot = scheduler.getAppsInQueue("root");
+    assertTrue(appsInRoot.contains(appAttemptId));
+    assertEquals(4, appsInRoot.size());
+
+    List<ApplicationAttemptId> appsInB1 = scheduler.getAppsInQueue("b1");
+    assertTrue(appsInB1.isEmpty());
+
+    List<ApplicationAttemptId> appsInB = scheduler.getAppsInQueue("b");
+    assertTrue(appsInB.isEmpty());
+
+    UsersManager um =
+        (UsersManager) scheduler.getQueue("a1").getAbstractUsersManager();
+
+    assertEquals(4, um.getNumActiveUsers());
+    assertEquals(2, um.getNumActiveUsersWithOnlyPendingApps());
+
+    // now move the app
+    scheduler.moveAllApps("a1", "b1");
+
+    //Triggering this event so that user limit computation can
+    //happen again
+    for (int i = 0; i < 10; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+      Thread.sleep(500);
+    }
+
+    // check postconditions
+    appsInB1 = scheduler.getAppsInQueue("b1");
+
+    assertEquals(4, appsInB1.size());
+    queue =
+        scheduler.getApplicationAttempt(appsInB1.get(0)).getQueue()
+            .getQueueName();
+    Assert.assertEquals("b1", queue);
+
+    appsInB = scheduler.getAppsInQueue("b");
+    assertTrue(appsInB.contains(appAttemptId));
+    assertEquals(4, appsInB.size());
+
+    appsInRoot = scheduler.getAppsInQueue("root");
+    assertTrue(appsInRoot.contains(appAttemptId));
+    assertEquals(4, appsInRoot.size());
+
+    List<ApplicationAttemptId> oldAppsInA1 = scheduler.getAppsInQueue("a1");
+    assertEquals(0, oldAppsInA1.size());
+
+    UsersManager um_b1 =
+        (UsersManager) scheduler.getQueue("b1").getAbstractUsersManager();
+
+    assertEquals(2, um_b1.getNumActiveUsers());
+    assertEquals(2, um_b1.getNumActiveUsersWithOnlyPendingApps());
+
+    appsInB1 = scheduler.getAppsInQueue("b1");
+    assertEquals(4, appsInB1.size());
+    rm.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9485c9ae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
index 25e535a..b9bfc2a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
@@ -941,4 +941,47 @@ public class TestContainerAllocation {
 
     rm1.close();
   }
+
+  @Test
+  public void testActiveUsersWithOnlyPendingApps() throws Exception {
+
+    CapacitySchedulerConfiguration newConf =
+        new CapacitySchedulerConfiguration(conf);
+    newConf.setMaximumAMResourcePercentPerPartition(
+        CapacitySchedulerConfiguration.ROOT + ".default", "", 0.2f);
+    MockRM rm1 = new MockRM(newConf);
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
+
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "u1", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "u2", null, "default");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
+
+    RMApp app3 = rm1.submitApp(1 * GB, "app", "u3", null, "default");
+
+    RMApp app4 = rm1.submitApp(1 * GB, "app", "u4", null, "default");
+
+    // Each application asks 50 * 1GB containers
+    am1.allocate("*", 1 * GB, 50, null);
+    am2.allocate("*", 1 * GB, 50, null);
+
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    for (int i = 0; i < 10; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+      Thread.sleep(1000);
+    }
+    LeafQueue lq = (LeafQueue) cs.getQueue("default");
+    UsersManager um = (UsersManager) lq.getAbstractUsersManager();
+
+    Assert.assertEquals(4, um.getNumActiveUsers());
+    Assert.assertEquals(2, um.getNumActiveUsersWithOnlyPendingApps());
+    Assert.assertEquals(2, lq.getMetrics().getAppsPending());
+    rm1.close();
+  }
 }


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


[32/50] hadoop git commit: YARN-8566. Add diagnostic message for unschedulable containers (snemeth via rkanter)

Posted by eh...@apache.org.
YARN-8566. Add diagnostic message for unschedulable containers (snemeth via rkanter)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fecbac49
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fecbac49
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fecbac49

Branch: refs/heads/HDFS-12090
Commit: fecbac499e2ae6b3334773a997d454a518f43e01
Parents: b429f19
Author: Robert Kanter <rk...@apache.org>
Authored: Fri Jul 27 14:32:34 2018 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Fri Jul 27 14:32:34 2018 -0700

----------------------------------------------------------------------
 .../src/site/markdown/ResourceManagerRest.md    | 285 +++++++++++++++++++
 1 file changed, 285 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fecbac49/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
index a30677c..24c2319 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
@@ -2326,6 +2326,291 @@ Response Body:
 </appAttempts>
 ```
 
+Containers for an Application Attempt API
+-----------------------------------------
+
+With Containers for an Application Attempt API you can obtain the list of containers, which belongs to an Application Attempt.
+
+### URI
+
+      * http://rm-http-address:port/ws/v1/cluster/apps/{appid}/appattempts/{appAttemptId}/containers
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Supported
+
+      None
+
+### Elements of the *containers* object
+
+When you make a request for the list of containers, the information will be returned as an array of container objects.
+
+containers:
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| containers | array of app container objects(JSON)/zero or more container objects(XML) | The collection of app container objects |
+
+### Elements of the *container* object
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| containerId | string | The container id |
+| allocatedMB | long | The amount of memory allocated for the container in MB |
+| allocatedVCores | int | The amount of virtual cores allocated for the container |
+| assignedNodeId | string | The node id of the node the attempt ran on |
+| priority | int | Allocated priority of the container |
+| startedTime | long | The start time of the attempt (in ms since epoch) |
+| finishedTime | long | The finish time of the attempt (in ms since epoch) 0 if not finished |
+| elapsedTime | long | The elapsed time in ms since the startedTime |
+| logUrl | string | The web URL that can be used to check the log for the container |
+| containerExitStatus | int | Final exit status of the container |
+| containerState | string | State of the container, can be NEW, RUNNING, or COMPLETE |
+| nodeHttpAddress | string | The node http address of the node the attempt ran on ||
+| nodeId | string | The node id of the node the attempt ran on |
+| allocatedResources |array of resource(JSON)/zero or more resource objects(XML) | Allocated resources for the container |
+
+### Elements of the *resource* object
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| memory | int | The maximum memory for the container |
+| vCores | int | The maximum number of vcores for the container |
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://rm-http-address:port/ws/v1/cluster/apps/{appid}/appattempts/{appAttemptId}/containers
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+  "containers" : {
+    "container": [
+      {
+      "containerId": "container_1531404209605_0008_01_000001",
+      "allocatedMB": "1536",
+      "allocatedVCores": "1",
+      "assignedNodeId": "host.domain.com:37814",
+      "priority": "0",
+      "startedTime": "1531405909444",
+      "finishedTime": "0",
+      "elapsedTime": "4112",
+      "logUrl": "http://host.domain.com:8042/node/containerlogs/container_1531404209605_0008_01_000001/systest",
+      "containerExitStatus": "0",
+      "containerState": "RUNNING",
+      "nodeHttpAddress": "http://host.domain.com:8042",
+      "nodeId": "host.domain.com:37814",
+      "allocatedResources": [
+         {
+            "key": "memory-mb",
+            "value": "1536"
+         },
+         {
+            "key": "vcores",
+            "value": "1"
+         }
+       ]
+      }
+    ]
+  }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      GET http://rm-http-address:port/ws/v1/cluster/apps/{appid}/appattempts/{appAttemptId}/containers
+      Accept: application/xml
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 1104
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<containers>
+  <container>
+    <containerId>container_1531404209605_0008_01_000001</containerId>
+    <allocatedMB>1536</allocatedMB>
+    <allocatedVCores>1</allocatedVCores>
+    <assignedNodeId>host.domain.com:37814</assignedNodeId>
+    <priority>0</priority>
+    <startedTime>1531405909444</startedTime>
+    <finishedTime>0</finishedTime>
+    <elapsedTime>4112</elapsedTime>
+    <logUrl>
+    http://host.domain.com:8042/node/containerlogs/container_1531404209605_0008_01_000001/systest
+    </logUrl>
+    <containerExitStatus>0</containerExitStatus>
+    <containerState>RUNNING</containerState>
+    <nodeHttpAddress>http://host.domain.com:8042</nodeHttpAddress>
+    <nodeId>host.domain.com:37814</nodeId>
+    <allocatedResources>
+      <entry>
+        <key>memory-mb</key>
+        <value>1536</value>
+      </entry>
+      <entry>
+        <key>vcores</key>
+        <value>1</value>
+      </entry>
+    </allocatedResources>
+  </container>
+</containers>
+```
+
+Specific Container for an Application Attempt API
+-------------------------------------------------
+
+With Specific Container for an Application Attempt API you can obtain information about a specific container, which belongs to an Application Attempt and selected by the container id.
+
+### URI
+
+      * http://rm-http-address:port/ws/v1/cluster/apps/{appid}/appattempts/{appAttemptId}/containers/{containerId}
+
+### HTTP Operations Supported
+
+      * GET
+
+### Query Parameters Supported
+
+      None
+
+### Elements of the *container* object
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| containerId | string | The container id |
+| allocatedMB | long | The amount of memory allocated for the container in MB |
+| allocatedVCores | int | The amount of virtual cores allocated for the container |
+| assignedNodeId | string | The node id of the node the attempt ran on |
+| priority | int | Allocated priority of the container |
+| startedTime | long | The start time of the attempt (in ms since epoch) |
+| finishedTime | long | The finish time of the attempt (in ms since epoch) 0 if not finished |
+| elapsedTime | long | The elapsed time in ms since the startedTime |
+| logUrl | string | The web URL that can be used to check the log for the container |
+| containerExitStatus | int | Final exit status of the container |
+| containerState | string | State of the container, can be NEW, RUNNING, or COMPLETE |
+| nodeHttpAddress | string | The node http address of the node the attempt ran on ||
+| nodeId | string | The node id of the node the attempt ran on |
+| allocatedResources |array of resource(JSON)/zero or more resource objects(XML) | Allocated resources for the container |
+
+### Elements of the *resource* object
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| memory | int | The maximum memory for the container |
+| vCores | int | The maximum number of vcores for the container |
+
+**JSON response**
+
+HTTP Request:
+
+      GET http://rm-http-address:port/ws/v1/cluster/apps/{appid}/appattempts/{appAttemptId}/containers/{containerId}
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/json
+      Transfer-Encoding: chunked
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```json
+{
+  "container": {
+    "containerId": "container_1531404209605_0008_01_000001",
+    "allocatedMB": "1536",
+    "allocatedVCores": "1",
+    "assignedNodeId": "host.domain.com:37814",
+    "priority": "0",
+    "startedTime": "1531405909444",
+    "finishedTime": "0",
+    "elapsedTime": "4112",
+    "logUrl": "http://host.domain.com:8042/node/containerlogs/container_1531404209605_0008_01_000001/systest",
+    "containerExitStatus": "0",
+    "containerState": "RUNNING",
+    "nodeHttpAddress": "http://host.domain.com:8042",
+    "nodeId": "host.domain.com:37814",
+    "allocatedResources": [
+       {
+          "key": "memory-mb",
+          "value": "1536"
+       },
+       {
+          "key": "vcores",
+          "value": "1"
+       }
+    ]
+  }
+}
+```
+
+**XML response**
+
+HTTP Request:
+
+      GET http://rm-http-address:port/ws/v1/cluster/apps/{appid}/appattempts/{appAttemptId}/containers/{containerId}
+      Accept: application/xml
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 1104
+      Server: Jetty(6.1.26)
+
+Response Body:
+
+```xml
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+
+<container>
+  <containerId>container_1531404209605_0008_01_000001</containerId>
+  <allocatedMB>1536</allocatedMB>
+  <allocatedVCores>1</allocatedVCores>
+  <assignedNodeId>host.domain.com:37814</assignedNodeId>
+  <priority>0</priority>
+  <startedTime>1531405909444</startedTime>
+  <finishedTime>0</finishedTime>
+  <elapsedTime>4112</elapsedTime>
+  <logUrl>
+  http://host.domain.com:8042/node/containerlogs/container_1531404209605_0008_01_000001/systest
+  </logUrl>
+  <containerExitStatus>0</containerExitStatus>
+  <containerState>RUNNING</containerState>
+  <nodeHttpAddress>http://host.domain.com:8042</nodeHttpAddress>
+  <nodeId>host.domain.com:37814</nodeId>
+  <allocatedResources>
+    <entry>
+      <key>memory-mb</key>
+      <value>1536</value>
+    </entry>
+    <entry>
+      <key>vcores</key>
+      <value>1</value>
+    </entry>
+  </allocatedResources>
+</container>
+```
+
 Cluster Nodes API
 -----------------
 


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


[09/50] hadoop git commit: HADOOP-15609. Retry KMS calls when SSLHandshakeException occurs. Contributed by Kitti Nanasi.

Posted by eh...@apache.org.
HADOOP-15609. Retry KMS calls when SSLHandshakeException occurs. Contributed by Kitti Nanasi.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/81d59506
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/81d59506
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/81d59506

Branch: refs/heads/HDFS-12090
Commit: 81d59506e539673edde12e19c0df5c2edd9d02ad
Parents: 2686447
Author: Xiao Chen <xi...@apache.org>
Authored: Tue Jul 24 21:45:14 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Tue Jul 24 21:45:43 2018 -0700

----------------------------------------------------------------------
 .../key/kms/LoadBalancingKMSClientProvider.java | 17 ++++-
 .../kms/TestLoadBalancingKMSClientProvider.java | 79 ++++++++++++++++++++
 2 files changed, 92 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/81d59506/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
index 1ac3fd3..23cdc50 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.crypto.key.kms;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
+import java.net.ConnectException;
 import java.security.GeneralSecurityException;
 import java.security.NoSuchAlgorithmException;
 import java.util.Arrays;
@@ -27,6 +28,8 @@ import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import javax.net.ssl.SSLHandshakeException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
@@ -115,7 +118,6 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
     if (providers.length == 0) {
       throw new IOException("No providers configured !");
     }
-    IOException ex = null;
     int numFailovers = 0;
     for (int i = 0;; i++, numFailovers++) {
       KMSClientProvider provider = providers[(currPos + i) % providers.length];
@@ -130,8 +132,15 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
       } catch (IOException ioe) {
         LOG.warn("KMS provider at [{}] threw an IOException: ",
             provider.getKMSUrl(), ioe);
-        ex = ioe;
-
+        // SSLHandshakeException can occur here because of lost connection
+        // with the KMS server, creating a ConnectException from it,
+        // so that the FailoverOnNetworkExceptionRetry policy will retry
+        if (ioe instanceof SSLHandshakeException) {
+          Exception cause = ioe;
+          ioe = new ConnectException("SSLHandshakeException: "
+              + cause.getMessage());
+          ioe.initCause(cause);
+        }
         RetryAction action = null;
         try {
           action = retryPolicy.shouldRetry(ioe, 0, numFailovers, false);
@@ -153,7 +162,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
                   CommonConfigurationKeysPublic.
                   KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, providers.length),
               providers.length);
-          throw ex;
+          throw ioe;
         }
         if (((numFailovers + 1) % providers.length) == 0) {
           // Sleep only after we try all the providers for every cycle.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81d59506/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
index bd68dca..4e7aed9 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.crypto.key.kms;
 
 import static org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -26,12 +27,15 @@ import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.verify;
 
 import java.io.IOException;
+import java.net.ConnectException;
 import java.net.NoRouteToHostException;
 import java.net.URI;
 import java.net.UnknownHostException;
 import java.security.GeneralSecurityException;
 import java.security.NoSuchAlgorithmException;
 
+import javax.net.ssl.SSLHandshakeException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProvider.Options;
@@ -44,13 +48,18 @@ import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.junit.After;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 import org.mockito.Mockito;
 
 import com.google.common.collect.Sets;
 
 public class TestLoadBalancingKMSClientProvider {
 
+  @Rule
+  public Timeout testTimeout = new Timeout(30 * 1000);
+
   @BeforeClass
   public static void setup() throws IOException {
     SecurityUtil.setTokenServiceUseIp(false);
@@ -638,4 +647,74 @@ public class TestLoadBalancingKMSClientProvider {
     verify(p2, Mockito.times(1)).createKey(Mockito.eq("test3"),
             Mockito.any(Options.class));
   }
+
+  /**
+   * Tests the operation succeeds second time after SSLHandshakeException.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesWithSSLHandshakeExceptionSucceedsSecondTime()
+      throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(
+        CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 3);
+    final String keyName = "test";
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new SSLHandshakeException("p1"))
+        .thenReturn(new KMSClientProvider.KMSKeyVersion(keyName, "v1",
+            new byte[0]));
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new ConnectException("p2"));
+
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2}, 0, conf);
+
+    kp.createKey(keyName, new Options(conf));
+    verify(p1, Mockito.times(2)).createKey(Mockito.eq(keyName),
+        Mockito.any(Options.class));
+    verify(p2, Mockito.times(1)).createKey(Mockito.eq(keyName),
+        Mockito.any(Options.class));
+  }
+
+  /**
+   * Tests the operation fails at every attempt after SSLHandshakeException.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesWithSSLHandshakeExceptionFailsAtEveryAttempt()
+      throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(
+        CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 2);
+    final String keyName = "test";
+    final String exceptionMessage = "p1 exception message";
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    Exception originalSslEx = new SSLHandshakeException(exceptionMessage);
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(originalSslEx);
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new ConnectException("p2 exception message"));
+
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2}, 0, conf);
+
+    Exception interceptedEx = intercept(ConnectException.class,
+        "SSLHandshakeException: " + exceptionMessage,
+        ()-> kp.createKey(keyName, new Options(conf)));
+    assertEquals(originalSslEx, interceptedEx.getCause());
+
+    verify(p1, Mockito.times(2)).createKey(Mockito.eq(keyName),
+        Mockito.any(Options.class));
+    verify(p2, Mockito.times(1)).createKey(Mockito.eq(keyName),
+        Mockito.any(Options.class));
+  }
 }
\ No newline at end of file


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


[46/50] hadoop git commit: HADOOP-15637. LocalFs#listLocatedStatus does not filter out hidden .crc files. Contributed by Erik Krogen.

Posted by eh...@apache.org.
HADOOP-15637. LocalFs#listLocatedStatus does not filter out hidden .crc files. Contributed by Erik Krogen.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e8f952ef
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e8f952ef
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e8f952ef

Branch: refs/heads/HDFS-12090
Commit: e8f952ef06ae05d2b504300d6f19beb8a052b6f1
Parents: 3517a47
Author: Chen Liang <cl...@apache.org>
Authored: Mon Jul 30 10:25:07 2018 -0700
Committer: Chen Liang <cl...@apache.org>
Committed: Mon Jul 30 10:25:07 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/ChecksumFs.java   | 37 +++++++++++++++++++
 .../fs/FileContextMainOperationsBaseTest.java   | 38 ++++++++++++++++++++
 2 files changed, 75 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8f952ef/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java
index 75622ad..c56f6e0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java
@@ -27,10 +27,12 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
 
+import java.util.NoSuchElementException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
 import org.slf4j.Logger;
@@ -527,4 +529,39 @@ public abstract class ChecksumFs extends FilterFs {
     }
     return results.toArray(new FileStatus[results.size()]);
   }
+
+  @Override
+  public RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path f)
+      throws AccessControlException, FileNotFoundException,
+             UnresolvedLinkException, IOException {
+    final RemoteIterator<LocatedFileStatus> iter =
+        getMyFs().listLocatedStatus(f);
+    return new RemoteIterator<LocatedFileStatus>() {
+
+      private LocatedFileStatus next = null;
+
+      @Override
+      public boolean hasNext() throws IOException {
+        while (next == null && iter.hasNext()) {
+          LocatedFileStatus unfilteredNext = iter.next();
+          if (!isChecksumFile(unfilteredNext.getPath())) {
+            next = unfilteredNext;
+          }
+        }
+        return next != null;
+      }
+
+      @Override
+      public LocatedFileStatus next() throws IOException {
+        if (!hasNext()) {
+          throw new NoSuchElementException();
+        }
+        LocatedFileStatus tmp = next;
+        next = null;
+        return tmp;
+      }
+
+    };
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8f952ef/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
index 62ecd9f..c07a6ff 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
@@ -369,6 +369,44 @@ public abstract class FileContextMainOperationsBaseTest  {
     pathsIterator = fc.listStatus(getTestRootPath(fc, "test/hadoop/a"));
     Assert.assertFalse(pathsIterator.hasNext());
   }
+
+  @Test
+  public void testListFiles() throws Exception {
+    Path[] testDirs = {
+        getTestRootPath(fc, "test/dir1"),
+        getTestRootPath(fc, "test/dir1/dir1"),
+        getTestRootPath(fc, "test/dir2")
+    };
+    Path[] testFiles = {
+        new Path(testDirs[0], "file1"),
+        new Path(testDirs[0], "file2"),
+        new Path(testDirs[1], "file2"),
+        new Path(testDirs[2], "file1")
+    };
+
+    for (Path path : testDirs) {
+      fc.mkdir(path, FsPermission.getDefault(), true);
+    }
+    for (Path p : testFiles) {
+      FSDataOutputStream out = fc.create(p).build();
+      out.writeByte(0);
+      out.close();
+    }
+
+    RemoteIterator<LocatedFileStatus> filesIterator =
+        fc.util().listFiles(getTestRootPath(fc, "test"), true);
+    LocatedFileStatus[] fileStats =
+        new LocatedFileStatus[testFiles.length];
+    for (int i = 0; i < fileStats.length; i++) {
+      assertTrue(filesIterator.hasNext());
+      fileStats[i] = filesIterator.next();
+    }
+    assertFalse(filesIterator.hasNext());
+
+    for (Path p : testFiles) {
+      assertTrue(containsPath(p, fileStats));
+    }
+  }
   
   @Test
   public void testListStatusFilterWithNoMatches() throws Exception {


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


[47/50] hadoop git commit: YARN-8584. Several typos in Log Aggregation related classes. Contributed by Szilard Nemeth.

Posted by eh...@apache.org.
YARN-8584. Several typos in Log Aggregation related classes. Contributed by Szilard Nemeth.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2b39ad26
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2b39ad26
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2b39ad26

Branch: refs/heads/HDFS-12090
Commit: 2b39ad26984d641bad57db2cfcc0b7515ef95f46
Parents: e8f952e
Author: bibinchundatt <bi...@apache.org>
Authored: Mon Jul 30 23:25:19 2018 +0530
Committer: bibinchundatt <bi...@apache.org>
Committed: Mon Jul 30 23:25:19 2018 +0530

----------------------------------------------------------------------
 .../AggregatedLogDeletionService.java           |  4 +--
 .../logaggregation/AggregatedLogFormat.java     |  8 +++---
 .../LogAggregationFileController.java           |  6 ++---
 .../ifile/IndexedFileAggregatedLogsBlock.java   |  6 ++---
 .../LogAggregationIndexedFileController.java    | 26 ++++++++++----------
 .../tfile/LogAggregationTFileController.java    |  2 +-
 .../TestAggregatedLogDeletionService.java       |  6 ++---
 .../logaggregation/AppLogAggregatorImpl.java    |  2 +-
 .../logaggregation/LogAggregationService.java   |  6 ++---
 .../tracker/NMLogAggregationStatusTracker.java  |  4 +--
 10 files changed, 35 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b39ad26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java
index 562bd2c..841b870 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java
@@ -258,7 +258,7 @@ public class AggregatedLogDeletionService extends AbstractService {
       return;
     }
     setLogAggCheckIntervalMsecs(retentionSecs);
-    task = new LogDeletionTask(conf, retentionSecs, creatRMClient());
+    task = new LogDeletionTask(conf, retentionSecs, createRMClient());
     timer = new Timer();
     timer.scheduleAtFixedRate(task, 0, checkIntervalMsecs);
   }
@@ -281,7 +281,7 @@ public class AggregatedLogDeletionService extends AbstractService {
   // We have already marked ApplicationClientProtocol.getApplicationReport
   // as @Idempotent, it will automatically take care of RM restart/failover.
   @VisibleForTesting
-  protected ApplicationClientProtocol creatRMClient() throws IOException {
+  protected ApplicationClientProtocol createRMClient() throws IOException {
     return ClientRMProxy.createRMProxy(getConfig(),
       ApplicationClientProtocol.class);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b39ad26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
index 4ee5c8a..d9b4c1e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
@@ -178,7 +178,7 @@ public class AggregatedLogFormat {
      * The set of log files that are older than retention policy that will
      * not be uploaded but ready for deletion.
      */
-    private final Set<File> obseleteRetentionLogFiles = new HashSet<File>();
+    private final Set<File> obsoleteRetentionLogFiles = new HashSet<File>();
 
     // TODO Maybe add a version string here. Instead of changing the version of
     // the entire k-v format
@@ -324,7 +324,7 @@ public class AggregatedLogFormat {
       // if log files are older than retention policy, do not upload them.
       // but schedule them for deletion.
       if(logRetentionContext != null && !logRetentionContext.shouldRetainLog()){
-        obseleteRetentionLogFiles.addAll(candidates);
+        obsoleteRetentionLogFiles.addAll(candidates);
         candidates.clear();
         return candidates;
       }
@@ -396,9 +396,9 @@ public class AggregatedLogFormat {
       return info;
     }
 
-    public Set<Path> getObseleteRetentionLogFiles() {
+    public Set<Path> getObsoleteRetentionLogFiles() {
       Set<Path> path = new HashSet<Path>();
-      for(File file: this.obseleteRetentionLogFiles) {
+      for(File file: this.obsoleteRetentionLogFiles) {
         path.add(new Path(file.getAbsolutePath()));
       }
       return path;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b39ad26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
index 5005b39..b047b1c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java
@@ -115,16 +115,16 @@ public abstract class LogAggregationFileController {
    */
   public void initialize(Configuration conf, String controllerName) {
     this.conf = conf;
-    int configuredRentionSize = conf.getInt(
+    int configuredRetentionSize = conf.getInt(
         YarnConfiguration.NM_LOG_AGGREGATION_NUM_LOG_FILES_SIZE_PER_APP,
         YarnConfiguration
             .DEFAULT_NM_LOG_AGGREGATION_NUM_LOG_FILES_SIZE_PER_APP);
-    if (configuredRentionSize <= 0) {
+    if (configuredRetentionSize <= 0) {
       this.retentionSize =
           YarnConfiguration
               .DEFAULT_NM_LOG_AGGREGATION_NUM_LOG_FILES_SIZE_PER_APP;
     } else {
-      this.retentionSize = configuredRentionSize;
+      this.retentionSize = configuredRetentionSize;
     }
     this.fileControllerName = controllerName;
     initInternal(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b39ad26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java
index c53ffcc..4ef429d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java
@@ -187,8 +187,8 @@ public class IndexedFileAggregatedLogsBlock extends LogAggregationHtmlBlock {
         FSDataInputStream fsin = fileContext.open(thisNodeFile.getPath());
         int bufferSize = 65536;
         for (IndexedFileLogMeta candidate : candidates) {
-          if (candidate.getLastModificatedTime() < startTime
-              || candidate.getLastModificatedTime() > endTime) {
+          if (candidate.getLastModifiedTime() < startTime
+              || candidate.getLastModifiedTime() > endTime) {
             continue;
           }
           byte[] cbuf = new byte[bufferSize];
@@ -205,7 +205,7 @@ public class IndexedFileAggregatedLogsBlock extends LogAggregationHtmlBlock {
             html.pre().__("\n\n").__();
             html.p().__("Log Type: " + candidate.getFileName()).__();
             html.p().__("Log Upload Time: " + Times.format(
-                candidate.getLastModificatedTime())).__();
+                candidate.getLastModifiedTime())).__();
             html.p().__("Log Length: " + Long.toString(
                 logLength)).__();
             long startIndex = start < 0

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b39ad26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java
index 59b8e2c..78b0c13 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java
@@ -404,7 +404,7 @@ public class LogAggregationIndexedFileController
         meta.setStartIndex(outputStreamState.getStartPos());
         meta.setFileSize(fileLength);
       }
-      meta.setLastModificatedTime(logFile.lastModified());
+      meta.setLastModifiedTime(logFile.lastModified());
       metas.add(meta);
     }
     logsMetaInThisCycle.addContainerLogMeta(containerId, metas);
@@ -499,12 +499,12 @@ public class LogAggregationIndexedFileController
         .getRemoteNodeFileDir(conf, appId, logRequest.getAppOwner(),
         this.remoteRootLogDir, this.remoteRootLogDirSuffix);
     if (!nodeFiles.hasNext()) {
-      throw new IOException("There is no available log fils for "
+      throw new IOException("There is no available log file for "
           + "application:" + appId);
     }
     List<FileStatus> allFiles = getAllNodeFiles(nodeFiles, appId);
     if (allFiles.isEmpty()) {
-      throw new IOException("There is no available log fils for "
+      throw new IOException("There is no available log file for "
           + "application:" + appId);
     }
     Map<String, Long> checkSumFiles = parseCheckSumFiles(allFiles);
@@ -581,7 +581,7 @@ public class LogAggregationIndexedFileController
               decompressor, getFSInputBufferSize(conf));
           LogToolUtils.outputContainerLog(candidate.getContainerId(),
               nodeName, candidate.getFileName(), candidate.getFileSize(), size,
-              Times.format(candidate.getLastModificatedTime()),
+              Times.format(candidate.getLastModifiedTime()),
               in, os, buf, ContainerLogAggregationType.AGGREGATED);
           byte[] b = aggregatedLogSuffix(candidate.getFileName())
               .getBytes(Charset.forName("UTF-8"));
@@ -618,12 +618,12 @@ public class LogAggregationIndexedFileController
         .getRemoteNodeFileDir(conf, appId, appOwner, this.remoteRootLogDir,
         this.remoteRootLogDirSuffix);
     if (!nodeFiles.hasNext()) {
-      throw new IOException("There is no available log fils for "
+      throw new IOException("There is no available log file for "
           + "application:" + appId);
     }
     List<FileStatus> allFiles = getAllNodeFiles(nodeFiles, appId);
     if (allFiles.isEmpty()) {
-      throw new IOException("There is no available log fils for "
+      throw new IOException("There is no available log file for "
           + "application:" + appId);
     }
     Map<String, Long> checkSumFiles = parseCheckSumFiles(allFiles);
@@ -660,7 +660,7 @@ public class LogAggregationIndexedFileController
             for (IndexedFileLogMeta aMeta : log.getValue()) {
               meta.addLogMeta(aMeta.getFileName(), Long.toString(
                   aMeta.getFileSize()),
-                  Times.format(aMeta.getLastModificatedTime()));
+                  Times.format(aMeta.getLastModifiedTime()));
             }
             containersLogMeta.add(meta);
           }
@@ -671,7 +671,7 @@ public class LogAggregationIndexedFileController
               logMeta.getContainerLogMeta(containerIdStr)) {
             meta.addLogMeta(log.getFileName(), Long.toString(
                 log.getFileSize()),
-                Times.format(log.getLastModificatedTime()));
+                Times.format(log.getLastModifiedTime()));
           }
           containersLogMeta.add(meta);
         }
@@ -1002,7 +1002,7 @@ public class LogAggregationIndexedFileController
     private String fileName;
     private long fileSize;
     private long fileCompressedSize;
-    private long lastModificatedTime;
+    private long lastModifiedTime;
     private long startIndex;
 
     public String getFileName() {
@@ -1026,11 +1026,11 @@ public class LogAggregationIndexedFileController
       this.fileCompressedSize = fileCompressedSize;
     }
 
-    public long getLastModificatedTime() {
-      return lastModificatedTime;
+    public long getLastModifiedTime() {
+      return lastModifiedTime;
     }
-    public void setLastModificatedTime(long lastModificatedTime) {
-      this.lastModificatedTime = lastModificatedTime;
+    public void setLastModifiedTime(long lastModifiedTime) {
+      this.lastModifiedTime = lastModifiedTime;
     }
 
     public long getStartIndex() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b39ad26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
index e87af7f..b3103d2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
@@ -275,7 +275,7 @@ public class LogAggregationTFileController
     RemoteIterator<FileStatus> nodeFiles = LogAggregationUtils
         .getRemoteNodeFileDir(conf, appId, appOwner);
     if (nodeFiles == null) {
-      throw new IOException("There is no available log fils for "
+      throw new IOException("There is no available log file for "
           + "application:" + appId);
     }
     while (nodeFiles.hasNext()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b39ad26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java
index 4e2d302..f36ebf4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java
@@ -160,7 +160,7 @@ public class TestAggregatedLogDeletionService {
     AggregatedLogDeletionService deletionService =
         new AggregatedLogDeletionService() {
           @Override
-          protected ApplicationClientProtocol creatRMClient()
+          protected ApplicationClientProtocol createRMClient()
               throws IOException {
             try {
               return createMockRMClient(finishedApplications,
@@ -262,7 +262,7 @@ public class TestAggregatedLogDeletionService {
         return conf;
       }
       @Override
-      protected ApplicationClientProtocol creatRMClient()
+      protected ApplicationClientProtocol createRMClient()
           throws IOException {
         try {
           return createMockRMClient(finishedApplications, null);
@@ -353,7 +353,7 @@ public class TestAggregatedLogDeletionService {
     AggregatedLogDeletionService deletionSvc =
         new AggregatedLogDeletionService() {
       @Override
-      protected ApplicationClientProtocol creatRMClient()
+      protected ApplicationClientProtocol createRMClient()
           throws IOException {
         try {
           return createMockRMClient(finishedApplications, null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b39ad26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
index 5956823..6630ba6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
@@ -632,7 +632,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
 
       // need to return files uploaded or older-than-retention clean up.
       return Sets.union(logValue.getCurrentUpLoadedFilesPath(),
-          logValue.getObseleteRetentionLogFiles());
+          logValue.getObsoleteRetentionLogFiles());
 
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b39ad26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java
index 4938939..dcc165f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java
@@ -117,9 +117,9 @@ public class LogAggregationService extends AbstractService implements
         LOG.info("Log aggregation debug mode enabled. rollingMonitorInterval = "
             + rollingMonitorInterval);
       } else {
-        LOG.warn("rollingMonitorIntervall should be more than or equal to "
-            + MIN_LOG_ROLLING_INTERVAL + " seconds. Using "
-            + MIN_LOG_ROLLING_INTERVAL + " seconds instead.");
+        LOG.warn("rollingMonitorInterval should be more than or equal to {} " +
+                "seconds. Using {} seconds instead.",
+                MIN_LOG_ROLLING_INTERVAL, MIN_LOG_ROLLING_INTERVAL);
         this.rollingMonitorInterval = MIN_LOG_ROLLING_INTERVAL;
       }
     } else if (rollingMonitorInterval <= 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b39ad26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/logaggregation/tracker/NMLogAggregationStatusTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/logaggregation/tracker/NMLogAggregationStatusTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/logaggregation/tracker/NMLogAggregationStatusTracker.java
index 510d6d8..eb2aaf5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/logaggregation/tracker/NMLogAggregationStatusTracker.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/logaggregation/tracker/NMLogAggregationStatusTracker.java
@@ -110,7 +110,7 @@ public class NMLogAggregationStatusTracker extends CompositeService {
       LogAggregationStatus logAggregationStatus, long updateTime,
       String diagnosis, boolean finalized) {
     if (disabled) {
-      LOG.warn("The log aggregation is diabled. No need to update "
+      LOG.warn("The log aggregation is disabled. No need to update "
           + "the log aggregation status");
     }
     // In NM, each application has exactly one appLogAggregator thread
@@ -164,7 +164,7 @@ public class NMLogAggregationStatusTracker extends CompositeService {
   public List<LogAggregationReport> pullCachedLogAggregationReports() {
     List<LogAggregationReport> reports = new ArrayList<>();
     if (disabled) {
-      LOG.warn("The log aggregation is diabled."
+      LOG.warn("The log aggregation is disabled."
           + "There is no cached log aggregation status.");
       return reports;
     }


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


[33/50] hadoop git commit: YARN-8517. getContainer and getContainers ResourceManager REST API methods are not documented (snemeth via rkanter)

Posted by eh...@apache.org.
YARN-8517. getContainer and getContainers ResourceManager REST API methods are not documented (snemeth via rkanter)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2cccf406
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2cccf406
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2cccf406

Branch: refs/heads/HDFS-12090
Commit: 2cccf4061cc4021c48e29879700dbc94f832b7d1
Parents: fecbac4
Author: Robert Kanter <rk...@apache.org>
Authored: Fri Jul 27 14:35:03 2018 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Fri Jul 27 14:35:03 2018 -0700

----------------------------------------------------------------------
 .../InvalidResourceRequestException.java        |  36 ++
 .../resourcemanager/DefaultAMSProcessor.java    |  23 +-
 .../scheduler/SchedulerUtils.java               |  55 +-
 .../scheduler/TestSchedulerUtils.java           | 630 ++++++++++---------
 4 files changed, 430 insertions(+), 314 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2cccf406/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/InvalidResourceRequestException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/InvalidResourceRequestException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/InvalidResourceRequestException.java
index f4fd2fa..1ea9eef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/InvalidResourceRequestException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/InvalidResourceRequestException.java
@@ -30,19 +30,55 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
  * 
  */
 public class InvalidResourceRequestException extends YarnException {
+  public static final String LESS_THAN_ZERO_RESOURCE_MESSAGE_TEMPLATE =
+          "Invalid resource request! Cannot allocate containers as "
+                  + "requested resource is less than 0! "
+                  + "Requested resource type=[%s], " + "Requested resource=%s";
+
+  public static final String GREATER_THAN_MAX_RESOURCE_MESSAGE_TEMPLATE =
+          "Invalid resource request! Cannot allocate containers as "
+                  + "requested resource is greater than " +
+                  "maximum allowed allocation. "
+                  + "Requested resource type=[%s], "
+                  + "Requested resource=%s, maximum allowed allocation=%s, "
+                  + "please note that maximum allowed allocation is calculated "
+                  + "by scheduler based on maximum resource of registered "
+                  + "NodeManagers, which might be less than configured "
+                  + "maximum allocation=%s";
+
+  public static final String UNKNOWN_REASON_MESSAGE_TEMPLATE =
+          "Invalid resource request! "
+                  + "Cannot allocate containers for an unknown reason! "
+                  + "Requested resource type=[%s], Requested resource=%s";
+
+  public enum InvalidResourceType {
+    LESS_THAN_ZERO, GREATER_THEN_MAX_ALLOCATION, UNKNOWN;
+  }
 
   private static final long serialVersionUID = 13498237L;
+  private final InvalidResourceType invalidResourceType;
 
   public InvalidResourceRequestException(Throwable cause) {
     super(cause);
+    this.invalidResourceType = InvalidResourceType.UNKNOWN;
   }
 
   public InvalidResourceRequestException(String message) {
+    this(message, InvalidResourceType.UNKNOWN);
+  }
+
+  public InvalidResourceRequestException(String message,
+      InvalidResourceType invalidResourceType) {
     super(message);
+    this.invalidResourceType = invalidResourceType;
   }
 
   public InvalidResourceRequestException(String message, Throwable cause) {
     super(message, cause);
+    this.invalidResourceType = InvalidResourceType.UNKNOWN;
   }
 
+  public InvalidResourceType getInvalidResourceType() {
+    return invalidResourceType;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2cccf406/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java
index 71558a7..43f73e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java
@@ -53,6 +53,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.InvalidContainerReleaseException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceBlacklistRequestException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
+import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException
+        .InvalidResourceType;
 import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -89,6 +91,12 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.hadoop.yarn.exceptions
+        .InvalidResourceRequestException.InvalidResourceType
+        .GREATER_THEN_MAX_ALLOCATION;
+import static org.apache.hadoop.yarn.exceptions
+        .InvalidResourceRequestException.InvalidResourceType.LESS_THAN_ZERO;
+
 /**
  * This is the default Application Master Service processor. It has be the
  * last processor in the @{@link AMSProcessingChain}.
@@ -231,8 +239,8 @@ final class DefaultAMSProcessor implements ApplicationMasterServiceProcessor {
           maximumCapacity, app.getQueue(),
           getScheduler(), getRmContext());
     } catch (InvalidResourceRequestException e) {
-      LOG.warn("Invalid resource ask by application " + appAttemptId, e);
-      throw e;
+      RMAppAttempt rmAppAttempt = app.getRMAppAttempt(appAttemptId);
+      handleInvalidResourceException(e, rmAppAttempt);
     }
 
     try {
@@ -336,6 +344,17 @@ final class DefaultAMSProcessor implements ApplicationMasterServiceProcessor {
         allocation.getPreviousAttemptContainers());
   }
 
+  private void handleInvalidResourceException(InvalidResourceRequestException e,
+          RMAppAttempt rmAppAttempt) throws InvalidResourceRequestException {
+    if (e.getInvalidResourceType() == LESS_THAN_ZERO ||
+            e.getInvalidResourceType() == GREATER_THEN_MAX_ALLOCATION) {
+      rmAppAttempt.updateAMLaunchDiagnostics(e.getMessage());
+    }
+    LOG.warn("Invalid resource ask by application " +
+            rmAppAttempt.getAppAttemptId(), e);
+    throw e;
+  }
+
   private void handleNodeUpdates(RMApp app, AllocateResponse allocateResponse) {
     Map<RMNode, NodeUpdateType> updatedNodes = new HashMap<>();
     if(app.pullRMNodeUpdates(updatedNodes) > 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2cccf406/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
index 844057e..9b07d37 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
@@ -45,6 +45,8 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.InvalidLabelResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
+import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException
+        .InvalidResourceType;
 import org.apache.hadoop.yarn.exceptions
         .SchedulerInvalidResoureRequestException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -61,6 +63,15 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import static org.apache.hadoop.yarn.exceptions
+        .InvalidResourceRequestException
+        .GREATER_THAN_MAX_RESOURCE_MESSAGE_TEMPLATE;
+import static org.apache.hadoop.yarn.exceptions
+        .InvalidResourceRequestException
+        .LESS_THAN_ZERO_RESOURCE_MESSAGE_TEMPLATE;
+import static org.apache.hadoop.yarn.exceptions
+        .InvalidResourceRequestException.UNKNOWN_REASON_MESSAGE_TEMPLATE;
+
 /**
  * Utilities shared by schedulers. 
  */
@@ -257,9 +268,9 @@ public class SchedulerUtils {
   }
 
 
-  public static void normalizeAndValidateRequest(ResourceRequest resReq,
-      Resource maximumResource, String queueName, YarnScheduler scheduler,
-      boolean isRecovery, RMContext rmContext, QueueInfo queueInfo)
+  private static void normalizeAndValidateRequest(ResourceRequest resReq,
+          Resource maximumResource, String queueName, YarnScheduler scheduler,
+          boolean isRecovery, RMContext rmContext, QueueInfo queueInfo)
       throws InvalidResourceRequestException {
     Configuration conf = rmContext.getYarnConfiguration();
     // If Node label is not enabled throw exception
@@ -384,13 +395,13 @@ public class SchedulerUtils {
 
       if (requestedRI.getValue() < 0) {
         throwInvalidResourceException(reqResource, availableResource,
-            reqResourceName);
+            reqResourceName, InvalidResourceType.LESS_THAN_ZERO);
       }
 
       boolean valid = checkResource(requestedRI, availableResource);
       if (!valid) {
         throwInvalidResourceException(reqResource, availableResource,
-            reqResourceName);
+            reqResourceName, InvalidResourceType.GREATER_THEN_MAX_ALLOCATION);
       }
     }
   }
@@ -470,18 +481,30 @@ public class SchedulerUtils {
   }
 
   private static void throwInvalidResourceException(Resource reqResource,
-      Resource availableResource, String reqResourceName)
+          Resource maxAllowedAllocation, String reqResourceName,
+          InvalidResourceType invalidResourceType)
       throws InvalidResourceRequestException {
-    throw new InvalidResourceRequestException(
-        "Invalid resource request, requested resource type=[" + reqResourceName
-            + "] < 0 or greater than maximum allowed allocation. Requested "
-            + "resource=" + reqResource + ", maximum allowed allocation="
-            + availableResource
-            + ", please note that maximum allowed allocation is calculated "
-            + "by scheduler based on maximum resource of registered "
-            + "NodeManagers, which might be less than configured "
-            + "maximum allocation="
-            + ResourceUtils.getResourceTypesMaximumAllocation());
+    final String message;
+
+    if (invalidResourceType == InvalidResourceType.LESS_THAN_ZERO) {
+      message = String.format(LESS_THAN_ZERO_RESOURCE_MESSAGE_TEMPLATE,
+          reqResourceName, reqResource);
+    } else if (invalidResourceType ==
+            InvalidResourceType.GREATER_THEN_MAX_ALLOCATION) {
+      message = String.format(GREATER_THAN_MAX_RESOURCE_MESSAGE_TEMPLATE,
+          reqResourceName, reqResource, maxAllowedAllocation,
+          ResourceUtils.getResourceTypesMaximumAllocation());
+    } else if (invalidResourceType == InvalidResourceType.UNKNOWN) {
+      message = String.format(UNKNOWN_REASON_MESSAGE_TEMPLATE, reqResourceName,
+          reqResource);
+    } else {
+      throw new IllegalArgumentException(String.format(
+          "InvalidResourceType argument should be either " + "%s, %s or %s",
+          InvalidResourceType.LESS_THAN_ZERO,
+          InvalidResourceType.GREATER_THEN_MAX_ALLOCATION,
+          InvalidResourceType.UNKNOWN));
+    }
+    throw new InvalidResourceRequestException(message, invalidResourceType);
   }
 
   private static void checkQueueLabelInLabelManager(String labelExpression,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2cccf406/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
index 15cfdb0..2ec2de2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
@@ -18,6 +18,11 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
+import static org.apache.hadoop.yarn.exceptions
+        .InvalidResourceRequestException.InvalidResourceType
+        .GREATER_THEN_MAX_ALLOCATION;
+import static org.apache.hadoop.yarn.exceptions
+        .InvalidResourceRequestException.InvalidResourceType.LESS_THAN_ZERO;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -67,6 +72,8 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.InvalidLabelResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceBlacklistRequestException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
+import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException
+        .InvalidResourceType;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
@@ -145,7 +152,7 @@ public class TestSchedulerUtils {
   private void initResourceTypes() {
     Configuration yarnConf = new Configuration();
     yarnConf.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
-        CustomResourceTypesConfigurationProvider.class.getName());
+            CustomResourceTypesConfigurationProvider.class.getName());
     ResourceUtils.resetResourceTypes(yarnConf);
   }
 
@@ -162,51 +169,51 @@ public class TestSchedulerUtils {
                     .build());
   }
 
-  @Test (timeout = 30000)
+  @Test(timeout = 30000)
   public void testNormalizeRequest() {
     ResourceCalculator resourceCalculator = new DefaultResourceCalculator();
-    
+
     final int minMemory = 1024;
     final int maxMemory = 8192;
     Resource minResource = Resources.createResource(minMemory, 0);
     Resource maxResource = Resources.createResource(maxMemory, 0);
-    
+
     ResourceRequest ask = new ResourceRequestPBImpl();
 
     // case negative memory
     ask.setCapability(Resources.createResource(-1024));
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, minResource,
-        maxResource);
+            maxResource);
     assertEquals(minMemory, ask.getCapability().getMemorySize());
 
     // case zero memory
     ask.setCapability(Resources.createResource(0));
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, minResource,
-        maxResource);
+            maxResource);
     assertEquals(minMemory, ask.getCapability().getMemorySize());
 
     // case memory is a multiple of minMemory
     ask.setCapability(Resources.createResource(2 * minMemory));
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, minResource,
-        maxResource);
+            maxResource);
     assertEquals(2 * minMemory, ask.getCapability().getMemorySize());
 
     // case memory is not a multiple of minMemory
     ask.setCapability(Resources.createResource(minMemory + 10));
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, minResource,
-        maxResource);
+            maxResource);
     assertEquals(2 * minMemory, ask.getCapability().getMemorySize());
 
     // case memory is equal to max allowed
     ask.setCapability(Resources.createResource(maxMemory));
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, minResource,
-        maxResource);
+            maxResource);
     assertEquals(maxMemory, ask.getCapability().getMemorySize());
 
     // case memory is just less than max
     ask.setCapability(Resources.createResource(maxMemory - 10));
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, minResource,
-        maxResource);
+            maxResource);
     assertEquals(maxMemory, ask.getCapability().getMemorySize());
 
     // max is not a multiple of min
@@ -214,39 +221,39 @@ public class TestSchedulerUtils {
     ask.setCapability(Resources.createResource(maxMemory - 100));
     // multiple of minMemory > maxMemory, then reduce to maxMemory
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, minResource,
-        maxResource);
+            maxResource);
     assertEquals(maxResource.getMemorySize(),
-        ask.getCapability().getMemorySize());
+            ask.getCapability().getMemorySize());
 
     // ask is more than max
     maxResource = Resources.createResource(maxMemory, 0);
     ask.setCapability(Resources.createResource(maxMemory + 100));
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, minResource,
-        maxResource);
+            maxResource);
     assertEquals(maxResource.getMemorySize(),
-        ask.getCapability().getMemorySize());
+            ask.getCapability().getMemorySize());
   }
 
-  @Test (timeout = 30000)
+  @Test(timeout = 30000)
   public void testNormalizeRequestWithDominantResourceCalculator() {
     ResourceCalculator resourceCalculator = new DominantResourceCalculator();
-    
+
     Resource minResource = Resources.createResource(1024, 1);
     Resource maxResource = Resources.createResource(10240, 10);
     Resource clusterResource = Resources.createResource(10 * 1024, 10);
-    
+
     ResourceRequest ask = new ResourceRequestPBImpl();
 
     // case negative memory/vcores
     ask.setCapability(Resources.createResource(-1024, -1));
     SchedulerUtils.normalizeRequest(
-        ask, resourceCalculator, minResource, maxResource);
+            ask, resourceCalculator, minResource, maxResource);
     assertEquals(minResource, ask.getCapability());
 
     // case zero memory/vcores
     ask.setCapability(Resources.createResource(0, 0));
     SchedulerUtils.normalizeRequest(
-        ask, resourceCalculator, minResource, maxResource);
+            ask, resourceCalculator, minResource, maxResource);
     assertEquals(minResource, ask.getCapability());
     assertEquals(1, ask.getCapability().getVirtualCores());
     assertEquals(1024, ask.getCapability().getMemorySize());
@@ -254,28 +261,28 @@ public class TestSchedulerUtils {
     // case non-zero memory & zero cores
     ask.setCapability(Resources.createResource(1536, 0));
     SchedulerUtils.normalizeRequest(
-        ask, resourceCalculator, minResource, maxResource);
+            ask, resourceCalculator, minResource, maxResource);
     assertEquals(Resources.createResource(2048, 1), ask.getCapability());
     assertEquals(1, ask.getCapability().getVirtualCores());
     assertEquals(2048, ask.getCapability().getMemorySize());
   }
-  
+
   @Test(timeout = 30000)
   public void testValidateResourceRequestWithErrorLabelsPermission()
-      throws IOException {
+          throws IOException {
     // mock queue and scheduler
     YarnScheduler scheduler = mock(YarnScheduler.class);
     Set<String> queueAccessibleNodeLabels = Sets.newHashSet();
     QueueInfo queueInfo = mock(QueueInfo.class);
     when(queueInfo.getQueueName()).thenReturn("queue");
     when(queueInfo.getAccessibleNodeLabels())
-        .thenReturn(queueAccessibleNodeLabels);
+            .thenReturn(queueAccessibleNodeLabels);
     when(scheduler.getQueueInfo(any(String.class), anyBoolean(), anyBoolean()))
-        .thenReturn(queueInfo);
+            .thenReturn(queueInfo);
 
     Resource maxResource = Resources.createResource(
-        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
-        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
 
     // queue has labels, success cases
     try {
@@ -283,36 +290,36 @@ public class TestSchedulerUtils {
       queueAccessibleNodeLabels.clear();
       queueAccessibleNodeLabels.addAll(Arrays.asList("x", "y"));
       rmContext.getNodeLabelManager().addToCluserNodeLabels(
-          ImmutableSet.of(NodeLabel.newInstance("x"),
-              NodeLabel.newInstance("y")));
+              ImmutableSet.of(NodeLabel.newInstance("x"),
+                      NodeLabel.newInstance("y")));
       Resource resource = Resources.createResource(
-          0,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              0,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
-          mock(Priority.class), ResourceRequest.ANY, resource, 1);
+              mock(Priority.class), ResourceRequest.ANY, resource, 1);
       resReq.setNodeLabelExpression("x");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
+              scheduler, rmContext);
 
       resReq.setNodeLabelExpression("y");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
-      
+              scheduler, rmContext);
+
       resReq.setNodeLabelExpression("");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
-      
+              scheduler, rmContext);
+
       resReq.setNodeLabelExpression(" ");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
+              scheduler, rmContext);
     } catch (InvalidResourceRequestException e) {
       e.printStackTrace();
       fail("Should be valid when request labels is a subset of queue labels");
     } finally {
       rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
-          Arrays.asList("x", "y"));
+              Arrays.asList("x", "y"));
     }
-    
+
     // same as above, but cluster node labels don't contains label being
     // requested. should fail
     try {
@@ -320,42 +327,42 @@ public class TestSchedulerUtils {
       queueAccessibleNodeLabels.clear();
       queueAccessibleNodeLabels.addAll(Arrays.asList("x", "y"));
       Resource resource = Resources.createResource(
-          0,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              0,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
-          mock(Priority.class), ResourceRequest.ANY, resource, 1);
+              mock(Priority.class), ResourceRequest.ANY, resource, 1);
       resReq.setNodeLabelExpression("x");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
-      
+              scheduler, rmContext);
+
       fail("Should fail");
     } catch (InvalidResourceRequestException e) {
     }
-    
+
     // queue has labels, failed cases (when ask a label not included by queue)
     try {
       // set queue accessible node labesl to [x, y]
       queueAccessibleNodeLabels.clear();
       queueAccessibleNodeLabels.addAll(Arrays.asList("x", "y"));
       rmContext.getNodeLabelManager().addToCluserNodeLabels(
-          ImmutableSet.of(NodeLabel.newInstance("x"),
-              NodeLabel.newInstance("y")));
-      
+              ImmutableSet.of(NodeLabel.newInstance("x"),
+                      NodeLabel.newInstance("y")));
+
       Resource resource = Resources.createResource(
-          0,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              0,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
-          mock(Priority.class), ResourceRequest.ANY, resource, 1);
+              mock(Priority.class), ResourceRequest.ANY, resource, 1);
       resReq.setNodeLabelExpression("z");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
+              scheduler, rmContext);
       fail("Should fail");
     } catch (InvalidResourceRequestException e) {
     } finally {
       rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
-          Arrays.asList("x", "y"));
+              Arrays.asList("x", "y"));
     }
-    
+
     // we don't allow specify more than two node labels in a single expression
     // now
     try {
@@ -363,225 +370,225 @@ public class TestSchedulerUtils {
       queueAccessibleNodeLabels.clear();
       queueAccessibleNodeLabels.addAll(Arrays.asList("x", "y"));
       rmContext.getNodeLabelManager().addToCluserNodeLabels(
-          ImmutableSet.of(NodeLabel.newInstance("x"),
-              NodeLabel.newInstance("y")));
-      
+              ImmutableSet.of(NodeLabel.newInstance("x"),
+                      NodeLabel.newInstance("y")));
+
       Resource resource = Resources.createResource(
-          0,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              0,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
-          mock(Priority.class), ResourceRequest.ANY, resource, 1);
+              mock(Priority.class), ResourceRequest.ANY, resource, 1);
       resReq.setNodeLabelExpression("x && y");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
+              scheduler, rmContext);
       fail("Should fail");
     } catch (InvalidResourceRequestException e) {
     } finally {
       rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
-          Arrays.asList("x", "y"));
+              Arrays.asList("x", "y"));
     }
-    
+
     // queue doesn't have label, succeed (when request no label)
     queueAccessibleNodeLabels.clear();
     try {
       // set queue accessible node labels to empty
       queueAccessibleNodeLabels.clear();
-      
+
       Resource resource = Resources.createResource(
-          0,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              0,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
-          mock(Priority.class), ResourceRequest.ANY, resource, 1);
+              mock(Priority.class), ResourceRequest.ANY, resource, 1);
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
-      
+              scheduler, rmContext);
+
       resReq.setNodeLabelExpression("");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
-      
+              scheduler, rmContext);
+
       resReq.setNodeLabelExpression("  ");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
+              scheduler, rmContext);
     } catch (InvalidResourceRequestException e) {
       e.printStackTrace();
       fail("Should be valid when request labels is empty");
     }
-    boolean invalidlabelexception=false;
+    boolean invalidlabelexception = false;
     // queue doesn't have label, failed (when request any label)
     try {
       // set queue accessible node labels to empty
       queueAccessibleNodeLabels.clear();
-      
+
       rmContext.getNodeLabelManager().addToCluserNodeLabels(
-          ImmutableSet.of(NodeLabel.newInstance("x")));
-      
+              ImmutableSet.of(NodeLabel.newInstance("x")));
+
       Resource resource = Resources.createResource(
-          0,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              0,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
-          mock(Priority.class), ResourceRequest.ANY, resource, 1);
+              mock(Priority.class), ResourceRequest.ANY, resource, 1);
       resReq.setNodeLabelExpression("x");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
+              scheduler, rmContext);
       fail("Should fail");
     } catch (InvalidLabelResourceRequestException e) {
-      invalidlabelexception=true;
+      invalidlabelexception = true;
     } catch (InvalidResourceRequestException e) {
     } finally {
       rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
-          Arrays.asList("x"));
+              Arrays.asList("x"));
     }
     Assert.assertTrue("InvalidLabelResourceRequestException expected",
-        invalidlabelexception);
+            invalidlabelexception);
     // queue is "*", always succeeded
     try {
       // set queue accessible node labels to empty
       queueAccessibleNodeLabels.clear();
       queueAccessibleNodeLabels.add(RMNodeLabelsManager.ANY);
-      
+
       rmContext.getNodeLabelManager().addToCluserNodeLabels(
-          ImmutableSet.of(NodeLabel.newInstance("x"),
-              NodeLabel.newInstance("y"), NodeLabel.newInstance("z")));
-      
+              ImmutableSet.of(NodeLabel.newInstance("x"),
+                      NodeLabel.newInstance("y"), NodeLabel.newInstance("z")));
+
       Resource resource = Resources.createResource(
-          0,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              0,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
-          mock(Priority.class), ResourceRequest.ANY, resource, 1);
+              mock(Priority.class), ResourceRequest.ANY, resource, 1);
       resReq.setNodeLabelExpression("x");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
-      
+              scheduler, rmContext);
+
       resReq.setNodeLabelExpression("y");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
-      
+              scheduler, rmContext);
+
       resReq.setNodeLabelExpression("z");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
+              scheduler, rmContext);
     } catch (InvalidResourceRequestException e) {
       e.printStackTrace();
       fail("Should be valid when queue can access any labels");
     } finally {
       rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
-          Arrays.asList("x", "y", "z"));
+              Arrays.asList("x", "y", "z"));
     }
-    
+
     // same as above, but cluster node labels don't contains label, should fail
     try {
       // set queue accessible node labels to empty
       queueAccessibleNodeLabels.clear();
       queueAccessibleNodeLabels.add(RMNodeLabelsManager.ANY);
-      
+
       Resource resource = Resources.createResource(
-          0,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              0,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
-          mock(Priority.class), ResourceRequest.ANY, resource, 1);
+              mock(Priority.class), ResourceRequest.ANY, resource, 1);
       resReq.setNodeLabelExpression("x");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
+              scheduler, rmContext);
       fail("Should fail");
     } catch (InvalidResourceRequestException e) {
     }
-    
+
     // we don't allow resource name other than ANY and specify label
     try {
       // set queue accessible node labesl to [x, y]
       queueAccessibleNodeLabels.clear();
       queueAccessibleNodeLabels.addAll(Arrays.asList("x", "y"));
       rmContext.getNodeLabelManager().addToCluserNodeLabels(
-          ImmutableSet.of(NodeLabel.newInstance("x"),
-              NodeLabel.newInstance("y")));
-      
+              ImmutableSet.of(NodeLabel.newInstance("x"),
+                      NodeLabel.newInstance("y")));
+
       Resource resource = Resources.createResource(
-          0,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              0,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
-          mock(Priority.class), "rack", resource, 1);
+              mock(Priority.class), "rack", resource, 1);
       resReq.setNodeLabelExpression("x");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
+              scheduler, rmContext);
       fail("Should fail");
     } catch (InvalidResourceRequestException e) {
     } finally {
       rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
-          Arrays.asList("x", "y"));
+              Arrays.asList("x", "y"));
     }
-    
+
     // we don't allow resource name other than ANY and specify label even if
     // queue has accessible label = *
     try {
       // set queue accessible node labesl to *
       queueAccessibleNodeLabels.clear();
       queueAccessibleNodeLabels.addAll(Arrays
-          .asList(CommonNodeLabelsManager.ANY));
+              .asList(CommonNodeLabelsManager.ANY));
       rmContext.getNodeLabelManager().addToCluserNodeLabels(
-          ImmutableSet.of(NodeLabel.newInstance("x")));
-      
+              ImmutableSet.of(NodeLabel.newInstance("x")));
+
       Resource resource = Resources.createResource(
-          0,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              0,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
-          mock(Priority.class), "rack", resource, 1);
+              mock(Priority.class), "rack", resource, 1);
       resReq.setNodeLabelExpression("x");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
+              scheduler, rmContext);
       fail("Should fail");
     } catch (InvalidResourceRequestException e) {
     } finally {
       rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
-          Arrays.asList("x"));
+              Arrays.asList("x"));
     }
     try {
       Resource resource = Resources.createResource(0,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq1 = BuilderUtils
-          .newResourceRequest(mock(Priority.class), "*", resource, 1, "x");
+              .newResourceRequest(mock(Priority.class), "*", resource, 1, "x");
       SchedulerUtils.normalizeAndvalidateRequest(resReq1, maxResource, "queue",
-          scheduler, rmContext);
+              scheduler, rmContext);
       fail("Should fail");
     } catch (InvalidResourceRequestException e) {
       assertEquals("Invalid label resource request, cluster do not contain , "
-          + "label= x", e.getMessage());
+              + "label= x", e.getMessage());
     }
 
     try {
       rmContext.getYarnConfiguration()
-          .set(YarnConfiguration.NODE_LABELS_ENABLED, "false");
+              .set(YarnConfiguration.NODE_LABELS_ENABLED, "false");
       Resource resource = Resources.createResource(0,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq1 = BuilderUtils
-          .newResourceRequest(mock(Priority.class), "*", resource, 1, "x");
+              .newResourceRequest(mock(Priority.class), "*", resource, 1, "x");
       SchedulerUtils.normalizeAndvalidateRequest(resReq1, maxResource, "queue",
-          scheduler, rmContext);
+              scheduler, rmContext);
       Assert.assertEquals(RMNodeLabelsManager.NO_LABEL,
-          resReq1.getNodeLabelExpression());
+              resReq1.getNodeLabelExpression());
     } catch (InvalidResourceRequestException e) {
       assertEquals("Invalid resource request, node label not enabled but "
-          + "request contains label expression", e.getMessage());
+              + "request contains label expression", e.getMessage());
     }
   }
 
-  @Test (timeout = 30000)
+  @Test(timeout = 30000)
   public void testValidateResourceRequest() {
     YarnScheduler mockScheduler = mock(YarnScheduler.class);
 
     Resource maxResource =
-        Resources.createResource(
-            YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
-            YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
+            Resources.createResource(
+                    YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+                    YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
 
     // zero memory
     try {
       Resource resource =
-          Resources.createResource(0,
-              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              Resources.createResource(0,
+                      YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq =
-          BuilderUtils.newResourceRequest(mock(Priority.class),
-              ResourceRequest.ANY, resource, 1);
+              BuilderUtils.newResourceRequest(mock(Priority.class),
+                      ResourceRequest.ANY, resource, 1);
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
-          mockScheduler, rmContext);
+              mockScheduler, rmContext);
     } catch (InvalidResourceRequestException e) {
       fail("Zero memory should be accepted");
     }
@@ -589,13 +596,13 @@ public class TestSchedulerUtils {
     // zero vcores
     try {
       Resource resource =
-          Resources.createResource(
-              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 0);
+              Resources.createResource(
+                      YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 0);
       ResourceRequest resReq =
-          BuilderUtils.newResourceRequest(mock(Priority.class),
-              ResourceRequest.ANY, resource, 1);
+              BuilderUtils.newResourceRequest(mock(Priority.class),
+                      ResourceRequest.ANY, resource, 1);
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
-          mockScheduler, rmContext);
+              mockScheduler, rmContext);
     } catch (InvalidResourceRequestException e) {
       fail("Zero vcores should be accepted");
     }
@@ -603,14 +610,14 @@ public class TestSchedulerUtils {
     // max memory
     try {
       Resource resource =
-          Resources.createResource(
-              YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
-              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              Resources.createResource(
+                      YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+                      YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq =
-          BuilderUtils.newResourceRequest(mock(Priority.class),
-              ResourceRequest.ANY, resource, 1);
+              BuilderUtils.newResourceRequest(mock(Priority.class),
+                      ResourceRequest.ANY, resource, 1);
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
-          mockScheduler, rmContext);
+              mockScheduler, rmContext);
     } catch (InvalidResourceRequestException e) {
       fail("Max memory should be accepted");
     }
@@ -618,14 +625,14 @@ public class TestSchedulerUtils {
     // max vcores
     try {
       Resource resource =
-          Resources.createResource(
-              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
-              YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
+              Resources.createResource(
+                      YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+                      YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq =
-          BuilderUtils.newResourceRequest(mock(Priority.class),
-              ResourceRequest.ANY, resource, 1);
+              BuilderUtils.newResourceRequest(mock(Priority.class),
+                      ResourceRequest.ANY, resource, 1);
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
-          mockScheduler, rmContext);
+              mockScheduler, rmContext);
     } catch (InvalidResourceRequestException e) {
       fail("Max vcores should not be accepted");
     }
@@ -633,77 +640,77 @@ public class TestSchedulerUtils {
     // negative memory
     try {
       Resource resource =
-          Resources.createResource(-1,
-              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              Resources.createResource(-1,
+                      YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq =
-          BuilderUtils.newResourceRequest(mock(Priority.class),
-              ResourceRequest.ANY, resource, 1);
+              BuilderUtils.newResourceRequest(mock(Priority.class),
+                      ResourceRequest.ANY, resource, 1);
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
-          mockScheduler, rmContext);
+              mockScheduler, rmContext);
       fail("Negative memory should not be accepted");
     } catch (InvalidResourceRequestException e) {
-      // expected
+      assertEquals(LESS_THAN_ZERO, e.getInvalidResourceType());
     }
 
     // negative vcores
     try {
       Resource resource =
-          Resources.createResource(
-              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB, -1);
+              Resources.createResource(
+                      YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB, -1);
       ResourceRequest resReq =
-          BuilderUtils.newResourceRequest(mock(Priority.class),
-              ResourceRequest.ANY, resource, 1);
+              BuilderUtils.newResourceRequest(mock(Priority.class),
+                      ResourceRequest.ANY, resource, 1);
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
-          mockScheduler, rmContext);
+              mockScheduler, rmContext);
       fail("Negative vcores should not be accepted");
     } catch (InvalidResourceRequestException e) {
-      // expected
+      assertEquals(LESS_THAN_ZERO, e.getInvalidResourceType());
     }
 
     // more than max memory
     try {
       Resource resource =
-          Resources.createResource(
-              YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB + 1,
-              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              Resources.createResource(
+                      YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB + 1,
+                      YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq =
-          BuilderUtils.newResourceRequest(mock(Priority.class),
-              ResourceRequest.ANY, resource, 1);
+              BuilderUtils.newResourceRequest(mock(Priority.class),
+                      ResourceRequest.ANY, resource, 1);
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
-          mockScheduler, rmContext);
+              mockScheduler, rmContext);
       fail("More than max memory should not be accepted");
     } catch (InvalidResourceRequestException e) {
-      // expected
+      assertEquals(GREATER_THEN_MAX_ALLOCATION, e.getInvalidResourceType());
     }
 
     // more than max vcores
     try {
       Resource resource = Resources.createResource(
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES + 1);
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES + 1);
       ResourceRequest resReq =
-          BuilderUtils.newResourceRequest(mock(Priority.class),
-              ResourceRequest.ANY, resource, 1);
+              BuilderUtils.newResourceRequest(mock(Priority.class),
+                      ResourceRequest.ANY, resource, 1);
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
-          mockScheduler, rmContext);
+              mockScheduler, rmContext);
       fail("More than max vcores should not be accepted");
     } catch (InvalidResourceRequestException e) {
-      // expected
+      assertEquals(GREATER_THEN_MAX_ALLOCATION, e.getInvalidResourceType());
     }
   }
-  
+
   @Test
   public void testValidateResourceBlacklistRequest() throws Exception {
 
     MyContainerManager containerManager = new MyContainerManager();
     final MockRMWithAMS rm =
-        new MockRMWithAMS(new YarnConfiguration(), containerManager);
+            new MockRMWithAMS(new YarnConfiguration(), containerManager);
     rm.start();
 
     MockNM nm1 = rm.registerNode("localhost:1234", 5120);
 
     Map<ApplicationAccessType, String> acls =
-        new HashMap<ApplicationAccessType, String>(2);
+            new HashMap<ApplicationAccessType, String>(2);
     acls.put(ApplicationAccessType.VIEW_APP, "*");
     RMApp app = rm.submitApp(1024, "appname", "appuser", acls);
 
@@ -718,33 +725,33 @@ public class TestSchedulerUtils {
     final YarnRPC rpc = YarnRPC.create(yarnConf);
 
     UserGroupInformation currentUser =
-        UserGroupInformation.createRemoteUser(applicationAttemptId.toString());
+            UserGroupInformation.createRemoteUser(applicationAttemptId.toString());
     Credentials credentials = containerManager.getContainerCredentials();
     final InetSocketAddress rmBindAddress =
-        rm.getApplicationMasterService().getBindAddress();
+            rm.getApplicationMasterService().getBindAddress();
     Token<? extends TokenIdentifier> amRMToken =
-        MockRMWithAMS.setupAndReturnAMRMToken(rmBindAddress,
-          credentials.getAllTokens());
+            MockRMWithAMS.setupAndReturnAMRMToken(rmBindAddress,
+                    credentials.getAllTokens());
     currentUser.addToken(amRMToken);
     ApplicationMasterProtocol client =
-        currentUser.doAs(new PrivilegedAction<ApplicationMasterProtocol>() {
-          @Override
-          public ApplicationMasterProtocol run() {
-            return (ApplicationMasterProtocol) rpc.getProxy(
-              ApplicationMasterProtocol.class, rmBindAddress, yarnConf);
-          }
-        });
+            currentUser.doAs(new PrivilegedAction<ApplicationMasterProtocol>() {
+              @Override
+              public ApplicationMasterProtocol run() {
+                return (ApplicationMasterProtocol) rpc.getProxy(
+                        ApplicationMasterProtocol.class, rmBindAddress, yarnConf);
+              }
+            });
 
     RegisterApplicationMasterRequest request = Records
-        .newRecord(RegisterApplicationMasterRequest.class);
+            .newRecord(RegisterApplicationMasterRequest.class);
     client.registerApplicationMaster(request);
 
     ResourceBlacklistRequest blacklistRequest =
-        ResourceBlacklistRequest.newInstance(
-            Collections.singletonList(ResourceRequest.ANY), null);
+            ResourceBlacklistRequest.newInstance(
+                    Collections.singletonList(ResourceRequest.ANY), null);
 
     AllocateRequest allocateRequest =
-        AllocateRequest.newInstance(0, 0.0f, null, null, blacklistRequest);
+            AllocateRequest.newInstance(0, 0.0f, null, null, blacklistRequest);
     boolean error = false;
     try {
       client.allocate(allocateRequest);
@@ -753,26 +760,26 @@ public class TestSchedulerUtils {
     }
 
     rm.stop();
-    
+
     Assert.assertTrue(
-        "Didn't not catch InvalidResourceBlacklistRequestException", error);
+            "Didn't not catch InvalidResourceBlacklistRequestException", error);
   }
 
   private void waitForLaunchedState(RMAppAttempt attempt)
-      throws InterruptedException {
+          throws InterruptedException {
     int waitCount = 0;
     while (attempt.getAppAttemptState() != RMAppAttemptState.LAUNCHED
-        && waitCount++ < 20) {
+            && waitCount++ < 20) {
       LOG.info("Waiting for AppAttempt to reach LAUNCHED state. "
-          + "Current state is " + attempt.getAppAttemptState());
+              + "Current state is " + attempt.getAppAttemptState());
       Thread.sleep(1000);
     }
     Assert.assertEquals(attempt.getAppAttemptState(),
-        RMAppAttemptState.LAUNCHED);
+            RMAppAttemptState.LAUNCHED);
   }
 
   @Test
-  public void testComparePriorities(){
+  public void testComparePriorities() {
     Priority high = Priority.newInstance(1);
     Priority low = Priority.newInstance(2);
     assertTrue(high.compareTo(low) > 0);
@@ -781,22 +788,22 @@ public class TestSchedulerUtils {
   @Test
   public void testCreateAbnormalContainerStatus() {
     ContainerStatus cd = SchedulerUtils.createAbnormalContainerStatus(
-        ContainerId.newContainerId(ApplicationAttemptId.newInstance(
-          ApplicationId.newInstance(System.currentTimeMillis(), 1), 1), 1), "x");
+            ContainerId.newContainerId(ApplicationAttemptId.newInstance(
+                    ApplicationId.newInstance(System.currentTimeMillis(), 1), 1), 1), "x");
     Assert.assertEquals(ContainerExitStatus.ABORTED, cd.getExitStatus());
   }
 
   @Test
   public void testCreatePreemptedContainerStatus() {
     ContainerStatus cd = SchedulerUtils.createPreemptedContainerStatus(
-        ContainerId.newContainerId(ApplicationAttemptId.newInstance(
-          ApplicationId.newInstance(System.currentTimeMillis(), 1), 1), 1), "x");
+            ContainerId.newContainerId(ApplicationAttemptId.newInstance(
+                    ApplicationId.newInstance(System.currentTimeMillis(), 1), 1), 1), "x");
     Assert.assertEquals(ContainerExitStatus.PREEMPTED, cd.getExitStatus());
   }
-  
-  @Test (timeout = 30000)
+
+  @Test(timeout = 30000)
   public void testNormalizeNodeLabelExpression()
-      throws IOException {
+          throws IOException {
     // mock queue and scheduler
     YarnScheduler scheduler = mock(YarnScheduler.class);
     Set<String> queueAccessibleNodeLabels = Sets.newHashSet();
@@ -805,11 +812,11 @@ public class TestSchedulerUtils {
     when(queueInfo.getAccessibleNodeLabels()).thenReturn(queueAccessibleNodeLabels);
     when(queueInfo.getDefaultNodeLabelExpression()).thenReturn(" x ");
     when(scheduler.getQueueInfo(any(String.class), anyBoolean(), anyBoolean()))
-        .thenReturn(queueInfo);
-    
+            .thenReturn(queueInfo);
+
     Resource maxResource = Resources.createResource(
-        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
-        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
 
     // queue has labels, success cases
     try {
@@ -817,156 +824,163 @@ public class TestSchedulerUtils {
       queueAccessibleNodeLabels.clear();
       queueAccessibleNodeLabels.addAll(Arrays.asList("x", "y"));
       rmContext.getNodeLabelManager().addToCluserNodeLabels(
-          ImmutableSet.of(NodeLabel.newInstance("x"),
-              NodeLabel.newInstance("y")));
+              ImmutableSet.of(NodeLabel.newInstance("x"),
+                      NodeLabel.newInstance("y")));
       Resource resource = Resources.createResource(
-          0,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+              0,
+              YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
       ResourceRequest resReq = BuilderUtils.newResourceRequest(
-          mock(Priority.class), ResourceRequest.ANY, resource, 1);
+              mock(Priority.class), ResourceRequest.ANY, resource, 1);
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
+              scheduler, rmContext);
       Assert.assertEquals("x", resReq.getNodeLabelExpression());
-      
+
       resReq.setNodeLabelExpression(" y ");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
-          scheduler, rmContext);
+              scheduler, rmContext);
       Assert.assertEquals("y", resReq.getNodeLabelExpression());
     } catch (InvalidResourceRequestException e) {
       e.printStackTrace();
       fail("Should be valid when request labels is a subset of queue labels");
     } finally {
       rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
-          Arrays.asList("x", "y"));
+              Arrays.asList("x", "y"));
     }
   }
 
   @Test
   public void testCustomResourceRequestedUnitIsSmallerThanAvailableUnit()
-      throws InvalidResourceRequestException {
+          throws InvalidResourceRequestException {
     Resource requestedResource =
-        ResourceTypesTestHelper.newResource(1, 1,
-                ImmutableMap.of("custom-resource-1", "11"));
+            ResourceTypesTestHelper.newResource(1, 1,
+                    ImmutableMap.of("custom-resource-1", "11"));
 
     Resource availableResource =
-        ResourceTypesTestHelper.newResource(1, 1,
-                ImmutableMap.of("custom-resource-1", "0G"));
+            ResourceTypesTestHelper.newResource(1, 1,
+                    ImmutableMap.of("custom-resource-1", "0G"));
 
     exception.expect(InvalidResourceRequestException.class);
     exception.expectMessage(InvalidResourceRequestExceptionMessageGenerator
-        .create().withRequestedResourceType("custom-resource-1")
-        .withRequestedResource(requestedResource)
-        .withAvailableAllocation(availableResource)
-        .withMaxAllocation(configuredMaxAllocation).build());
+            .create().withRequestedResourceType("custom-resource-1")
+            .withRequestedResource(requestedResource)
+            .withAvailableAllocation(availableResource)
+            .withMaxAllocation(configuredMaxAllocation)
+            .withInvalidResourceType(GREATER_THEN_MAX_ALLOCATION)
+            .build());
 
     SchedulerUtils.checkResourceRequestAgainstAvailableResource(
-        requestedResource, availableResource);
+            requestedResource, availableResource);
   }
 
   @Test
   public void testCustomResourceRequestedUnitIsSmallerThanAvailableUnit2() {
     Resource requestedResource =
-        ResourceTypesTestHelper.newResource(1, 1,
-                ImmutableMap.of("custom-resource-1", "11"));
+            ResourceTypesTestHelper.newResource(1, 1,
+                    ImmutableMap.of("custom-resource-1", "11"));
 
     Resource availableResource =
-        ResourceTypesTestHelper.newResource(1, 1,
-                ImmutableMap.of("custom-resource-1", "1G"));
+            ResourceTypesTestHelper.newResource(1, 1,
+                    ImmutableMap.of("custom-resource-1", "1G"));
 
     try {
       SchedulerUtils.checkResourceRequestAgainstAvailableResource(
-          requestedResource, availableResource);
+              requestedResource, availableResource);
     } catch (InvalidResourceRequestException e) {
       fail(String.format(
-          "Resource request should be accepted. Requested: %s, available: %s",
-          requestedResource, availableResource));
+              "Resource request should be accepted. Requested: %s, available: %s",
+              requestedResource, availableResource));
     }
   }
 
   @Test
   public void testCustomResourceRequestedUnitIsGreaterThanAvailableUnit()
-      throws InvalidResourceRequestException {
+          throws InvalidResourceRequestException {
     Resource requestedResource =
-        ResourceTypesTestHelper.newResource(1, 1,
-                ImmutableMap.of("custom-resource-1", "1M"));
+            ResourceTypesTestHelper.newResource(1, 1,
+                    ImmutableMap.of("custom-resource-1", "1M"));
 
     Resource availableResource = ResourceTypesTestHelper.newResource(1, 1,
-        ImmutableMap.<String, String> builder().put("custom-resource-1", "120k")
-            .build());
+            ImmutableMap.<String, String>builder().put("custom-resource-1",
+                    "120k")
+                    .build());
 
     exception.expect(InvalidResourceRequestException.class);
     exception.expectMessage(InvalidResourceRequestExceptionMessageGenerator
-        .create().withRequestedResourceType("custom-resource-1")
-        .withRequestedResource(requestedResource)
-        .withAvailableAllocation(availableResource)
-        .withMaxAllocation(configuredMaxAllocation).build());
+            .create().withRequestedResourceType("custom-resource-1")
+            .withRequestedResource(requestedResource)
+            .withAvailableAllocation(availableResource)
+            .withMaxAllocation(configuredMaxAllocation)
+            .withInvalidResourceType(GREATER_THEN_MAX_ALLOCATION)
+            .build());
     SchedulerUtils.checkResourceRequestAgainstAvailableResource(
-        requestedResource, availableResource);
+            requestedResource, availableResource);
   }
 
   @Test
   public void testCustomResourceRequestedUnitIsGreaterThanAvailableUnit2() {
     Resource requestedResource = ResourceTypesTestHelper.newResource(1, 1,
-        ImmutableMap.<String, String> builder().put("custom-resource-1", "11M")
-            .build());
+            ImmutableMap.<String, String>builder().put("custom-resource-1", "11M")
+                    .build());
 
     Resource availableResource =
-        ResourceTypesTestHelper.newResource(1, 1,
-                ImmutableMap.of("custom-resource-1", "1G"));
+            ResourceTypesTestHelper.newResource(1, 1,
+                    ImmutableMap.of("custom-resource-1", "1G"));
 
     try {
       SchedulerUtils.checkResourceRequestAgainstAvailableResource(
-          requestedResource, availableResource);
+              requestedResource, availableResource);
     } catch (InvalidResourceRequestException e) {
       fail(String.format(
-          "Resource request should be accepted. Requested: %s, available: %s",
-          requestedResource, availableResource));
+              "Resource request should be accepted. Requested: %s, available: %s",
+              requestedResource, availableResource));
     }
   }
 
   @Test
   public void testCustomResourceRequestedUnitIsSameAsAvailableUnit() {
     Resource requestedResource = ResourceTypesTestHelper.newResource(1, 1,
-        ImmutableMap.of("custom-resource-1", "11M"));
+            ImmutableMap.of("custom-resource-1", "11M"));
 
     Resource availableResource = ResourceTypesTestHelper.newResource(1, 1,
-        ImmutableMap.of("custom-resource-1", "100M"));
+            ImmutableMap.of("custom-resource-1", "100M"));
 
     try {
       SchedulerUtils.checkResourceRequestAgainstAvailableResource(
-          requestedResource, availableResource);
+              requestedResource, availableResource);
     } catch (InvalidResourceRequestException e) {
       fail(String.format(
-          "Resource request should be accepted. Requested: %s, available: %s",
-          requestedResource, availableResource));
+              "Resource request should be accepted. Requested: %s, available: %s",
+              requestedResource, availableResource));
     }
   }
 
   @Test
   public void testCustomResourceRequestedUnitIsSameAsAvailableUnit2()
-      throws InvalidResourceRequestException {
+          throws InvalidResourceRequestException {
     Resource requestedResource = ResourceTypesTestHelper.newResource(1, 1,
-        ImmutableMap.of("custom-resource-1", "110M"));
+            ImmutableMap.of("custom-resource-1", "110M"));
 
     Resource availableResource = ResourceTypesTestHelper.newResource(1, 1,
-        ImmutableMap.of("custom-resource-1", "100M"));
+            ImmutableMap.of("custom-resource-1", "100M"));
 
     exception.expect(InvalidResourceRequestException.class);
     exception.expectMessage(InvalidResourceRequestExceptionMessageGenerator
-        .create().withRequestedResourceType("custom-resource-1")
-        .withRequestedResource(requestedResource)
-        .withAvailableAllocation(availableResource)
-        .withMaxAllocation(configuredMaxAllocation).build());
+            .create().withRequestedResourceType("custom-resource-1")
+            .withRequestedResource(requestedResource)
+            .withAvailableAllocation(availableResource)
+            .withInvalidResourceType(GREATER_THEN_MAX_ALLOCATION)
+            .withMaxAllocation(configuredMaxAllocation)
+            .build());
 
     SchedulerUtils.checkResourceRequestAgainstAvailableResource(
-        requestedResource, availableResource);
+            requestedResource, availableResource);
   }
 
   public static void waitSchedulerApplicationAttemptStopped(
-      AbstractYarnScheduler ys,
-      ApplicationAttemptId attemptId) throws InterruptedException {
+          AbstractYarnScheduler ys,
+          ApplicationAttemptId attemptId) throws InterruptedException {
     SchedulerApplicationAttempt schedulerApp =
-        ys.getApplicationAttempt(attemptId);
+            ys.getApplicationAttempt(attemptId);
     if (null == schedulerApp) {
       return;
     }
@@ -986,35 +1000,35 @@ public class TestSchedulerUtils {
   }
 
   public static SchedulerApplication<SchedulerApplicationAttempt>
-      verifyAppAddedAndRemovedFromScheduler(
+  verifyAppAddedAndRemovedFromScheduler(
           Map<ApplicationId, SchedulerApplication<SchedulerApplicationAttempt>> applications,
           EventHandler<SchedulerEvent> handler, String queueName) {
 
     ApplicationId appId =
-        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+            ApplicationId.newInstance(System.currentTimeMillis(), 1);
     AppAddedSchedulerEvent appAddedEvent =
-        new AppAddedSchedulerEvent(appId, queueName, "user");
+            new AppAddedSchedulerEvent(appId, queueName, "user");
     handler.handle(appAddedEvent);
     SchedulerApplication<SchedulerApplicationAttempt> app =
-        applications.get(appId);
+            applications.get(appId);
     // verify application is added.
     Assert.assertNotNull(app);
     Assert.assertEquals("user", app.getUser());
 
     AppRemovedSchedulerEvent appRemoveEvent =
-        new AppRemovedSchedulerEvent(appId, RMAppState.FINISHED);
+            new AppRemovedSchedulerEvent(appId, RMAppState.FINISHED);
     handler.handle(appRemoveEvent);
     Assert.assertNull(applications.get(appId));
     return app;
   }
-  
+
   private static RMContext getMockRMContext() {
     RMContext rmContext = mock(RMContext.class);
     RMNodeLabelsManager nlm = new NullRMNodeLabelsManager();
     nlm.init(new Configuration(false));
     when(rmContext.getYarnConfiguration()).thenReturn(conf);
     rmContext.getYarnConfiguration().set(YarnConfiguration.NODE_LABELS_ENABLED,
-        "true");
+            "true");
     when(rmContext.getNodeLabelManager()).thenReturn(nlm);
     return rmContext;
   }
@@ -1026,6 +1040,7 @@ public class TestSchedulerUtils {
     private Resource availableAllocation;
     private Resource configuredMaxAllowedAllocation;
     private String resourceType;
+    private InvalidResourceType invalidResourceType;
 
     InvalidResourceRequestExceptionMessageGenerator(StringBuilder sb) {
       this.sb = sb;
@@ -1033,7 +1048,7 @@ public class TestSchedulerUtils {
 
     public static InvalidResourceRequestExceptionMessageGenerator create() {
       return new InvalidResourceRequestExceptionMessageGenerator(
-          new StringBuilder());
+              new StringBuilder());
     }
 
     InvalidResourceRequestExceptionMessageGenerator withRequestedResource(
@@ -1055,23 +1070,46 @@ public class TestSchedulerUtils {
     }
 
     InvalidResourceRequestExceptionMessageGenerator withMaxAllocation(
-        Resource r) {
+            Resource r) {
       this.configuredMaxAllowedAllocation = r;
       return this;
     }
 
+    InvalidResourceRequestExceptionMessageGenerator
+    withInvalidResourceType(InvalidResourceType invalidResourceType) {
+      this.invalidResourceType = invalidResourceType;
+      return this;
+    }
+
     public String build() {
-      return sb
-          .append("Invalid resource request, requested resource type=[")
-          .append(resourceType).append("]")
-          .append(" < 0 or greater than maximum allowed allocation. ")
-          .append("Requested resource=").append(requestedResource).append(", ")
-          .append("maximum allowed allocation=").append(availableAllocation)
-          .append(", please note that maximum allowed allocation is calculated "
-              + "by scheduler based on maximum resource of " +
-                  "registered NodeManagers, which might be less than " +
-                  "configured maximum allocation=")
-          .append(configuredMaxAllowedAllocation).toString();
+      if (invalidResourceType == LESS_THAN_ZERO) {
+        return sb.append("Invalid resource request! " +
+                "Cannot allocate containers as " +
+                "requested resource is less than 0! ")
+                .append("Requested resource type=[")
+                .append(resourceType).append("]")
+                .append(", Requested resource=")
+                .append(requestedResource).toString();
+
+      } else if (invalidResourceType == GREATER_THEN_MAX_ALLOCATION) {
+        return sb.append("Invalid resource request! " +
+                "Cannot allocate containers as "
+                + "requested resource is greater than " +
+                "maximum allowed allocation. ")
+                .append("Requested resource type=[").append(resourceType)
+                .append("], ")
+                .append("Requested resource=").append(requestedResource)
+                .append(", maximum allowed allocation=")
+                .append(availableAllocation)
+                .append(", please note that maximum allowed allocation is " +
+                        "calculated by scheduler based on maximum resource " +
+                        "of registered NodeManagers, which might be less " +
+                        "than configured maximum allocation=")
+                .append(configuredMaxAllowedAllocation)
+                .toString();
+      }
+      throw new IllegalStateException("Wrong type of InvalidResourceType is " +
+              "detected!");
     }
   }
 }


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


[30/50] hadoop git commit: HDFS-13765. Fix javadoc for FSDirMkdirOp#createParentDirectories. Contributed by Lokesh Jain.

Posted by eh...@apache.org.
HDFS-13765. Fix javadoc for FSDirMkdirOp#createParentDirectories. Contributed by Lokesh Jain.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1c40bc28
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1c40bc28
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1c40bc28

Branch: refs/heads/HDFS-12090
Commit: 1c40bc283645db5a661dc9f004a0bf34832a0902
Parents: 3cc7ce8
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Jul 27 10:14:01 2018 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Jul 27 10:14:01 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java    | 5 +----
 1 file changed, 1 insertion(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c40bc28/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java
index 45bb6b4..2f0a0fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java
@@ -110,10 +110,7 @@ class FSDirMkdirOp {
    * Create all ancestor directories and return the parent inodes.
    *
    * @param fsd FSDirectory
-   * @param existing The INodesInPath instance containing all the existing
-   *                 ancestral INodes
-   * @param children The relative path from the parent towards children,
-   *                 starting with "/"
+   * @param iip inodes in path to the fs directory
    * @param perm the permission of the directory. Note that all ancestors
    *             created along the path has implicit {@code u+wx} permissions.
    * @param inheritPerms if the ancestor directories should inherit permissions


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


[02/50] hadoop git commit: YARN-7748. TestContainerResizing.testIncreaseContainerUnreservedWhenApplicationCompleted fails due to multiple container fail events. Contributed by Weiwei Yang.

Posted by eh...@apache.org.
YARN-7748. TestContainerResizing.testIncreaseContainerUnreservedWhenApplicationCompleted fails due to multiple container fail events. Contributed by Weiwei Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/35ce6eb1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/35ce6eb1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/35ce6eb1

Branch: refs/heads/HDFS-12090
Commit: 35ce6eb1f526ce3db7e015fb1761eee15604100c
Parents: 773d312
Author: Sunil G <su...@apache.org>
Authored: Tue Jul 24 22:20:06 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Tue Jul 24 22:20:17 2018 +0530

----------------------------------------------------------------------
 .../scheduler/capacity/TestContainerResizing.java | 18 +++++++++++++-----
 1 file changed, 13 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ce6eb1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java
index eacbf6e..307d5ae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
@@ -58,7 +59,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica
     .FiCaSchedulerNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.CandidateNodeSet;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -740,11 +740,14 @@ public class TestContainerResizing {
   @Test
   public void testIncreaseContainerUnreservedWhenApplicationCompleted()
       throws Exception {
+    // Disable relaunch app attempt on failure, in order to check
+    // resource usages for current app only.
+    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
     /**
      * Similar to testIncreaseContainerUnreservedWhenContainerCompleted, when
      * application finishes, reserved increase container should be cancelled
      */
-    MockRM rm1 = new MockRM() {
+    MockRM rm1 = new MockRM(conf) {
       @Override
       public RMNodeLabelsManager createNodeLabelManager() {
         return mgr;
@@ -807,9 +810,14 @@ public class TestContainerResizing {
     Assert.assertEquals(6 * GB,
         app.getAppAttemptResourceUsage().getReserved().getMemorySize());
 
-    // Kill the application
-    cs.handle(new AppAttemptRemovedSchedulerEvent(am1.getApplicationAttemptId(),
-        RMAppAttemptState.KILLED, false));
+    // Kill the application by killing the AM container
+    ContainerId amContainer =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
+    cs.killContainer(cs.getRMContainer(amContainer));
+    rm1.waitForState(am1.getApplicationAttemptId(),
+        RMAppAttemptState.FAILED);
+    rm1.waitForState(am1.getApplicationAttemptId().getApplicationId(),
+        RMAppState.FAILED);
 
     /* Check statuses after reservation satisfied */
     // Increase request should be unreserved


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


[45/50] hadoop git commit: HDDS-287. Add Close ContainerAction to Datanode#StateContext when the container gets full. Contributed by Nanda kumar.

Posted by eh...@apache.org.
HDDS-287. Add Close ContainerAction to Datanode#StateContext when the container gets full. Contributed by Nanda kumar.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3517a478
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3517a478
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3517a478

Branch: refs/heads/HDFS-12090
Commit: 3517a47897457c11096ab57a4cb0b096a838a3ec
Parents: 952dc2f
Author: Nanda kumar <na...@apache.org>
Authored: Mon Jul 30 21:18:42 2018 +0530
Committer: Nanda kumar <na...@apache.org>
Committed: Mon Jul 30 21:18:42 2018 +0530

----------------------------------------------------------------------
 .../container/common/impl/HddsDispatcher.java   |  63 +++++++-
 .../statemachine/DatanodeStateMachine.java      |   2 +-
 .../common/statemachine/StateContext.java       |  14 +-
 .../container/ozoneimpl/OzoneContainer.java     |   6 +-
 .../common/impl/TestHddsDispatcher.java         | 152 +++++++++++++++++++
 .../container/common/impl/package-info.java     |  22 +++
 .../common/interfaces/TestHandler.java          |   4 +-
 .../container/ozoneimpl/TestOzoneContainer.java |   2 +-
 .../ozone/container/common/TestEndPoint.java    |  12 +-
 .../common/impl/TestCloseContainerHandler.java  |   2 +-
 .../container/metrics/TestContainerMetrics.java |   2 +-
 .../container/ozoneimpl/TestOzoneContainer.java |   2 +-
 .../container/server/TestContainerServer.java   |   2 +-
 .../genesis/BenchMarkDatanodeDispatcher.java    |   6 +-
 14 files changed, 270 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3517a478/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
index 3d418e5..ee232db 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
@@ -21,12 +21,21 @@ package org.apache.hadoop.ozone.container.common.impl;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
+import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.HddsUtils;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerInfo;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerAction;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.common.interfaces.Handler;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
@@ -35,11 +44,14 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandResponseProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerType;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerLifeCycleState;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.Map;
+import java.util.Optional;
 
 /**
  * Ozone Container dispatcher takes a call from the netty server and routes it
@@ -53,6 +65,8 @@ public class HddsDispatcher implements ContainerDispatcher {
   private final Configuration conf;
   private final ContainerSet containerSet;
   private final VolumeSet volumeSet;
+  private final StateContext context;
+  private final float containerCloseThreshold;
   private String scmID;
   private ContainerMetrics metrics;
 
@@ -61,10 +75,11 @@ public class HddsDispatcher implements ContainerDispatcher {
    * XceiverServerHandler.
    */
   public HddsDispatcher(Configuration config, ContainerSet contSet,
-      VolumeSet volumes) {
+      VolumeSet volumes, StateContext context) {
     this.conf = config;
     this.containerSet = contSet;
     this.volumeSet = volumes;
+    this.context = context;
     this.handlers = Maps.newHashMap();
     this.metrics = ContainerMetrics.create(conf);
     for (ContainerType containerType : ContainerType.values()) {
@@ -72,6 +87,9 @@ public class HddsDispatcher implements ContainerDispatcher {
           Handler.getHandlerForContainerType(
               containerType, conf, containerSet, volumeSet, metrics));
     }
+    this.containerCloseThreshold = conf.getFloat(
+        ScmConfigKeys.OZONE_SCM_CONTAINER_CLOSE_THRESHOLD,
+        ScmConfigKeys.OZONE_SCM_CONTAINER_CLOSE_THRESHOLD_DEFAULT);
 
   }
 
@@ -113,7 +131,11 @@ public class HddsDispatcher implements ContainerDispatcher {
     } catch (StorageContainerException ex) {
       return ContainerUtils.logAndReturnError(LOG, ex, msg);
     }
-
+    // Small performance optimization. We check if the operation is of type
+    // write before trying to send CloseContainerAction.
+    if (!HddsUtils.isReadOnly(msg)) {
+      sendCloseContainerActionIfNeeded(container);
+    }
     Handler handler = getHandler(containerType);
     if (handler == null) {
       StorageContainerException ex = new StorageContainerException("Invalid " +
@@ -130,6 +152,43 @@ public class HddsDispatcher implements ContainerDispatcher {
     }
   }
 
+  /**
+   * If the container usage reaches the close threshold we send Close
+   * ContainerAction to SCM.
+   *
+   * @param container current state of container
+   */
+  private void sendCloseContainerActionIfNeeded(Container container) {
+    // We have to find a more efficient way to close a container.
+    Boolean isOpen = Optional.ofNullable(container)
+        .map(cont -> cont.getContainerState() == ContainerLifeCycleState.OPEN)
+        .orElse(Boolean.FALSE);
+    if (isOpen) {
+      ContainerData containerData = container.getContainerData();
+      double containerUsedPercentage = 1.0f * containerData.getBytesUsed() /
+          StorageUnit.GB.toBytes(containerData.getMaxSizeGB());
+      if (containerUsedPercentage >= containerCloseThreshold) {
+
+        ContainerInfo containerInfo = ContainerInfo.newBuilder()
+            .setContainerID(containerData.getContainerID())
+            .setReadCount(containerData.getReadCount())
+            .setWriteCount(containerData.getWriteCount())
+            .setReadBytes(containerData.getReadBytes())
+            .setWriteBytes(containerData.getWriteBytes())
+            .setUsed(containerData.getBytesUsed())
+            .setState(HddsProtos.LifeCycleState.OPEN)
+            .build();
+
+        ContainerAction action = ContainerAction.newBuilder()
+            .setContainer(containerInfo)
+            .setAction(ContainerAction.Action.CLOSE)
+            .setReason(ContainerAction.Reason.CONTAINER_FULL)
+            .build();
+        context.addContainerActionIfAbsent(action);
+      }
+    }
+  }
+
   @Override
   public Handler getHandler(ContainerProtos.ContainerType containerType) {
     return handlers.get(containerType);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3517a478/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
index 69a243e..1ac42dd 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
@@ -89,7 +89,7 @@ public class DatanodeStateMachine implements Closeable {
     heartbeatFrequency = TimeUnit.SECONDS.toMillis(
         getScmHeartbeatInterval(conf));
     container = new OzoneContainer(this.datanodeDetails,
-        new OzoneConfiguration(conf));
+        new OzoneConfiguration(conf), context);
     nextHB = new AtomicLong(Time.monotonicNow());
 
      // When we add new handlers just adding a new handler here should do the

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3517a478/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
index 7862cc6..19c9496 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Queue;
@@ -213,6 +212,19 @@ public class StateContext {
   }
 
   /**
+   * Add ContainerAction to ContainerAction queue if it's not present.
+   *
+   * @param containerAction ContainerAction to be added
+   */
+  public void addContainerActionIfAbsent(ContainerAction containerAction) {
+    synchronized (containerActions) {
+      if (!containerActions.contains(containerAction)) {
+        containerActions.add(containerAction);
+      }
+    }
+  }
+
+  /**
    * Returns all the pending ContainerActions from the ContainerAction queue,
    * or empty list if the queue is empty.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3517a478/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
index 30fe113..85c947f 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
 import org.apache.hadoop.ozone.container.common.transport.server.XceiverServer;
 import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerGrpc;
 import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi;
@@ -70,7 +71,7 @@ public class OzoneContainer {
    * @throws IOException
    */
   public OzoneContainer(DatanodeDetails datanodeDetails, OzoneConfiguration
-      conf) throws IOException {
+      conf, StateContext context) throws IOException {
     this.dnDetails = datanodeDetails;
     this.config = conf;
     this.volumeSet = new VolumeSet(datanodeDetails.getUuidString(), conf);
@@ -79,7 +80,8 @@ public class OzoneContainer {
         ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_KEY,
         ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_DEFAULT);
     buildContainerSet();
-    hddsDispatcher = new HddsDispatcher(config, containerSet, volumeSet);
+    hddsDispatcher = new HddsDispatcher(config, containerSet, volumeSet,
+        context);
     server = new XceiverServerSpi[]{
         useGrpc ? new XceiverServerGrpc(datanodeDetails, this.config, this
             .hddsDispatcher) :

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3517a478/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java
new file mode 100644
index 0000000..b107782
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java
@@ -0,0 +1,152 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.impl;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto
+    .ContainerProtos.ContainerCommandResponseProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .WriteChunkRequestProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerAction;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.UUID;
+
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Test-cases to verify the functionality of HddsDispatcher.
+ */
+public class TestHddsDispatcher {
+
+  @Test
+  public void testContainerCloseActionWhenFull() throws IOException {
+    String testDir = GenericTestUtils.getTempPath(
+        TestHddsDispatcher.class.getSimpleName());
+    try {
+      UUID scmId = UUID.randomUUID();
+      OzoneConfiguration conf = new OzoneConfiguration();
+      conf.set(HDDS_DATANODE_DIR_KEY, testDir);
+      DatanodeDetails dd = randomDatanodeDetails();
+      ContainerSet containerSet = new ContainerSet();
+      VolumeSet volumeSet = new VolumeSet(dd.getUuidString(), conf);
+      StateContext context = Mockito.mock(StateContext.class);
+      KeyValueContainerData containerData = new KeyValueContainerData(1L, 1);
+      Container container = new KeyValueContainer(containerData, conf);
+      container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(),
+          scmId.toString());
+      containerSet.addContainer(container);
+      HddsDispatcher hddsDispatcher = new HddsDispatcher(
+          conf, containerSet, volumeSet, context);
+      hddsDispatcher.setScmId(scmId.toString());
+      ContainerCommandResponseProto responseOne = hddsDispatcher.dispatch(
+          getWriteChunkRequest(dd.getUuidString(), 1L, 1L));
+      Assert.assertEquals(ContainerProtos.Result.SUCCESS,
+          responseOne.getResult());
+      verify(context, times(0))
+          .addContainerActionIfAbsent(Mockito.any(ContainerAction.class));
+      containerData.setBytesUsed(Double.valueOf(
+          StorageUnit.MB.toBytes(950)).longValue());
+      ContainerCommandResponseProto responseTwo = hddsDispatcher.dispatch(
+          getWriteChunkRequest(dd.getUuidString(), 1L, 2L));
+      Assert.assertEquals(ContainerProtos.Result.SUCCESS,
+          responseTwo.getResult());
+      verify(context, times(1))
+          .addContainerActionIfAbsent(Mockito.any(ContainerAction.class));
+
+    } finally {
+      FileUtils.deleteDirectory(new File(testDir));
+    }
+
+  }
+
+  // This method has to be removed once we move scm/TestUtils.java
+  // from server-scm project to container-service or to common project.
+  private static DatanodeDetails randomDatanodeDetails() {
+    DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.STANDALONE, 0);
+    DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.RATIS, 0);
+    DatanodeDetails.Port restPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.REST, 0);
+    DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
+    builder.setUuid(UUID.randomUUID().toString())
+        .setHostName("localhost")
+        .setIpAddress("127.0.0.1")
+        .addPort(containerPort)
+        .addPort(ratisPort)
+        .addPort(restPort);
+    return builder.build();
+  }
+
+  private ContainerCommandRequestProto getWriteChunkRequest(
+      String datanodeId, Long containerId, Long localId) {
+
+    ByteString data = ByteString.copyFrom(
+        UUID.randomUUID().toString().getBytes());
+    ContainerProtos.ChunkInfo chunk = ContainerProtos.ChunkInfo
+        .newBuilder()
+        .setChunkName(
+            DigestUtils.md5Hex("dummy-key") + "_stream_"
+                + containerId + "_chunk_" + localId)
+        .setOffset(0)
+        .setLen(data.size())
+        .build();
+
+    WriteChunkRequestProto.Builder writeChunkRequest = WriteChunkRequestProto
+        .newBuilder()
+        .setBlockID(new BlockID(containerId, localId)
+            .getDatanodeBlockIDProtobuf())
+        .setChunkData(chunk)
+        .setData(data);
+
+    return ContainerCommandRequestProto
+        .newBuilder()
+        .setContainerID(containerId)
+        .setCmdType(ContainerProtos.Type.WriteChunk)
+        .setTraceID(UUID.randomUUID().toString())
+        .setDatanodeUuid(datanodeId)
+        .setWriteChunk(writeChunkRequest)
+        .build();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3517a478/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/package-info.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/package-info.java
new file mode 100644
index 0000000..07c78c0
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/package-info.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+/**
+ * Datanode container related test-cases.
+ */
+package org.apache.hadoop.ozone.container.common.impl;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3517a478/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
index 6660e9b..c9733f8 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
@@ -52,15 +52,13 @@ public class TestHandler {
   private VolumeSet volumeSet;
   private Handler handler;
 
-  private final static String DATANODE_UUID = UUID.randomUUID().toString();
-
   @Before
   public void setup() throws Exception {
     this.conf = new Configuration();
     this.containerSet = Mockito.mock(ContainerSet.class);
     this.volumeSet = Mockito.mock(VolumeSet.class);
 
-    this.dispatcher = new HddsDispatcher(conf, containerSet, volumeSet);
+    this.dispatcher = new HddsDispatcher(conf, containerSet, volumeSet, null);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3517a478/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
index 284ffa3..19ec6a2 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
@@ -86,7 +86,7 @@ public class TestOzoneContainer {
     // When OzoneContainer is started, the containers from disk should be
     // loaded into the containerSet.
     OzoneContainer ozoneContainer = new
-        OzoneContainer(datanodeDetails, conf);
+        OzoneContainer(datanodeDetails, conf, null);
     ContainerSet containerset = ozoneContainer.getContainerSet();
     assertEquals(10, containerset.containerCount());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3517a478/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
index e24e73e..e9359b8 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
@@ -151,8 +151,8 @@ public class TestEndPoint {
     OzoneConfiguration conf = SCMTestUtils.getConf();
     try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
         serverAddress, 1000)) {
-      OzoneContainer ozoneContainer = new OzoneContainer(TestUtils.randomDatanodeDetails(),
-          conf);
+      OzoneContainer ozoneContainer = new OzoneContainer(
+          TestUtils.randomDatanodeDetails(), conf, null);
       rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
       VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
           conf, ozoneContainer);
@@ -176,7 +176,7 @@ public class TestEndPoint {
       GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
           .captureLogs(VersionEndpointTask.LOG);
       OzoneContainer ozoneContainer = new OzoneContainer(TestUtils
-          .randomDatanodeDetails(), conf);
+          .randomDatanodeDetails(), conf, null);
       rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
       VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
           conf, ozoneContainer);
@@ -228,7 +228,7 @@ public class TestEndPoint {
         nonExistentServerAddress, 1000)) {
       rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
       OzoneContainer ozoneContainer = new OzoneContainer(TestUtils.randomDatanodeDetails(),
-          conf);
+          conf, null);
       VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
           conf, ozoneContainer);
       EndpointStateMachine.EndPointStates newState = versionTask.call();
@@ -254,8 +254,8 @@ public class TestEndPoint {
     try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
         serverAddress, (int) rpcTimeout)) {
       rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
-      OzoneContainer ozoneContainer = new OzoneContainer(TestUtils.randomDatanodeDetails(),
-          conf);
+      OzoneContainer ozoneContainer = new OzoneContainer(
+          TestUtils.randomDatanodeDetails(), conf, null);
       VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
           conf, ozoneContainer);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3517a478/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
index d67cf88..73fa70d 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
@@ -86,7 +86,7 @@ public class TestCloseContainerHandler {
             .setHostName("localhost").setIpAddress("127.0.0.1").build();
     volumeSet = new VolumeSet(datanodeDetails.getUuidString(), conf);
 
-    dispatcher = new HddsDispatcher(conf, containerSet, volumeSet);
+    dispatcher = new HddsDispatcher(conf, containerSet, volumeSet, null);
     handler = (KeyValueHandler) dispatcher
         .getHandler(ContainerProtos.ContainerType.KeyValueContainer);
     openContainerBlockMap = handler.getOpenContainerBlockMap();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3517a478/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
index 13ed192..19b561a 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
@@ -78,7 +78,7 @@ public class TestContainerMetrics {
           datanodeDetails.getUuidString(), conf);
       ContainerSet containerSet = new ContainerSet();
       HddsDispatcher dispatcher = new HddsDispatcher(conf, containerSet,
-          volumeSet);
+          volumeSet, null);
       dispatcher.setScmId(UUID.randomUUID().toString());
 
       server = new XceiverServer(datanodeDetails, conf, dispatcher);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3517a478/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
index d271ed3..215dd21 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
@@ -72,7 +72,7 @@ public class TestOzoneContainer {
       conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, false);
 
       container = new OzoneContainer(TestUtils.randomDatanodeDetails(),
-          conf);
+          conf, null);
       //Setting scmId, as we start manually ozone container.
       container.getDispatcher().setScmId(UUID.randomUUID().toString());
       container.start();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3517a478/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
index bdb26fb..ebcc930 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
@@ -214,7 +214,7 @@ public class TestContainerServer {
               .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
 
       HddsDispatcher dispatcher = new HddsDispatcher(
-          conf, mock(ContainerSet.class), mock(VolumeSet.class));
+          conf, mock(ContainerSet.class), mock(VolumeSet.class), null);
       dispatcher.init();
       DatanodeDetails datanodeDetails = TestUtils.randomDatanodeDetails();
       server = new XceiverServer(datanodeDetails, conf, dispatcher);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3517a478/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
index e757a7f..3c49fb6 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
@@ -20,6 +20,9 @@ package org.apache.hadoop.ozone.genesis;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
+import org.apache.hadoop.ozone.container.common.statemachine
+    .DatanodeStateMachine.DatanodeStates;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.ratis.shaded.com.google.protobuf.ByteString;
 import org.apache.commons.codec.digest.DigestUtils;
@@ -104,7 +107,8 @@ public class BenchMarkDatanodeDispatcher {
     ContainerSet containerSet = new ContainerSet();
     VolumeSet volumeSet = new VolumeSet(datanodeUuid, conf);
 
-    dispatcher = new HddsDispatcher(conf, containerSet, volumeSet);
+    dispatcher = new HddsDispatcher(conf, containerSet, volumeSet,
+        new StateContext(conf, DatanodeStates.RUNNING, null));
     dispatcher.init();
 
     containerCount = new AtomicInteger();


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


[18/50] hadoop git commit: HDFS-11060. make DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED configurable. Contributed by Lantao Jin.

Posted by eh...@apache.org.
HDFS-11060. make DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED configurable. Contributed by Lantao Jin.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e95c5e9f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e95c5e9f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e95c5e9f

Branch: refs/heads/HDFS-12090
Commit: e95c5e9f62452ee848875ec2f8642eab4992cd23
Parents: 9485c9a
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Wed Jul 25 11:04:18 2018 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Wed Jul 25 11:04:18 2018 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java | 3 +++
 .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java    | 8 ++++++--
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml         | 9 +++++++++
 .../hdfs/server/namenode/TestListCorruptFileBlocks.java     | 2 +-
 4 files changed, 19 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e95c5e9f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index cc902b0..5a1266c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -241,6 +241,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_DEFAULT
       = 1;
 
+  public static final String  DFS_NAMENODE_MAX_CORRUPT_FILE_BLOCKS_RETURNED_KEY = "dfs.namenode.max-corrupt-file-blocks-returned";
+  public static final int     DFS_NAMENODE_MAX_CORRUPT_FILE_BLOCKS_RETURNED_DEFAULT = 100;
+
   public static final String  DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY =
       HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY;
   public static final int     DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e95c5e9f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 66bc567..8c95f7d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -425,7 +425,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   public static final Log auditLog = LogFactory.getLog(
       FSNamesystem.class.getName() + ".audit");
 
-  static final int DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED = 100;
+  private final int maxCorruptFileBlocksReturn;
   static int BLOCK_DELETION_INCREMENT = 1000;
   private final boolean isPermissionEnabled;
   private final UserGroupInformation fsOwner;
@@ -831,6 +831,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           DFSConfigKeys.DFS_NAMENODE_FILE_CLOSE_NUM_COMMITTED_ALLOWED_KEY,
           DFSConfigKeys.DFS_NAMENODE_FILE_CLOSE_NUM_COMMITTED_ALLOWED_DEFAULT);
 
+      this.maxCorruptFileBlocksReturn = conf.getInt(
+          DFSConfigKeys.DFS_NAMENODE_MAX_CORRUPT_FILE_BLOCKS_RETURNED_KEY,
+          DFSConfigKeys.DFS_NAMENODE_MAX_CORRUPT_FILE_BLOCKS_RETURNED_DEFAULT);
+
       this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
       
       this.standbyShouldCheckpoint = conf.getBoolean(
@@ -5508,7 +5512,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           if (src.startsWith(path)){
             corruptFiles.add(new CorruptFileBlockInfo(src, blk));
             count++;
-            if (count >= DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED)
+            if (count >= maxCorruptFileBlocksReturn)
               break;
           }
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e95c5e9f/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 384cedf..a10be27 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -616,6 +616,15 @@
 </property>
 
 <property>
+  <name>dfs.namenode.max-corrupt-file-blocks-returned</name>
+  <value>100</value>
+  <description>
+      The maximum number of corrupt file blocks listed by NameNode Web UI,
+      JMX and other client request.
+  </description>
+</property>
+
+<property>
   <name>dfs.blocksize</name>
   <value>134217728</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e95c5e9f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java
index 1f31bdc..e1c8ae3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java
@@ -452,7 +452,7 @@ public class TestListCorruptFileBlocks {
       cluster = new MiniDFSCluster.Builder(conf).build();
       FileSystem fs = cluster.getFileSystem();
       final int maxCorruptFileBlocks = 
-        FSNamesystem.DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED;
+        conf.getInt(DFSConfigKeys.DFS_NAMENODE_MAX_CORRUPT_FILE_BLOCKS_RETURNED_KEY, 100);
 
       // create 110 files with one block each
       DFSTestUtil util = new DFSTestUtil.Builder().setName("testMaxCorruptFiles").


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


[24/50] hadoop git commit: HDDS-291. Initialize hadoop metrics system in standalone hdds datanodes. Contributed by Elek Marton.

Posted by eh...@apache.org.
HDDS-291. Initialize hadoop metrics system in standalone hdds datanodes. Contributed by Elek Marton.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d70d8457
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d70d8457
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d70d8457

Branch: refs/heads/HDFS-12090
Commit: d70d84570575574b7e3ad0f00baf54f1dde76d97
Parents: fd31cb6
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Thu Jul 26 13:17:37 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Thu Jul 26 13:17:37 2018 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java | 2 ++
 .../ozone/container/common/statemachine/SCMConnectionManager.java  | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d70d8457/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
index ddeec87..f359e72 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hdds.HddsUtils;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.statemachine
     .DatanodeStateMachine;
@@ -241,6 +242,7 @@ public class HddsDatanodeService implements ServicePlugin {
         System.exit(1);
       }
       StringUtils.startupShutdownMessage(HddsDatanodeService.class, args, LOG);
+      DefaultMetricsSystem.initialize("HddsDatanode");
       HddsDatanodeService hddsDatanodeService =
           createHddsDatanodeService(conf);
       hddsDatanodeService.start(null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d70d8457/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java
index 19722f0..85fb580 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java
@@ -67,7 +67,7 @@ public class SCMConnectionManager
     this.rpcTimeout = timeOut.intValue();
     this.scmMachines = new HashMap<>();
     this.conf = conf;
-    jmxBean = MBeans.register("OzoneDataNode",
+    jmxBean = MBeans.register("HddsDatanode",
         "SCMConnectionManager",
         this);
   }


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


[13/50] hadoop git commit: HDDS-266. Integrate checksum into .container file. Contributed by Hanisha Koneru.

Posted by eh...@apache.org.
HDDS-266. Integrate checksum into .container file. Contributed by Hanisha Koneru.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b507f83e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b507f83e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b507f83e

Branch: refs/heads/HDFS-12090
Commit: b507f83e15b47163724d550dfeb41627f26fd551
Parents: 43db0cb
Author: Nanda kumar <na...@apache.org>
Authored: Wed Jul 25 18:55:26 2018 +0530
Committer: Nanda kumar <na...@apache.org>
Committed: Wed Jul 25 18:55:26 2018 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/ozone/OzoneConsts.java    |   2 +-
 .../common/helpers/ContainerUtils.java          |  59 ++++++++-
 .../container/common/impl/ContainerData.java    |  66 +++++++++-
 .../common/impl/ContainerDataYaml.java          |  98 +++++++++-----
 .../container/keyvalue/KeyValueContainer.java   | 121 +++++------------
 .../keyvalue/KeyValueContainerData.java         |  30 ++---
 .../helpers/KeyValueContainerLocationUtil.java  |  32 ++---
 .../keyvalue/helpers/KeyValueContainerUtil.java | 130 +++----------------
 .../container/ozoneimpl/ContainerReader.java    |  92 +++++++------
 .../common/impl/TestContainerDataYaml.java      | 107 +++++++++++----
 .../keyvalue/TestKeyValueContainer.java         |  21 ++-
 .../test/resources/additionalfields.container   |   3 +-
 .../test/resources/incorrect.checksum.container |  11 ++
 .../src/test/resources/incorrect.container      |   3 +-
 hadoop-hdds/pom.xml                             |   1 +
 .../common/impl/TestContainerPersistence.java   |   8 +-
 .../container/ozoneimpl/TestOzoneContainer.java |   7 +
 17 files changed, 430 insertions(+), 361 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index 25b68e0..f912f02 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -98,7 +98,6 @@ public final class OzoneConsts {
   public static final String OM_DB_NAME = "om.db";
 
   public static final String STORAGE_DIR_CHUNKS = "chunks";
-  public static final String CONTAINER_FILE_CHECKSUM_EXTENSION = ".chksm";
 
   /**
    * Supports Bucket Versioning.
@@ -190,4 +189,5 @@ public final class OzoneConsts {
   public static final String METADATA_PATH = "metadataPath";
   public static final String CHUNKS_PATH = "chunksPath";
   public static final String CONTAINER_DB_TYPE = "containerDBType";
+  public static final String CHECKSUM = "checksum";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
index 1d5dfc5..77a891a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
@@ -19,9 +19,11 @@
 package org.apache.hadoop.ozone.container.common.helpers;
 
 import com.google.common.base.Preconditions;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
@@ -31,8 +33,8 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.ozone.OzoneConsts;
-import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -43,8 +45,17 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import org.yaml.snakeyaml.Yaml;
 
 import static org.apache.commons.io.FilenameUtils.removeExtension;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.CONTAINER_CHECKSUM_ERROR;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.NO_SUCH_ALGORITHM;
+import static org.apache.hadoop.ozone.container.common.impl.ContainerData
+    .CHARSET_ENCODING;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
+
 
 /**
  * A set of helper functions to create proper responses.
@@ -245,4 +256,48 @@ public final class ContainerUtils {
           + path.getAbsolutePath(), e);
     }
   }
+
+  /**
+   * Verify that the checksum stored in containerData is equal to the
+   * computed checksum.
+   * @param containerData
+   * @throws IOException
+   */
+  public static void verifyChecksum(ContainerData containerData)
+      throws IOException {
+    String storedChecksum = containerData.getChecksum();
+
+    Yaml yaml = ContainerDataYaml.getYamlForContainerType(
+        containerData.getContainerType());
+    containerData.computeAndSetChecksum(yaml);
+    String computedChecksum = containerData.getChecksum();
+
+    if (storedChecksum == null || !storedChecksum.equals(computedChecksum)) {
+      throw new StorageContainerException("Container checksum error for " +
+          "ContainerID: " + containerData.getContainerID() + ". " +
+          "\nStored Checksum: " + storedChecksum +
+          "\nExpected Checksum: " + computedChecksum,
+          CONTAINER_CHECKSUM_ERROR);
+    }
+  }
+
+  /**
+   * Return the SHA-256 chesksum of the containerData.
+   * @param containerDataYamlStr ContainerData as a Yaml String
+   * @return Checksum of the container data
+   * @throws StorageContainerException
+   */
+  public static String getChecksum(String containerDataYamlStr)
+      throws StorageContainerException {
+    MessageDigest sha;
+    try {
+      sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
+      sha.update(containerDataYamlStr.getBytes(CHARSET_ENCODING));
+      return DigestUtils.sha256Hex(sha.digest());
+    } catch (NoSuchAlgorithmException e) {
+      throw new StorageContainerException("Unable to create Message Digest, " +
+          "usually this is a java configuration issue.", NO_SUCH_ALGORITHM);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
index a7e2b55..5803628 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
@@ -18,22 +18,33 @@
 package org.apache.hadoop.ozone.container.common.impl;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.nio.charset.Charset;
 import java.util.List;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
     ContainerType;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
     ContainerLifeCycleState;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 
-import java.io.IOException;
 import java.util.Collections;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
+import org.yaml.snakeyaml.Yaml;
 
 import static java.lang.Math.max;
+import static org.apache.hadoop.ozone.OzoneConsts.CHECKSUM;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_ID;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_TYPE;
+import static org.apache.hadoop.ozone.OzoneConsts.LAYOUTVERSION;
+import static org.apache.hadoop.ozone.OzoneConsts.MAX_SIZE_GB;
+import static org.apache.hadoop.ozone.OzoneConsts.METADATA;
+import static org.apache.hadoop.ozone.OzoneConsts.STATE;
 
 /**
  * ContainerData is the in-memory representation of container metadata and is
@@ -72,6 +83,23 @@ public abstract class ContainerData {
 
   private long deleteTransactionId;
 
+  private String checksum;
+  public static final Charset CHARSET_ENCODING = Charset.forName("UTF-8");
+  private static final String DUMMY_CHECKSUM = new String(new byte[64],
+      CHARSET_ENCODING);
+
+  // Common Fields need to be stored in .container file.
+  protected static final List<String> YAML_FIELDS =
+      Collections.unmodifiableList(Lists.newArrayList(
+      CONTAINER_TYPE,
+      CONTAINER_ID,
+      LAYOUTVERSION,
+      STATE,
+      METADATA,
+      MAX_SIZE_GB,
+      CHECKSUM));
+
+
   /**
    * Number of pending deletion blocks in container.
    */
@@ -113,6 +141,7 @@ public abstract class ContainerData {
     this.maxSizeGB = size;
     this.numPendingDeletionBlocks = new AtomicInteger(0);
     this.deleteTransactionId = 0;
+    setChecksumTo0ByteArray();
   }
 
   /**
@@ -400,6 +429,41 @@ public abstract class ContainerData {
     return this.numPendingDeletionBlocks.get();
   }
 
+  public void setChecksumTo0ByteArray() {
+    this.checksum = DUMMY_CHECKSUM;
+  }
+
+  public void setChecksum(String checkSum) {
+    this.checksum = checkSum;
+  }
+
+  public String getChecksum() {
+    return this.checksum;
+  }
+
+  /**
+   * Compute the checksum for ContainerData using the specified Yaml (based
+   * on ContainerType) and set the checksum.
+   *
+   * Checksum of ContainerData is calculated by setting the
+   * {@link ContainerData#checksum} field to a 64-byte array with all 0's -
+   * {@link ContainerData#DUMMY_CHECKSUM}. After the checksum is calculated,
+   * the checksum field is updated with this value.
+   *
+   * @param yaml Yaml for ContainerType to get the ContainerData as Yaml String
+   * @throws IOException
+   */
+  public void computeAndSetChecksum(Yaml yaml) throws IOException {
+    // Set checksum to dummy value - 0 byte array, to calculate the checksum
+    // of rest of the data.
+    setChecksumTo0ByteArray();
+
+    // Dump yaml data into a string to compute its checksum
+    String containerDataYamlStr = yaml.dump(this);
+
+    this.checksum = ContainerUtils.getChecksum(containerDataYamlStr);
+  }
+
   /**
    * Returns a ProtoBuf Message from ContainerData.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
index 90af24f..aed75d3 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
@@ -20,10 +20,14 @@ package org.apache.hadoop.ozone.container.common.impl;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerType;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.yaml.snakeyaml.Yaml;
 
 import java.beans.IntrospectionException;
@@ -59,9 +63,13 @@ import static org.apache.hadoop.ozone.container.keyvalue
 
 public final class ContainerDataYaml {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContainerDataYaml.class);
+
   private ContainerDataYaml() {
 
   }
+
   /**
    * Creates a .container file in yaml format.
    *
@@ -69,38 +77,29 @@ public final class ContainerDataYaml {
    * @param containerData
    * @throws IOException
    */
-  public static void createContainerFile(ContainerProtos.ContainerType
-                                             containerType, File containerFile,
-                                         ContainerData containerData) throws
-      IOException {
-
-    Preconditions.checkNotNull(containerFile, "yamlFile cannot be null");
-    Preconditions.checkNotNull(containerData, "containerData cannot be null");
-    Preconditions.checkNotNull(containerType, "containerType cannot be null");
-
-    PropertyUtils propertyUtils = new PropertyUtils();
-    propertyUtils.setBeanAccess(BeanAccess.FIELD);
-    propertyUtils.setAllowReadOnlyProperties(true);
-
-    switch(containerType) {
-    case KeyValueContainer:
-      Representer representer = new ContainerDataRepresenter();
-      representer.setPropertyUtils(propertyUtils);
-      representer.addClassTag(KeyValueContainerData.class,
-          KeyValueContainerData.KEYVALUE_YAML_TAG);
-
-      Constructor keyValueDataConstructor = new ContainerDataConstructor();
+  public static void createContainerFile(ContainerType containerType,
+      ContainerData containerData, File containerFile) throws IOException {
+    Writer writer = null;
+    try {
+      // Create Yaml for given container type
+      Yaml yaml = getYamlForContainerType(containerType);
+      // Compute Checksum and update ContainerData
+      containerData.computeAndSetChecksum(yaml);
 
-      Yaml yaml = new Yaml(keyValueDataConstructor, representer);
-      Writer writer = new OutputStreamWriter(new FileOutputStream(
+      // Write the ContainerData with checksum to Yaml file.
+      writer = new OutputStreamWriter(new FileOutputStream(
           containerFile), "UTF-8");
       yaml.dump(containerData, writer);
-      writer.close();
-      break;
-    default:
-      throw new StorageContainerException("Unrecognized container Type " +
-          "format " + containerType, ContainerProtos.Result
-          .UNKNOWN_CONTAINER_TYPE);
+
+    } finally {
+      try {
+        if (writer != null) {
+          writer.close();
+        }
+      } catch (IOException ex) {
+        LOG.warn("Error occurred during closing the writer. ContainerID: " +
+            containerData.getContainerID());
+      }
     }
   }
 
@@ -141,6 +140,39 @@ public final class ContainerDataYaml {
   }
 
   /**
+   * Given a ContainerType this method returns a Yaml representation of
+   * the container properties.
+   *
+   * @param containerType type of container
+   * @return Yamal representation of container properties
+   *
+   * @throws StorageContainerException if the type is unrecognized
+   */
+  public static Yaml getYamlForContainerType(ContainerType containerType)
+      throws StorageContainerException {
+    PropertyUtils propertyUtils = new PropertyUtils();
+    propertyUtils.setBeanAccess(BeanAccess.FIELD);
+    propertyUtils.setAllowReadOnlyProperties(true);
+
+    switch (containerType) {
+    case KeyValueContainer:
+      Representer representer = new ContainerDataRepresenter();
+      representer.setPropertyUtils(propertyUtils);
+      representer.addClassTag(
+          KeyValueContainerData.class,
+          KeyValueContainerData.KEYVALUE_YAML_TAG);
+
+      Constructor keyValueDataConstructor = new ContainerDataConstructor();
+
+      return new Yaml(keyValueDataConstructor, representer);
+    default:
+      throw new StorageContainerException("Unrecognized container Type " +
+          "format " + containerType, ContainerProtos.Result
+          .UNKNOWN_CONTAINER_TYPE);
+    }
+  }
+
+  /**
    * Representer class to define which fields need to be stored in yaml file.
    */
   private static class ContainerDataRepresenter extends Representer {
@@ -192,8 +224,9 @@ public final class ContainerDataYaml {
         int maxSize = (int) size;
 
         //When a new field is added, it needs to be added here.
-        KeyValueContainerData kvData = new KeyValueContainerData((long) nodes
-            .get(OzoneConsts.CONTAINER_ID), lv, maxSize);
+        KeyValueContainerData kvData = new KeyValueContainerData(
+            (long) nodes.get(OzoneConsts.CONTAINER_ID), lv, maxSize);
+
         kvData.setContainerDBType((String)nodes.get(
             OzoneConsts.CONTAINER_DB_TYPE));
         kvData.setMetadataPath((String) nodes.get(
@@ -201,6 +234,7 @@ public final class ContainerDataYaml {
         kvData.setChunksPath((String) nodes.get(OzoneConsts.CHUNKS_PATH));
         Map<String, String> meta = (Map) nodes.get(OzoneConsts.METADATA);
         kvData.setMetadata(meta);
+        kvData.setChecksum((String) nodes.get(OzoneConsts.CHECKSUM));
         String state = (String) nodes.get(OzoneConsts.STATE);
         switch (state) {
         case "OPEN":
@@ -215,7 +249,7 @@ public final class ContainerDataYaml {
         default:
           throw new IllegalStateException("Unexpected " +
               "ContainerLifeCycleState " + state + " for the containerId " +
-              (long) nodes.get(OzoneConsts.CONTAINER_ID));
+              nodes.get(OzoneConsts.CONTAINER_ID));
         }
         return kvData;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index f381e24..14f731a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -24,12 +24,12 @@ import java.nio.file.StandardCopyOption;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerLifeCycleState;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerType;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -49,10 +49,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
 
 import java.util.Map;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -113,26 +110,24 @@ public class KeyValueContainer implements Container {
           .getVolumesList(), maxSize);
       String hddsVolumeDir = containerVolume.getHddsRootDir().toString();
 
-      long containerId = containerData.getContainerID();
-      String containerName = Long.toString(containerId);
+      long containerID = containerData.getContainerID();
 
       containerMetaDataPath = KeyValueContainerLocationUtil
-          .getContainerMetaDataPath(hddsVolumeDir, scmId, containerId);
+          .getContainerMetaDataPath(hddsVolumeDir, scmId, containerID);
       File chunksPath = KeyValueContainerLocationUtil.getChunksLocationPath(
-          hddsVolumeDir, scmId, containerId);
+          hddsVolumeDir, scmId, containerID);
+
       File containerFile = KeyValueContainerLocationUtil.getContainerFile(
-          containerMetaDataPath, containerName);
-      File containerCheckSumFile = KeyValueContainerLocationUtil
-          .getContainerCheckSumFile(containerMetaDataPath, containerName);
+          containerMetaDataPath, containerID);
       File dbFile = KeyValueContainerLocationUtil.getContainerDBFile(
-          containerMetaDataPath, containerName);
+          containerMetaDataPath, containerID);
 
       // Check if it is new Container.
       ContainerUtils.verifyIsNewContainer(containerMetaDataPath);
 
       //Create Metadata path chunks path and metadata db
       KeyValueContainerUtil.createContainerMetaData(containerMetaDataPath,
-          chunksPath, dbFile, containerName, config);
+          chunksPath, dbFile, config);
 
       String impl = config.getTrimmed(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL,
           OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_DEFAULT);
@@ -144,9 +139,8 @@ public class KeyValueContainer implements Container {
       containerData.setDbFile(dbFile);
       containerData.setVolume(containerVolume);
 
-      // Create .container file and .chksm file
-      writeToContainerFile(containerFile, containerCheckSumFile, true);
-
+      // Create .container file
+      writeToContainerFile(containerFile, true);
 
     } catch (StorageContainerException ex) {
       if (containerMetaDataPath != null && containerMetaDataPath.getParentFile()
@@ -176,97 +170,64 @@ public class KeyValueContainer implements Container {
    * Creates .container file and checksum file.
    *
    * @param containerFile
-   * @param checksumFile
    * @param isCreate true if we are creating a new container file and false if
    *                we are updating an existing container file.
    * @throws StorageContainerException
    */
-  private void writeToContainerFile(File containerFile, File
-      checksumFile, boolean isCreate)
+  private void writeToContainerFile(File containerFile, boolean isCreate)
       throws StorageContainerException {
     File tempContainerFile = null;
-    File tempChecksumFile = null;
-    FileOutputStream containerCheckSumStream = null;
-    Writer writer = null;
     long containerId = containerData.getContainerID();
     try {
       tempContainerFile = createTempFile(containerFile);
-      tempChecksumFile = createTempFile(checksumFile);
-      ContainerDataYaml.createContainerFile(ContainerProtos.ContainerType
-              .KeyValueContainer, tempContainerFile, containerData);
-
-      //Compute Checksum for container file
-      String checksum = KeyValueContainerUtil.computeCheckSum(containerId,
-          tempContainerFile);
-      containerCheckSumStream = new FileOutputStream(tempChecksumFile);
-      writer = new OutputStreamWriter(containerCheckSumStream, "UTF-8");
-      writer.write(checksum);
-      writer.flush();
+      ContainerDataYaml.createContainerFile(
+          ContainerType.KeyValueContainer, containerData, tempContainerFile);
 
       if (isCreate) {
         // When creating a new container, .container file should not exist
         // already.
         NativeIO.renameTo(tempContainerFile, containerFile);
-        NativeIO.renameTo(tempChecksumFile, checksumFile);
       } else {
         // When updating a container, the .container file should exist. If
         // not, the container is in an inconsistent state.
         Files.move(tempContainerFile.toPath(), containerFile.toPath(),
             StandardCopyOption.REPLACE_EXISTING);
-        Files.move(tempChecksumFile.toPath(), checksumFile.toPath(),
-            StandardCopyOption.REPLACE_EXISTING);
       }
 
     } catch (IOException ex) {
       throw new StorageContainerException("Error during creation of " +
-          "required files(.container, .chksm) for container. ContainerID: "
-          + containerId, ex, CONTAINER_FILES_CREATE_ERROR);
+          ".container file. ContainerID: " + containerId, ex,
+          CONTAINER_FILES_CREATE_ERROR);
     } finally {
-      IOUtils.closeStream(containerCheckSumStream);
       if (tempContainerFile != null && tempContainerFile.exists()) {
         if (!tempContainerFile.delete()) {
           LOG.warn("Unable to delete container temporary file: {}.",
               tempContainerFile.getAbsolutePath());
         }
       }
-      if (tempChecksumFile != null && tempChecksumFile.exists()) {
-        if (!tempChecksumFile.delete()) {
-          LOG.warn("Unable to delete container temporary checksum file: {}.",
-              tempContainerFile.getAbsolutePath());
-        }
-      }
-      try {
-        if (writer != null) {
-          writer.close();
-        }
-      } catch (IOException ex) {
-        LOG.warn("Error occurred during closing the writer.  Container " +
-            "Name:" + containerId);
-      }
-
     }
   }
 
 
-  private void updateContainerFile(File containerFile, File
-      checksumFile) throws StorageContainerException {
+  private void updateContainerFile(File containerFile)
+      throws StorageContainerException {
 
     long containerId = containerData.getContainerID();
 
-    if (containerFile.exists() && checksumFile.exists()) {
-      try {
-        writeToContainerFile(containerFile, checksumFile, false);
-      } catch (IOException e) {
-        //TODO : Container update failure is not handled currently. Might
-        // lead to loss of .container file. When Update container feature
-        // support is added, this failure should also be handled.
-        throw new StorageContainerException("Container update failed. " +
-            "ContainerID: " + containerId, CONTAINER_FILES_CREATE_ERROR);
-      }
-    } else {
+    if (!containerFile.exists()) {
       throw new StorageContainerException("Container is an Inconsistent " +
-          "state, missing required files(.container, .chksm). ContainerID: " +
-          containerId, INVALID_CONTAINER_STATE);
+          "state, missing .container file. ContainerID: " + containerId,
+          INVALID_CONTAINER_STATE);
+    }
+
+    try {
+      writeToContainerFile(containerFile, false);
+    } catch (IOException e) {
+      //TODO : Container update failure is not handled currently. Might
+      // lead to loss of .container file. When Update container feature
+      // support is added, this failure should also be handled.
+      throw new StorageContainerException("Container update failed. " +
+          "ContainerID: " + containerId, CONTAINER_FILES_CREATE_ERROR);
     }
   }
 
@@ -305,10 +266,9 @@ public class KeyValueContainer implements Container {
       }
       containerData.closeContainer();
       File containerFile = getContainerFile();
-      File containerCheckSumFile = getContainerCheckSumFile();
 
       // update the new container data to .container File
-      updateContainerFile(containerFile, containerCheckSumFile);
+      updateContainerFile(containerFile);
 
     } catch (StorageContainerException ex) {
       throw ex;
@@ -340,8 +300,8 @@ public class KeyValueContainer implements Container {
   }
 
   @Override
-  public ContainerProtos.ContainerType getContainerType() {
-    return ContainerProtos.ContainerType.KeyValueContainer;
+  public ContainerType getContainerType() {
+    return ContainerType.KeyValueContainer;
   }
 
   @Override
@@ -369,10 +329,10 @@ public class KeyValueContainer implements Container {
       for (Map.Entry<String, String> entry : metadata.entrySet()) {
         containerData.addMetadata(entry.getKey(), entry.getValue());
       }
+
       File containerFile = getContainerFile();
-      File containerCheckSumFile = getContainerCheckSumFile();
       // update the new container data to .container File
-      updateContainerFile(containerFile, containerCheckSumFile);
+      updateContainerFile(containerFile);
     } catch (StorageContainerException  ex) {
       // TODO:
       // On error, reset the metadata.
@@ -461,15 +421,6 @@ public class KeyValueContainer implements Container {
   }
 
   /**
-   * Returns container checksum file.
-   * @return container checksum file
-   */
-  private File getContainerCheckSumFile() {
-    return new File(containerData.getMetadataPath(), containerData
-        .getContainerID() + OzoneConsts.CONTAINER_FILE_CHECKSUM_EXTENSION);
-  }
-
-  /**
    * Creates a temporary file.
    * @param file
    * @return

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
index 3e3cc77..34035c8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.container.keyvalue;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
+import java.util.Collections;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -34,13 +35,7 @@ import java.util.Map;
 
 import static org.apache.hadoop.ozone.OzoneConsts.CHUNKS_PATH;
 import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_TYPE;
-import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_ID;
-import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_TYPE;
-import static org.apache.hadoop.ozone.OzoneConsts.LAYOUTVERSION;
-import static org.apache.hadoop.ozone.OzoneConsts.MAX_SIZE_GB;
-import static org.apache.hadoop.ozone.OzoneConsts.METADATA;
 import static org.apache.hadoop.ozone.OzoneConsts.METADATA_PATH;
-import static org.apache.hadoop.ozone.OzoneConsts.STATE;
 
 /**
  * This class represents the KeyValueContainer metadata, which is the
@@ -53,17 +48,7 @@ public class KeyValueContainerData extends ContainerData {
   public static final Tag KEYVALUE_YAML_TAG = new Tag("KeyValueContainerData");
 
   // Fields need to be stored in .container file.
-  private static final List<String> YAML_FIELDS =
-      Lists.newArrayList(
-          CONTAINER_TYPE,
-          CONTAINER_ID,
-          LAYOUTVERSION,
-          STATE,
-          METADATA,
-          METADATA_PATH,
-          CHUNKS_PATH,
-          CONTAINER_DB_TYPE,
-          MAX_SIZE_GB);
+  private static final List<String> KV_YAML_FIELDS;
 
   // Path to Container metadata Level DB/RocksDB Store and .container file.
   private String metadataPath;
@@ -76,6 +61,15 @@ public class KeyValueContainerData extends ContainerData {
 
   private File dbFile = null;
 
+  static {
+    // Initialize YAML fields
+    KV_YAML_FIELDS = Lists.newArrayList();
+    KV_YAML_FIELDS.addAll(YAML_FIELDS);
+    KV_YAML_FIELDS.add(METADATA_PATH);
+    KV_YAML_FIELDS.add(CHUNKS_PATH);
+    KV_YAML_FIELDS.add(CONTAINER_DB_TYPE);
+  }
+
   /**
    * Constructs KeyValueContainerData object.
    * @param id - ContainerId
@@ -210,7 +204,7 @@ public class KeyValueContainerData extends ContainerData {
   }
 
   public static List<String> getYamlFields() {
-    return YAML_FIELDS;
+    return Collections.unmodifiableList(KV_YAML_FIELDS);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
index 868b9f4..02a8e73 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
@@ -101,42 +101,26 @@ public final class KeyValueContainerLocationUtil {
   /**
    * Returns containerFile.
    * @param containerMetaDataPath
-   * @param containerName
+   * @param containerID
    * @return .container File name
    */
-  public static File getContainerFile(File containerMetaDataPath, String
-      containerName) {
+  public static File getContainerFile(File containerMetaDataPath,
+      long containerID) {
     Preconditions.checkNotNull(containerMetaDataPath);
-    Preconditions.checkNotNull(containerName);
-    return new File(containerMetaDataPath, containerName +
+    return new File(containerMetaDataPath, containerID +
         OzoneConsts.CONTAINER_EXTENSION);
   }
 
   /**
    * Return containerDB File.
    * @param containerMetaDataPath
-   * @param containerName
+   * @param containerID
    * @return containerDB File name
    */
-  public static File getContainerDBFile(File containerMetaDataPath, String
-      containerName) {
+  public static File getContainerDBFile(File containerMetaDataPath,
+      long containerID) {
     Preconditions.checkNotNull(containerMetaDataPath);
-    Preconditions.checkNotNull(containerName);
-    return new File(containerMetaDataPath, containerName + OzoneConsts
+    return new File(containerMetaDataPath, containerID + OzoneConsts
         .DN_CONTAINER_DB);
   }
-
-  /**
-   * Returns container checksum file.
-   * @param containerMetaDataPath
-   * @param containerName
-   * @return container checksum file
-   */
-  public static File getContainerCheckSumFile(File containerMetaDataPath,
-                                              String containerName) {
-    Preconditions.checkNotNull(containerMetaDataPath);
-    Preconditions.checkNotNull(containerName);
-    return new File(containerMetaDataPath, containerName + OzoneConsts
-        .CONTAINER_FILE_CHECKSUM_EXTENSION);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
index 1856111..2352cf6 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
@@ -18,23 +18,16 @@
 package org.apache.hadoop.ozone.container.keyvalue.helpers;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandResponseProto;
-import org.apache.hadoop.hdds.scm.container.common.helpers
-    .StorageContainerException;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
-import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.utils.MetadataKeyFilters;
 import org.apache.hadoop.utils.MetadataStore;
@@ -43,18 +36,12 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.IOException;
-import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.*;
-
 /**
  * Class which defines utility methods for KeyValueContainer.
  */
@@ -77,10 +64,8 @@ public final class KeyValueContainerUtil {
    * @throws IOException
    */
   public static void createContainerMetaData(File containerMetaDataPath, File
-      chunksPath, File dbFile, String containerName, Configuration conf) throws
-      IOException {
+      chunksPath, File dbFile, Configuration conf) throws IOException {
     Preconditions.checkNotNull(containerMetaDataPath);
-    Preconditions.checkNotNull(containerName);
     Preconditions.checkNotNull(conf);
 
     if (!containerMetaDataPath.mkdirs()) {
@@ -166,106 +151,31 @@ public final class KeyValueContainerUtil {
   }
 
   /**
-   * Compute checksum of the .container file.
-   * @param containerId
-   * @param containerFile
-   * @throws StorageContainerException
-   */
-  public static String computeCheckSum(long containerId, File
-      containerFile) throws StorageContainerException {
-    Preconditions.checkNotNull(containerFile, "containerFile cannot be null");
-    MessageDigest sha;
-    FileInputStream containerFileStream = null;
-    try {
-      sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
-    } catch (NoSuchAlgorithmException e) {
-      throw new StorageContainerException("Unable to create Message Digest, " +
-          "usually this is a java configuration issue.", NO_SUCH_ALGORITHM);
-    }
-    try {
-      containerFileStream = new FileInputStream(containerFile);
-      byte[] byteArray = new byte[1024];
-      int bytesCount = 0;
-      while ((bytesCount = containerFileStream.read(byteArray)) != -1) {
-        sha.update(byteArray, 0, bytesCount);
-      }
-      String checksum = DigestUtils.sha256Hex(sha.digest());
-      return checksum;
-    } catch (IOException ex) {
-      throw new StorageContainerException("Error during computing checksum: " +
-          "for container " + containerId, ex, CONTAINER_CHECKSUM_ERROR);
-    } finally {
-      IOUtils.closeStream(containerFileStream);
-    }
-  }
-
-  /**
-   * Verify checksum of the container.
-   * @param containerId
-   * @param checksumFile
-   * @param checksum
-   * @throws StorageContainerException
-   */
-  public static void verifyCheckSum(long containerId, File checksumFile,
-                                    String checksum)
-      throws StorageContainerException {
-    try {
-      Preconditions.checkNotNull(checksum);
-      Preconditions.checkNotNull(checksumFile);
-      Path path = Paths.get(checksumFile.getAbsolutePath());
-      List<String> fileCheckSum = Files.readAllLines(path);
-      Preconditions.checkState(fileCheckSum.size() == 1, "checksum " +
-          "should be 32 byte string");
-      if (!checksum.equals(fileCheckSum.get(0))) {
-        LOG.error("Checksum mismatch for the container {}", containerId);
-        throw new StorageContainerException("Checksum mismatch for " +
-            "the container " + containerId, CHECKSUM_MISMATCH);
-      }
-    } catch (StorageContainerException ex) {
-      throw ex;
-    } catch (IOException ex) {
-      LOG.error("Error during verify checksum for container {}", containerId);
-      throw new StorageContainerException("Error during verify checksum" +
-          " for container " + containerId, IO_EXCEPTION);
-    }
-  }
-
-  /**
    * Parse KeyValueContainerData and verify checksum.
-   * @param containerData
-   * @param containerFile
-   * @param checksumFile
-   * @param dbFile
+   * @param kvContainerData
    * @param config
    * @throws IOException
    */
-  public static void parseKeyValueContainerData(
-      KeyValueContainerData containerData, File containerFile, File
-      checksumFile, File dbFile, OzoneConfiguration config) throws IOException {
+  public static void parseKVContainerData(KeyValueContainerData kvContainerData,
+      OzoneConfiguration config) throws IOException {
 
-    Preconditions.checkNotNull(containerData, "containerData cannot be null");
-    Preconditions.checkNotNull(containerFile, "containerFile cannot be null");
-    Preconditions.checkNotNull(checksumFile, "checksumFile cannot be null");
-    Preconditions.checkNotNull(dbFile, "dbFile cannot be null");
-    Preconditions.checkNotNull(config, "ozone config cannot be null");
-
-    long containerId = containerData.getContainerID();
-    String containerName = String.valueOf(containerId);
-    File metadataPath = new File(containerData.getMetadataPath());
-
-    Preconditions.checkNotNull(containerName, "container Name cannot be " +
-        "null");
-    Preconditions.checkNotNull(metadataPath, "metadata path cannot be " +
-        "null");
+    long containerID = kvContainerData.getContainerID();
+    File metadataPath = new File(kvContainerData.getMetadataPath());
 
     // Verify Checksum
-    String checksum = KeyValueContainerUtil.computeCheckSum(
-        containerData.getContainerID(), containerFile);
-    KeyValueContainerUtil.verifyCheckSum(containerId, checksumFile, checksum);
-
-    containerData.setDbFile(dbFile);
+    ContainerUtils.verifyChecksum(kvContainerData);
+
+    File dbFile = KeyValueContainerLocationUtil.getContainerDBFile(
+        metadataPath, containerID);
+    if (!dbFile.exists()) {
+      LOG.error("Container DB file is missing for ContainerID {}. " +
+          "Skipping loading of this container.", containerID);
+      // Don't further process this container, as it is missing db file.
+      return;
+    }
+    kvContainerData.setDbFile(dbFile);
 
-    MetadataStore metadata = KeyUtils.getDB(containerData, config);
+    MetadataStore metadata = KeyUtils.getDB(kvContainerData, config);
     long bytesUsed = 0;
     List<Map.Entry<byte[], byte[]>> liveKeys = metadata
         .getRangeKVs(null, Integer.MAX_VALUE,
@@ -279,8 +189,8 @@ public final class KeyValueContainerUtil {
         return 0L;
       }
     }).sum();
-    containerData.setBytesUsed(bytesUsed);
-    containerData.setKeyCount(liveKeys.size());
+    kvContainerData.setBytesUsed(bytesUsed);
+    kvContainerData.setKeyCount(liveKeys.size());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
index c1595b2..dc33f2e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
@@ -20,6 +20,9 @@ package org.apache.hadoop.ozone.container.ozoneimpl;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .StorageContainerException;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.Storage;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
@@ -38,7 +41,6 @@ import java.io.File;
 import java.io.FileFilter;
 import java.io.IOException;
 
-
 /**
  * Class used to read .container files from Volume and build container map.
  *
@@ -46,15 +48,19 @@ import java.io.IOException;
  *
  * ../hdds/VERSION
  * ../hdds/<<scmUuid>>/current/<<containerDir>>/<<containerID>/metadata/<<containerID>>.container
- * ../hdds/<<scmUuid>>/current/<<containerDir>>/<<containerID>/metadata/<<containerID>>.checksum
- * ../hdds/<<scmUuid>>/current/<<containerDir>>/<<containerID>/metadata/<<containerID>>.db
  * ../hdds/<<scmUuid>>/current/<<containerDir>>/<<containerID>/<<dataPath>>
  *
+ * Some ContainerTypes will have extra metadata other than the .container
+ * file. For example, KeyValueContainer will have a .db file. This .db file
+ * will also be stored in the metadata folder along with the .container file.
+ *
+ * ../hdds/<<scmUuid>>/current/<<containerDir>>/<<KVcontainerID>/metadata/<<KVcontainerID>>.db
+ *
  * Note that the <<dataPath>> is dependent on the ContainerType.
  * For KeyValueContainers, the data is stored in a "chunks" folder. As such,
  * the <<dataPath>> layout for KeyValueContainers is
  *
- * ../hdds/<<scmUuid>>/current/<<containerDir>>/<<containerID>/chunks/<<chunksFile>>
+ * ../hdds/<<scmUuid>>/current/<<containerDir>>/<<KVcontainerID>/chunks/<<chunksFile>>
  *
  */
 public class ContainerReader implements Runnable {
@@ -124,22 +130,19 @@ public class ContainerReader implements Runnable {
               for (File containerDir : containerDirs) {
                 File metadataPath = new File(containerDir + File.separator +
                     OzoneConsts.CONTAINER_META_PATH);
-                String containerName = containerDir.getName();
+                long containerID = Long.parseLong(containerDir.getName());
                 if (metadataPath.exists()) {
                   File containerFile = KeyValueContainerLocationUtil
-                      .getContainerFile(metadataPath, containerName);
-                  File checksumFile = KeyValueContainerLocationUtil
-                      .getContainerCheckSumFile(metadataPath, containerName);
-                  if (containerFile.exists() && checksumFile.exists()) {
-                    verifyContainerFile(containerName, containerFile,
-                        checksumFile);
+                      .getContainerFile(metadataPath, containerID);
+                  if (containerFile.exists()) {
+                    verifyContainerFile(containerID, containerFile);
                   } else {
-                    LOG.error("Missing container metadata files for " +
-                        "Container: {}", containerName);
+                    LOG.error("Missing .container file for ContainerID: {}",
+                        containerID);
                   }
                 } else {
                   LOG.error("Missing container metadata directory for " +
-                      "Container: {}", containerName);
+                      "ContainerID: {}", containerID);
                 }
               }
             }
@@ -149,39 +152,46 @@ public class ContainerReader implements Runnable {
     }
   }
 
-  private void verifyContainerFile(String containerName, File containerFile,
-                                   File checksumFile) {
+  private void verifyContainerFile(long containerID, File containerFile) {
     try {
-      ContainerData containerData =  ContainerDataYaml.readContainerFile(
+      ContainerData containerData = ContainerDataYaml.readContainerFile(
           containerFile);
+      if (containerID != containerData.getContainerID()) {
+        LOG.error("Invalid ContainerID in file {}. " +
+            "Skipping loading of this container.", containerFile);
+        return;
+      }
+      verifyContainerData(containerData);
+    } catch (IOException ex) {
+      LOG.error("Failed to parse ContainerFile for ContainerID: {}",
+          containerID, ex);
+    }
+  }
 
-      switch (containerData.getContainerType()) {
-      case KeyValueContainer:
-        KeyValueContainerData keyValueContainerData = (KeyValueContainerData)
+  public void verifyContainerData(ContainerData containerData)
+      throws IOException {
+    switch (containerData.getContainerType()) {
+    case KeyValueContainer:
+      if (containerData instanceof KeyValueContainerData) {
+        KeyValueContainerData kvContainerData = (KeyValueContainerData)
             containerData;
         containerData.setVolume(hddsVolume);
-        File dbFile = KeyValueContainerLocationUtil
-            .getContainerDBFile(new File(containerFile.getParent()),
-                containerName);
-        if (!dbFile.exists()) {
-          LOG.error("Container DB file is missing for Container {}, skipping " +
-                  "this", containerName);
-          // Don't further process this container, as it is missing db file.
-          return;
-        }
-        KeyValueContainerUtil.parseKeyValueContainerData(keyValueContainerData,
-            containerFile, checksumFile, dbFile, config);
-        KeyValueContainer keyValueContainer = new KeyValueContainer(
-            keyValueContainerData, config);
-        containerSet.addContainer(keyValueContainer);
-        break;
-      default:
-        LOG.error("Unrecognized ContainerType {} format during verify " +
-            "ContainerFile", containerData.getContainerType());
+
+        KeyValueContainerUtil.parseKVContainerData(kvContainerData, config);
+        KeyValueContainer kvContainer = new KeyValueContainer(
+            kvContainerData, config);
+        containerSet.addContainer(kvContainer);
+      } else {
+        throw new StorageContainerException("Container File is corrupted. " +
+            "ContainerType is KeyValueContainer but cast to " +
+            "KeyValueContainerData failed. ",
+            ContainerProtos.Result.CONTAINER_METADATA_ERROR);
       }
-    } catch (IOException ex) {
-      LOG.error("Error during reading container file {}", containerFile);
+      break;
+    default:
+      throw new StorageContainerException("Unrecognized ContainerType " +
+          containerData.getContainerType(),
+          ContainerProtos.Result.UNKNOWN_CONTAINER_TYPE);
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
index d734271..fd51db3 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.ozone.container.common.impl;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
@@ -37,39 +38,58 @@ import static org.junit.Assert.fail;
  */
 public class TestContainerDataYaml {
 
+  private static long testContainerID = 1234;
+
+  private static String testRoot = new FileSystemTestHelper().getTestRootDir();
+
   private static final int MAXSIZE = 5;
-  @Test
-  public void testCreateContainerFile() throws IOException {
-    String path = new FileSystemTestHelper().getTestRootDir();
-    String containerPath = "1.container";
 
-    File filePath = new File(new FileSystemTestHelper().getTestRootDir());
-    filePath.mkdirs();
+  /**
+   * Creates a .container file. cleanup() should be called at the end of the
+   * test when container file is created.
+   */
+  private File createContainerFile(long containerID) throws IOException {
+    new File(testRoot).mkdirs();
+
+    String containerPath = containerID + ".container";
 
     KeyValueContainerData keyValueContainerData = new KeyValueContainerData(
-        Long.MAX_VALUE, MAXSIZE);
+        containerID, MAXSIZE);
     keyValueContainerData.setContainerDBType("RocksDB");
-    keyValueContainerData.setMetadataPath(path);
-    keyValueContainerData.setChunksPath(path);
+    keyValueContainerData.setMetadataPath(testRoot);
+    keyValueContainerData.setChunksPath(testRoot);
 
-    File containerFile = new File(filePath, containerPath);
+    File containerFile = new File(testRoot, containerPath);
 
     // Create .container file with ContainerData
     ContainerDataYaml.createContainerFile(ContainerProtos.ContainerType
-            .KeyValueContainer, containerFile, keyValueContainerData);
+        .KeyValueContainer, keyValueContainerData, containerFile);
 
     //Check .container file exists or not.
     assertTrue(containerFile.exists());
 
+    return containerFile;
+  }
+
+  private void cleanup() {
+    FileUtil.fullyDelete(new File(testRoot));
+  }
+
+  @Test
+  public void testCreateContainerFile() throws IOException {
+    long containerID = testContainerID++;
+
+    File containerFile = createContainerFile(containerID);
+
     // Read from .container file, and verify data.
     KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
         .readContainerFile(containerFile);
-    assertEquals(Long.MAX_VALUE, kvData.getContainerID());
+    assertEquals(containerID, kvData.getContainerID());
     assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
         .getContainerType());
     assertEquals("RocksDB", kvData.getContainerDBType());
-    assertEquals(path, kvData.getMetadataPath());
-    assertEquals(path, kvData.getChunksPath());
+    assertEquals(containerFile.getParent(), kvData.getMetadataPath());
+    assertEquals(containerFile.getParent(), kvData.getChunksPath());
     assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN, kvData
         .getState());
     assertEquals(1, kvData.getLayOutVersion());
@@ -82,22 +102,20 @@ public class TestContainerDataYaml {
     kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED);
 
 
-    // Update .container file with new ContainerData.
-    containerFile = new File(filePath, containerPath);
     ContainerDataYaml.createContainerFile(ContainerProtos.ContainerType
-            .KeyValueContainer, containerFile, kvData);
+            .KeyValueContainer, kvData, containerFile);
 
     // Reading newly updated data from .container file
     kvData =  (KeyValueContainerData) ContainerDataYaml.readContainerFile(
         containerFile);
 
     // verify data.
-    assertEquals(Long.MAX_VALUE, kvData.getContainerID());
+    assertEquals(containerID, kvData.getContainerID());
     assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
         .getContainerType());
     assertEquals("RocksDB", kvData.getContainerDBType());
-    assertEquals(path, kvData.getMetadataPath());
-    assertEquals(path, kvData.getChunksPath());
+    assertEquals(containerFile.getParent(), kvData.getMetadataPath());
+    assertEquals(containerFile.getParent(), kvData.getChunksPath());
     assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, kvData
         .getState());
     assertEquals(1, kvData.getLayOutVersion());
@@ -105,19 +123,15 @@ public class TestContainerDataYaml {
     assertEquals("hdfs", kvData.getMetadata().get("VOLUME"));
     assertEquals("ozone", kvData.getMetadata().get("OWNER"));
     assertEquals(MAXSIZE, kvData.getMaxSizeGB());
-
-    FileUtil.fullyDelete(filePath);
-
-
   }
 
   @Test
   public void testIncorrectContainerFile() throws IOException{
     try {
-      String path = "incorrect.container";
+      String containerFile = "incorrect.container";
       //Get file from resources folder
       ClassLoader classLoader = getClass().getClassLoader();
-      File file = new File(classLoader.getResource(path).getFile());
+      File file = new File(classLoader.getResource(containerFile).getFile());
       KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
           .readContainerFile(file);
       fail("testIncorrectContainerFile failed");
@@ -137,12 +151,13 @@ public class TestContainerDataYaml {
     // created or not.
 
     try {
-      String path = "additionalfields.container";
+      String containerFile = "additionalfields.container";
       //Get file from resources folder
       ClassLoader classLoader = getClass().getClassLoader();
-      File file = new File(classLoader.getResource(path).getFile());
+      File file = new File(classLoader.getResource(containerFile).getFile());
       KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
           .readContainerFile(file);
+      ContainerUtils.verifyChecksum(kvData);
 
       //Checking the Container file data is consistent or not
       assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, kvData
@@ -159,9 +174,45 @@ public class TestContainerDataYaml {
       assertEquals(2, kvData.getMetadata().size());
 
     } catch (Exception ex) {
+      ex.printStackTrace();
       fail("testCheckBackWardCompatabilityOfContainerFile failed");
     }
   }
 
+  /**
+   * Test to verify {@link ContainerUtils#verifyChecksum(ContainerData)}.
+   */
+  @Test
+  public void testChecksumInContainerFile() throws IOException {
+    long containerID = testContainerID++;
 
+    File containerFile = createContainerFile(containerID);
+
+    // Read from .container file, and verify data.
+    KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
+        .readContainerFile(containerFile);
+    ContainerUtils.verifyChecksum(kvData);
+
+    cleanup();
+  }
+
+  /**
+   * Test to verify incorrect checksum is detected.
+   */
+  @Test
+  public void testIncorrectChecksum() {
+    try {
+      String containerFile = "incorrect.checksum.container";
+      //Get file from resources folder
+      ClassLoader classLoader = getClass().getClassLoader();
+      File file = new File(classLoader.getResource(containerFile).getFile());
+      KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
+          .readContainerFile(file);
+      ContainerUtils.verifyChecksum(kvData);
+      fail("testIncorrectChecksum failed");
+    } catch (Exception ex) {
+      GenericTestUtils.assertExceptionContains("Container checksum error for " +
+          "ContainerID:", ex);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
index 4f00507..2bf41e5 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
@@ -69,8 +69,7 @@ public class TestKeyValueContainer {
   private String scmId = UUID.randomUUID().toString();
   private VolumeSet volumeSet;
   private RoundRobinVolumeChoosingPolicy volumeChoosingPolicy;
-  private long containerId = 1L;
-  private String containerName = String.valueOf(containerId);
+  private long containerID = 1L;
   private KeyValueContainerData keyValueContainerData;
   private KeyValueContainer keyValueContainer;
 
@@ -111,16 +110,12 @@ public class TestKeyValueContainer {
     assertTrue(chunksPath != null);
     File containerMetaDataLoc = new File(containerMetaDataPath);
 
-    //Check whether container file, check sum file and container db file exists
-    // or not.
+    //Check whether container file and container db file exists or not.
     assertTrue(KeyValueContainerLocationUtil.getContainerFile(
-        containerMetaDataLoc, containerName).exists(), ".Container File does" +
+        containerMetaDataLoc, containerID).exists(), ".Container File does" +
         " not exist");
-    assertTrue(KeyValueContainerLocationUtil.getContainerCheckSumFile(
-        containerMetaDataLoc, containerName).exists(), "Container check sum " +
-        "File does" + " not exist");
     assertTrue(KeyValueContainerLocationUtil.getContainerDBFile(
-        containerMetaDataLoc, containerName).exists(), "Container DB does " +
+        containerMetaDataLoc, containerID).exists(), "Container DB does " +
         "not exist");
   }
 
@@ -172,10 +167,10 @@ public class TestKeyValueContainer {
 
     assertFalse("Container File still exists",
         KeyValueContainerLocationUtil.getContainerFile(containerMetaDataLoc,
-            containerName).exists());
+            containerID).exists());
     assertFalse("Container DB file still exists",
         KeyValueContainerLocationUtil.getContainerDBFile(containerMetaDataLoc,
-            containerName).exists());
+            containerID).exists());
   }
 
 
@@ -195,7 +190,7 @@ public class TestKeyValueContainer {
         .getMetadataPath();
     File containerMetaDataLoc = new File(containerMetaDataPath);
     File containerFile = KeyValueContainerLocationUtil.getContainerFile(
-        containerMetaDataLoc, containerName);
+        containerMetaDataLoc, containerID);
 
     keyValueContainerData = (KeyValueContainerData) ContainerDataYaml
         .readContainerFile(containerFile);
@@ -236,7 +231,7 @@ public class TestKeyValueContainer {
         .getMetadataPath();
     File containerMetaDataLoc = new File(containerMetaDataPath);
     File containerFile = KeyValueContainerLocationUtil.getContainerFile(
-        containerMetaDataLoc, containerName);
+        containerMetaDataLoc, containerID);
 
     keyValueContainerData = (KeyValueContainerData) ContainerDataYaml
         .readContainerFile(containerFile);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/test/resources/additionalfields.container
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/resources/additionalfields.container b/hadoop-hdds/container-service/src/test/resources/additionalfields.container
index f437a95..73cf5f3 100644
--- a/hadoop-hdds/container-service/src/test/resources/additionalfields.container
+++ b/hadoop-hdds/container-service/src/test/resources/additionalfields.container
@@ -8,4 +8,5 @@ layOutVersion: 1
 maxSizeGB: 5
 metadata: {OWNER: ozone, VOLUME: hdfs}
 state: CLOSED
-aclEnabled: true
\ No newline at end of file
+aclEnabled: true
+checksum: 1bbff32aeaa8fadc0b80c5c1e0597036e96acd8ae4bddbed188a2162762251a2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/test/resources/incorrect.checksum.container
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/resources/incorrect.checksum.container b/hadoop-hdds/container-service/src/test/resources/incorrect.checksum.container
new file mode 100644
index 0000000..feeeadc
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/resources/incorrect.checksum.container
@@ -0,0 +1,11 @@
+!<KeyValueContainerData>
+containerDBType: RocksDB
+chunksPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
+containerID: 9223372036854775807
+containerType: KeyValueContainer
+metadataPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
+layOutVersion: 1
+maxSizeGB: 5
+metadata: {OWNER: ozone, VOLUME: hdfs}
+state: OPEN
+checksum: 08bc9d390f9183aeed3cf33c789e2a07310bba60f3cf55941caccc939db8670f
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/test/resources/incorrect.container
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/resources/incorrect.container b/hadoop-hdds/container-service/src/test/resources/incorrect.container
index 38a8857..8aeb30c 100644
--- a/hadoop-hdds/container-service/src/test/resources/incorrect.container
+++ b/hadoop-hdds/container-service/src/test/resources/incorrect.container
@@ -7,4 +7,5 @@ metadataPath: /hdds/current/aed-fg4-hji-jkl/containerDir0/1
 layOutVersion: 1
 maxSizeGB: 5
 metadata: {OWNER: ozone, VOLUME: hdfs}
-state: INVALID
\ No newline at end of file
+state: INVALID
+checksum: 08bc9d390f9183aeed3cf33c789e2a07310bba60f3cf55941caccc939db8670f
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/pom.xml b/hadoop-hdds/pom.xml
index 09fac33..f655c2f 100644
--- a/hadoop-hdds/pom.xml
+++ b/hadoop-hdds/pom.xml
@@ -104,6 +104,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/main/resources/webapps/static/nvd3-1.8.5.min.js.map</exclude>
             <exclude>src/test/resources/incorrect.container</exclude>
             <exclude>src/test/resources/additionalfields.container</exclude>
+            <exclude>src/test/resources/incorrect.checksum.container</exclude>
           </excludes>
         </configuration>
       </plugin>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
index d29937e..c2e1645 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
@@ -716,7 +716,7 @@ public class TestContainerPersistence {
 
     File orgContainerFile = KeyValueContainerLocationUtil.getContainerFile(
         new File(container.getContainerData().getMetadataPath()),
-        String.valueOf(testContainerID));
+        testContainerID);
     Assert.assertTrue(orgContainerFile.exists());
 
     Map<String, String> newMetadata = Maps.newHashMap();
@@ -740,7 +740,7 @@ public class TestContainerPersistence {
     // Verify container data on disk
     File newContainerFile = KeyValueContainerLocationUtil.getContainerFile(
         new File(actualNewData.getMetadataPath()),
-        String.valueOf(testContainerID));
+        testContainerID);
     Assert.assertTrue("Container file should exist.",
         newContainerFile.exists());
     Assert.assertEquals("Container file should be in same location.",
@@ -780,8 +780,8 @@ public class TestContainerPersistence {
 
     // Update a non-existing container
     exception.expect(StorageContainerException.class);
-    exception.expectMessage("Container is an Inconsistent state, missing " +
-        "required files(.container, .chksm).");
+    exception.expectMessage("Container is an Inconsistent " +
+        "state, missing .container file.");
     Container nonExistentContainer = new KeyValueContainer(
         new KeyValueContainerData(RandomUtils.nextLong(),
             ContainerTestHelper.CONTAINER_MAX_SIZE_GB), conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
index 1522283..d271ed3 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
@@ -33,11 +33,14 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 import org.junit.rules.Timeout;
 
 import java.util.*;
 import java.util.concurrent.CompletableFuture;
 
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
+
 /**
  * Tests ozone containers.
  */
@@ -48,6 +51,9 @@ public class TestOzoneContainer {
   @Rule
   public Timeout testTimeout = new Timeout(300000);
 
+  @Rule
+  public TemporaryFolder tempFolder = new TemporaryFolder();
+
   @Test
   public void testCreateOzoneContainer() throws Exception {
     long containerID = ContainerTestHelper.getTestContainerID();
@@ -60,6 +66,7 @@ public class TestOzoneContainer {
       // We don't start Ozone Container via data node, we will do it
       // independently in our test path.
       Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline();
+      conf.set(HDDS_DATANODE_DIR_KEY, tempFolder.getRoot().getPath());
       conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, pipeline.getLeader()
               .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
       conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, false);


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


[40/50] hadoop git commit: HDDS-248. Refactor DatanodeContainerProtocol.proto Contributed by Hanisha Koneru.

Posted by eh...@apache.org.
HDDS-248. Refactor DatanodeContainerProtocol.proto Contributed by Hanisha Koneru.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/007e6f51
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/007e6f51
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/007e6f51

Branch: refs/heads/HDFS-12090
Commit: 007e6f51135adb5864f6bfc258010fd09576387b
Parents: feb795b
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Sat Jul 28 14:50:43 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Sat Jul 28 14:57:11 2018 -0700

----------------------------------------------------------------------
 .../scm/storage/ContainerProtocolCalls.java     | 37 +++++---
 .../main/proto/DatanodeContainerProtocol.proto  | 96 +++++++++-----------
 .../container/common/impl/HddsDispatcher.java   | 51 +----------
 .../CloseContainerCommandHandler.java           |  8 +-
 .../server/ratis/ContainerStateMachine.java     |  6 +-
 .../keyvalue/KeyValueContainerData.java         |  9 --
 .../container/keyvalue/KeyValueHandler.java     | 16 +---
 .../container/ozoneimpl/OzoneContainer.java     |  2 +-
 .../container/keyvalue/TestKeyValueHandler.java | 12 ++-
 .../scm/cli/container/InfoContainerHandler.java |  1 -
 .../ozone/container/ContainerTestHelper.java    | 59 ++++++------
 .../common/impl/TestCloseContainerHandler.java  | 18 ++--
 .../genesis/BenchMarkDatanodeDispatcher.java    | 19 ++--
 13 files changed, 148 insertions(+), 186 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/007e6f51/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
index 36cdfc9..abad9e3 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
@@ -29,6 +29,8 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandResponseProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .CloseContainerRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .DatanodeBlockID;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .GetKeyRequestProto;
@@ -86,15 +88,18 @@ public final class ContainerProtocolCalls  {
         .newBuilder()
         .setBlockID(datanodeBlockID);
     String id = xceiverClient.getPipeline().getLeader().getUuidString();
+
     ContainerCommandRequestProto request = ContainerCommandRequestProto
         .newBuilder()
         .setCmdType(Type.GetKey)
+        .setContainerID(datanodeBlockID.getContainerID())
         .setTraceID(traceID)
         .setDatanodeUuid(id)
         .setGetKey(readKeyRequest)
         .build();
     ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
     validateContainerResponse(response);
+
     return response.getGetKey();
   }
 
@@ -118,7 +123,9 @@ public final class ContainerProtocolCalls  {
     String id = xceiverClient.getPipeline().getLeader().getUuidString();
     ContainerCommandRequestProto request =
         ContainerCommandRequestProto.newBuilder()
-            .setCmdType(Type.GetCommittedBlockLength).setTraceID(traceID)
+            .setCmdType(Type.GetCommittedBlockLength)
+            .setContainerID(blockID.getContainerID())
+            .setTraceID(traceID)
             .setDatanodeUuid(id)
             .setGetCommittedBlockLength(getBlockLengthRequestBuilder).build();
     ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
@@ -143,6 +150,7 @@ public final class ContainerProtocolCalls  {
     ContainerCommandRequestProto request = ContainerCommandRequestProto
         .newBuilder()
         .setCmdType(Type.PutKey)
+        .setContainerID(containerKeyData.getBlockID().getContainerID())
         .setTraceID(traceID)
         .setDatanodeUuid(id)
         .setPutKey(createKeyRequest)
@@ -171,6 +179,7 @@ public final class ContainerProtocolCalls  {
     ContainerCommandRequestProto request = ContainerCommandRequestProto
         .newBuilder()
         .setCmdType(Type.ReadChunk)
+        .setContainerID(blockID.getContainerID())
         .setTraceID(traceID)
         .setDatanodeUuid(id)
         .setReadChunk(readChunkRequest)
@@ -202,6 +211,7 @@ public final class ContainerProtocolCalls  {
     ContainerCommandRequestProto request = ContainerCommandRequestProto
         .newBuilder()
         .setCmdType(Type.WriteChunk)
+        .setContainerID(blockID.getContainerID())
         .setTraceID(traceID)
         .setDatanodeUuid(id)
         .setWriteChunk(writeChunkRequest)
@@ -250,6 +260,7 @@ public final class ContainerProtocolCalls  {
     ContainerCommandRequestProto request =
         ContainerCommandRequestProto.newBuilder()
             .setCmdType(Type.PutSmallFile)
+            .setContainerID(blockID.getContainerID())
             .setTraceID(traceID)
             .setDatanodeUuid(id)
             .setPutSmallFile(putSmallFileRequest)
@@ -270,7 +281,6 @@ public final class ContainerProtocolCalls  {
     ContainerProtos.CreateContainerRequestProto.Builder createRequest =
         ContainerProtos.CreateContainerRequestProto
             .newBuilder();
-    createRequest.setContainerID(containerID);
     createRequest.setContainerType(ContainerProtos.ContainerType
         .KeyValueContainer);
 
@@ -278,6 +288,7 @@ public final class ContainerProtocolCalls  {
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.CreateContainer);
+    request.setContainerID(containerID);
     request.setCreateContainer(createRequest.build());
     request.setDatanodeUuid(id);
     request.setTraceID(traceID);
@@ -298,12 +309,13 @@ public final class ContainerProtocolCalls  {
       boolean force, String traceID) throws IOException {
     ContainerProtos.DeleteContainerRequestProto.Builder deleteRequest =
         ContainerProtos.DeleteContainerRequestProto.newBuilder();
-    deleteRequest.setContainerID(containerID);
     deleteRequest.setForceDelete(force);
     String id = client.getPipeline().getLeader().getUuidString();
+
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.DeleteContainer);
+    request.setContainerID(containerID);
     request.setDeleteContainer(deleteRequest);
     request.setTraceID(traceID);
     request.setDatanodeUuid(id);
@@ -322,15 +334,13 @@ public final class ContainerProtocolCalls  {
    */
   public static void closeContainer(XceiverClientSpi client,
       long containerID, String traceID) throws IOException {
-    ContainerProtos.CloseContainerRequestProto.Builder closeRequest =
-        ContainerProtos.CloseContainerRequestProto.newBuilder();
-    closeRequest.setContainerID(containerID);
-
     String id = client.getPipeline().getLeader().getUuidString();
+
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(Type.CloseContainer);
-    request.setCloseContainer(closeRequest);
+    request.setContainerID(containerID);
+    request.setCloseContainer(CloseContainerRequestProto.getDefaultInstance());
     request.setTraceID(traceID);
     request.setDatanodeUuid(id);
     ContainerCommandResponseProto response =
@@ -348,19 +358,19 @@ public final class ContainerProtocolCalls  {
   public static ReadContainerResponseProto readContainer(
       XceiverClientSpi client, long containerID,
       String traceID) throws IOException {
-    ReadContainerRequestProto.Builder readRequest =
-        ReadContainerRequestProto.newBuilder();
-    readRequest.setContainerID(containerID);
     String id = client.getPipeline().getLeader().getUuidString();
+
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(Type.ReadContainer);
-    request.setReadContainer(readRequest);
+    request.setContainerID(containerID);
+    request.setReadContainer(ReadContainerRequestProto.getDefaultInstance());
     request.setDatanodeUuid(id);
     request.setTraceID(traceID);
     ContainerCommandResponseProto response =
         client.sendCommand(request.build());
     validateContainerResponse(response);
+
     return response.getReadContainer();
   }
 
@@ -383,15 +393,18 @@ public final class ContainerProtocolCalls  {
             .newBuilder().setKey(getKey)
             .build();
     String id = client.getPipeline().getLeader().getUuidString();
+
     ContainerCommandRequestProto request = ContainerCommandRequestProto
         .newBuilder()
         .setCmdType(Type.GetSmallFile)
+        .setContainerID(blockID.getContainerID())
         .setTraceID(traceID)
         .setDatanodeUuid(id)
         .setGetSmallFile(getSmallFileRequest)
         .build();
     ContainerCommandResponseProto response = client.sendCommand(request);
     validateContainerResponse(response);
+
     return response.getGetSmallFile();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/007e6f51/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
index 6969fa6..af06346 100644
--- a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
+++ b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
@@ -165,80 +165,81 @@ enum ContainerLifeCycleState {
 }
 
 message ContainerCommandRequestProto {
-  required Type cmdType = 1; // Type of the command
+  required   Type cmdType = 1; // Type of the command
 
   // A string that identifies this command, we generate  Trace ID in Ozone
   // frontend and this allows us to trace that command all over ozone.
-  optional string traceID = 2;
+  optional   string traceID = 2;
+
+  required   int64 containerID = 3;
+  required   string datanodeUuid = 4;
 
   // One of the following command is available when the corresponding
   // cmdType is set. At the protocol level we allow only
   // one command in each packet.
   // TODO : Upgrade to Protobuf 2.6 or later.
-  optional   CreateContainerRequestProto createContainer = 3;
-  optional   ReadContainerRequestProto readContainer = 4;
-  optional   UpdateContainerRequestProto updateContainer = 5;
-  optional   DeleteContainerRequestProto deleteContainer = 6;
-  optional   ListContainerRequestProto listContainer = 7;
+  optional   CreateContainerRequestProto createContainer = 5;
+  optional   ReadContainerRequestProto readContainer = 6;
+  optional   UpdateContainerRequestProto updateContainer = 7;
+  optional   DeleteContainerRequestProto deleteContainer = 8;
+  optional   ListContainerRequestProto listContainer = 9;
+  optional   CloseContainerRequestProto closeContainer = 10;
+
+  optional   PutKeyRequestProto putKey = 11;
+  optional   GetKeyRequestProto getKey = 12;
+  optional   DeleteKeyRequestProto deleteKey = 13;
+  optional   ListKeyRequestProto listKey = 14;
 
-  optional   PutKeyRequestProto putKey = 8;
-  optional   GetKeyRequestProto getKey = 9;
-  optional   DeleteKeyRequestProto deleteKey = 10;
-  optional   ListKeyRequestProto listKey = 11;
+  optional   ReadChunkRequestProto readChunk = 15;
+  optional   WriteChunkRequestProto writeChunk = 16;
+  optional   DeleteChunkRequestProto deleteChunk = 17;
+  optional   ListChunkRequestProto listChunk = 18;
 
-  optional   ReadChunkRequestProto readChunk = 12;
-  optional   WriteChunkRequestProto writeChunk = 13;
-  optional   DeleteChunkRequestProto deleteChunk = 14;
-  optional   ListChunkRequestProto listChunk = 15;
+  optional   PutSmallFileRequestProto putSmallFile = 19;
+  optional   GetSmallFileRequestProto getSmallFile = 20;
 
-  optional   PutSmallFileRequestProto putSmallFile = 16;
-  optional   GetSmallFileRequestProto getSmallFile = 17;
-  optional   CloseContainerRequestProto closeContainer = 18;
-  optional   GetCommittedBlockLengthRequestProto getCommittedBlockLength = 19;
-  required   string datanodeUuid = 20;
+  optional   GetCommittedBlockLengthRequestProto getCommittedBlockLength = 21;
 }
 
 message ContainerCommandResponseProto {
-  required Type cmdType = 1;
-  optional string traceID = 2;
+  required   Type cmdType = 1;
+  optional   string traceID = 2;
 
-  optional   CreateContainerResponseProto createContainer = 3;
-  optional   ReadContainerResponseProto readContainer = 4;
-  optional   UpdateContainerResponseProto updateContainer = 5;
-  optional   DeleteContainerResponseProto deleteContainer = 6;
-  optional   ListContainerResponseProto listContainer = 7;
+  required   Result result = 3;
+  optional   string message = 4;
 
-  optional   PutKeyResponseProto putKey = 8;
-  optional   GetKeyResponseProto getKey = 9;
-  optional   DeleteKeyResponseProto deleteKey = 10;
-  optional   ListKeyResponseProto listKey = 11;
+  optional   CreateContainerResponseProto createContainer = 5;
+  optional   ReadContainerResponseProto readContainer = 6;
+  optional   UpdateContainerResponseProto updateContainer = 7;
+  optional   DeleteContainerResponseProto deleteContainer = 8;
+  optional   ListContainerResponseProto listContainer = 9;
+  optional   CloseContainerResponseProto closeContainer = 10;
 
-  optional  WriteChunkResponseProto writeChunk = 12;
-  optional  ReadChunkResponseProto readChunk = 13;
-  optional  DeleteChunkResponseProto deleteChunk = 14;
-  optional  ListChunkResponseProto listChunk = 15;
+  optional   PutKeyResponseProto putKey = 11;
+  optional   GetKeyResponseProto getKey = 12;
+  optional   DeleteKeyResponseProto deleteKey = 13;
+  optional   ListKeyResponseProto listKey = 14;
 
-  required Result result = 17;
-  optional string message = 18;
+  optional   WriteChunkResponseProto writeChunk = 15;
+  optional   ReadChunkResponseProto readChunk = 16;
+  optional   DeleteChunkResponseProto deleteChunk = 17;
+  optional   ListChunkResponseProto listChunk = 18;
 
-  optional PutSmallFileResponseProto putSmallFile = 19;
-  optional GetSmallFileResponseProto getSmallFile = 20;
-  optional CloseContainerResponseProto closeContainer = 21;
-  optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 22;
+  optional   PutSmallFileResponseProto putSmallFile = 19;
+  optional   GetSmallFileResponseProto getSmallFile = 20;
 
+  optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 21;
 }
 
 message ContainerData {
   required int64 containerID = 1;
   repeated KeyValue metadata = 2;
-  optional string dbPath = 3;
   optional string containerPath = 4;
   optional int64 bytesUsed = 6;
   optional int64 size = 7;
-  optional int64 keyCount = 8;
+  optional int64 blockCount = 8;
   optional ContainerLifeCycleState state = 9 [default = OPEN];
   optional ContainerType containerType = 10 [default = KeyValueContainer];
-  optional string containerDBType = 11;
 }
 
 enum ContainerType {
@@ -248,7 +249,6 @@ enum ContainerType {
 
 // Container Messages.
 message  CreateContainerRequestProto {
-  required int64 containerID = 1;
   repeated KeyValue metadata = 2;
   optional ContainerType containerType = 3 [default = KeyValueContainer];
 }
@@ -257,7 +257,6 @@ message  CreateContainerResponseProto {
 }
 
 message  ReadContainerRequestProto {
-  required int64 containerID = 1;
 }
 
 message  ReadContainerResponseProto {
@@ -265,7 +264,6 @@ message  ReadContainerResponseProto {
 }
 
 message  UpdateContainerRequestProto {
-  required int64 containerID = 1;
   repeated KeyValue metadata = 2;
   optional bool forceUpdate = 3 [default = false];
 }
@@ -274,7 +272,6 @@ message  UpdateContainerResponseProto {
 }
 
 message  DeleteContainerRequestProto {
-  required int64 containerID = 1;
   optional bool forceDelete = 2 [default = false];
 }
 
@@ -282,7 +279,6 @@ message  DeleteContainerResponseProto {
 }
 
 message  ListContainerRequestProto {
-  required int64 startContainerID = 1;
   optional uint32 count = 2; // Max Results to return
 }
 
@@ -291,7 +287,6 @@ message  ListContainerResponseProto {
 }
 
 message CloseContainerRequestProto {
-  required int64 containerID = 1;
 }
 
 message CloseContainerResponseProto {
@@ -341,7 +336,6 @@ message   DeleteKeyResponseProto {
 }
 
 message  ListKeyRequestProto {
-  required int64 containerID = 1;
   optional int64 startLocalID = 2;
   required uint32 count = 3;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/007e6f51/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
index 6d11abb..3d418e5 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
@@ -98,13 +98,16 @@ public class HddsDispatcher implements ContainerDispatcher {
     long startTime = System.nanoTime();
     ContainerProtos.Type cmdType = msg.getCmdType();
     try {
-      long containerID = getContainerID(msg);
+      long containerID = msg.getContainerID();
 
       metrics.incContainerOpsMetrics(cmdType);
       if (cmdType != ContainerProtos.Type.CreateContainer) {
         container = getContainer(containerID);
         containerType = getContainerType(container);
       } else {
+        if (!msg.hasCreateContainer()) {
+          return ContainerUtils.malformedRequest(msg);
+        }
         containerType = msg.getCreateContainer().getContainerType();
       }
     } catch (StorageContainerException ex) {
@@ -143,52 +146,6 @@ public class HddsDispatcher implements ContainerDispatcher {
     }
   }
 
-  private long getContainerID(ContainerCommandRequestProto request)
-      throws StorageContainerException {
-    ContainerProtos.Type cmdType = request.getCmdType();
-
-    switch(cmdType) {
-    case CreateContainer:
-      return request.getCreateContainer().getContainerID();
-    case ReadContainer:
-      return request.getReadContainer().getContainerID();
-    case UpdateContainer:
-      return request.getUpdateContainer().getContainerID();
-    case DeleteContainer:
-      return request.getDeleteContainer().getContainerID();
-    case ListContainer:
-      return request.getListContainer().getStartContainerID();
-    case CloseContainer:
-      return request.getCloseContainer().getContainerID();
-    case PutKey:
-      return request.getPutKey().getKeyData().getBlockID().getContainerID();
-    case GetKey:
-      return request.getGetKey().getBlockID().getContainerID();
-    case DeleteKey:
-      return request.getDeleteKey().getBlockID().getContainerID();
-    case ListKey:
-      return request.getListKey().getContainerID();
-    case ReadChunk:
-      return request.getReadChunk().getBlockID().getContainerID();
-    case DeleteChunk:
-      return request.getDeleteChunk().getBlockID().getContainerID();
-    case WriteChunk:
-      return request.getWriteChunk().getBlockID().getContainerID();
-    case ListChunk:
-      return request.getListChunk().getBlockID().getContainerID();
-    case PutSmallFile:
-      return request.getPutSmallFile().getKey().getKeyData().getBlockID()
-          .getContainerID();
-    case GetSmallFile:
-      return request.getGetSmallFile().getKey().getBlockID().getContainerID();
-    case GetCommittedBlockLength:
-      return request.getGetCommittedBlockLength().getBlockID().getContainerID();
-    }
-
-    throw new StorageContainerException(
-        ContainerProtos.Result.UNSUPPORTED_REQUEST);
-  }
-
   @VisibleForTesting
   public Container getContainer(long containerID)
       throws StorageContainerException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/007e6f51/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java
index f58cbae..a3bddfc 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java
@@ -76,14 +76,12 @@ public class CloseContainerCommandHandler implements CommandHandler {
       HddsProtos.ReplicationType replicationType =
           closeContainerProto.getReplicationType();
 
-      ContainerProtos.CloseContainerRequestProto.Builder closeRequest =
-          ContainerProtos.CloseContainerRequestProto.newBuilder();
-      closeRequest.setContainerID(containerID);
-
       ContainerProtos.ContainerCommandRequestProto.Builder request =
           ContainerProtos.ContainerCommandRequestProto.newBuilder();
       request.setCmdType(ContainerProtos.Type.CloseContainer);
-      request.setCloseContainer(closeRequest);
+      request.setContainerID(containerID);
+      request.setCloseContainer(
+          ContainerProtos.CloseContainerRequestProto.getDefaultInstance());
       request.setTraceID(UUID.randomUUID().toString());
       request.setDatanodeUuid(
           context.getParent().getDatanodeDetails().getUuidString());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/007e6f51/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
index fc7635e..ac7aa57 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
@@ -207,8 +207,7 @@ public class ContainerStateMachine extends BaseStateMachine {
 
   private CompletableFuture<Message> handleCreateContainer(
       ContainerCommandRequestProto requestProto) {
-    long containerID =
-        requestProto.getCreateContainer().getContainerID();
+    long containerID = requestProto.getContainerID();
     createContainerFutureMap.
         computeIfAbsent(containerID, k -> new CompletableFuture<>());
     return CompletableFuture.completedFuture(() -> ByteString.EMPTY);
@@ -264,8 +263,7 @@ public class ContainerStateMachine extends BaseStateMachine {
       } else {
         Message message = runCommand(requestProto);
         if (cmdType == ContainerProtos.Type.CreateContainer) {
-          long containerID =
-              requestProto.getCreateContainer().getContainerID();
+          long containerID = requestProto.getContainerID();
           createContainerFutureMap.remove(containerID).complete(message);
         }
         return CompletableFuture.completedFuture(message);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/007e6f51/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
index 34035c8..0705cf4 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
@@ -177,7 +177,6 @@ public class KeyValueContainerData extends ContainerData {
     ContainerProtos.ContainerData.Builder builder = ContainerProtos
         .ContainerData.newBuilder();
     builder.setContainerID(this.getContainerID());
-    builder.setDbPath(this.getDbFile().getPath());
     builder.setContainerPath(this.getMetadataPath());
     builder.setState(this.getState());
 
@@ -196,10 +195,6 @@ public class KeyValueContainerData extends ContainerData {
       builder.setContainerType(ContainerProtos.ContainerType.KeyValueContainer);
     }
 
-    if(this.getContainerDBType() != null) {
-      builder.setContainerDBType(containerDBType);
-    }
-
     return builder.build();
   }
 
@@ -239,10 +234,6 @@ public class KeyValueContainerData extends ContainerData {
       data.setBytesUsed(protoData.getBytesUsed());
     }
 
-    if(protoData.hasContainerDBType()) {
-      data.setContainerDBType(protoData.getContainerDBType());
-    }
-
     return data;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/007e6f51/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index 0b26a14..a4e124b 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -231,13 +231,7 @@ public class KeyValueHandler extends Handler {
     // container would be created here.
     Preconditions.checkArgument(kvContainer == null);
 
-    CreateContainerRequestProto createContainerReq =
-        request.getCreateContainer();
-    long containerID = createContainerReq.getContainerID();
-    if (createContainerReq.hasContainerType()) {
-      Preconditions.checkArgument(createContainerReq.getContainerType()
-          .equals(ContainerType.KeyValueContainer));
-    }
+    long containerID = request.getContainerID();
 
     KeyValueContainerData newContainerData = new KeyValueContainerData(
         containerID, maxContainerSizeGB);
@@ -381,15 +375,15 @@ public class KeyValueHandler extends Handler {
     try {
       checkContainerOpen(kvContainer);
 
+      KeyValueContainerData kvData = kvContainer.getContainerData();
+
       // remove the container from open block map once, all the blocks
       // have been committed and the container is closed
-      kvContainer.getContainerData()
-          .setState(ContainerProtos.ContainerLifeCycleState.CLOSING);
+      kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSING);
       commitPendingKeys(kvContainer);
       kvContainer.close();
       // make sure the the container open keys from BlockMap gets removed
-      openContainerBlockMap.removeContainer(
-          request.getCloseContainer().getContainerID());
+      openContainerBlockMap.removeContainer(kvData.getContainerID());
     } catch (StorageContainerException ex) {
       return ContainerUtils.logAndReturnError(LOG, ex, request);
     } catch (IOException ex) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/007e6f51/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
index 8f067d9..30fe113 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
@@ -196,7 +196,7 @@ public class OzoneContainer {
     ContainerProtos.Type type = request.getCmdType();
     switch (type) {
     case CloseContainer:
-      return request.getCloseContainer().getContainerID();
+      return request.getContainerID();
       // Right now, we handle only closeContainer via queuing it over the
       // over the XceiVerServer. For all other commands we throw Illegal
       // argument exception here. Will need to extend the switch cases

http://git-wip-us.apache.org/repos/asf/hadoop/blob/007e6f51/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
index 42ec54f..8e8a1be 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
 import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestRule;
@@ -55,6 +56,7 @@ public class TestKeyValueHandler {
   private final String baseDir = MiniDFSCluster.getBaseDirectory();
   private final String volume = baseDir + "disk1";
 
+  private static final long DUMMY_CONTAINER_ID = 9999;
 
   @Test
   /**
@@ -74,8 +76,13 @@ public class TestKeyValueHandler {
 
     // Test Create Container Request handling
     ContainerCommandRequestProto createContainerRequest =
-        getDummyCommandRequestProto(ContainerProtos.Type.CreateContainer);
-
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.CreateContainer)
+            .setContainerID(DUMMY_CONTAINER_ID)
+            .setDatanodeUuid(DATANODE_UUID)
+            .setCreateContainer(ContainerProtos.CreateContainerRequestProto
+                .getDefaultInstance())
+            .build();
     dispatcher.dispatch(createContainerRequest);
     Mockito.verify(handler, times(1)).handleCreateContainer(
         any(ContainerCommandRequestProto.class), any());
@@ -191,6 +198,7 @@ public class TestKeyValueHandler {
     ContainerCommandRequestProto request =
         ContainerProtos.ContainerCommandRequestProto.newBuilder()
             .setCmdType(cmdType)
+            .setContainerID(DUMMY_CONTAINER_ID)
             .setDatanodeUuid(DATANODE_UUID)
             .build();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/007e6f51/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java
index 3716ace..89215fa 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java
@@ -81,7 +81,6 @@ public class InfoContainerHandler extends OzoneCommandHandler {
         containerData.getState() == ContainerLifeCycleState.OPEN ? "OPEN" :
             "CLOSED";
     logOut("Container State: %s", openStatus);
-    logOut("Container DB Path: %s", containerData.getDbPath());
     logOut("Container Path: %s", containerData.getContainerPath());
 
     // Output meta data.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/007e6f51/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
index d25b73e..f3980a5 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
@@ -220,6 +220,7 @@ public final class ContainerTestHelper {
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.WriteChunk);
+    request.setContainerID(blockID.getContainerID());
     request.setWriteChunk(writeRequest);
     request.setTraceID(UUID.randomUUID().toString());
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
@@ -261,6 +262,7 @@ public final class ContainerTestHelper {
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.PutSmallFile);
+    request.setContainerID(blockID.getContainerID());
     request.setPutSmallFile(smallFileRequest);
     request.setTraceID(UUID.randomUUID().toString());
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
@@ -279,6 +281,7 @@ public final class ContainerTestHelper {
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.GetSmallFile);
+    request.setContainerID(getKey.getGetKey().getBlockID().getContainerID());
     request.setGetSmallFile(smallFileRequest);
     request.setTraceID(UUID.randomUUID().toString());
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
@@ -308,6 +311,7 @@ public final class ContainerTestHelper {
     ContainerCommandRequestProto.Builder newRequest =
         ContainerCommandRequestProto.newBuilder();
     newRequest.setCmdType(ContainerProtos.Type.ReadChunk);
+    newRequest.setContainerID(readRequest.getBlockID().getContainerID());
     newRequest.setReadChunk(readRequest);
     newRequest.setTraceID(UUID.randomUUID().toString());
     newRequest.setDatanodeUuid(pipeline.getLeader().getUuidString());
@@ -340,6 +344,7 @@ public final class ContainerTestHelper {
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.DeleteChunk);
+    request.setContainerID(writeRequest.getBlockID().getContainerID());
     request.setDeleteChunk(deleteRequest);
     request.setTraceID(UUID.randomUUID().toString());
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
@@ -356,15 +361,12 @@ public final class ContainerTestHelper {
       long containerID, Pipeline pipeline) throws IOException {
     LOG.trace("addContainer: {}", containerID);
 
-    ContainerProtos.CreateContainerRequestProto.Builder createRequest =
-        ContainerProtos.CreateContainerRequestProto
-            .newBuilder();
-    createRequest.setContainerID(containerID);
-
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.CreateContainer);
-    request.setCreateContainer(createRequest);
+    request.setContainerID(containerID);
+    request.setCreateContainer(
+        ContainerProtos.CreateContainerRequestProto.getDefaultInstance());
     request.setTraceID(UUID.randomUUID().toString());
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
 
@@ -385,7 +387,6 @@ public final class ContainerTestHelper {
       long containerID, Map<String, String> metaData) throws IOException {
     ContainerProtos.UpdateContainerRequestProto.Builder updateRequestBuilder =
         ContainerProtos.UpdateContainerRequestProto.newBuilder();
-    updateRequestBuilder.setContainerID(containerID);
     String[] keys = metaData.keySet().toArray(new String[]{});
     for(int i=0; i<keys.length; i++) {
       KeyValue.Builder kvBuilder = KeyValue.newBuilder();
@@ -399,6 +400,7 @@ public final class ContainerTestHelper {
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.UpdateContainer);
+    request.setContainerID(containerID);
     request.setUpdateContainer(updateRequestBuilder.build());
     request.setTraceID(UUID.randomUUID().toString());
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
@@ -412,14 +414,13 @@ public final class ContainerTestHelper {
    */
   public static ContainerCommandResponseProto
       getCreateContainerResponse(ContainerCommandRequestProto request) {
-    ContainerProtos.CreateContainerResponseProto.Builder createResponse =
-        ContainerProtos.CreateContainerResponseProto.newBuilder();
 
     ContainerCommandResponseProto.Builder response =
         ContainerCommandResponseProto.newBuilder();
     response.setCmdType(ContainerProtos.Type.CreateContainer);
     response.setTraceID(request.getTraceID());
-    response.setCreateContainer(createResponse.build());
+    response.setCreateContainer(
+        ContainerProtos.CreateContainerResponseProto.getDefaultInstance());
     response.setResult(ContainerProtos.Result.SUCCESS);
     return response.build();
   }
@@ -448,6 +449,7 @@ public final class ContainerTestHelper {
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.PutKey);
+    request.setContainerID(keyData.getContainerID());
     request.setPutKey(putRequest);
     request.setTraceID(UUID.randomUUID().toString());
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
@@ -474,6 +476,7 @@ public final class ContainerTestHelper {
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.GetKey);
+    request.setContainerID(blockID.getContainerID());
     request.setGetKey(getRequest);
     request.setTraceID(UUID.randomUUID().toString());
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
@@ -501,14 +504,16 @@ public final class ContainerTestHelper {
    */
   public static ContainerCommandRequestProto getDeleteKeyRequest(
       Pipeline pipeline, ContainerProtos.PutKeyRequestProto putKeyRequest) {
-    LOG.trace("deleteKey: name={}",
-        putKeyRequest.getKeyData().getBlockID());
+    ContainerProtos.DatanodeBlockID blockID = putKeyRequest.getKeyData()
+        .getBlockID();
+    LOG.trace("deleteKey: name={}", blockID);
     ContainerProtos.DeleteKeyRequestProto.Builder delRequest =
         ContainerProtos.DeleteKeyRequestProto.newBuilder();
-    delRequest.setBlockID(putKeyRequest.getKeyData().getBlockID());
+    delRequest.setBlockID(blockID);
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.DeleteKey);
+    request.setContainerID(blockID.getContainerID());
     request.setDeleteKey(delRequest);
     request.setTraceID(UUID.randomUUID().toString());
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
@@ -523,12 +528,12 @@ public final class ContainerTestHelper {
    */
   public static ContainerCommandRequestProto getCloseContainer(
       Pipeline pipeline, long containerID) {
-    ContainerProtos.CloseContainerRequestProto closeRequest =
-        ContainerProtos.CloseContainerRequestProto.newBuilder().
-            setContainerID(containerID).build();
     ContainerProtos.ContainerCommandRequestProto cmd =
-        ContainerCommandRequestProto.newBuilder().setCmdType(ContainerProtos
-            .Type.CloseContainer).setCloseContainer(closeRequest)
+        ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.CloseContainer)
+            .setContainerID(containerID)
+            .setCloseContainer(
+                ContainerProtos.CloseContainerRequestProto.getDefaultInstance())
             .setTraceID(UUID.randomUUID().toString())
             .setDatanodeUuid(pipeline.getLeader().getUuidString())
             .build();
@@ -545,14 +550,14 @@ public final class ContainerTestHelper {
   public static ContainerCommandRequestProto getRequestWithoutTraceId(
       Pipeline pipeline, long containerID) {
     Preconditions.checkNotNull(pipeline);
-    ContainerProtos.CloseContainerRequestProto closeRequest =
-            ContainerProtos.CloseContainerRequestProto.newBuilder().
-                setContainerID(containerID).build();
     ContainerProtos.ContainerCommandRequestProto cmd =
-            ContainerCommandRequestProto.newBuilder().setCmdType(ContainerProtos
-                    .Type.CloseContainer).setCloseContainer(closeRequest)
-                    .setDatanodeUuid(pipeline.getLeader().getUuidString())
-                    .build();
+        ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.CloseContainer)
+            .setContainerID(containerID)
+            .setCloseContainer(
+                ContainerProtos.CloseContainerRequestProto.getDefaultInstance())
+            .setDatanodeUuid(pipeline.getLeader().getUuidString())
+            .build();
     return cmd;
   }
 
@@ -566,10 +571,12 @@ public final class ContainerTestHelper {
     Preconditions.checkNotNull(pipeline);
     ContainerProtos.DeleteContainerRequestProto deleteRequest =
         ContainerProtos.DeleteContainerRequestProto.newBuilder().
-            setContainerID(containerID).
             setForceDelete(forceDelete).build();
     return ContainerCommandRequestProto.newBuilder()
         .setCmdType(ContainerProtos.Type.DeleteContainer)
+        .setContainerID(containerID)
+        .setDeleteContainer(
+            ContainerProtos.DeleteContainerRequestProto.getDefaultInstance())
         .setDeleteContainer(deleteRequest)
         .setTraceID(UUID.randomUUID().toString())
         .setDatanodeUuid(pipeline.getLeader().getUuidString())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/007e6f51/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
index 6d1c086..d67cf88 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
@@ -108,16 +108,14 @@ public class TestCloseContainerHandler {
 
   private long createContainer() {
     long testContainerId = ContainerTestHelper.getTestContainerID();
-    ContainerProtos.CreateContainerRequestProto createReq =
-        ContainerProtos.CreateContainerRequestProto.newBuilder()
-            .setContainerID(testContainerId)
-            .build();
 
     ContainerProtos.ContainerCommandRequestProto request =
         ContainerProtos.ContainerCommandRequestProto.newBuilder()
             .setCmdType(ContainerProtos.Type.CreateContainer)
+            .setContainerID(testContainerId)
             .setDatanodeUuid(DATANODE_UUID)
-            .setCreateContainer(createReq)
+            .setCreateContainer(ContainerProtos.CreateContainerRequestProto
+                .getDefaultInstance())
             .build();
 
     dispatcher.dispatch(request);
@@ -143,6 +141,7 @@ public class TestCloseContainerHandler {
       ContainerProtos.ContainerCommandRequestProto.Builder request =
           ContainerProtos.ContainerCommandRequestProto.newBuilder();
       request.setCmdType(ContainerProtos.Type.WriteChunk);
+      request.setContainerID(blockID.getContainerID());
       request.setWriteChunk(writeRequest);
       request.setTraceID(UUID.randomUUID().toString());
       request.setDatanodeUuid(pipeline.getLeader().getUuidString());
@@ -177,6 +176,7 @@ public class TestCloseContainerHandler {
     ContainerProtos.ContainerCommandRequestProto.Builder request =
         ContainerProtos.ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.PutKey);
+    request.setContainerID(blockID.getContainerID());
     request.setPutKey(putKeyRequestProto);
     request.setTraceID(UUID.randomUUID().toString());
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
@@ -213,6 +213,7 @@ public class TestCloseContainerHandler {
     ContainerProtos.ContainerCommandRequestProto.Builder request =
         ContainerProtos.ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.DeleteChunk);
+    request.setContainerID(blockID.getContainerID());
     request.setDeleteChunk(deleteChunkProto);
     request.setWriteChunk(writeRequest);
     request.setTraceID(UUID.randomUUID().toString());
@@ -242,13 +243,12 @@ public class TestCloseContainerHandler {
             .get(blockID.getLocalID()));
     Assert.assertTrue(
         keyData.getChunks().size() == chunkList.size());
-    ContainerProtos.CloseContainerRequestProto.Builder closeContainerProto =
-        ContainerProtos.CloseContainerRequestProto.newBuilder();
-    closeContainerProto.setContainerID(blockID.getContainerID());
     ContainerProtos.ContainerCommandRequestProto.Builder request =
         ContainerProtos.ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.CloseContainer);
-    request.setCloseContainer(closeContainerProto);
+    request.setContainerID(blockID.getContainerID());
+    request.setCloseContainer(
+        ContainerProtos.CloseContainerRequestProto.getDefaultInstance());
     request.setTraceID(UUID.randomUUID().toString());
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
     dispatcher.dispatch(request.build());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/007e6f51/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
index 93e7ef1..e757a7f 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
@@ -51,8 +51,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .CreateContainerRequestProto;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ReadChunkRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .WriteChunkRequestProto;
@@ -156,15 +154,14 @@ public class BenchMarkDatanodeDispatcher {
     FileUtils.deleteDirectory(new File(baseDir));
   }
 
-  private ContainerCommandRequestProto getCreateContainerCommand(long containerID) {
-    CreateContainerRequestProto.Builder createRequest =
-        CreateContainerRequestProto.newBuilder();
-    createRequest.setContainerID(containerID).build();
-
+  private ContainerCommandRequestProto getCreateContainerCommand(
+      long containerID) {
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.CreateContainer);
-    request.setCreateContainer(createRequest);
+    request.setContainerID(containerID);
+    request.setCreateContainer(
+        ContainerProtos.CreateContainerRequestProto.getDefaultInstance());
     request.setDatanodeUuid(datanodeUuid);
     request.setTraceID(containerID + "-trace");
     return request.build();
@@ -181,6 +178,7 @@ public class BenchMarkDatanodeDispatcher {
     ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto
         .newBuilder();
     request.setCmdType(ContainerProtos.Type.WriteChunk)
+        .setContainerID(blockID.getContainerID())
         .setTraceID(getBlockTraceID(blockID))
         .setDatanodeUuid(datanodeUuid)
         .setWriteChunk(writeChunkRequest);
@@ -193,9 +191,11 @@ public class BenchMarkDatanodeDispatcher {
         .newBuilder()
         .setBlockID(blockID.getDatanodeBlockIDProtobuf())
         .setChunkData(getChunkInfo(blockID, chunkName));
+
     ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto
         .newBuilder();
     request.setCmdType(ContainerProtos.Type.ReadChunk)
+        .setContainerID(blockID.getContainerID())
         .setTraceID(getBlockTraceID(blockID))
         .setDatanodeUuid(datanodeUuid)
         .setReadChunk(readChunkRequest);
@@ -219,9 +219,11 @@ public class BenchMarkDatanodeDispatcher {
     PutKeyRequestProto.Builder putKeyRequest = PutKeyRequestProto
         .newBuilder()
         .setKeyData(getKeyData(blockID, chunkKey));
+
     ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto
         .newBuilder();
     request.setCmdType(ContainerProtos.Type.PutKey)
+        .setContainerID(blockID.getContainerID())
         .setTraceID(getBlockTraceID(blockID))
         .setDatanodeUuid(datanodeUuid)
         .setPutKey(putKeyRequest);
@@ -234,6 +236,7 @@ public class BenchMarkDatanodeDispatcher {
     ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto
         .newBuilder()
         .setCmdType(ContainerProtos.Type.GetKey)
+        .setContainerID(blockID.getContainerID())
         .setTraceID(getBlockTraceID(blockID))
         .setDatanodeUuid(datanodeUuid)
         .setGetKey(readKeyRequest);


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


[08/50] hadoop git commit: HDFS-13761. Add toString Method to AclFeature Class. Contributed by Shweta.

Posted by eh...@apache.org.
HDFS-13761. Add toString Method to AclFeature Class. Contributed by Shweta.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/26864471
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/26864471
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/26864471

Branch: refs/heads/HDFS-12090
Commit: 26864471c24bf389ab8fc913decc3d069404688b
Parents: 849c45d
Author: Xiao Chen <xi...@apache.org>
Authored: Tue Jul 24 21:42:00 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Tue Jul 24 21:42:47 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/26864471/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
index 97d4759..6d546f8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
@@ -73,6 +73,11 @@ public class AclFeature implements INode.Feature, ReferenceCounter {
   }
 
   @Override
+  public String toString() {
+    return "AclFeature : " + Integer.toHexString(hashCode()) + " Size of entries : " + entries.length;
+  }
+
+  @Override
   public int hashCode() {
     return Arrays.hashCode(entries);
   }


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


[15/50] hadoop git commit: HDDS-288. Fix bugs in OpenContainerBlockMap. Contributed by Tsz Wo Nicholas Sze.

Posted by eh...@apache.org.
HDDS-288. Fix bugs in OpenContainerBlockMap. Contributed by Tsz Wo Nicholas Sze.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3c4fbc63
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3c4fbc63
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3c4fbc63

Branch: refs/heads/HDFS-12090
Commit: 3c4fbc635e8ed81cfbec00793a3767bb47f6d176
Parents: 3d3158c
Author: Nanda kumar <na...@apache.org>
Authored: Wed Jul 25 20:27:03 2018 +0530
Committer: Nanda kumar <na...@apache.org>
Committed: Wed Jul 25 20:27:03 2018 +0530

----------------------------------------------------------------------
 .../common/impl/OpenContainerBlockMap.java      | 139 +++++++------------
 .../container/keyvalue/KeyValueHandler.java     |  10 +-
 .../common/impl/TestCloseContainerHandler.java  |  30 ++--
 3 files changed, 73 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c4fbc63/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
index ab5f861..6a93c9d 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
@@ -21,22 +21,52 @@ package org.apache.hadoop.ozone.container.common.impl;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
+import java.util.Optional;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.stream.Collectors;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
 
 /**
+ * Map: containerId -> (localId -> KeyData).
+ * The outer container map does not entail locking for a better performance.
+ * The inner {@link KeyDataMap} is synchronized.
+ *
  * This class will maintain list of open keys per container when closeContainer
  * command comes, it should autocommit all open keys of a open container before
  * marking the container as closed.
  */
 public class OpenContainerBlockMap {
+  /**
+   * Map: localId -> KeyData.
+   *
+   * In order to support {@link #getAll()}, the update operations are synchronized.
+   */
+  static class KeyDataMap {
+    private final ConcurrentMap<Long, KeyData> blocks = new ConcurrentHashMap<>();
+
+    KeyData get(long localId) {
+      return blocks.get(localId);
+    }
+
+    synchronized int removeAndGetSize(long localId) {
+      blocks.remove(localId);
+      return blocks.size();
+    }
+
+    synchronized KeyData computeIfAbsent(long localId, Function<Long, KeyData> f) {
+      return blocks.computeIfAbsent(localId, f);
+    }
+
+    synchronized List<KeyData> getAll() {
+      return new ArrayList<>(blocks.values());
+    }
+  }
 
   /**
    * TODO : We may construct the openBlockMap by reading the Block Layout
@@ -46,89 +76,36 @@ public class OpenContainerBlockMap {
    *
    * For now, we will track all open blocks of a container in the blockMap.
    */
-  private final ConcurrentHashMap<Long, HashMap<Long, KeyData>>
-      openContainerBlockMap;
+  private final ConcurrentMap<Long, KeyDataMap> containers = new ConcurrentHashMap<>();
 
   /**
-   * Constructs OpenContainerBlockMap.
-   */
-  public OpenContainerBlockMap() {
-     openContainerBlockMap = new ConcurrentHashMap<>();
-  }
-  /**
    * Removes the Container matching with specified containerId.
    * @param containerId containerId
    */
   public void removeContainer(long containerId) {
     Preconditions
         .checkState(containerId >= 0, "Container Id cannot be negative.");
-    openContainerBlockMap.computeIfPresent(containerId, (k, v) -> null);
-  }
-
-  /**
-   * updates the chunkInfoList in case chunk is added or deleted
-   * @param blockID id of the block.
-   * @param info - Chunk Info
-   * @param remove if true, deletes the chunkInfo list otherwise appends to the
-   *               chunkInfo List
-   * @throws IOException
-   */
-  public synchronized void updateOpenKeyMap(BlockID blockID,
-      ContainerProtos.ChunkInfo info, boolean remove) throws IOException {
-    if (remove) {
-      deleteChunkFromMap(blockID, info);
-    } else {
-      addChunkToMap(blockID, info);
-    }
+    containers.remove(containerId);
   }
 
-  private KeyData getKeyData(ContainerProtos.ChunkInfo info, BlockID blockID)
-      throws IOException {
-    KeyData keyData = new KeyData(blockID);
-    keyData.addMetadata("TYPE", "KEY");
-    keyData.addChunk(info);
-    return keyData;
-  }
-
-  private void addChunkToMap(BlockID blockID, ContainerProtos.ChunkInfo info)
-      throws IOException {
+  public void addChunk(BlockID blockID, ChunkInfo info) {
     Preconditions.checkNotNull(info);
-    long containerId = blockID.getContainerID();
-    long localID = blockID.getLocalID();
-
-    KeyData keyData = openContainerBlockMap.computeIfAbsent(containerId,
-        emptyMap -> new LinkedHashMap<Long, KeyData>())
-        .putIfAbsent(localID, getKeyData(info, blockID));
-    // KeyData != null means the block already exist
-    if (keyData != null) {
-      HashMap<Long, KeyData> keyDataSet =
-          openContainerBlockMap.get(containerId);
-      keyDataSet.putIfAbsent(blockID.getLocalID(), getKeyData(info, blockID));
-      keyDataSet.computeIfPresent(blockID.getLocalID(), (key, value) -> {
-        value.addChunk(info);
-        return value;
-      });
-    }
+    containers.computeIfAbsent(blockID.getContainerID(), id -> new KeyDataMap())
+        .computeIfAbsent(blockID.getLocalID(), id -> new KeyData(blockID))
+        .addChunk(info);
   }
 
   /**
-   * removes the chunks from the chunkInfo list for the given block.
+   * Removes the chunk from the chunkInfo list for the given block.
    * @param blockID id of the block
    * @param chunkInfo chunk info.
    */
-  private synchronized void deleteChunkFromMap(BlockID blockID,
-      ContainerProtos.ChunkInfo chunkInfo) {
+  public void removeChunk(BlockID blockID, ChunkInfo chunkInfo) {
     Preconditions.checkNotNull(chunkInfo);
     Preconditions.checkNotNull(blockID);
-    HashMap<Long, KeyData> keyDataMap =
-        openContainerBlockMap.get(blockID.getContainerID());
-    if (keyDataMap != null) {
-      long localId = blockID.getLocalID();
-      KeyData keyData = keyDataMap.get(localId);
-      if (keyData != null) {
-        keyData.removeChunk(chunkInfo);
-      }
-    }
+    Optional.ofNullable(containers.get(blockID.getContainerID()))
+        .map(blocks -> blocks.get(blockID.getLocalID()))
+        .ifPresent(keyData -> keyData.removeChunk(chunkInfo));
   }
 
   /**
@@ -137,31 +114,23 @@ public class OpenContainerBlockMap {
    * @return List of open Keys(blocks)
    */
   public List<KeyData> getOpenKeys(long containerId) {
-    HashMap<Long, KeyData> keyDataHashMap =
-        openContainerBlockMap.get(containerId);
-    return keyDataHashMap == null ? null :
-        keyDataHashMap.values().stream().collect(Collectors.toList());
+    return Optional.ofNullable(containers.get(containerId))
+        .map(KeyDataMap::getAll)
+        .orElseGet(Collections::emptyList);
   }
 
   /**
    * removes the block from the block map.
    * @param blockID
    */
-  public synchronized void removeFromKeyMap(BlockID blockID) {
+  public void removeFromKeyMap(BlockID blockID) {
     Preconditions.checkNotNull(blockID);
-    HashMap<Long, KeyData> keyDataMap =
-        openContainerBlockMap.get(blockID.getContainerID());
-    if (keyDataMap != null) {
-      keyDataMap.remove(blockID.getLocalID());
-      if (keyDataMap.size() == 0) {
-        removeContainer(blockID.getContainerID());
-      }
-    }
+    containers.computeIfPresent(blockID.getContainerID(), (containerId, blocks)
+        -> blocks.removeAndGetSize(blockID.getLocalID()) == 0? null: blocks);
   }
 
   @VisibleForTesting
-  public ConcurrentHashMap<Long,
-      HashMap<Long, KeyData>> getContainerOpenKeyMap() {
-    return openContainerBlockMap;
+  KeyDataMap getKeyDataMap(long containerId) {
+    return containers.get(containerId);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c4fbc63/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index 4123dc8..b08e128 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -435,10 +435,8 @@ public class KeyValueHandler extends Handler {
     long containerId = kvContainer.getContainerData().getContainerID();
     List<KeyData> pendingKeys =
         this.openContainerBlockMap.getOpenKeys(containerId);
-    if (pendingKeys != null) {
-      for (KeyData keyData : pendingKeys) {
-        commitKey(keyData, kvContainer);
-      }
+    for(KeyData keyData : pendingKeys) {
+      commitKey(keyData, kvContainer);
     }
   }
 
@@ -598,7 +596,7 @@ public class KeyValueHandler extends Handler {
       Preconditions.checkNotNull(chunkInfo);
 
       chunkManager.deleteChunk(kvContainer, blockID, chunkInfo);
-      openContainerBlockMap.updateOpenKeyMap(blockID, chunkInfoProto, true);
+      openContainerBlockMap.removeChunk(blockID, chunkInfoProto);
     } catch (StorageContainerException ex) {
       return ContainerUtils.logAndReturnError(LOG, ex, request);
     } catch (IOException ex) {
@@ -648,7 +646,7 @@ public class KeyValueHandler extends Handler {
             .getChunkData().getLen());
         // the openContainerBlockMap should be updated only while writing data
         // not during COMMIT_STAGE of handling write chunk request.
-        openContainerBlockMap.updateOpenKeyMap(blockID, chunkInfoProto, false);
+        openContainerBlockMap.addChunk(blockID, chunkInfoProto);
       }
     } catch (StorageContainerException ex) {
       return ContainerUtils.logAndReturnError(LOG, ex, request);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c4fbc63/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
index 3ab593e..6d1c086 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
@@ -162,9 +162,9 @@ public class TestCloseContainerHandler {
     Pipeline pipeline = createSingleNodePipeline();
     List<ChunkInfo> chunkList = writeChunkBuilder(blockID, pipeline, 3);
     // the key should exist in the map
-    Assert.assertTrue(
-        openContainerBlockMap.getContainerOpenKeyMap().get(testContainerID)
-            .containsKey(blockID.getLocalID()));
+    Assert.assertNotNull(
+        openContainerBlockMap.getKeyDataMap(testContainerID)
+            .get(blockID.getLocalID()));
     KeyData keyData = new KeyData(blockID);
     List<ContainerProtos.ChunkInfo> chunkProtoList = new LinkedList<>();
     for (ChunkInfo i : chunkList) {
@@ -184,7 +184,7 @@ public class TestCloseContainerHandler {
 
     //the open key should be removed from Map
     Assert.assertNull(
-        openContainerBlockMap.getContainerOpenKeyMap().get(testContainerID));
+        openContainerBlockMap.getKeyDataMap(testContainerID));
   }
 
   @Test
@@ -196,11 +196,11 @@ public class TestCloseContainerHandler {
     Pipeline pipeline = createSingleNodePipeline();
     List<ChunkInfo> chunkList = writeChunkBuilder(blockID, pipeline, 3);
     // the key should exist in the map
+    Assert.assertNotNull(
+        openContainerBlockMap.getKeyDataMap(testContainerID)
+            .get(blockID.getLocalID()));
     Assert.assertTrue(
-        openContainerBlockMap.getContainerOpenKeyMap().get(testContainerID)
-            .containsKey(blockID.getLocalID()));
-    Assert.assertTrue(
-        openContainerBlockMap.getContainerOpenKeyMap().get(testContainerID)
+        openContainerBlockMap.getKeyDataMap(testContainerID)
             .get(blockID.getLocalID()).getChunks().size() == 3);
     ContainerProtos.DeleteChunkRequestProto.Builder deleteChunkProto =
         ContainerProtos.DeleteChunkRequestProto.newBuilder();
@@ -219,7 +219,7 @@ public class TestCloseContainerHandler {
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
     dispatcher.dispatch(request.build());
     Assert.assertTrue(
-        openContainerBlockMap.getContainerOpenKeyMap().get(testContainerID)
+        openContainerBlockMap.getKeyDataMap(testContainerID)
             .get(blockID.getLocalID()).getChunks().size() == 2);
 
   }
@@ -234,12 +234,12 @@ public class TestCloseContainerHandler {
     List<ChunkInfo> chunkList = writeChunkBuilder(blockID, pipeline, 3);
 
     Container container = containerSet.getContainer(testContainerID);
-    KeyData keyData = openContainerBlockMap.getContainerOpenKeyMap().
-        get(testContainerID).get(blockID.getLocalID());
+    KeyData keyData = openContainerBlockMap.
+        getKeyDataMap(testContainerID).get(blockID.getLocalID());
     // the key should exist in the map
-    Assert.assertTrue(
-        openContainerBlockMap.getContainerOpenKeyMap().get(testContainerID)
-            .containsKey(blockID.getLocalID()));
+    Assert.assertNotNull(
+        openContainerBlockMap.getKeyDataMap(testContainerID)
+            .get(blockID.getLocalID()));
     Assert.assertTrue(
         keyData.getChunks().size() == chunkList.size());
     ContainerProtos.CloseContainerRequestProto.Builder closeContainerProto =
@@ -253,7 +253,7 @@ public class TestCloseContainerHandler {
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
     dispatcher.dispatch(request.build());
     Assert.assertNull(
-        openContainerBlockMap.getContainerOpenKeyMap().get(testContainerID));
+        openContainerBlockMap.getKeyDataMap(testContainerID));
     // Make sure the key got committed
     Assert.assertNotNull(handler.getKeyManager().getKey(container, blockID));
   }


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