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 ha...@apache.org on 2018/05/30 21:11:51 UTC

[01/50] [abbrv] hadoop git commit: HDDS-45. Removal of old OzoneRestClient. Contributed by Lokesh Jain. [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/HDDS-48 978eaf102 -> 6cd19b45e (forced update)


http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneRestWithMiniCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneRestWithMiniCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneRestWithMiniCluster.java
index 5b67657..a9b8175 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneRestWithMiniCluster.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneRestWithMiniCluster.java
@@ -23,23 +23,31 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 import static org.apache.hadoop.ozone.OzoneConsts.CHUNK_SIZE;
 import static org.junit.Assert.*;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.RandomStringUtils;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.web.client.OzoneRestClient;
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.web.client.OzoneBucket;
-import org.apache.hadoop.ozone.web.client.OzoneVolume;
-import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.junit.rules.Timeout;
 
+import java.io.IOException;
+import java.io.InputStream;
+
 /**
  * End-to-end testing of Ozone REST operations.
  */
@@ -52,7 +60,9 @@ public class TestOzoneRestWithMiniCluster {
 
   private static MiniOzoneCluster cluster;
   private static OzoneConfiguration conf;
-  private static OzoneRestClient ozoneClient;
+  private static ClientProtocol client;
+  private static ReplicationFactor replicationFactor = ReplicationFactor.ONE;
+  private static ReplicationType replicationType = ReplicationType.STAND_ALONE;
 
   @Rule
   public ExpectedException exception = ExpectedException.none();
@@ -62,180 +72,125 @@ public class TestOzoneRestWithMiniCluster {
     conf = new OzoneConfiguration();
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
-    int port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
-    ozoneClient = new OzoneRestClient(
-        String.format("http://localhost:%d", port));
-    ozoneClient.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
+    client = new RpcClient(conf);
   }
 
   @AfterClass
-  public static void shutdown() throws InterruptedException {
+  public static void shutdown() throws InterruptedException, IOException {
     if (cluster != null) {
       cluster.shutdown();
     }
-    IOUtils.cleanupWithLogger(null, ozoneClient);
+    client.close();
   }
 
   @Test
   public void testCreateAndGetVolume() throws Exception {
-    String volumeName = nextId("volume");
-    OzoneVolume volume = ozoneClient.createVolume(volumeName, "bilbo", "100TB");
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
-    volume = ozoneClient.getVolume(volumeName);
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
+    createAndGetVolume();
   }
 
   @Test
   public void testCreateAndGetBucket() throws Exception {
-    String volumeName = nextId("volume");
-    String bucketName = nextId("bucket");
-    OzoneVolume volume = ozoneClient.createVolume(volumeName, "bilbo", "100TB");
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
-    OzoneBucket bucket = volume.createBucket(bucketName);
-    assertNotNull(bucket);
-    assertEquals(bucketName, bucket.getBucketName());
-    bucket = volume.getBucket(bucketName);
-    assertNotNull(bucket);
-    assertEquals(bucketName, bucket.getBucketName());
+    OzoneVolume volume = createAndGetVolume();
+    createAndGetBucket(volume);
   }
 
   @Test
   public void testPutAndGetKey() throws Exception {
-    String volumeName = nextId("volume");
-    String bucketName = nextId("bucket");
     String keyName = nextId("key");
     String keyData = nextId("data");
-    OzoneVolume volume = ozoneClient.createVolume(volumeName, "bilbo", "100TB");
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
-    OzoneBucket bucket = volume.createBucket(bucketName);
-    assertNotNull(bucket);
-    assertEquals(bucketName, bucket.getBucketName());
-    bucket.putKey(keyName, keyData);
-    assertEquals(keyData, bucket.getKey(keyName));
+    OzoneVolume volume = createAndGetVolume();
+    OzoneBucket bucket = createAndGetBucket(volume);
+    putKey(bucket, keyName, keyData);
+  }
+
+  private void putKey(OzoneBucket bucket, String keyName, String keyData) throws IOException {
+    try (
+        OzoneOutputStream ozoneOutputStream = bucket
+            .createKey(keyName, 0, replicationType, replicationFactor);
+        InputStream inputStream = IOUtils.toInputStream(keyData, UTF_8)) {
+      IOUtils.copy(inputStream, ozoneOutputStream);
+    }
+    try (
+        InputStream inputStream = IOUtils.toInputStream(keyData, UTF_8);
+        OzoneInputStream ozoneInputStream = bucket.readKey(keyName)) {
+      IOUtils.contentEquals(ozoneInputStream, inputStream);
+    }
   }
 
   @Test
   public void testPutAndGetEmptyKey() throws Exception {
-    String volumeName = nextId("volume");
-    String bucketName = nextId("bucket");
     String keyName = nextId("key");
     String keyData = "";
-    OzoneVolume volume = ozoneClient.createVolume(volumeName, "bilbo", "100TB");
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
-    OzoneBucket bucket = volume.createBucket(bucketName);
-    assertNotNull(bucket);
-    assertEquals(bucketName, bucket.getBucketName());
-    bucket.putKey(keyName, keyData);
-    assertEquals(keyData, bucket.getKey(keyName));
+    OzoneVolume volume = createAndGetVolume();
+    OzoneBucket bucket = createAndGetBucket(volume);
+    putKey(bucket, keyName, keyData);
   }
 
   @Test
   public void testPutAndGetMultiChunkKey() throws Exception {
-    String volumeName = nextId("volume");
-    String bucketName = nextId("bucket");
     String keyName = nextId("key");
     int keyDataLen = 3 * CHUNK_SIZE;
     String keyData = buildKeyData(keyDataLen);
-    OzoneVolume volume = ozoneClient.createVolume(volumeName, "bilbo", "100TB");
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
-    OzoneBucket bucket = volume.createBucket(bucketName);
-    assertNotNull(bucket);
-    assertEquals(bucketName, bucket.getBucketName());
-    bucket.putKey(keyName, keyData);
-    assertEquals(keyData, bucket.getKey(keyName));
+    OzoneVolume volume = createAndGetVolume();
+    OzoneBucket bucket = createAndGetBucket(volume);
+    putKey(bucket, keyName, keyData);
   }
 
   @Test
   public void testPutAndGetMultiChunkKeyLastChunkPartial() throws Exception {
-    String volumeName = nextId("volume");
-    String bucketName = nextId("bucket");
     String keyName = nextId("key");
     int keyDataLen = (int)(2.5 * CHUNK_SIZE);
     String keyData = buildKeyData(keyDataLen);
-    OzoneVolume volume = ozoneClient.createVolume(volumeName, "bilbo", "100TB");
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
-    OzoneBucket bucket = volume.createBucket(bucketName);
-    assertNotNull(bucket);
-    assertEquals(bucketName, bucket.getBucketName());
-    bucket.putKey(keyName, keyData);
-    assertEquals(keyData, bucket.getKey(keyName));
+    OzoneVolume volume = createAndGetVolume();
+    OzoneBucket bucket = createAndGetBucket(volume);
+    putKey(bucket, keyName, keyData);
   }
 
   @Test
   public void testReplaceKey() throws Exception {
-    String volumeName = nextId("volume");
-    String bucketName = nextId("bucket");
     String keyName = nextId("key");
     int keyDataLen = (int)(2.5 * CHUNK_SIZE);
     String keyData = buildKeyData(keyDataLen);
-    OzoneVolume volume = ozoneClient.createVolume(volumeName, "bilbo", "100TB");
-    assertNotNull(volume);
-    assertEquals(volumeName, volume.getVolumeName());
-    assertEquals(ozoneClient.getUserAuth(), volume.getCreatedby());
-    assertEquals("bilbo", volume.getOwnerName());
-    assertNotNull(volume.getQuota());
-    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
-        volume.getQuota().sizeInBytes());
-    OzoneBucket bucket = volume.createBucket(bucketName);
-    assertNotNull(bucket);
-    assertEquals(bucketName, bucket.getBucketName());
-    bucket.putKey(keyName, keyData);
-    assertEquals(keyData, bucket.getKey(keyName));
+    OzoneVolume volume = createAndGetVolume();
+    OzoneBucket bucket = createAndGetBucket(volume);
+    putKey(bucket, keyName, keyData);
 
     // Replace key with data consisting of fewer chunks.
     keyDataLen = (int)(1.5 * CHUNK_SIZE);
     keyData = buildKeyData(keyDataLen);
-    bucket.putKey(keyName, keyData);
-    assertEquals(keyData, bucket.getKey(keyName));
+    putKey(bucket, keyName, keyData);
 
     // Replace key with data consisting of more chunks.
     keyDataLen = (int)(3.5 * CHUNK_SIZE);
     keyData = buildKeyData(keyDataLen);
-    bucket.putKey(keyName, keyData);
-    assertEquals(keyData, bucket.getKey(keyName));
+    putKey(bucket, keyName, keyData);
+  }
+
+  private OzoneVolume createAndGetVolume() throws IOException {
+    String volumeName = nextId("volume");
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .setAdmin("hdfs")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume volume = client.getVolumeDetails(volumeName);
+    assertEquals(volumeName, volume.getName());
+    assertNotNull(volume);
+    assertEquals("bilbo", volume.getOwner());
+    assertNotNull(volume.getQuota());
+    assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
+        volume.getQuota());
+    return volume;
+  }
+
+  private OzoneBucket createAndGetBucket(OzoneVolume vol) throws IOException {
+    String bucketName = nextId("bucket");
+    vol.createBucket(bucketName);
+    OzoneBucket bucket = vol.getBucket(bucketName);
+    assertNotNull(bucket);
+    assertEquals(bucketName, bucket.getName());
+    return bucket;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
index 64e5f71..684f4d3 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
@@ -17,12 +17,17 @@
  */
 package org.apache.hadoop.ozone.web.client;
 
-import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.BucketArgs;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -36,7 +41,10 @@ import org.junit.rules.Timeout;
 import java.io.IOException;
 import java.net.URISyntaxException;
 import java.text.ParseException;
+import java.util.Arrays;
+import java.util.Iterator;
 import java.util.List;
+import java.util.stream.Collectors;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -54,7 +62,7 @@ public class TestBuckets {
   public Timeout testTimeout = new Timeout(300000);
 
   private static MiniOzoneCluster cluster = null;
-  private static OzoneRestClient ozoneRestClient = null;
+  private static ClientProtocol client = null;
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -76,11 +84,10 @@ public class TestBuckets {
         OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
 
     conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
-    cluster = MiniOzoneCluster.newBuilder(conf).build();
-    final int port = cluster.getHddsDatanodes().get(0).getDatanodeDetails()
-        .getOzoneRestPort();
-    ozoneRestClient = new OzoneRestClient(
-        String.format("http://localhost:%d", port));
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(3)
+        .build();
+    client = new RpcClient(conf);
   }
 
   /**
@@ -95,110 +102,151 @@ public class TestBuckets {
 
   @Test
   public void testCreateBucket() throws Exception {
-    runTestCreateBucket(ozoneRestClient);
+    runTestCreateBucket(client);
   }
 
-  static void runTestCreateBucket(OzoneRestClient client)
+  static void runTestCreateBucket(ClientProtocol client)
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth("hdfs");
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .setAdmin("hdfs")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = client.getVolumeDetails(volumeName);
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
 
     // create 10 buckets under same volume
     for (int x = 0; x < 10; x++) {
       long currentTime = Time.now();
       String bucketName = OzoneUtils.getRequestID().toLowerCase();
-      OzoneBucket bucket =
-          vol.createBucket(bucketName, acls, StorageType.DEFAULT);
-      assertEquals(bucket.getBucketName(), bucketName);
+
+      List<OzoneAcl> aclList =
+          Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
+              .collect(Collectors.toList());
+      BucketArgs bucketArgs = BucketArgs.newBuilder()
+          .setAcls(aclList)
+          .build();
+      vol.createBucket(bucketName, bucketArgs);
+      OzoneBucket bucket = vol.getBucket(bucketName);
+      assertEquals(bucket.getName(), bucketName);
 
       // verify the bucket creation time
-      assertTrue((OzoneUtils.formatDate(bucket.getCreatedOn())
-          / 1000) >= (currentTime / 1000));
+      assertTrue((bucket.getCreationTime() / 1000) >= (currentTime / 1000));
     }
     client.close();
 
-    assertEquals(vol.getVolumeName(), volumeName);
-    assertEquals(vol.getCreatedby(), "hdfs");
-    assertEquals(vol.getOwnerName(), "bilbo");
-    assertEquals(vol.getQuota().getUnit(), OzoneQuota.Units.TB);
-    assertEquals(vol.getQuota().getSize(), 100);
+    assertEquals(vol.getName(), volumeName);
+    assertEquals(vol.getAdmin(), "hdfs");
+    assertEquals(vol.getOwner(), "bilbo");
+    assertEquals(vol.getQuota(), OzoneQuota.parseQuota("100TB").sizeInBytes());
 
     // Test create a bucket with invalid bucket name,
     // not use Rule here because the test method is static.
     try {
       String invalidBucketName = "#" + OzoneUtils.getRequestID().toLowerCase();
-      vol.createBucket(invalidBucketName, acls, StorageType.DEFAULT);
+      vol.createBucket(invalidBucketName);
       fail("Except the bucket creation to be failed because the"
           + " bucket name starts with an invalid char #");
     } catch (Exception e) {
-      assertTrue(e instanceof OzoneClientException);
-      assertTrue(e.getMessage().contains("Bucket or Volume name"
-          + " has an unsupported character : #"));
+      assertTrue(e.getMessage()
+          .contains("Bucket or Volume name has an unsupported character : #"));
     }
   }
 
   @Test
   public void testAddBucketAcls() throws Exception {
-    runTestAddBucketAcls(ozoneRestClient);
+    runTestAddBucketAcls(client);
   }
 
-  static void runTestAddBucketAcls(OzoneRestClient client)
+  static void runTestAddBucketAcls(ClientProtocol client)
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth("hdfs");
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .setAdmin("hdfs")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = client.getVolumeDetails(volumeName);
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
     String bucketName = OzoneUtils.getRequestID().toLowerCase();
     vol.createBucket(bucketName);
-    vol.addAcls(bucketName, acls);
+    OzoneBucket bucket = vol.getBucket(bucketName);
+    List<OzoneAcl> aclList =
+        Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
+            .collect(Collectors.toList());
+    int numAcls = bucket.getAcls().size();
+    bucket.addAcls(aclList);
     OzoneBucket updatedBucket = vol.getBucket(bucketName);
-    assertEquals(updatedBucket.getAcls().size(), 2);
+    assertEquals(updatedBucket.getAcls().size(), 2 + numAcls);
     // verify if the creation time is missing after update operation
     assertTrue(
-        (OzoneUtils.formatDate(updatedBucket.getCreatedOn()) / 1000) >= 0);
+        (updatedBucket.getCreationTime()) / 1000 >= 0);
     client.close();
   }
 
   @Test
   public void testRemoveBucketAcls() throws Exception {
-    runTestRemoveBucketAcls(ozoneRestClient);
+    runTestRemoveBucketAcls(client);
   }
 
-  static void runTestRemoveBucketAcls(OzoneRestClient client)
+  static void runTestRemoveBucketAcls(ClientProtocol client)
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth("hdfs");
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .setAdmin("hdfs")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = client.getVolumeDetails(volumeName);
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
     String bucketName = OzoneUtils.getRequestID().toLowerCase();
-    OzoneBucket bucket = vol.createBucket(bucketName, acls);
-    assertEquals(bucket.getAcls().size(), 2);
-    vol.removeAcls(bucketName, acls);
+    List<OzoneAcl> aclList =
+        Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
+            .collect(Collectors.toList());
+    vol.createBucket(bucketName);
+    OzoneBucket bucket = vol.getBucket(bucketName);
+    int numAcls = bucket.getAcls().size();
+    bucket.addAcls(aclList);
+    assertEquals(bucket.getAcls().size(), 2 + numAcls);
+    bucket.removeAcls(aclList);
     OzoneBucket updatedBucket = vol.getBucket(bucketName);
 
     // We removed all acls
-    assertEquals(updatedBucket.getAcls().size(), 0);
+    assertEquals(updatedBucket.getAcls().size(), numAcls);
     // verify if the creation time is missing after update operation
     assertTrue(
-        (OzoneUtils.formatDate(updatedBucket.getCreatedOn()) / 1000) >= 0);
+        (updatedBucket.getCreationTime() / 1000) >= 0);
     client.close();
   }
 
   @Test
   public void testDeleteBucket() throws OzoneException, IOException {
-    runTestDeleteBucket(ozoneRestClient);
+    runTestDeleteBucket(client);
   }
 
-  static void runTestDeleteBucket(OzoneRestClient client)
+  static void runTestDeleteBucket(ClientProtocol client)
       throws OzoneException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth("hdfs");
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .setAdmin("hdfs")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = client.getVolumeDetails(volumeName);
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
     String bucketName = OzoneUtils.getRequestID().toLowerCase();
-    vol.createBucket(bucketName, acls);
+    List<OzoneAcl> aclList =
+        Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
+            .collect(Collectors.toList());
+    BucketArgs bucketArgs = BucketArgs.newBuilder()
+        .setAcls(aclList)
+        .build();
+    vol.createBucket(bucketName, bucketArgs);
     vol.deleteBucket(bucketName);
     try {
       OzoneBucket updatedBucket = vol.getBucket(bucketName);
@@ -212,38 +260,57 @@ public class TestBuckets {
 
   @Test
   public void testListBucket() throws Exception {
-    runTestListBucket(ozoneRestClient);
+    runTestListBucket(client);
   }
 
-  static void runTestListBucket(OzoneRestClient client)
+  static void runTestListBucket(ClientProtocol client)
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth("hdfs");
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .setAdmin("hdfs")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = client.getVolumeDetails(volumeName);
     String[] acls = {"user:frodo:rw", "user:samwise:rw"};
+    List<OzoneAcl> aclList =
+        Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
+            .collect(Collectors.toList());
 
     long currentTime = Time.now();
     for (int x = 0; x < 10; x++) {
       String bucketName = "listbucket-test-" + x;
-      vol.createBucket(bucketName, acls);
+      BucketArgs bucketArgs = BucketArgs.newBuilder()
+          .setAcls(aclList)
+          .build();
+      vol.createBucket(bucketName, bucketArgs);
     }
-    List<OzoneBucket> bucketList = vol.listBuckets("100", null, null);
-    assertEquals(bucketList.size(), 10);
+    Iterator<OzoneBucket> bucketIterator = vol.listBuckets(null);
+    int count = 0;
 
-    for (OzoneBucket bucket : bucketList) {
-      assertTrue((OzoneUtils.formatDate(bucket.getCreatedOn())
+    while (bucketIterator.hasNext()) {
+      assertTrue((bucketIterator.next().getCreationTime()
           / 1000) >= (currentTime / 1000));
+      count++;
     }
+    assertEquals(count, 10);
 
-    bucketList = vol.listBuckets("3", null, null);
-    assertEquals(bucketList.size(), 3);
-
-    bucketList = vol.listBuckets("100", "listbucket-test-4", null);
-    assertEquals(bucketList.size(), 5);
+    bucketIterator = vol.listBuckets(null, "listbucket-test-4");
+    assertEquals(getSize(bucketIterator), 5);
 
-    bucketList = vol.listBuckets("100", null, "listbucket-test-3");
-    assertEquals(bucketList.size(), 1);
+    bucketIterator = vol.listBuckets(null, "listbucket-test-3");
+    assertEquals(getSize(bucketIterator), 6);
 
     client.close();
   }
+
+  private static int getSize(Iterator<OzoneBucket> bucketIterator) {
+    int count = 0;
+    while (bucketIterator.hasNext()) {
+      count++;
+      bucketIterator.next();
+    }
+    return count;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
index b913a86..9f80184 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.ozone.web.client;
 
 import org.apache.hadoop.ozone.RatisTestHelper;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -35,12 +36,12 @@ public class TestBucketsRatis {
   public Timeout testTimeout = new Timeout(300000);
 
   private static RatisTestHelper.RatisTestSuite suite;
-  private static OzoneRestClient ozoneRestClient;
+  private static ClientProtocol client;
 
   @BeforeClass
   public static void init() throws Exception {
     suite = new RatisTestHelper.RatisTestSuite(TestBucketsRatis.class);
-    ozoneRestClient = suite.newOzoneRestClient();
+    client = suite.newOzoneClient();
   }
 
   @AfterClass
@@ -52,25 +53,25 @@ public class TestBucketsRatis {
 
   @Test
   public void testCreateBucket() throws Exception {
-    TestBuckets.runTestCreateBucket(ozoneRestClient);
+    TestBuckets.runTestCreateBucket(client);
   }
 
   @Test
   public void testAddBucketAcls() throws Exception {
-    TestBuckets.runTestAddBucketAcls(ozoneRestClient);
+    TestBuckets.runTestAddBucketAcls(client);
   }
 
   @Test
   public void testRemoveBucketAcls() throws Exception {
-    TestBuckets.runTestRemoveBucketAcls(ozoneRestClient);
+    TestBuckets.runTestRemoveBucketAcls(client);
   }
 
   @Test
   public void testDeleteBucket() throws OzoneException, IOException {
-    TestBuckets.runTestDeleteBucket(ozoneRestClient);
+    TestBuckets.runTestDeleteBucket(client);
   }
   @Test
   public void testListBucket() throws Exception {
-    TestBuckets.runTestListBucket(ozoneRestClient);
+    TestBuckets.runTestListBucket(client);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
index 57d4287..f8df7dc 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
@@ -20,16 +20,29 @@ package org.apache.hadoop.ozone.web.client;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.collections.IteratorUtils;
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.commons.lang.math.RandomUtils;
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
-import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.ozone.client.BucketArgs;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
@@ -59,15 +72,18 @@ import org.junit.rules.Timeout;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
+import java.io.InputStream;
 import java.io.IOException;
 import java.net.URISyntaxException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -84,10 +100,13 @@ public class TestKeys {
   @Rule
   public Timeout testTimeout = new Timeout(300000);
 
+  private static OzoneConfiguration conf;
   private static MiniOzoneCluster ozoneCluster = null;
   private static String path;
-  private static OzoneRestClient ozoneRestClient = null;
+  private static ClientProtocol client = null;
   private static long currentTime;
+  private static ReplicationFactor replicationFactor = ReplicationFactor.ONE;
+  private static ReplicationType replicationType = ReplicationType.STAND_ALONE;
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -96,7 +115,7 @@ public class TestKeys {
    */
   @BeforeClass
   public static void init() throws Exception {
-    OzoneConfiguration conf = new OzoneConfiguration();
+    conf = new OzoneConfiguration();
 
     // Set short block deleting service interval to speed up deletions.
     conf.setTimeDuration(OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL,
@@ -105,12 +124,9 @@ public class TestKeys {
     path = GenericTestUtils.getTempPath(TestKeys.class.getSimpleName());
     Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
 
-    ozoneCluster = MiniOzoneCluster.newBuilder(conf).build();
+    ozoneCluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
     ozoneCluster.waitForClusterToBeReady();
-    final int port = ozoneCluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
-    ozoneRestClient = new OzoneRestClient(
-        String.format("http://localhost:%d", port));
+    client = new RpcClient(conf);
     currentTime = Time.now();
   }
 
@@ -166,7 +182,7 @@ public class TestKeys {
   }
 
   static class PutHelper {
-    private final OzoneRestClient client;
+    private final ClientProtocol client;
     private final String dir;
     private final String keyName;
 
@@ -174,11 +190,11 @@ public class TestKeys {
     private OzoneBucket bucket;
     private File file;
 
-    PutHelper(OzoneRestClient client, String dir) {
+    PutHelper(ClientProtocol client, String dir) {
       this(client, dir, OzoneUtils.getRequestID().toLowerCase());
     }
 
-    PutHelper(OzoneRestClient client, String dir, String key) {
+    PutHelper(ClientProtocol client, String dir, String key) {
       this.client = client;
       this.dir = dir;
       this.keyName = key;
@@ -202,111 +218,139 @@ public class TestKeys {
      * @return Returns the name of the new key that was created.
      * @throws OzoneException
      */
-    private KsmKeyArgs putKey() throws Exception {
+    private String putKey() throws Exception {
       String volumeName = OzoneUtils.getRequestID().toLowerCase();
-      client.setUserAuth("hdfs");
 
-      vol = client.createVolume(volumeName, "bilbo", "100TB");
+      VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+          .setOwner("bilbo")
+          .setQuota("100TB")
+          .setAdmin("hdfs")
+          .build();
+      client.createVolume(volumeName, volumeArgs);
+      vol = client.getVolumeDetails(volumeName);
       String[] acls = {"user:frodo:rw", "user:samwise:rw"};
 
       String bucketName = OzoneUtils.getRequestID().toLowerCase();
-      bucket = vol.createBucket(bucketName, acls, StorageType.DEFAULT);
+      List<OzoneAcl> aclList =
+          Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
+              .collect(Collectors.toList());
+      BucketArgs bucketArgs = BucketArgs.newBuilder()
+          .setAcls(aclList)
+          .build();
+      vol.createBucket(bucketName, bucketArgs);
+      bucket = vol.getBucket(bucketName);
 
       String fileName = OzoneUtils.getRequestID().toLowerCase();
 
       file = createRandomDataFile(dir, fileName, 1024);
 
-      bucket.putKey(keyName, file);
-      return new KsmKeyArgs.Builder()
-          .setKeyName(keyName)
-          .setVolumeName(volumeName)
-          .setBucketName(bucketName)
-          .setDataSize(1024)
-          .build();
+      try (
+          OzoneOutputStream ozoneOutputStream = bucket
+              .createKey(keyName, 0, replicationType, replicationFactor);
+          InputStream fileInputStream = new FileInputStream(file)) {
+        IOUtils.copy(fileInputStream, ozoneOutputStream);
+      }
+      return keyName;
     }
   }
 
   @Test
   public void testPutKey() throws Exception {
     // Test non-delimited keys
-    runTestPutKey(new PutHelper(ozoneRestClient, path));
+    runTestPutKey(new PutHelper(client, path));
     // Test key delimited by a random delimiter
     String delimiter = RandomStringUtils.randomAscii(1);
-    runTestPutKey(new PutHelper(ozoneRestClient, path,
+    runTestPutKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   static void runTestPutKey(PutHelper helper) throws Exception {
-    final OzoneRestClient client = helper.client;
+    final ClientProtocol client = helper.client;
     helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
-    List<OzoneKey> keyList = helper.getBucket().listKeys("100", null, null);
+    List<OzoneKey> keyList = client
+        .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
+            null, 10);
     Assert.assertEquals(1, keyList.size());
 
     // test list key using a more efficient call
     String newkeyName = OzoneUtils.getRequestID().toLowerCase();
-    client.putKey(helper.getVol().getVolumeName(),
-        helper.getBucket().getBucketName(), newkeyName, helper.getFile());
-    keyList = helper.getBucket().listKeys("100", null, null);
+    OzoneOutputStream ozoneOutputStream = client
+        .createKey(helper.getVol().getName(), helper.getBucket().getName(),
+            newkeyName, 0, replicationType, replicationFactor);
+    ozoneOutputStream.close();
+    keyList = client
+        .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
+            null, 10);
     Assert.assertEquals(2, keyList.size());
 
     // test new put key with invalid volume/bucket name
     try {
-      client.putKey("invalid-volume",
-          helper.getBucket().getBucketName(), newkeyName, helper.getFile());
+      ozoneOutputStream = client
+          .createKey("invalid-volume", helper.getBucket().getName(), newkeyName,
+              0, replicationType, replicationFactor);
+      ozoneOutputStream.close();
       fail("Put key should have thrown"
           + " when using invalid volume name.");
-    } catch (OzoneException e) {
+    } catch (IOException e) {
       GenericTestUtils.assertExceptionContains(
           Status.VOLUME_NOT_FOUND.toString(), e);
     }
 
     try {
-      client.putKey(helper.getVol().getVolumeName(), "invalid-bucket",
-          newkeyName, helper.getFile());
+      ozoneOutputStream = client
+          .createKey(helper.getVol().getName(), "invalid-bucket", newkeyName, 0,
+              replicationType, replicationFactor);
+      ozoneOutputStream.close();
       fail("Put key should have thrown "
           + "when using invalid bucket name.");
-    } catch (OzoneException e) {
+    } catch (IOException e) {
       GenericTestUtils.assertExceptionContains(
           Status.BUCKET_NOT_FOUND.toString(), e);
     }
   }
 
-  private static void restartDatanode(
-      MiniOzoneCluster cluster, int datanodeIdx, OzoneRestClient client)
+  private static void restartDatanode(MiniOzoneCluster cluster, int datanodeIdx)
       throws OzoneException, URISyntaxException {
     cluster.restartHddsDatanode(datanodeIdx);
-    // refresh the datanode endpoint uri after datanode restart
-    final int port = ozoneCluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
-    client.setEndPoint(String.format("http://localhost:%d", port));
   }
 
   @Test
   public void testPutAndGetKeyWithDnRestart() throws Exception {
     runTestPutAndGetKeyWithDnRestart(
-        new PutHelper(ozoneRestClient, path), ozoneCluster);
+        new PutHelper(client, path), ozoneCluster);
     String delimiter = RandomStringUtils.randomAscii(1);
     runTestPutAndGetKeyWithDnRestart(
-        new PutHelper(ozoneRestClient, path,
+        new PutHelper(client, path,
             getMultiPartKey(delimiter)), ozoneCluster);
   }
 
   static void runTestPutAndGetKeyWithDnRestart(
       PutHelper helper, MiniOzoneCluster cluster) throws Exception {
-    String keyName = helper.putKey().getKeyName();
+    String keyName = helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
 
     // restart the datanode
-    restartDatanode(cluster, 0, helper.client);
+    restartDatanode(cluster, 0);
+    // TODO: Try removing sleep and adding a join for the MiniOzoneCluster start
+    // The ozoneContainer is not started and its metrics are not initialized
+    // which leads to NullPointerException in Dispatcher.
+    Thread.sleep(1000);
+    ozoneCluster.waitForClusterToBeReady();
     // verify getKey after the datanode restart
     String newFileName = helper.dir + "/"
         + OzoneUtils.getRequestID().toLowerCase();
     Path newPath = Paths.get(newFileName);
-
-    helper.getBucket().getKey(keyName, newPath);
+    try (
+        FileOutputStream newOutputStream = new FileOutputStream(
+            newPath.toString());
+        OzoneInputStream ozoneInputStream = helper.client
+            .getKey(helper.getVol().getName(), helper.getBucket().getName(),
+                keyName)) {
+      IOUtils.copy(ozoneInputStream, newOutputStream);
+    }
 
     try (
         FileInputStream original = new FileInputStream(helper.getFile());
@@ -321,16 +365,16 @@ public class TestKeys {
 
   @Test
   public void testPutAndGetKey() throws Exception {
-    runTestPutAndGetKey(new PutHelper(ozoneRestClient, path));
+    runTestPutAndGetKey(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAscii(1);
-    runTestPutAndGetKey(new PutHelper(ozoneRestClient, path,
+    runTestPutAndGetKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   static void runTestPutAndGetKey(PutHelper helper) throws Exception {
-    final OzoneRestClient client = helper.client;
+    final ClientProtocol client = helper.client;
 
-    String keyName = helper.putKey().getKeyName();
+    String keyName = helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
 
@@ -342,10 +386,22 @@ public class TestKeys {
     Path newPath1 = Paths.get(newFileName1);
     Path newPath2 = Paths.get(newFileName2);
 
-    helper.getBucket().getKey(keyName, newPath1);
+    try (
+        FileOutputStream newOutputStream = new FileOutputStream(
+            newPath1.toString());
+        OzoneInputStream ozoneInputStream = helper.getBucket()
+            .readKey(keyName)) {
+      IOUtils.copy(ozoneInputStream, newOutputStream);
+    }
+
     // test get key using a more efficient call
-    client.getKey(helper.getVol().getVolumeName(),
-        helper.getBucket().getBucketName(), keyName, newPath2);
+    try (
+        FileOutputStream newOutputStream = new FileOutputStream(
+            newPath2.toString());
+        OzoneInputStream ozoneInputStream = helper.getBucket()
+            .readKey(keyName)) {
+      IOUtils.copy(ozoneInputStream, newOutputStream);
+    }
 
     try (FileInputStream original = new FileInputStream(helper.getFile());
         FileInputStream downloaded1 = new FileInputStream(newPath1.toFile());
@@ -363,19 +419,17 @@ public class TestKeys {
 
       // test new get key with invalid volume/bucket name
       try {
-        client.getKey("invalid-volume", helper.getBucket().getBucketName(),
-            keyName, newPath1);
+        client.getKey("invalid-volume", helper.getBucket().getName(), keyName);
         fail("Get key should have thrown " + "when using invalid volume name.");
-      } catch (OzoneException e) {
+      } catch (IOException e) {
         GenericTestUtils
             .assertExceptionContains(Status.KEY_NOT_FOUND.toString(), e);
       }
 
       try {
-        client.getKey(helper.getVol().getVolumeName(), "invalid-bucket",
-            keyName, newPath1);
+        client.getKey(helper.getVol().getName(), "invalid-bucket", keyName);
         fail("Get key should have thrown " + "when using invalid bucket name.");
-      } catch (OzoneException e) {
+      } catch (IOException e) {
         GenericTestUtils.assertExceptionContains(
             Status.KEY_NOT_FOUND.toString(), e);
       }
@@ -384,14 +438,14 @@ public class TestKeys {
 
   @Test
   public void testPutAndDeleteKey() throws Exception {
-    runTestPutAndDeleteKey(new PutHelper(ozoneRestClient, path));
+    runTestPutAndDeleteKey(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAscii(1);
-    runTestPutAndDeleteKey(new PutHelper(ozoneRestClient, path,
+    runTestPutAndDeleteKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   static void runTestPutAndDeleteKey(PutHelper helper) throws Exception {
-    String keyName = helper.putKey().getKeyName();
+    String keyName = helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
     helper.getBucket().deleteKey(keyName);
@@ -399,7 +453,7 @@ public class TestKeys {
     try {
       helper.getBucket().getKey(keyName);
       fail("Get Key on a deleted key should have thrown");
-    } catch (OzoneException ex) {
+    } catch (IOException ex) {
       GenericTestUtils.assertExceptionContains(
           Status.KEY_NOT_FOUND.toString(), ex);
     }
@@ -407,14 +461,14 @@ public class TestKeys {
 
   @Test
   public void testPutAndListKey() throws Exception {
-    runTestPutAndListKey(new PutHelper(ozoneRestClient, path));
+    runTestPutAndListKey(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAscii(1);
-    runTestPutAndListKey(new PutHelper(ozoneRestClient, path,
+    runTestPutAndListKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   static void runTestPutAndListKey(PutHelper helper) throws Exception {
-    final OzoneRestClient client = helper.client;
+    ClientProtocol client = helper.client;
     helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
@@ -422,13 +476,20 @@ public class TestKeys {
     // add keys [list-key0, list-key1, ..., list-key9]
     for (int x = 0; x < 10; x++) {
       String newkeyName = "list-key" + x;
-      helper.getBucket().putKey(newkeyName, helper.getFile());
+      try (
+          OzoneOutputStream ozoneOutputStream = helper.getBucket()
+              .createKey(newkeyName, 0, replicationType, replicationFactor);
+          InputStream fileInputStream = new FileInputStream(helper.getFile())) {
+        IOUtils.copy(fileInputStream, ozoneOutputStream);
+      }
     }
 
-    List<OzoneKey> keyList1 = helper.getBucket().listKeys("100", null, null);
+    List<OzoneKey> keyList1 =
+        IteratorUtils.toList(helper.getBucket().listKeys(null, null));
     // test list key using a more efficient call
-    List<OzoneKey> keyList2 = client.listKeys(helper.getVol().getVolumeName(),
-        helper.getBucket().getBucketName(), "100", null, null);
+    List<OzoneKey> keyList2 = client
+        .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
+            null, 100);
 
     Assert.assertEquals(11, keyList1.size());
     Assert.assertEquals(11, keyList2.size());
@@ -436,57 +497,56 @@ public class TestKeys {
     // second unit since the date string reparsed to millisecond will
     // lose precision.
     for (OzoneKey key : keyList1) {
-      assertTrue((OzoneUtils.formatDate(key.getObjectInfo().getCreatedOn())
-          / 1000) >= (currentTime / 1000));
-      assertTrue((OzoneUtils.formatDate(key.getObjectInfo().getModifiedOn())
-          / 1000) >= (currentTime / 1000));
+      assertTrue((key.getCreationTime() / 1000) >= (currentTime / 1000));
+      assertTrue((key.getModificationTime() / 1000) >= (currentTime / 1000));
     }
 
     for (OzoneKey key : keyList2) {
-      assertTrue((OzoneUtils.formatDate(key.getObjectInfo().getCreatedOn())
-          / 1000) >= (currentTime / 1000));
-      assertTrue((OzoneUtils.formatDate(key.getObjectInfo().getModifiedOn())
-          / 1000) >= (currentTime / 1000));
+      assertTrue((key.getCreationTime() / 1000) >= (currentTime / 1000));
+      assertTrue((key.getModificationTime() / 1000) >= (currentTime / 1000));
     }
 
     // test maxLength parameter of list keys
-    keyList1 = helper.getBucket().listKeys("1", null, null);
-    keyList2 = client.listKeys(helper.getVol().getVolumeName(),
-        helper.getBucket().getBucketName(), "1", null, null);
-    Assert.assertEquals(1, keyList1.size());
+    keyList2 = client
+        .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
+            null, 1);
     Assert.assertEquals(1, keyList2.size());
 
     // test startKey parameter of list keys
-    keyList1 = helper.getBucket().listKeys("100", "list-key4", "list-key");
-    keyList2 = client.listKeys(helper.getVol().getVolumeName(),
-        helper.getBucket().getBucketName(), "100", "list-key4", "list-key");
+    keyList1 = IteratorUtils
+        .toList(helper.getBucket().listKeys("list-key", "list-key4"));
+    keyList2 = client
+        .listKeys(helper.getVol().getName(), helper.getBucket().getName(),
+            "list-key", "list-key4", 100);
     Assert.assertEquals(5, keyList1.size());
     Assert.assertEquals(5, keyList2.size());
 
     // test prefix parameter of list keys
-    keyList1 = helper.getBucket().listKeys("100", null, "list-key2");
-    keyList2 = client.listKeys(helper.getVol().getVolumeName(),
-        helper.getBucket().getBucketName(), "100", null, "list-key2");
-    Assert.assertTrue(keyList1.size() == 1
-        && keyList1.get(0).getObjectInfo().getKeyName().equals("list-key2"));
-    Assert.assertTrue(keyList2.size() == 1
-        && keyList2.get(0).getObjectInfo().getKeyName().equals("list-key2"));
+    keyList1 =
+        IteratorUtils.toList(helper.getBucket().listKeys("list-key2", null));
+    keyList2 = client
+        .listKeys(helper.getVol().getName(), helper.getBucket().getName(),
+            "list-key2", null, 100);
+    Assert.assertTrue(
+        keyList1.size() == 1 && keyList1.get(0).getName().equals("list-key2"));
+    Assert.assertTrue(
+        keyList2.size() == 1 && keyList2.get(0).getName().equals("list-key2"));
 
     // test new list keys with invalid volume/bucket name
     try {
-      client.listKeys("invalid-volume", helper.getBucket().getBucketName(),
-          "100", null, null);
+      client.listKeys("invalid-volume", helper.getBucket().getName(),
+          null, null, 100);
       fail("List keys should have thrown when using invalid volume name.");
-    } catch (OzoneException e) {
+    } catch (IOException e) {
       GenericTestUtils.assertExceptionContains(
           Status.BUCKET_NOT_FOUND.toString(), e);
     }
 
     try {
-      client.listKeys(helper.getVol().getVolumeName(), "invalid-bucket", "100",
-          null, null);
+      client.listKeys(helper.getVol().getName(), "invalid-bucket", null,
+          null, 100);
       fail("List keys should have thrown when using invalid bucket name.");
-    } catch (OzoneException e) {
+    } catch (IOException e) {
       GenericTestUtils.assertExceptionContains(
           Status.BUCKET_NOT_FOUND.toString(), e);
     }
@@ -494,29 +554,27 @@ public class TestKeys {
 
   @Test
   public void testGetKeyInfo() throws Exception {
-    runTestGetKeyInfo(new PutHelper(ozoneRestClient, path));
+    runTestGetKeyInfo(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAscii(1);
-    runTestGetKeyInfo(new PutHelper(ozoneRestClient, path,
+    runTestGetKeyInfo(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   static void runTestGetKeyInfo(PutHelper helper) throws Exception {
-    String keyName = helper.putKey().getKeyName();
+    String keyName = helper.putKey();
     assertNotNull(helper.getBucket());
     assertNotNull(helper.getFile());
 
-    OzoneKey keyInfo = helper.getBucket().getKeyInfo(keyName);
-    assertNotNull(keyInfo.getObjectInfo());
-    assertEquals(keyName, keyInfo.getObjectInfo().getKeyName());
+    OzoneKey keyInfo = helper.getBucket().getKey(keyName);
+    assertNotNull(keyInfo);
+    assertEquals(keyName, keyInfo.getName());
 
     // Compare the time in second unit since the date string reparsed to
     // millisecond will lose precision.
+    Assert
+        .assertTrue((keyInfo.getCreationTime() / 1000) >= (currentTime / 1000));
     Assert.assertTrue(
-        (OzoneUtils.formatDate(keyInfo.getObjectInfo().getCreatedOn())
-            / 1000) >= (currentTime / 1000));
-    Assert.assertTrue(
-        (OzoneUtils.formatDate(keyInfo.getObjectInfo().getModifiedOn())
-            / 1000) >= (currentTime / 1000));
+        (keyInfo.getModificationTime() / 1000) >= (currentTime / 1000));
   }
 
   // Volume, bucket, keys info that helps for test create/delete keys.
@@ -593,12 +651,12 @@ public class TestKeys {
     int numOfExistedKeys = countKsmKeys(ksm);
 
     // Keep tracking bucket keys info while creating them
-    PutHelper helper = new PutHelper(ozoneRestClient, path);
+    PutHelper helper = new PutHelper(client, path);
     BucketKeys bucketKeys = new BucketKeys();
     for (int i = 0; i < 20; i++) {
-      KsmKeyArgs keyArgs = helper.putKey();
-      bucketKeys.addKey(keyArgs.getVolumeName(), keyArgs.getBucketName(),
-          keyArgs.getKeyName());
+      String keyName = helper.putKey();
+      bucketKeys.addKey(helper.getVol().getName(), helper.getBucket().getName(),
+          keyName);
     }
 
     // There should be 20 keys in the buckets we just created.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
index 802cc3d..645b866 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
@@ -21,6 +21,7 @@ import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.RatisTestHelper;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -52,7 +53,7 @@ public class TestKeysRatis {
   private static RatisTestHelper.RatisTestSuite suite;
   private static MiniOzoneCluster ozoneCluster = null;
   static private String path;
-  private static OzoneRestClient ozoneRestClient = null;
+  private static ClientProtocol client = null;
 
   @BeforeClass
   public static void init() throws Exception {
@@ -60,7 +61,7 @@ public class TestKeysRatis {
     path = suite.getConf().get(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT);
     ozoneCluster = suite.getCluster();
     ozoneCluster.waitForClusterToBeReady();
-    ozoneRestClient = suite.newOzoneRestClient();
+    client = suite.newOzoneClient();
   }
 
   /**
@@ -76,9 +77,9 @@ public class TestKeysRatis {
 
   @Test
   public void testPutKey() throws Exception {
-    runTestPutKey(new PutHelper(ozoneRestClient, path));
+    runTestPutKey(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAlphanumeric(1);
-    runTestPutKey(new PutHelper(ozoneRestClient, path,
+    runTestPutKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
@@ -86,42 +87,42 @@ public class TestKeysRatis {
   @Test
   public void testPutAndGetKeyWithDnRestart() throws Exception {
     runTestPutAndGetKeyWithDnRestart(
-        new PutHelper(ozoneRestClient, path), ozoneCluster);
+        new PutHelper(client, path), ozoneCluster);
     String delimiter = RandomStringUtils.randomAlphanumeric(1);
     runTestPutAndGetKeyWithDnRestart(
-        new PutHelper(ozoneRestClient, path, getMultiPartKey(delimiter)),
+        new PutHelper(client, path, getMultiPartKey(delimiter)),
         ozoneCluster);
   }
 
   @Test
   public void testPutAndGetKey() throws Exception {
-    runTestPutAndGetKey(new PutHelper(ozoneRestClient, path));
+    runTestPutAndGetKey(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAlphanumeric(1);
-    runTestPutAndGetKey(new PutHelper(ozoneRestClient, path,
+    runTestPutAndGetKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   @Test
   public void testPutAndDeleteKey() throws Exception  {
-    runTestPutAndDeleteKey(new PutHelper(ozoneRestClient, path));
+    runTestPutAndDeleteKey(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAlphanumeric(1);
-    runTestPutAndDeleteKey(new PutHelper(ozoneRestClient, path,
+    runTestPutAndDeleteKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   @Test
   public void testPutAndListKey() throws Exception {
-    runTestPutAndListKey(new PutHelper(ozoneRestClient, path));
+    runTestPutAndListKey(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAlphanumeric(1);
-    runTestPutAndListKey(new PutHelper(ozoneRestClient, path,
+    runTestPutAndListKey(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 
   @Test
   public void testGetKeyInfo() throws Exception {
-    runTestGetKeyInfo(new PutHelper(ozoneRestClient, path));
+    runTestGetKeyInfo(new PutHelper(client, path));
     String delimiter = RandomStringUtils.randomAlphanumeric(1);
-    runTestGetKeyInfo(new PutHelper(ozoneRestClient, path,
+    runTestGetKeyInfo(new PutHelper(client, path,
         getMultiPartKey(delimiter)));
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
index 2d3cea9..f8c7eec 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
@@ -20,50 +20,42 @@ package org.apache.hadoop.ozone.web.client;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.RandomStringUtils;
-import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hdds.client.OzoneQuota;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.Status;
-import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
-import org.apache.hadoop.ozone.web.request.OzoneQuota;
+import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
-import org.apache.http.client.methods.HttpUriRequest;
-import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
-import org.mockito.Mockito;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.Ignore;
 
 import java.io.File;
 import java.io.IOException;
 import java.text.ParseException;
-import java.util.ArrayList;
 import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
 
-import org.junit.BeforeClass;
-import org.junit.AfterClass;
-import org.junit.Test;
-import org.junit.Ignore;
-import static org.junit.Assert.fail;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
 
 /**
  * Test Ozone Volumes Lifecycle.
  */
 public class TestVolume {
   private static MiniOzoneCluster cluster = null;
-  private static OzoneRestClient ozoneRestClient = null;
+  private static ClientProtocol client = null;
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -92,8 +84,7 @@ public class TestVolume {
     final int port = cluster.getHddsDatanodes().get(0)
         .getDatanodeDetails().getOzoneRestPort();
 
-    ozoneRestClient = new OzoneRestClient(
-        String.format("http://localhost:%d", port));
+    client = new RpcClient(conf);
   }
 
   /**
@@ -108,128 +99,122 @@ public class TestVolume {
 
   @Test
   public void testCreateVolume() throws Exception {
-    runTestCreateVolume(ozoneRestClient);
+    runTestCreateVolume(client);
   }
 
-  static void runTestCreateVolume(OzoneRestClient client)
+  static void runTestCreateVolume(ClientProtocol client)
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
 
     long currentTime = Time.now();
-    OzoneRestClient mockClient = Mockito.spy(client);
-    List<CloseableHttpClient> mockedClients = mockHttpClients(mockClient);
-    OzoneVolume vol = mockClient.createVolume(volumeName, "bilbo", "100TB");
-    // Verify http clients are properly closed.
-    verifyHttpConnectionClosed(mockedClients);
-
-    assertEquals(vol.getVolumeName(), volumeName);
-    assertEquals(vol.getCreatedby(), "hdfs");
-    assertEquals(vol.getOwnerName(), "bilbo");
-    assertEquals(vol.getQuota().getUnit(), OzoneQuota.Units.TB);
-    assertEquals(vol.getQuota().getSize(), 100);
+
+    VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+        .setOwner("bilbo")
+        .setQuota("100TB")
+        .setAdmin("hdfs")
+        .build();
+    client.createVolume(volumeName, volumeArgs);
+    OzoneVolume vol = client.getVolumeDetails(volumeName);
+
+    assertEquals(vol.getName(), volumeName);
+    assertEquals(vol.getAdmin(), "hdfs");
+    assertEquals(vol.getOwner(), "bilbo");
+    assertEquals(vol.getQuota(), OzoneQuota.parseQuota("100TB").sizeInBytes());
 
     // verify the key creation time
-    assertTrue((OzoneUtils.formatDate(vol.getCreatedOn())
+    assertTrue((vol.getCreationTime()
         / 1000) >= (currentTime / 1000));
 
     // Test create a volume with invalid volume name,
     // not use Rule here because the test method is static.
     try {
       String invalidVolumeName = "#" + OzoneUtils.getRequestID().toLowerCase();
-      client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
-      mockClient.createVolume(invalidVolumeName, "bilbo", "100TB");
+      client.createVolume(invalidVolumeName);
+      /*
+      //TODO: RestClient and RpcClient should use HddsClientUtils to verify name
       fail("Except the volume creation be failed because the"
-          + " volume name starts with an invalid char #");
+          + " volume name starts with an invalid char #");*/
     } catch (Exception e) {
-      assertTrue(e instanceof OzoneClientException);
       assertTrue(e.getMessage().contains("Bucket or Volume name"
           + " has an unsupported character : #"));
     }
   }
 
   @Test
-  public void testCreateDuplicateVolume() throws OzoneException {
-    runTestCreateDuplicateVolume(ozoneRestClient);
+  public void testCreateDuplicateVolume() throws OzoneException, IOException {
+    runTestCreateDuplicateVolume(client);
   }
 
-  static void runTestCreateDuplicateVolume(OzoneRestClient client)
-      throws OzoneException {
+  static void runTestCreateDuplicateVolume(ClientProtocol client)
+      throws OzoneException, IOException {
     try {
-      client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
-      client.createVolume("testvol", "bilbo", "100TB");
-      client.createVolume("testvol", "bilbo", "100TB");
+      client.createVolume("testvol");
+      client.createVolume("testvol");
       assertFalse(true);
-    } catch (OzoneException ex) {
-      // Ozone will throw saying volume already exists
-      GenericTestUtils.assertExceptionContains(
-          Status.VOLUME_ALREADY_EXISTS.toString(), ex);
+    } catch (IOException ioe) {
+      Assert.assertTrue(ioe.getMessage()
+          .contains("Volume creation failed, error:VOLUME_ALREADY_EXISTS"));
     }
   }
 
   @Test
-  public void testDeleteVolume() throws OzoneException {
-    runTestDeleteVolume(ozoneRestClient);
+  public void testDeleteVolume() throws OzoneException, IOException {
+    runTestDeleteVolume(client);
   }
 
-  static void runTestDeleteVolume(OzoneRestClient client)
-      throws OzoneException {
+  static void runTestDeleteVolume(ClientProtocol client)
+      throws OzoneException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
-    client.deleteVolume(vol.getVolumeName());
+    client.createVolume(volumeName);
+    client.deleteVolume(volumeName);
   }
 
   @Test
   public void testChangeOwnerOnVolume() throws Exception {
-    runTestChangeOwnerOnVolume(ozoneRestClient);
+    runTestChangeOwnerOnVolume(client);
   }
 
-  static void runTestChangeOwnerOnVolume(OzoneRestClient client)
-      throws OzoneException, ParseException {
+  static void runTestChangeOwnerOnVolume(ClientProtocol client)
+      throws OzoneException, ParseException, IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
+    client.createVolume(volumeName);
+    client.getVolumeDetails(volumeName);
     client.setVolumeOwner(volumeName, "frodo");
-    OzoneVolume newVol = client.getVolume(volumeName);
-    assertEquals(newVol.getOwnerName(), "frodo");
+    OzoneVolume newVol = client.getVolumeDetails(volumeName);
+    assertEquals(newVol.getOwner(), "frodo");
     // verify if the creation time is missing after setting owner operation
-    assertTrue(OzoneUtils.formatDate(newVol.getCreatedOn()) > 0);
+    assertTrue(newVol.getCreationTime() > 0);
   }
 
   @Test
   public void testChangeQuotaOnVolume() throws Exception {
-    runTestChangeQuotaOnVolume(ozoneRestClient);
+    runTestChangeQuotaOnVolume(client);
   }
 
-  static void runTestChangeQuotaOnVolume(OzoneRestClient client)
+  static void runTestChangeQuotaOnVolume(ClientProtocol client)
       throws OzoneException, IOException, ParseException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
-    OzoneVolume vol = client.createVolume(volumeName, "bilbo", "100TB");
-    client.setVolumeQuota(volumeName, "1000MB");
-    OzoneVolume newVol = client.getVolume(volumeName);
-    assertEquals(newVol.getQuota().getSize(), 1000);
-    assertEquals(newVol.getQuota().getUnit(), OzoneQuota.Units.MB);
+    client.createVolume(volumeName);
+    client.setVolumeQuota(volumeName, OzoneQuota.parseQuota("1000MB"));
+    OzoneVolume newVol = client.getVolumeDetails(volumeName);
+    assertEquals(newVol.getQuota(), OzoneQuota.parseQuota("1000MB").sizeInBytes());
     // verify if the creation time is missing after setting quota operation
-    assertTrue(OzoneUtils.formatDate(newVol.getCreatedOn()) > 0);
+    assertTrue(newVol.getCreationTime() > 0);
   }
 
   @Test
   public void testListVolume() throws OzoneException, IOException {
-    runTestListVolume(ozoneRestClient);
+    runTestListVolume(client);
   }
 
-  static void runTestListVolume(OzoneRestClient client)
+  static void runTestListVolume(ClientProtocol client)
       throws OzoneException, IOException {
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
     for (int x = 0; x < 10; x++) {
       String volumeName = OzoneUtils.getRequestID().toLowerCase();
-      OzoneVolume vol = client.createVolume(volumeName, "frodo", "100TB");
-      assertNotNull(vol);
+      client.createVolume(volumeName);
     }
 
-    List<OzoneVolume> ovols = client.listVolumes("frodo");
+    List<OzoneVolume> ovols = client.listVolumes(null, null, 100);
     assertTrue(ovols.size() >= 10);
   }
 
@@ -237,27 +222,24 @@ public class TestVolume {
   @Ignore("Takes 3m to run, disable for now.")
   @Test
   public void testListVolumePagination() throws OzoneException, IOException {
-    runTestListVolumePagination(ozoneRestClient);
+    runTestListVolumePagination(client);
   }
 
-  static void runTestListVolumePagination(OzoneRestClient client)
+  static void runTestListVolumePagination(ClientProtocol client)
       throws OzoneException, IOException {
     final int volCount = 2000;
     final int step = 100;
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
     for (int x = 0; x < volCount; x++) {
       String volumeName = OzoneUtils.getRequestID().toLowerCase();
-      OzoneVolume vol = client.createVolume(volumeName, "frodo", "100TB");
-      assertNotNull(vol);
+      client.createVolume(volumeName);
     }
-    OzoneVolume prevKey = null;
+    String prevKey = null;
     int count = 0;
     int pagecount = 0;
     while (count < volCount) {
-      List<OzoneVolume> ovols = client.listVolumes("frodo", null, step,
-          prevKey);
+      List<OzoneVolume> ovols = client.listVolumes(null, prevKey, step);
       count += ovols.size();
-      prevKey = ovols.get(ovols.size() - 1);
+      prevKey = ovols.get(ovols.size() - 1).getName();
       pagecount++;
     }
     assertEquals(volCount / step, pagecount);
@@ -267,30 +249,35 @@ public class TestVolume {
   @Ignore
   @Test
   public void testListAllVolumes() throws OzoneException, IOException {
-    runTestListAllVolumes(ozoneRestClient);
+    runTestListAllVolumes(client);
   }
 
-  static void runTestListAllVolumes(OzoneRestClient client)
+  static void runTestListAllVolumes(ClientProtocol client)
       throws OzoneException, IOException {
     final int volCount = 200;
     final int step = 10;
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
     for (int x = 0; x < volCount; x++) {
       String userName =
           "frodo" + RandomStringUtils.randomAlphabetic(5).toLowerCase();
       String volumeName =
           "vol" + RandomStringUtils.randomAlphabetic(5).toLowerCase();
-      OzoneVolume vol = client.createVolume(volumeName, userName, "100TB");
+      VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+          .setOwner(userName)
+          .setQuota("100TB")
+          .setAdmin("hdfs")
+          .build();
+      client.createVolume(volumeName, volumeArgs);
+      OzoneVolume vol = client.getVolumeDetails(volumeName);
       assertNotNull(vol);
     }
-    OzoneVolume prevKey = null;
+    String prevKey = null;
     int count = 0;
     int pagecount = 0;
     while (count < volCount) {
-      List<OzoneVolume> ovols = client.listAllVolumes(null, step, prevKey);
+      List<OzoneVolume> ovols = client.listVolumes(null, prevKey, step);
       count += ovols.size();
       if (ovols.size() > 0) {
-        prevKey = ovols.get(ovols.size() - 1);
+        prevKey = ovols.get(ovols.size() - 1).getName();
       }
       pagecount++;
     }
@@ -301,17 +288,16 @@ public class TestVolume {
 
   @Test
   public void testListVolumes() throws Exception {
-    runTestListVolumes(ozoneRestClient);
+    runTestListVolumes(client);
   }
 
-  static void runTestListVolumes(OzoneRestClient client)
+  static void runTestListVolumes(ClientProtocol client)
       throws OzoneException, IOException, ParseException {
     final int volCount = 20;
     final String user1 = "test-user-a";
     final String user2 = "test-user-b";
 
     long currentTime = Time.now();
-    client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
     // Create 20 volumes, 10 for user1 and another 10 for user2.
     for (int x = 0; x < volCount; x++) {
       String volumeName;
@@ -326,109 +312,40 @@ public class TestVolume {
         userName = user2;
         volumeName = "test-vol" + x;
       }
-      OzoneVolume vol = client.createVolume(volumeName, userName, "100TB");
+      VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+          .setOwner(userName)
+          .setQuota("100TB")
+          .setAdmin("hdfs")
+          .build();
+      client.createVolume(volumeName, volumeArgs);
+      OzoneVolume vol = client.getVolumeDetails(volumeName);
       assertNotNull(vol);
     }
 
     // list all the volumes belong to user1
-    List<OzoneVolume> volumeList = client.listVolumes(user1,
-        null, 100, StringUtils.EMPTY);
+    List<OzoneVolume> volumeList = client.listVolumes(user1, null, null, 100);
     assertEquals(10, volumeList.size());
     // verify the owner name and creation time of volume
     for (OzoneVolume vol : volumeList) {
-      assertTrue(vol.getOwnerName().equals(user1));
-      assertTrue((OzoneUtils.formatDate(vol.getCreatedOn())
+      assertTrue(vol.getOwner().equals(user1));
+      assertTrue((vol.getCreationTime()
           / 1000) >= (currentTime / 1000));
     }
 
     // test max key parameter of listing volumes
-    volumeList = client.listVolumes(user1, null, 2, StringUtils.EMPTY);
+    volumeList = client.listVolumes(user1, null, null, 2);
     assertEquals(2, volumeList.size());
 
     // test prefix parameter of listing volumes
-    volumeList = client.listVolumes(user1, "test-vol10", 100,
-        StringUtils.EMPTY);
+    volumeList = client.listVolumes(user1, "test-vol10", null, 10);
     assertTrue(volumeList.size() == 1
-        && volumeList.get(0).getVolumeName().equals("test-vol10"));
+        && volumeList.get(0).getName().equals("test-vol10"));
 
-    volumeList = client.listVolumes(user1, "test-vol1",
-        100, StringUtils.EMPTY);
+    volumeList = client.listVolumes(user1, "test-vol1", null, 10);
     assertEquals(5, volumeList.size());
 
     // test start key parameter of listing volumes
-    volumeList = client.listVolumes(user2, null, 100, "test-vol15");
+    volumeList = client.listVolumes(user2, null, "test-vol15", 10);
     assertEquals(2, volumeList.size());
   }
-
-  /**
-   * Returns a list of mocked {@link CloseableHttpClient} used for testing.
-   * The mocked client replaces the actual calls in
-   * {@link OzoneRestClient#newHttpClient()}, it is used to verify
-   * if the invocation of this client is expected. <b>Note</b>, the output
-   * of this method is always used as the input of
-   * {@link TestVolume#verifyHttpConnectionClosed(List)}.
-   *
-   * @param mockedClient mocked ozone client.
-   * @return a list of mocked {@link CloseableHttpClient}.
-   * @throws IOException
-   */
-  private static List<CloseableHttpClient> mockHttpClients(
-      OzoneRestClient mockedClient)
-      throws IOException {
-    List<CloseableHttpClient> spyHttpClients = new ArrayList<>();
-    for (int i = 0; i < 5; i++) {
-      CloseableHttpClient spyHttpClient = Mockito
-          .spy(HddsClientUtils.newHttpClient());
-      spyHttpClients.add(spyHttpClient);
-    }
-
-    List<CloseableHttpClient> nextReturns =
-        new ArrayList<>(spyHttpClients.subList(1, spyHttpClients.size()));
-    Mockito.when(mockedClient.newHttpClient()).thenReturn(
-        spyHttpClients.get(0),
-        nextReturns.toArray(new CloseableHttpClient[nextReturns.size()]));
-    return spyHttpClients;
-  }
-
-  /**
-   * This method is used together with
-   * {@link TestVolume#mockHttpClients(OzoneRestClient)} to verify
-   * if the http client is properly closed. It verifies that as long as
-   * a client calls {@link CloseableHttpClient#execute(HttpUriRequest)} to
-   * send request, then it must calls {@link CloseableHttpClient#close()}
-   * close the http connection.
-   *
-   * @param mockedHttpClients
-   */
-  private static void verifyHttpConnectionClosed(
-      List<CloseableHttpClient> mockedHttpClients) {
-    final AtomicInteger totalCalled = new AtomicInteger();
-    assertTrue(mockedHttpClients.stream().allMatch(closeableHttpClient -> {
-      boolean clientUsed = false;
-      try {
-        verify(closeableHttpClient, times(1)).execute(Mockito.any());
-        totalCalled.incrementAndGet();
-        clientUsed = true;
-      } catch (Throwable e) {
-        // There might be some redundant instances in mockedHttpClients,
-        // it is allowed that a client is not used.
-        return true;
-      }
-
-      if (clientUsed) {
-        try {
-          // If a client is used, ensure the close function is called.
-          verify(closeableHttpClient, times(1)).close();
-          return true;
-        } catch (IOException e) {
-          return false;
-        }
-      } else {
-        return true;
-      }
-    }));
-    System.out.println("Successful connections " + totalCalled.get());
-    assertTrue("The mocked http client should be called at least once.",
-        totalCalled.get() > 0);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
index 8314851..bc4ba25 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
@@ -23,7 +23,9 @@ import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -38,7 +40,7 @@ import java.io.IOException;
 public class TestVolumeRatis {
   @Rule
   public Timeout testTimeout = new Timeout(300000);
-  private static OzoneRestClient ozoneClient;
+  private static ClientProtocol client;
   private static MiniOzoneCluster cluster;
 
   @BeforeClass
@@ -63,8 +65,7 @@ public class TestVolumeRatis {
     final int port = cluster.getHddsDatanodes().get(0)
         .getDatanodeDetails().getOzoneRestPort();
 
-    ozoneClient = new OzoneRestClient(
-        String.format("http://localhost:%d", port));
+    client = new RpcClient(conf);
   }
 
   @AfterClass
@@ -77,53 +78,53 @@ public class TestVolumeRatis {
 
   @Test
   public void testCreateVolume() throws Exception {
-    TestVolume.runTestCreateVolume(ozoneClient);
+    TestVolume.runTestCreateVolume(client);
   }
 
   @Test
-  public void testCreateDuplicateVolume() throws OzoneException {
-    TestVolume.runTestCreateDuplicateVolume(ozoneClient);
+  public void testCreateDuplicateVolume() throws OzoneException, IOException {
+    TestVolume.runTestCreateDuplicateVolume(client);
   }
 
   @Test
-  public void testDeleteVolume() throws OzoneException {
-    TestVolume.runTestDeleteVolume(ozoneClient);
+  public void testDeleteVolume() throws OzoneException, IOException {
+    TestVolume.runTestDeleteVolume(client);
   }
 
   @Test
   public void testChangeOwnerOnVolume() throws Exception {
-    TestVolume.runTestChangeOwnerOnVolume(ozoneClient);
+    TestVolume.runTestChangeOwnerOnVolume(client);
   }
 
   @Test
   public void testChangeQuotaOnVolume() throws Exception {
-    TestVolume.runTestChangeQuotaOnVolume(ozoneClient);
+    TestVolume.runTestChangeQuotaOnVolume(client);
   }
 
   // TODO: remove @Ignore below once the problem has been resolved.
   @Ignore("listVolumes not implemented in DistributedStorageHandler")
   @Test
   public void testListVolume() throws OzoneException, IOException {
-    TestVolume.runTestListVolume(ozoneClient);
+    TestVolume.runTestListVolume(client);
   }
 
   // TODO: remove @Ignore below once the problem has been resolved.
   @Ignore("See TestVolume.testListVolumePagination()")
   @Test
   public void testListVolumePagination() throws OzoneException, IOException {
-    TestVolume.runTestListVolumePagination(ozoneClient);
+    TestVolume.runTestListVolumePagination(client);
   }
 
   // TODO: remove @Ignore below once the problem has been resolved.
   @Ignore("See TestVolume.testListAllVolumes()")
   @Test
   public void testListAllVolumes() throws Exception {
-    TestVolume.runTestListAllVolumes(ozoneClient);
+    TestVolume.runTestListAllVolumes(client);
   }
 
   @Ignore("Disabling Ratis tests for pipeline work.")
   @Test
   public void testListVolumes() throws Exception {
-    TestVolume.runTestListVolumes(ozoneClient);
+    TestVolume.runTestListVolumes(client);
   }
 }


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


[30/50] [abbrv] hadoop git commit: YARN-8339. Service AM should localize static/archive resource types to container working directory instead of 'resources'. (Suma Shivaprasad via wangda)

Posted by ha...@apache.org.
YARN-8339. Service AM should localize static/archive resource types to container working directory instead of 'resources'. (Suma Shivaprasad via wangda)

Change-Id: I9f8e8f621650347f6c2f9e3420edee9eb2f356a4


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

Branch: refs/heads/HDDS-48
Commit: 3061bfcde53210d2032df3814243498b27a997b7
Parents: 3c75f8e
Author: Wangda Tan <wa...@apache.org>
Authored: Tue May 29 09:23:11 2018 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Tue May 29 09:23:11 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/yarn/service/provider/ProviderUtils.java | 3 +--
 .../apache/hadoop/yarn/service/provider/TestProviderUtils.java | 6 +++---
 2 files changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3061bfcd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.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/ProviderUtils.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/ProviderUtils.java
index 1ad5fd8..ac90992 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/ProviderUtils.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/ProviderUtils.java
@@ -298,8 +298,7 @@ public class ProviderUtils implements YarnServiceConstants {
         destFile = new Path(staticFile.getDestFile());
       }
 
-      String symlink = APP_RESOURCES_DIR + "/" + destFile.getName();
-      addLocalResource(launcher, symlink, localResource, destFile);
+      addLocalResource(launcher, destFile.getName(), localResource, destFile);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3061bfcd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/provider/TestProviderUtils.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/provider/TestProviderUtils.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/provider/TestProviderUtils.java
index 6e8bc43..5d794d2 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/provider/TestProviderUtils.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/provider/TestProviderUtils.java
@@ -154,11 +154,11 @@ public class TestProviderUtils {
 
     ProviderUtils.handleStaticFilesForLocalization(launcher, sfs,
         compLaunchCtx);
-    Mockito.verify(launcher).addLocalResource(Mockito.eq("resources/destFile1"),
+    Mockito.verify(launcher).addLocalResource(Mockito.eq("destFile1"),
         any(LocalResource.class));
     Mockito.verify(launcher).addLocalResource(
-        Mockito.eq("resources/destFile_2"), any(LocalResource.class));
+        Mockito.eq("destFile_2"), any(LocalResource.class));
     Mockito.verify(launcher).addLocalResource(
-        Mockito.eq("resources/sourceFile4"), any(LocalResource.class));
+        Mockito.eq("sourceFile4"), any(LocalResource.class));
   }
 }


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


[10/50] [abbrv] hadoop git commit: YARN-8292: Fix the dominant resource preemption cannot happen when some of the resource vector becomes negative. Contributed by Wangda Tan.

Posted by ha...@apache.org.
YARN-8292: Fix the dominant resource preemption cannot happen when some of the resource vector becomes negative. Contributed by Wangda Tan.


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

Branch: refs/heads/HDDS-48
Commit: 8d5509c68156faaa6641f4e747fc9ff80adccf88
Parents: bddfe79
Author: Eric E Payne <er...@oath.com>
Authored: Fri May 25 16:06:09 2018 +0000
Committer: Eric E Payne <er...@oath.com>
Committed: Fri May 25 16:06:09 2018 +0000

----------------------------------------------------------------------
 .../resource/DefaultResourceCalculator.java     |  15 ++-
 .../resource/DominantResourceCalculator.java    |  39 ++++---
 .../yarn/util/resource/ResourceCalculator.java  |  13 ++-
 .../hadoop/yarn/util/resource/Resources.java    |   5 -
 .../AbstractPreemptableResourceCalculator.java  |  58 ++++++++---
 .../CapacitySchedulerPreemptionUtils.java       |  61 +++++++++--
 .../capacity/FifoCandidatesSelector.java        |   8 +-
 .../FifoIntraQueuePreemptionPlugin.java         |   4 +-
 .../capacity/IntraQueueCandidatesSelector.java  |   2 +-
 .../capacity/PreemptableResourceCalculator.java |   6 +-
 .../monitor/capacity/TempQueuePerPartition.java |   8 +-
 ...alCapacityPreemptionPolicyMockFramework.java |  30 ++++++
 .../TestPreemptionForQueueWithPriorities.java   | 103 ++++++++++++-------
 ...pacityPreemptionPolicyInterQueueWithDRF.java |  60 ++++++++++-
 14 files changed, 312 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d5509c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
index 6375c4a..ab6d7f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
@@ -136,13 +136,18 @@ public class DefaultResourceCalculator extends ResourceCalculator {
   }
 
   @Override
-  public boolean isAnyMajorResourceZero(Resource resource) {
-    return resource.getMemorySize() == 0f;
-  }
-
-  @Override
   public Resource normalizeDown(Resource r, Resource stepFactor) {
     return Resources.createResource(
         roundDown((r.getMemorySize()), stepFactor.getMemorySize()));
   }
+
+  @Override
+  public boolean isAnyMajorResourceZeroOrNegative(Resource resource) {
+    return resource.getMemorySize() <= 0;
+  }
+
+  @Override
+  public boolean isAnyMajorResourceAboveZero(Resource resource) {
+    return resource.getMemorySize() > 0;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d5509c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
index 6fed23b..2e85ebc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
@@ -577,19 +577,6 @@ public class DominantResourceCalculator extends ResourceCalculator {
   }
 
   @Override
-  public boolean isAnyMajorResourceZero(Resource resource) {
-    int maxLength = ResourceUtils.getNumberOfKnownResourceTypes();
-    for (int i = 0; i < maxLength; i++) {
-      ResourceInformation resourceInformation = resource
-          .getResourceInformation(i);
-      if (resourceInformation.getValue() == 0L) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  @Override
   public Resource normalizeDown(Resource r, Resource stepFactor) {
     Resource ret = Resource.newInstance(r);
     int maxLength = ResourceUtils.getNumberOfKnownResourceTypes();
@@ -613,4 +600,30 @@ public class DominantResourceCalculator extends ResourceCalculator {
     }
     return ret;
   }
+
+  @Override
+  public boolean isAnyMajorResourceZeroOrNegative(Resource resource) {
+    int maxLength = ResourceUtils.getNumberOfKnownResourceTypes();
+    for (int i = 0; i < maxLength; i++) {
+      ResourceInformation resourceInformation = resource.getResourceInformation(
+          i);
+      if (resourceInformation.getValue() <= 0L) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public boolean isAnyMajorResourceAboveZero(Resource resource) {
+    int maxLength = ResourceUtils.getNumberOfKnownResourceTypes();
+    for (int i = 0; i < maxLength; i++) {
+      ResourceInformation resourceInformation = resource.getResourceInformation(
+          i);
+      if (resourceInformation.getValue() > 0) {
+        return true;
+      }
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d5509c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
index 1c42126..51078cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
@@ -239,12 +239,12 @@ public abstract class ResourceCalculator {
 
   /**
    * Check if resource has any major resource types (which are all NodeManagers
-   * included) a zero value.
+   * included) a zero value or negative value.
    *
    * @param resource resource
    * @return returns true if any resource is zero.
    */
-  public abstract boolean isAnyMajorResourceZero(Resource resource);
+  public abstract boolean isAnyMajorResourceZeroOrNegative(Resource resource);
 
   /**
    * Get resource <code>r</code>and normalize down using step-factor
@@ -257,4 +257,13 @@ public abstract class ResourceCalculator {
    * @return resulting normalized resource
    */
   public abstract Resource normalizeDown(Resource r, Resource stepFactor);
+
+  /**
+   * Check if resource has any major resource types (which are all NodeManagers
+   * included) has a >0 value.
+   *
+   * @param resource resource
+   * @return returns true if any resource is >0
+   */
+  public abstract boolean isAnyMajorResourceAboveZero(Resource resource);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d5509c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
index 1c08844..7826f51 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
@@ -547,11 +547,6 @@ public class Resources {
     return ret;
   }
 
-  public static boolean isAnyMajorResourceZero(ResourceCalculator rc,
-      Resource resource) {
-    return rc.isAnyMajorResourceZero(resource);
-  }
-
   public static Resource normalizeDown(ResourceCalculator calculator,
       Resource resource, Resource factor) {
     return calculator.normalizeDown(resource, factor);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d5509c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/AbstractPreemptableResourceCalculator.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/monitor/capacity/AbstractPreemptableResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/AbstractPreemptableResourceCalculator.java
index 2589970..64b3615 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/AbstractPreemptableResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/AbstractPreemptableResourceCalculator.java
@@ -18,12 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.PriorityQueue;
-
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy.PriorityUtilizationQueueOrderingPolicy;
@@ -32,6 +26,12 @@ 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.PriorityQueue;
+
 /**
  * Calculate how much resources need to be preempted for each queue,
  * will be used by {@link PreemptionCandidatesSelector}.
@@ -40,7 +40,8 @@ public class AbstractPreemptableResourceCalculator {
 
   protected final CapacitySchedulerPreemptionContext context;
   protected final ResourceCalculator rc;
-  private boolean isReservedPreemptionCandidatesSelector;
+  protected boolean isReservedPreemptionCandidatesSelector;
+  private Resource stepFactor;
 
   static class TQComparator implements Comparator<TempQueuePerPartition> {
     private ResourceCalculator rc;
@@ -90,6 +91,11 @@ public class AbstractPreemptableResourceCalculator {
     rc = preemptionContext.getResourceCalculator();
     this.isReservedPreemptionCandidatesSelector =
         isReservedPreemptionCandidatesSelector;
+
+    stepFactor = Resource.newInstance(0, 0);
+    for (ResourceInformation ri : stepFactor.getResources()) {
+      ri.setValue(1);
+    }
   }
 
   /**
@@ -122,23 +128,24 @@ public class AbstractPreemptableResourceCalculator {
     TQComparator tqComparator = new TQComparator(rc, totGuarant);
     PriorityQueue<TempQueuePerPartition> orderedByNeed = new PriorityQueue<>(10,
         tqComparator);
-    for (Iterator<TempQueuePerPartition> i = qAlloc.iterator(); i.hasNext();) {
+    for (Iterator<TempQueuePerPartition> i = qAlloc.iterator(); i.hasNext(); ) {
       TempQueuePerPartition q = i.next();
       Resource used = q.getUsed();
 
       Resource initIdealAssigned;
       if (Resources.greaterThan(rc, totGuarant, used, q.getGuaranteed())) {
-        initIdealAssigned =
-            Resources.add(q.getGuaranteed(), q.untouchableExtra);
-      } else {
+        initIdealAssigned = Resources.add(
+            Resources.componentwiseMin(q.getGuaranteed(), q.getUsed()),
+            q.untouchableExtra);
+      } else{
         initIdealAssigned = Resources.clone(used);
       }
 
       // perform initial assignment
       initIdealAssignment(totGuarant, q, initIdealAssigned);
 
-
       Resources.subtractFrom(unassigned, q.idealAssigned);
+
       // If idealAssigned < (allocated + used + pending), q needs more
       // resources, so
       // add it to the list of underserved queues, ordered by need.
@@ -152,7 +159,6 @@ public class AbstractPreemptableResourceCalculator {
     // left
     while (!orderedByNeed.isEmpty() && Resources.greaterThan(rc, totGuarant,
         unassigned, Resources.none())) {
-      Resource wQassigned = Resource.newInstance(0, 0);
       // we compute normalizedGuarantees capacity based on currently active
       // queues
       resetCapacity(unassigned, orderedByNeed, ignoreGuarantee);
@@ -166,11 +172,26 @@ public class AbstractPreemptableResourceCalculator {
       Collection<TempQueuePerPartition> underserved = getMostUnderservedQueues(
           orderedByNeed, tqComparator);
 
+      // This value will be used in every round to calculate ideal allocation.
+      // So make a copy to avoid it changed during calculation.
+      Resource dupUnassignedForTheRound = Resources.clone(unassigned);
+
       for (Iterator<TempQueuePerPartition> i = underserved.iterator(); i
           .hasNext();) {
+        if (!rc.isAnyMajorResourceAboveZero(unassigned)) {
+          break;
+        }
+
         TempQueuePerPartition sub = i.next();
-        Resource wQavail = Resources.multiplyAndNormalizeUp(rc, unassigned,
-            sub.normalizedGuarantee, Resource.newInstance(1, 1));
+
+        // How much resource we offer to the queue (to increase its ideal_alloc
+        Resource wQavail = Resources.multiplyAndNormalizeUp(rc,
+            dupUnassignedForTheRound,
+            sub.normalizedGuarantee, this.stepFactor);
+
+        // Make sure it is not beyond unassigned
+        wQavail = Resources.componentwiseMin(wQavail, unassigned);
+
         Resource wQidle = sub.offer(wQavail, rc, totGuarant,
             isReservedPreemptionCandidatesSelector);
         Resource wQdone = Resources.subtract(wQavail, wQidle);
@@ -180,9 +201,12 @@ public class AbstractPreemptableResourceCalculator {
           // queue, recalculating its order based on need.
           orderedByNeed.add(sub);
         }
-        Resources.addTo(wQassigned, wQdone);
+
+        Resources.subtractFrom(unassigned, wQdone);
+
+        // Make sure unassigned is always larger than 0
+        unassigned = Resources.componentwiseMax(unassigned, Resources.none());
       }
-      Resources.subtractFrom(unassigned, wQassigned);
     }
 
     // Sometimes its possible that, all queues are properly served. So intra

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d5509c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.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/monitor/capacity/CapacitySchedulerPreemptionUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
index f097e9c..5396d61 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@@ -132,6 +133,16 @@ public class CapacitySchedulerPreemptionUtils {
    *          map to hold preempted containers
    * @param totalPreemptionAllowed
    *          total preemption allowed per round
+   * @param conservativeDRF
+   *          should we do conservativeDRF preemption or not.
+   *          When true:
+   *            stop preempt container when any major resource type <= 0 for to-
+   *            preempt.
+   *            This is default preemption behavior of intra-queue preemption
+   *          When false:
+   *            stop preempt container when: all major resource type <= 0 for
+   *            to-preempt.
+   *            This is default preemption behavior of inter-queue preemption
    * @return should we preempt rmContainer. If we should, deduct from
    *         <code>resourceToObtainByPartition</code>
    */
@@ -140,7 +151,7 @@ public class CapacitySchedulerPreemptionUtils {
       Map<String, Resource> resourceToObtainByPartitions,
       RMContainer rmContainer, Resource clusterResource,
       Map<ApplicationAttemptId, Set<RMContainer>> preemptMap,
-      Resource totalPreemptionAllowed) {
+      Resource totalPreemptionAllowed, boolean conservativeDRF) {
     ApplicationAttemptId attemptId = rmContainer.getApplicationAttemptId();
 
     // We will not account resource of a container twice or more
@@ -152,13 +163,49 @@ public class CapacitySchedulerPreemptionUtils {
         rmContainer.getAllocatedNode());
     Resource toObtainByPartition = resourceToObtainByPartitions
         .get(nodePartition);
+    if (null == toObtainByPartition) {
+      return false;
+    }
+
+    // If a toObtain resource type == 0, set it to -1 to avoid 0 resource
+    // type affect following doPreemption check: isAnyMajorResourceZero
+    for (ResourceInformation ri : toObtainByPartition.getResources()) {
+      if (ri.getValue() == 0) {
+        ri.setValue(-1);
+      }
+    }
+
+    if (rc.isAnyMajorResourceAboveZero(toObtainByPartition) && Resources.fitsIn(
+        rc, rmContainer.getAllocatedResource(), totalPreemptionAllowed)) {
+      boolean doPreempt;
+
+      // How much resource left after preemption happen.
+      Resource toObtainAfterPreemption = Resources.subtract(toObtainByPartition,
+          rmContainer.getAllocatedResource());
+
+      if (conservativeDRF) {
+        doPreempt = !rc.isAnyMajorResourceZeroOrNegative(toObtainByPartition);
+      } else {
+        // When we want to do more aggressive preemption, we will do preemption
+        // only if:
+        // - The preempt of the container makes positive contribution to the
+        //   to-obtain resource. Positive contribution means any positive
+        //   resource type decreases.
+        //
+        //   This is example of positive contribution:
+        //     * before: <30, 10, 5>, after <20, 10, -10>
+        //   But this not positive contribution:
+        //     * before: <30, 10, 0>, after <30, 10, -15>
+        doPreempt = Resources.lessThan(rc, clusterResource,
+            Resources
+                .componentwiseMax(toObtainAfterPreemption, Resources.none()),
+            Resources.componentwiseMax(toObtainByPartition, Resources.none()));
+      }
+
+      if (!doPreempt) {
+        return false;
+      }
 
-    if (null != toObtainByPartition
-        && Resources.greaterThan(rc, clusterResource, toObtainByPartition,
-            Resources.none())
-        && Resources.fitsIn(rc, rmContainer.getAllocatedResource(),
-            totalPreemptionAllowed)
-        && !Resources.isAnyMajorResourceZero(rc, toObtainByPartition)) {
       Resources.subtractFrom(toObtainByPartition,
           rmContainer.getAllocatedResource());
       Resources.subtractFrom(totalPreemptionAllowed,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d5509c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.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/monitor/capacity/FifoCandidatesSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.java
index 748548a..3b2fcbb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.java
@@ -111,7 +111,7 @@ public class FifoCandidatesSelector
                   .tryPreemptContainerAndDeductResToObtain(rc,
                       preemptionContext, resToObtainByPartition, c,
                       clusterResource, selectedCandidates,
-                      totalPreemptionAllowed);
+                      totalPreemptionAllowed, false);
               if (!preempted) {
                 continue;
               }
@@ -187,7 +187,7 @@ public class FifoCandidatesSelector
       boolean preempted = CapacitySchedulerPreemptionUtils
           .tryPreemptContainerAndDeductResToObtain(rc, preemptionContext,
               resToObtainByPartition, c, clusterResource, preemptMap,
-              totalPreemptionAllowed);
+              totalPreemptionAllowed, false);
       if (preempted) {
         Resources.subtractFrom(skippedAMSize, c.getAllocatedResource());
       }
@@ -221,7 +221,7 @@ public class FifoCandidatesSelector
       // Try to preempt this container
       CapacitySchedulerPreemptionUtils.tryPreemptContainerAndDeductResToObtain(
           rc, preemptionContext, resToObtainByPartition, c, clusterResource,
-          selectedContainers, totalPreemptionAllowed);
+          selectedContainers, totalPreemptionAllowed, false);
 
       if (!preemptionContext.isObserveOnly()) {
         preemptionContext.getRMContext().getDispatcher().getEventHandler()
@@ -264,7 +264,7 @@ public class FifoCandidatesSelector
       // Try to preempt this container
       CapacitySchedulerPreemptionUtils.tryPreemptContainerAndDeductResToObtain(
           rc, preemptionContext, resToObtainByPartition, c, clusterResource,
-          selectedContainers, totalPreemptionAllowed);
+          selectedContainers, totalPreemptionAllowed, false);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d5509c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoIntraQueuePreemptionPlugin.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/monitor/capacity/FifoIntraQueuePreemptionPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoIntraQueuePreemptionPlugin.java
index 1776bd4..40f333f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoIntraQueuePreemptionPlugin.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoIntraQueuePreemptionPlugin.java
@@ -278,8 +278,8 @@ public class FifoIntraQueuePreemptionPlugin
 
       // Once unallocated resource is 0, we can stop assigning ideal per app.
       if (Resources.lessThanOrEqual(rc, clusterResource,
-          queueReassignableResource, Resources.none())
-          || Resources.isAnyMajorResourceZero(rc, queueReassignableResource)) {
+          queueReassignableResource, Resources.none()) || rc
+          .isAnyMajorResourceZeroOrNegative(queueReassignableResource)) {
         continue;
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d5509c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/IntraQueueCandidatesSelector.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/monitor/capacity/IntraQueueCandidatesSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/IntraQueueCandidatesSelector.java
index 5b6932e..a91fac7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/IntraQueueCandidatesSelector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/IntraQueueCandidatesSelector.java
@@ -230,7 +230,7 @@ public class IntraQueueCandidatesSelector extends PreemptionCandidatesSelector {
       boolean ret = CapacitySchedulerPreemptionUtils
           .tryPreemptContainerAndDeductResToObtain(rc, preemptionContext,
               resToObtainByPartition, c, clusterResource, selectedCandidates,
-              totalPreemptedResourceAllowed);
+              totalPreemptedResourceAllowed, true);
 
       // Subtract from respective user's resource usage once a container is
       // selected for preemption.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d5509c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/PreemptableResourceCalculator.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/monitor/capacity/PreemptableResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/PreemptableResourceCalculator.java
index 676c14f..08d834e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/PreemptableResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/PreemptableResourceCalculator.java
@@ -41,8 +41,6 @@ public class PreemptableResourceCalculator
   private static final Log LOG =
       LogFactory.getLog(PreemptableResourceCalculator.class);
 
-  private boolean isReservedPreemptionCandidatesSelector;
-
   /**
    * PreemptableResourceCalculator constructor
    *
@@ -95,8 +93,8 @@ public class PreemptableResourceCalculator
     }
 
     // first compute the allocation as a fixpoint based on guaranteed capacity
-    computeFixpointAllocation(tot_guarant, nonZeroGuarQueues, unassigned,
-        false);
+    computeFixpointAllocation(tot_guarant, new HashSet<>(nonZeroGuarQueues),
+        unassigned, false);
 
     // if any capacity is left unassigned, distributed among zero-guarantee
     // queues uniformly (i.e., not based on guaranteed capacity, as this is zero)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d5509c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.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/monitor/capacity/TempQueuePerPartition.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java
index 9d8297d..4214acc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java
@@ -151,7 +151,7 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
     //               # This is for leaf queue only.
     //               max(guaranteed, used) - assigned}
     // remain = avail - accepted
-    Resource accepted = Resources.min(rc, clusterResource,
+    Resource accepted = Resources.componentwiseMin(
         absMaxCapIdealAssignedDelta,
         Resources.min(rc, clusterResource, avail, Resources
             /*
@@ -186,6 +186,12 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
 
     accepted = acceptedByLocality(rc, accepted);
 
+    // accept should never be < 0
+    accepted = Resources.componentwiseMax(accepted, Resources.none());
+
+    // or more than offered
+    accepted = Resources.componentwiseMin(accepted, avail);
+
     Resource remain = Resources.subtract(avail, accepted);
     Resources.addTo(idealAssigned, accepted);
     return remain;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d5509c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.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/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java
index a8e2697..a972584 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceInformation;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@@ -59,6 +60,7 @@ import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 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 org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.mockito.ArgumentMatcher;
@@ -104,10 +106,32 @@ public class ProportionalCapacityPreemptionPolicyMockFramework {
   EventHandler<Event> mDisp = null;
   ProportionalCapacityPreemptionPolicy policy = null;
   Resource clusterResource = null;
+  // Initialize resource map
+  Map<String, ResourceInformation> riMap = new HashMap<>();
+
+  private void resetResourceInformationMap() {
+    // Initialize mandatory resources
+    ResourceInformation memory = ResourceInformation.newInstance(
+        ResourceInformation.MEMORY_MB.getName(),
+        ResourceInformation.MEMORY_MB.getUnits(),
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
+    ResourceInformation vcores = ResourceInformation.newInstance(
+        ResourceInformation.VCORES.getName(),
+        ResourceInformation.VCORES.getUnits(),
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
+    riMap.put(ResourceInformation.MEMORY_URI, memory);
+    riMap.put(ResourceInformation.VCORES_URI, vcores);
+
+    ResourceUtils.initializeResourcesFromResourceInformationMap(riMap);
+  }
 
   @SuppressWarnings("unchecked")
   @Before
   public void setup() {
+    resetResourceInformationMap();
+
     org.apache.log4j.Logger.getRootLogger().setLevel(
         org.apache.log4j.Level.DEBUG);
 
@@ -142,6 +166,12 @@ public class ProportionalCapacityPreemptionPolicyMockFramework {
     partitionToResource = new HashMap<>();
     nodeIdToSchedulerNodes = new HashMap<>();
     nameToCSQueues = new HashMap<>();
+    clusterResource = Resource.newInstance(0, 0);
+  }
+
+  @After
+  public void cleanup() {
+    resetResourceInformationMap();
   }
 
   public void buildEnv(String labelsConfig, String nodesConfig,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d5509c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestPreemptionForQueueWithPriorities.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/monitor/capacity/TestPreemptionForQueueWithPriorities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestPreemptionForQueueWithPriorities.java
index e9a8116..6a953cf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestPreemptionForQueueWithPriorities.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestPreemptionForQueueWithPriorities.java
@@ -20,44 +20,25 @@ package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity;
 
 import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
 import org.apache.hadoop.yarn.api.records.ResourceInformation;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
 
 import static org.mockito.Matchers.argThat;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 public class TestPreemptionForQueueWithPriorities
     extends ProportionalCapacityPreemptionPolicyMockFramework {
-  // Initialize resource map
-  private Map<String, ResourceInformation> riMap = new HashMap<>();
-
   @Before
   public void setup() {
-
-    // Initialize mandatory resources
-    ResourceInformation memory = ResourceInformation.newInstance(
-        ResourceInformation.MEMORY_MB.getName(),
-        ResourceInformation.MEMORY_MB.getUnits(),
-        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
-        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
-    ResourceInformation vcores = ResourceInformation.newInstance(
-        ResourceInformation.VCORES.getName(),
-        ResourceInformation.VCORES.getUnits(),
-        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
-        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
-    riMap.put(ResourceInformation.MEMORY_URI, memory);
-    riMap.put(ResourceInformation.VCORES_URI, vcores);
-
-    ResourceUtils.initializeResourcesFromResourceInformationMap(riMap);
-
+    rc = new DefaultResourceCalculator();
     super.setup();
     policy = new ProportionalCapacityPreemptionPolicy(rmContext, cs, mClock);
   }
@@ -340,8 +321,8 @@ public class TestPreemptionForQueueWithPriorities
      *   - a2 (capacity=60), p=1
      * - b (capacity=30), p=1
      *   - b1 (capacity=50), p=1
-     *   - b1 (capacity=50), p=2
-     * - c (capacity=40), p=2
+     *   - b2 (capacity=50), p=2
+     * - c (capacity=40), p=1
      * </pre>
      */
     String labelsConfig = "=100,true"; // default partition
@@ -349,11 +330,11 @@ public class TestPreemptionForQueueWithPriorities
     String queuesConfig =
         // guaranteed,max,used,pending
         "root(=[100 100 100 100]);" + //root
-            "-a(=[30 100 40 50]){priority=1};" + // a
+            "-a(=[29 100 40 50]){priority=1};" + // a
             "--a1(=[12 100 20 50]){priority=1};" + // a1
-            "--a2(=[18 100 20 50]){priority=1};" + // a2
-            "-b(=[30 100 59 50]){priority=1};" + // b
-            "--b1(=[15 100 30 50]){priority=1};" + // b1
+            "--a2(=[17 100 20 50]){priority=1};" + // a2
+            "-b(=[31 100 59 50]){priority=1};" + // b
+            "--b1(=[16 100 30 50]){priority=1};" + // b1
             "--b2(=[15 100 29 50]){priority=2};" + // b2
             "-c(=[40 100 1 30]){priority=1}";   // c
     String appsConfig =
@@ -362,7 +343,7 @@ public class TestPreemptionForQueueWithPriorities
             "a2\t(1,1,n1,,20,false);" + // app2 in a2
             "b1\t(1,1,n1,,30,false);" + // app3 in b1
             "b2\t(1,1,n1,,29,false);" + // app4 in b2
-            "c\t(1,1,n1,,29,false)"; // app5 in c
+            "c\t(1,1,n1,,1,false)"; // app5 in c
 
 
     buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
@@ -370,16 +351,16 @@ public class TestPreemptionForQueueWithPriorities
 
     // Preemption should first divide capacities between a / b, and b2 should
     // get less preemption than b1 (because b2 has higher priority)
-    verify(mDisp, times(5)).handle(argThat(
+    verify(mDisp, times(6)).handle(argThat(
         new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
             getAppAttemptId(1))));
-    verify(mDisp, never()).handle(argThat(
+    verify(mDisp, times(1)).handle(argThat(
         new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
             getAppAttemptId(2))));
-    verify(mDisp, times(15)).handle(argThat(
+    verify(mDisp, times(13)).handle(argThat(
         new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
             getAppAttemptId(3))));
-    verify(mDisp, times(9)).handle(argThat(
+    verify(mDisp, times(10)).handle(argThat(
         new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
             getAppAttemptId(4))));
   }
@@ -426,7 +407,7 @@ public class TestPreemptionForQueueWithPriorities
 
     // Preemption should first divide capacities between a / b, and b1 should
     // get less preemption than b2 (because b1 has higher priority)
-    verify(mDisp, never()).handle(argThat(
+    verify(mDisp, times(3)).handle(argThat(
         new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
             getAppAttemptId(1))));
     verify(mDisp, never()).handle(argThat(
@@ -505,4 +486,56 @@ public class TestPreemptionForQueueWithPriorities
             getAppAttemptId(3))));
   }
 
+  @Test
+  public void test3ResourceTypesInterQueuePreemption() throws IOException {
+    rc = new DominantResourceCalculator();
+    when(cs.getResourceCalculator()).thenReturn(rc);
+
+    // Initialize resource map
+    String RESOURCE_1 = "res1";
+    riMap.put(RESOURCE_1, ResourceInformation.newInstance(RESOURCE_1, "", 0,
+        ResourceTypes.COUNTABLE, 0, Integer.MAX_VALUE));
+
+    ResourceUtils.initializeResourcesFromResourceInformationMap(riMap);
+
+    /**
+     * Queue structure is:
+     *
+     * <pre>
+     *              root
+     *           /  \  \
+     *          a    b  c
+     * </pre>
+     *  A / B / C have 33.3 / 33.3 / 33.4 resources
+     *  Total cluster resource have mem=30, cpu=18, GPU=6
+     *  A uses mem=6, cpu=3, GPU=3
+     *  B uses mem=6, cpu=3, GPU=3
+     *  C is asking mem=1,cpu=1,GPU=1
+     *
+     *  We expect it can preempt from one of the jobs
+     */
+    String labelsConfig =
+        "=30:18:6,true;";
+    String nodesConfig =
+        "n1= res=30:18:6;"; // n1 is default partition
+    String queuesConfig =
+        // guaranteed,max,used,pending
+        "root(=[30:18:6 30:18:6 12:12:6 1:1:1]){priority=1};" + //root
+            "-a(=[10:6:2 10:6:2 6:6:3 0:0:0]){priority=1};" + // a
+            "-b(=[10:6:2 10:6:2 6:6:3 0:0:0]){priority=1};" + // b
+            "-c(=[10:6:2 10:6:2 0:0:0 1:1:1]){priority=2}"; // c
+    String appsConfig=
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a\t" // app1 in a1
+            + "(1,2:2:1,n1,,3,false);" +
+            "b\t" // app2 in b2
+            + "(1,2:2:1,n1,,3,false)";
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    verify(mDisp, times(1)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(1))));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d5509c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyInterQueueWithDRF.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/monitor/capacity/TestProportionalCapacityPreemptionPolicyInterQueueWithDRF.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyInterQueueWithDRF.java
index c8a1f0f..14a3a9a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyInterQueueWithDRF.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyInterQueueWithDRF.java
@@ -18,11 +18,16 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity;
 
+import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.junit.Before;
 import org.junit.Test;
 
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import java.io.IOException;
+
 import static org.mockito.Matchers.argThat;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
@@ -41,8 +46,7 @@ public class TestProportionalCapacityPreemptionPolicyInterQueueWithDRF
   }
 
   @Test
-  public void testInterQueuePreemptionWithMultipleResource()
-      throws Exception {
+  public void testInterQueuePreemptionWithMultipleResource() throws Exception {
     /**
      * Queue structure is:
      *
@@ -121,4 +125,52 @@ public class TestProportionalCapacityPreemptionPolicyInterQueueWithDRF
         new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
             getAppAttemptId(1))));
   }
-}
+
+  @Test
+  public void test3ResourceTypesInterQueuePreemption() throws IOException {
+    // Initialize resource map
+    String RESOURCE_1 = "res1";
+    riMap.put(RESOURCE_1, ResourceInformation
+        .newInstance(RESOURCE_1, "", 0, ResourceTypes.COUNTABLE, 0,
+            Integer.MAX_VALUE));
+
+    ResourceUtils.initializeResourcesFromResourceInformationMap(riMap);
+
+    /*
+     *              root
+     *           /  \  \
+     *          a    b  c
+     *
+     *  A / B / C have 33.3 / 33.3 / 33.4 resources
+     *  Total cluster resource have mem=30, cpu=18, GPU=6
+     *  A uses mem=6, cpu=3, GPU=3
+     *  B uses mem=6, cpu=3, GPU=3
+     *  C is asking mem=1,cpu=1,GPU=1
+     *
+     *  We expect it can preempt from one of the jobs
+     */
+    String labelsConfig = "=30:18:6,true;";
+    String nodesConfig = "n1= res=30:18:6;"; // n1 is default partition
+    String queuesConfig =
+        // guaranteed,max,used,pending
+        "root(=[30:18:6 30:18:6 12:12:6 1:1:1]);" + //root
+            "-a(=[10:7:2 10:6:3 6:6:3 0:0:0]);" + // a
+            "-b(=[10:6:2 10:6:3 6:6:3 0:0:0]);" + // b
+            "-c(=[10:5:2 10:6:2 0:0:0 1:1:1])"; // c
+    String appsConfig =
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a\t" // app1 in a1
+            + "(1,2:2:1,n1,,3,false);" + "b\t" // app2 in b2
+            + "(1,2:2:1,n1,,3,false)";
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    verify(mDisp, times(0)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(1))));
+    verify(mDisp, times(1)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(2))));
+  }
+}
\ 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


[39/50] [abbrv] hadoop git commit: HDDS-114. Ozone Datanode mbean registration fails for StorageLocation. Contributed by Elek, Marton.

Posted by ha...@apache.org.
HDDS-114. Ozone Datanode mbean registration fails for StorageLocation.
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/24169062
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/24169062
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/24169062

Branch: refs/heads/HDDS-48
Commit: 24169062e5f4e7798a47c5e6e3e94504cba73092
Parents: 30284d0
Author: Anu Engineer <ae...@apache.org>
Authored: Tue May 29 13:23:58 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Tue May 29 13:48:55 2018 -0700

----------------------------------------------------------------------
 .../common/impl/StorageLocationReport.java      | 52 +++++++++++---------
 .../ContainerLocationManagerMXBean.java         |  4 +-
 .../interfaces/StorageLocationReportMXBean.java | 40 +++++++++++++++
 3 files changed, 71 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/24169062/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java
index 87b9656..061d09b 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java
@@ -23,6 +23,8 @@ import org.apache.hadoop.hdds.protocol.proto.
     StorageContainerDatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdds.protocol.proto.
     StorageContainerDatanodeProtocolProtos.StorageTypeProto;
+import org.apache.hadoop.ozone.container.common.interfaces
+    .StorageLocationReportMXBean;
 
 import java.io.IOException;
 
@@ -30,7 +32,8 @@ import java.io.IOException;
  * Storage location stats of datanodes that provide back store for containers.
  *
  */
-public class StorageLocationReport {
+public final class StorageLocationReport implements
+    StorageLocationReportMXBean {
 
   private final String id;
   private final boolean failed;
@@ -76,6 +79,11 @@ public class StorageLocationReport {
     return storageLocation;
   }
 
+  @Override
+  public String getStorageTypeName() {
+    return storageType.name();
+  }
+
   public StorageType getStorageType() {
     return storageType;
   }
@@ -204,76 +212,76 @@ public class StorageLocationReport {
     /**
      * Sets the storageId.
      *
-     * @param id storageId
+     * @param idValue storageId
      * @return StorageLocationReport.Builder
      */
-    public Builder setId(String id) {
-      this.id = id;
+    public Builder setId(String idValue) {
+      this.id = idValue;
       return this;
     }
 
     /**
      * Sets whether the volume failed or not.
      *
-     * @param failed whether volume failed or not
+     * @param failedValue whether volume failed or not
      * @return StorageLocationReport.Builder
      */
-    public Builder setFailed(boolean failed) {
-      this.failed = failed;
+    public Builder setFailed(boolean failedValue) {
+      this.failed = failedValue;
       return this;
     }
 
     /**
      * Sets the capacity of volume.
      *
-     * @param capacity capacity
+     * @param capacityValue capacity
      * @return StorageLocationReport.Builder
      */
-    public Builder setCapacity(long capacity) {
-      this.capacity = capacity;
+    public Builder setCapacity(long capacityValue) {
+      this.capacity = capacityValue;
       return this;
     }
     /**
      * Sets the scmUsed Value.
      *
-     * @param scmUsed storage space used by scm
+     * @param scmUsedValue storage space used by scm
      * @return StorageLocationReport.Builder
      */
-    public Builder setScmUsed(long scmUsed) {
-      this.scmUsed = scmUsed;
+    public Builder setScmUsed(long scmUsedValue) {
+      this.scmUsed = scmUsedValue;
       return this;
     }
 
     /**
      * Sets the remaining free space value.
      *
-     * @param remaining remaining free space
+     * @param remainingValue remaining free space
      * @return StorageLocationReport.Builder
      */
-    public Builder setRemaining(long remaining) {
-      this.remaining = remaining;
+    public Builder setRemaining(long remainingValue) {
+      this.remaining = remainingValue;
       return this;
     }
 
     /**
      * Sets the storageType.
      *
-     * @param storageType type of the storage used
+     * @param storageTypeValue type of the storage used
      * @return StorageLocationReport.Builder
      */
-    public Builder setStorageType(StorageType storageType) {
-      this.storageType = storageType;
+    public Builder setStorageType(StorageType storageTypeValue) {
+      this.storageType = storageTypeValue;
       return this;
     }
 
     /**
      * Sets the storageLocation.
      *
-     * @param storageLocation location of the volume
+     * @param storageLocationValue location of the volume
      * @return StorageLocationReport.Builder
      */
-    public Builder setStorageLocation(String storageLocation) {
-      this.storageLocation = storageLocation;
+    public Builder setStorageLocation(String storageLocationValue) {
+      this.storageLocation = storageLocationValue;
       return this;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24169062/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerLocationManagerMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerLocationManagerMXBean.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerLocationManagerMXBean.java
index 88e6148..97d2dc3 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerLocationManagerMXBean.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerLocationManagerMXBean.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.ozone.container.common.interfaces;
 
-import org.apache.hadoop.ozone.container.common.impl.StorageLocationReport;
-
 import java.io.IOException;
 
 /**
@@ -31,6 +29,6 @@ public interface ContainerLocationManagerMXBean {
    *
    * @return storage location usage report.
    */
-  StorageLocationReport[] getLocationReport() throws IOException;
+  StorageLocationReportMXBean[] getLocationReport() throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24169062/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/StorageLocationReportMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/StorageLocationReportMXBean.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/StorageLocationReportMXBean.java
new file mode 100644
index 0000000..fd06367
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/StorageLocationReportMXBean.java
@@ -0,0 +1,40 @@
+/*
+ * 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.interfaces;
+
+/**
+ * Contract to define properties available on the JMX interface.
+ */
+public interface StorageLocationReportMXBean {
+
+  String getId();
+
+  boolean isFailed();
+
+  long getCapacity();
+
+  long getScmUsed();
+
+  long getRemaining();
+
+  String getStorageLocation();
+
+  String getStorageTypeName();
+
+}


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


[20/50] [abbrv] hadoop git commit: MAPREDUCE-7097. MapReduce JHS should honor yarn.webapp.filter-entity-list-by-user. Contributed by Sunil Govindan.

Posted by ha...@apache.org.
MAPREDUCE-7097. MapReduce JHS should honor yarn.webapp.filter-entity-list-by-user. Contributed by  Sunil Govindan.


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

Branch: refs/heads/HDDS-48
Commit: 88cbe57c069a1d2dd3bfb32e3ad742566470a10b
Parents: d14e26b
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Mon May 28 12:45:07 2018 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Mon May 28 14:05:49 2018 +0530

----------------------------------------------------------------------
 .../mapreduce/v2/hs/webapp/HsJobBlock.java      | 18 ++++++++++++++-
 .../mapreduce/v2/hs/webapp/TestHsJobBlock.java  | 20 ++++++++++++++--
 .../apache/hadoop/yarn/webapp/Controller.java   |  4 ++++
 .../org/apache/hadoop/yarn/webapp/View.java     | 24 +++++++++++++-------
 4 files changed, 55 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/88cbe57c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java
index 18040f0..9b845cd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java
@@ -27,6 +27,8 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI._TH;
 import java.util.Date;
 import java.util.List;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -39,8 +41,10 @@ import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobInfo;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.ResponseInfo;
 import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
@@ -56,9 +60,14 @@ import com.google.inject.Inject;
  */
 public class HsJobBlock extends HtmlBlock {
   final AppContext appContext;
+  private UserGroupInformation ugi;
+  private boolean isFilterAppListByUserEnabled;
 
-  @Inject HsJobBlock(AppContext appctx) {
+  @Inject HsJobBlock(Configuration conf, AppContext appctx, ViewContext ctx) {
+    super(ctx);
     appContext = appctx;
+    isFilterAppListByUserEnabled = conf
+        .getBoolean(YarnConfiguration.FILTER_ENTITY_LIST_BY_USER, false);
   }
 
   /*
@@ -78,6 +87,13 @@ public class HsJobBlock extends HtmlBlock {
       html.p().__("Sorry, ", jid, " not found.").__();
       return;
     }
+    ugi = getCallerUGI();
+    if (isFilterAppListByUserEnabled && ugi != null
+        && !j.checkAccess(ugi, JobACL.VIEW_JOB)) {
+      html.p().__("Sorry, ", jid, " could not be viewed for '",
+          ugi.getUserName(), "'.").__();
+      return;
+    }
     if(j instanceof UnparsedJob) {
       final int taskCount = j.getTotalMaps() + j.getTotalReduces();
       UnparsedJob oversizedJob = (UnparsedJob) j;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/88cbe57c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsJobBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsJobBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsJobBlock.java
index 7fa238e..48e3d3b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsJobBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsJobBlock.java
@@ -33,8 +33,10 @@ import org.apache.hadoop.mapreduce.v2.hs.UnparsedJob;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.util.StringHelper;
+import org.apache.hadoop.yarn.webapp.Controller;
 import org.apache.hadoop.yarn.webapp.ResponseInfo;
 import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.hadoop.yarn.webapp.View.ViewContext;
 import org.apache.hadoop.yarn.webapp.view.BlockForTest;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlockForTest;
@@ -49,6 +51,8 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Map;
 
+import javax.servlet.http.HttpServletRequest;
+
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -69,7 +73,13 @@ public class TestHsJobBlock {
         new JobHistoryStubWithAllOversizeJobs(maxAllowedTaskNum);
     jobHistory.init(config);
 
-    HsJobBlock jobBlock = new HsJobBlock(jobHistory) {
+    Controller.RequestContext rc = mock(Controller.RequestContext.class);
+    ViewContext view = mock(ViewContext.class);
+    HttpServletRequest req =mock(HttpServletRequest.class);
+    when(rc.getRequest()).thenReturn(req);
+    when(view.requestContext()).thenReturn(rc);
+
+    HsJobBlock jobBlock = new HsJobBlock(config, jobHistory, view) {
       // override this so that job block can fetch a job id.
       @Override
       public Map<String, String> moreParams() {
@@ -101,7 +111,13 @@ public class TestHsJobBlock {
     JobHistory jobHistory = new JobHitoryStubWithAllNormalSizeJobs();
     jobHistory.init(config);
 
-    HsJobBlock jobBlock = new HsJobBlock(jobHistory) {
+    Controller.RequestContext rc = mock(Controller.RequestContext.class);
+    ViewContext view = mock(ViewContext.class);
+    HttpServletRequest req =mock(HttpServletRequest.class);
+    when(rc.getRequest()).thenReturn(req);
+    when(view.requestContext()).thenReturn(rc);
+
+    HsJobBlock jobBlock = new HsJobBlock(config, jobHistory, view) {
       // override this so that the job block can fetch a job id.
       @Override
       public Map<String, String> moreParams() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/88cbe57c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Controller.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Controller.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Controller.java
index dc4eee2..1b25b84 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Controller.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Controller.java
@@ -108,6 +108,10 @@ public abstract class Controller implements Params {
     }
 
     public String prefix() { return prefix; }
+
+    public HttpServletRequest getRequest() {
+      return request;
+    }
   }
 
   private RequestContext context;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/88cbe57c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/View.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/View.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/View.java
index c16787d..666a0bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/View.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/View.java
@@ -96,28 +96,36 @@ public abstract class View implements Params {
     return vc;
   }
 
-  public Throwable error() { return context().rc.error; }
+  public Throwable error() {
+    return context().requestContext().error;
+  }
 
-  public int status() { return context().rc.status; }
+  public int status() {
+    return context().requestContext().status;
+  }
 
-  public boolean inDevMode() { return context().rc.devMode; }
+  public boolean inDevMode() {
+    return context().requestContext().devMode;
+  }
 
-  public Injector injector() { return context().rc.injector; }
+  public Injector injector() {
+    return context().requestContext().injector;
+  }
 
   public <T> T getInstance(Class<T> cls) {
     return injector().getInstance(cls);
   }
 
   public HttpServletRequest request() {
-    return context().rc.request;
+    return context().requestContext().getRequest();
   }
 
   public HttpServletResponse response() {
-    return context().rc.response;
+    return context().requestContext().response;
   }
 
   public Map<String, String> moreParams() {
-    return context().rc.moreParams();
+    return context().requestContext().moreParams();
   }
 
   /**
@@ -125,7 +133,7 @@ public abstract class View implements Params {
    * @return the cookies map
    */
   public Map<String, Cookie> cookies() {
-    return context().rc.cookies();
+    return context().requestContext().cookies();
   }
 
   public ServletOutputStream outputStream() {


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


[50/50] [abbrv] hadoop git commit: HDDS-90: Create ContainerData, Container classes. Contributed by Bharat Viswanadham

Posted by ha...@apache.org.
HDDS-90: Create ContainerData, Container classes. 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/6cd19b45
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6cd19b45
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6cd19b45

Branch: refs/heads/HDDS-48
Commit: 6cd19b45efbbcce6d6ca4b5c0eb3beb42d95e558
Parents: ee1e0e2
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Fri May 25 15:11:19 2018 -0700
Committer: Hanisha Koneru <ha...@apache.org>
Committed: Wed May 30 14:04:19 2018 -0700

----------------------------------------------------------------------
 .../main/proto/DatanodeContainerProtocol.proto  |   8 +
 .../common/impl/ChunkLayOutVersion.java         |  80 +++++++
 .../container/common/impl/ContainerData.java    | 234 +++++++++++++++++++
 .../common/impl/KeyValueContainer.java          |  74 ++++++
 .../common/impl/KeyValueContainerData.java      | 159 +++++++++++++
 .../container/common/interfaces/Container.java  |  75 ++++++
 .../common/TestChunkLayOutVersion.java          |  42 ++++
 .../common/TestKeyValueContainerData.java       | 119 ++++++++++
 8 files changed, 791 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd19b45/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 53da18a..72e1006 100644
--- a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
+++ b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
@@ -232,6 +232,14 @@ message ContainerData {
   optional string containerDBType = 11;
 }
 
+// This is used for create Container Request.
+message CreateContainerData {
+  required int64 containerId = 1;
+  repeated KeyValue metadata = 2;
+  optional ContainerType containerType = 3 [default = KeyValueContainer];
+}
+
+
 enum ContainerType {
   KeyValueContainer = 1;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd19b45/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkLayOutVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkLayOutVersion.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkLayOutVersion.java
new file mode 100644
index 0000000..fff68de6
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkLayOutVersion.java
@@ -0,0 +1,80 @@
+/*
+ * 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;
+
+
+/**
+ * Defines layout versions for the Chunks.
+ */
+
+public final class ChunkLayOutVersion {
+
+  private final static ChunkLayOutVersion[] CHUNK_LAYOUT_VERSION_INFOS =
+      {new ChunkLayOutVersion(1, "Data without checksums.")};
+
+  private int version;
+  private String description;
+
+
+  /**
+   * Never created outside this class.
+   *
+   * @param description -- description
+   * @param version     -- version number
+   */
+  private ChunkLayOutVersion(int version, String description) {
+    this.version = version;
+    this.description = description;
+  }
+
+  /**
+   * Returns all versions.
+   *
+   * @return Version info array.
+   */
+  public static ChunkLayOutVersion[] getAllVersions() {
+    return CHUNK_LAYOUT_VERSION_INFOS.clone();
+  }
+
+  /**
+   * Returns the latest version.
+   *
+   * @return versionInfo
+   */
+  public static ChunkLayOutVersion getLatestVersion() {
+    return CHUNK_LAYOUT_VERSION_INFOS[CHUNK_LAYOUT_VERSION_INFOS.length - 1];
+  }
+
+  /**
+   * Return version.
+   *
+   * @return int
+   */
+  public int getVersion() {
+    return version;
+  }
+
+  /**
+   * Returns description.
+   * @return String
+   */
+  public String getDescription() {
+    return description;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd19b45/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
new file mode 100644
index 0000000..a4b2130
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
@@ -0,0 +1,234 @@
+/*
+ * 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.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
+    ContainerType;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
+    ContainerLifeCycleState;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * ContainerData is the in-memory representation of container metadata and is
+ * represented on disk by the .container file.
+ */
+public class ContainerData {
+
+  //Type of the container.
+  // For now, we support only KeyValueContainer.
+  private final ContainerType containerType;
+
+  // Unique identifier for the container
+  private final long containerId;
+
+  // Layout version of the container data
+  private final ChunkLayOutVersion layOutVersion;
+
+  // Metadata of the container will be a key value pair.
+  // This can hold information like volume name, owner etc.,
+  private final Map<String, String> metadata;
+
+  // State of the Container
+  private ContainerLifeCycleState state;
+
+  /** parameters for read/write statistics on the container. **/
+  private final AtomicLong readBytes;
+  private final AtomicLong writeBytes;
+  private final AtomicLong readCount;
+  private final AtomicLong writeCount;
+
+
+  /**
+   * Creates a ContainerData Object, which holds metadata of the container.
+   * @param type - ContainerType
+   * @param containerId - ContainerId
+   */
+  public ContainerData(ContainerType type, long containerId) {
+    this.containerType = type;
+    this.containerId = containerId;
+    this.layOutVersion = ChunkLayOutVersion.getLatestVersion();
+    this.metadata = new TreeMap<>();
+    this.state = ContainerLifeCycleState.OPEN;
+    this.readCount = new AtomicLong(0L);
+    this.readBytes =  new AtomicLong(0L);
+    this.writeCount =  new AtomicLong(0L);
+    this.writeBytes =  new AtomicLong(0L);
+  }
+
+  /**
+   * Returns the containerId.
+   */
+  public long getContainerId() {
+    return containerId;
+  }
+
+  /**
+   * Returns the type of the container.
+   * @return ContainerType
+   */
+  public ContainerType getContainerType() {
+    return containerType;
+  }
+
+
+  /**
+   * Returns the state of the container.
+   * @return ContainerLifeCycleState
+   */
+  public synchronized ContainerLifeCycleState getState() {
+    return state;
+  }
+
+  /**
+   * Set the state of the container.
+   * @param state
+   */
+  public synchronized void setState(ContainerLifeCycleState state) {
+    this.state = state;
+  }
+
+  /**
+   * Returns the layOutVersion of the actual container data format.
+   * @return layOutVersion
+   */
+  public ChunkLayOutVersion getLayOutVersion() {
+    return layOutVersion;
+  }
+
+  /**
+   * Adds metadata.
+   */
+  public void addMetadata(String key, String value) throws IOException {
+    synchronized (this.metadata) {
+      if (this.metadata.containsKey(key)) {
+        throw new IOException("This key already exists. Key " + key);
+      }
+      metadata.put(key, value);
+    }
+  }
+
+  /**
+   * Retuns metadata of the container.
+   * @return metadata
+   */
+  public Map<String, String> getMetadata() {
+    synchronized (this.metadata) {
+      return Collections.unmodifiableMap(this.metadata);
+    }
+  }
+
+  /**
+   * checks if the container is open.
+   * @return - boolean
+   */
+  public synchronized  boolean isOpen() {
+    return ContainerLifeCycleState.OPEN == state;
+  }
+
+  /**
+   * checks if the container is invalid.
+   * @return - boolean
+   */
+  public synchronized boolean isValid() {
+    return !(ContainerLifeCycleState.INVALID == state);
+  }
+
+  /**
+   * checks if the container is closed.
+   * @return - boolean
+   */
+  public synchronized  boolean isClosed() {
+    return ContainerLifeCycleState.CLOSED == state;
+  }
+
+  /**
+   * Marks this container as closed.
+   */
+  public synchronized void closeContainer() {
+    // TODO: closed or closing here
+    setState(ContainerLifeCycleState.CLOSED);
+  }
+
+  /**
+   * Get the number of bytes read from the container.
+   * @return the number of bytes read from the container.
+   */
+  public long getReadBytes() {
+    return readBytes.get();
+  }
+
+  /**
+   * Increase the number of bytes read from the container.
+   * @param bytes number of bytes read.
+   */
+  public void incrReadBytes(long bytes) {
+    this.readBytes.addAndGet(bytes);
+  }
+
+  /**
+   * Get the number of times the container is read.
+   * @return the number of times the container is read.
+   */
+  public long getReadCount() {
+    return readCount.get();
+  }
+
+  /**
+   * Increase the number of container read count by 1.
+   */
+  public void incrReadCount() {
+    this.readCount.incrementAndGet();
+  }
+
+  /**
+   * Get the number of bytes write into the container.
+   * @return the number of bytes write into the container.
+   */
+  public long getWriteBytes() {
+    return writeBytes.get();
+  }
+
+  /**
+   * Increase the number of bytes write into the container.
+   * @param bytes the number of bytes write into the container.
+   */
+  public void incrWriteBytes(long bytes) {
+    this.writeBytes.addAndGet(bytes);
+  }
+
+  /**
+   * Get the number of writes into the container.
+   * @return the number of writes into the container.
+   */
+  public long getWriteCount() {
+    return writeCount.get();
+  }
+
+  /**
+   * Increase the number of writes into the container by 1.
+   */
+  public void incrWriteCount() {
+    this.writeCount.incrementAndGet();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd19b45/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java
new file mode 100644
index 0000000..956840b
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java
@@ -0,0 +1,74 @@
+/*
+ * 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.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+
+
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.security.NoSuchAlgorithmException;
+
+
+/**
+ * Class to perform KeyValue Container operations.
+ */
+public class KeyValueContainer implements Container {
+
+  static final Logger LOG =
+      LoggerFactory.getLogger(Container.class);
+
+  private KeyValueContainerData containerData;
+
+  public KeyValueContainer(KeyValueContainerData containerData) {
+    this.containerData = containerData;
+  }
+
+  @Override
+  public void create(ContainerData cData) throws StorageContainerException {
+
+  }
+
+  @Override
+  public void delete(boolean forceDelete)
+      throws StorageContainerException {
+
+  }
+
+  @Override
+  public void update(boolean forceUpdate)
+      throws StorageContainerException {
+
+  }
+
+  @Override
+  public ContainerData getContainerData() throws StorageContainerException {
+    return null;
+  }
+
+  @Override
+  public void close() throws StorageContainerException,
+      NoSuchAlgorithmException {
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd19b45/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
new file mode 100644
index 0000000..37eaa49
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
@@ -0,0 +1,159 @@
+/*
+ * 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.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+
+
+import java.io.IOException;
+
+/**
+ * This class represents the KeyValueContainer metadata, which is the
+ * in-memory representation of container metadata and is represented on disk
+ * by the .container file.
+ */
+public class KeyValueContainerData extends ContainerData {
+
+  // Path to Level DB/RocksDB Store.
+  private String dbPath;
+
+  // Path to Physical file system where container and checksum are stored.
+  private String containerFilePath;
+
+  //Type of DB used to store key to chunks mapping
+  private String containerDBType;
+
+  //Number of pending deletion blocks in container.
+  private int numPendingDeletionBlocks;
+
+  /**
+   * Constructs KeyValueContainerData object.
+   * @param type - containerType
+   * @param id - ContainerId
+   */
+  public KeyValueContainerData(ContainerProtos.ContainerType type, long id) {
+    super(type, id);
+    this.numPendingDeletionBlocks = 0;
+  }
+
+  /**
+   * Returns path.
+   *
+   * @return - path
+   */
+  public String getDBPath() {
+    return dbPath;
+  }
+
+  /**
+   * Sets path.
+   *
+   * @param path - String.
+   */
+  public void setDBPath(String path) {
+    this.dbPath = path;
+  }
+
+  /**
+   * Get container file path.
+   * @return - Physical path where container file and checksum is stored.
+   */
+  public String getContainerPath() {
+    return containerFilePath;
+  }
+
+  /**
+   * Set container Path.
+   * @param containerPath - File path.
+   */
+  public void setContainerPath(String containerPath) {
+    this.containerFilePath = containerPath;
+  }
+
+  /**
+   * Returns the DBType used for the container.
+   * @return containerDBType
+   */
+  public String getContainerDBType() {
+    return containerDBType;
+  }
+
+  /**
+   * Sets the DBType used for the container.
+   * @param containerDBType
+   */
+  public void setContainerDBType(String containerDBType) {
+    this.containerDBType = containerDBType;
+  }
+
+  /**
+   * Returns the number of pending deletion blocks in container.
+   * @return numPendingDeletionBlocks
+   */
+  public int getNumPendingDeletionBlocks() {
+    return numPendingDeletionBlocks;
+  }
+
+
+  /**
+   * Increase the count of pending deletion blocks.
+   *
+   * @param numBlocks increment number
+   */
+  public void incrPendingDeletionBlocks(int numBlocks) {
+    this.numPendingDeletionBlocks += numBlocks;
+  }
+
+  /**
+   * Decrease the count of pending deletion blocks.
+   *
+   * @param numBlocks decrement number
+   */
+  public void decrPendingDeletionBlocks(int numBlocks) {
+    this.numPendingDeletionBlocks -= numBlocks;
+  }
+
+
+  /**
+   * Constructs a KeyValueContainerData object from ProtoBuf classes.
+   *
+   * @param protoData - ProtoBuf Message
+   * @throws IOException
+   */
+  public static KeyValueContainerData getFromProtoBuf(
+      ContainerProtos.CreateContainerData protoData) throws IOException {
+
+    long containerID;
+    ContainerProtos.ContainerType containerType;
+
+    containerID = protoData.getContainerId();
+    containerType = protoData.getContainerType();
+
+    KeyValueContainerData keyValueContainerData = new KeyValueContainerData(
+        containerType, containerID);
+
+    for (int x = 0; x < protoData.getMetadataCount(); x++) {
+      keyValueContainerData.addMetadata(protoData.getMetadata(x).getKey(),
+          protoData.getMetadata(x).getValue());
+    }
+
+    return keyValueContainerData;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd19b45/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
new file mode 100644
index 0000000..a680e6a
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
@@ -0,0 +1,75 @@
+/**
+ * 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.interfaces;
+
+
+import org.apache.hadoop.hdds.scm.container.common.helpers.
+    StorageContainerException;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+
+import java.security.NoSuchAlgorithmException;
+
+/**
+ * Interface for Container Operations.
+ */
+public interface Container {
+
+  /**
+   * Creates a container.
+   *
+   * @throws StorageContainerException
+   */
+  void create(ContainerData containerData) throws StorageContainerException;
+
+  /**
+   * Deletes the container.
+   *
+   * @param forceDelete   - whether this container should be deleted forcibly.
+   * @throws StorageContainerException
+   */
+  void delete(boolean forceDelete) throws StorageContainerException;
+
+  /**
+   * Update the container.
+   *
+   * @param forceUpdate if true, update container forcibly.
+   * @throws StorageContainerException
+   */
+  void update(boolean forceUpdate)
+      throws StorageContainerException;
+
+  /**
+   * Get metadata about the container.
+   *
+   * @return ContainerData - Container Data.
+   * @throws StorageContainerException
+   */
+  ContainerData getContainerData() throws StorageContainerException;
+
+  /**
+   * Closes a open container, if it is already closed or does not exist a
+   * StorageContainerException is thrown.
+   *
+   * @throws StorageContainerException
+   */
+  void close() throws StorageContainerException,
+      NoSuchAlgorithmException;
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd19b45/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestChunkLayOutVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestChunkLayOutVersion.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestChunkLayOutVersion.java
new file mode 100644
index 0000000..a4e0028
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestChunkLayOutVersion.java
@@ -0,0 +1,42 @@
+/*
+ * 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;
+
+import org.apache.hadoop.ozone.container.common.impl.ChunkLayOutVersion;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * This class tests ChunkLayOutVersion.
+ */
+public class TestChunkLayOutVersion {
+
+  @Test
+  public void testChunkLayOutVersion() {
+
+    // Check Latest Version and description
+    Assert.assertEquals(1, ChunkLayOutVersion.getLatestVersion().getVersion());
+    Assert.assertEquals("Data without checksums.", ChunkLayOutVersion
+        .getLatestVersion().getDescription());
+
+    Assert.assertEquals(1, ChunkLayOutVersion.getAllVersions().length);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd19b45/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
new file mode 100644
index 0000000..1541921
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.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
+ *
+ *      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;
+
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This class is used to test the KeyValueContainerData.
+ */
+public class TestKeyValueContainerData {
+
+  @Test
+  public void testGetFromProtoBuf() throws IOException {
+
+    long containerId = 1L;
+    ContainerProtos.ContainerType containerType = ContainerProtos
+        .ContainerType.KeyValueContainer;
+    String path = "/tmp";
+    String containerDBType = "RocksDB";
+    int layOutVersion = 1;
+    ContainerProtos.ContainerLifeCycleState state = ContainerProtos
+        .ContainerLifeCycleState.OPEN;
+
+    ContainerProtos.KeyValue.Builder keyValBuilder =
+        ContainerProtos.KeyValue.newBuilder();
+    ContainerProtos.CreateContainerData containerData = ContainerProtos
+        .CreateContainerData.newBuilder()
+        .setContainerType(containerType)
+        .setContainerId(containerId)
+        .addMetadata(0, keyValBuilder.setKey("VOLUME").setValue("ozone")
+            .build())
+        .addMetadata(1, keyValBuilder.setKey("OWNER").setValue("hdfs")
+            .build()).build();
+
+    KeyValueContainerData kvData = KeyValueContainerData.getFromProtoBuf(
+        containerData);
+
+    assertEquals(containerType, kvData.getContainerType());
+    assertEquals(containerId, kvData.getContainerId());
+    assertEquals(layOutVersion, kvData.getLayOutVersion().getVersion());
+    assertEquals(state, kvData.getState());
+    assertEquals(2, kvData.getMetadata().size());
+    assertEquals("ozone", kvData.getMetadata().get("VOLUME"));
+    assertEquals("hdfs", kvData.getMetadata().get("OWNER"));
+
+  }
+
+  @Test
+  public void testKeyValueData() {
+    long containerId = 1L;
+    ContainerProtos.ContainerType containerType = ContainerProtos
+        .ContainerType.KeyValueContainer;
+    String path = "/tmp";
+    String containerDBType = "RocksDB";
+    int layOutVersion = 1;
+    ContainerProtos.ContainerLifeCycleState state = ContainerProtos
+        .ContainerLifeCycleState.CLOSED;
+    AtomicLong val = new AtomicLong(0);
+    AtomicLong updatedVal = new AtomicLong(100);
+
+    KeyValueContainerData kvData = new KeyValueContainerData(containerType,
+        containerId);
+
+    assertEquals(containerType, kvData.getContainerType());
+    assertEquals(containerId, kvData.getContainerId());
+    assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN, kvData
+        .getState());
+    assertEquals(0, kvData.getMetadata().size());
+    assertEquals(0, kvData.getNumPendingDeletionBlocks());
+    assertEquals(val.get(), kvData.getReadBytes());
+    assertEquals(val.get(), kvData.getWriteBytes());
+    assertEquals(val.get(), kvData.getReadCount());
+    assertEquals(val.get(), kvData.getWriteCount());
+
+    kvData.setState(state);
+    kvData.setContainerDBType(containerDBType);
+    kvData.setContainerPath(path);
+    kvData.setDBPath(path);
+    kvData.incrReadBytes(10);
+    kvData.incrWriteBytes(10);
+    kvData.incrReadCount();
+    kvData.incrWriteCount();
+
+    assertEquals(state, kvData.getState());
+    assertEquals(containerDBType, kvData.getContainerDBType());
+    assertEquals(path, kvData.getContainerPath());
+    assertEquals(path, kvData.getDBPath());
+
+    assertEquals(10, kvData.getReadBytes());
+    assertEquals(10, kvData.getWriteBytes());
+    assertEquals(1, kvData.getReadCount());
+    assertEquals(1, kvData.getWriteCount());
+
+  }
+
+}


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


[37/50] [abbrv] hadoop git commit: HDDS-81. Moving ContainerReport inside Datanode heartbeat. Contributed by Nanda Kumar.

Posted by ha...@apache.org.
HDDS-81. Moving ContainerReport inside Datanode heartbeat.
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/201440b9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/201440b9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/201440b9

Branch: refs/heads/HDDS-48
Commit: 201440b987d5ef3910c2045b2411c213ed6eec1f
Parents: 4827e9a
Author: Anu Engineer <ae...@apache.org>
Authored: Tue May 29 12:40:27 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Tue May 29 12:48:50 2018 -0700

----------------------------------------------------------------------
 .../common/impl/ContainerManagerImpl.java       |  22 +-
 .../common/impl/StorageLocationReport.java      |   8 +-
 .../common/interfaces/ContainerManager.java     |   8 +-
 .../statemachine/DatanodeStateMachine.java      |   7 +-
 .../common/statemachine/StateContext.java       |  16 +-
 .../CloseContainerCommandHandler.java           | 113 ++++++++
 .../commandhandler/CloseContainerHandler.java   | 113 --------
 .../commandhandler/CommandDispatcher.java       |   5 +-
 .../commandhandler/CommandHandler.java          |   8 +-
 .../DeleteBlocksCommandHandler.java             |  12 +-
 .../states/endpoint/HeartbeatEndpointTask.java  |  30 +-
 .../states/endpoint/RegisterEndpointTask.java   |  12 +-
 .../container/ozoneimpl/OzoneContainer.java     |  10 +-
 .../StorageContainerDatanodeProtocol.java       |  30 +-
 .../protocol/StorageContainerNodeProtocol.java  |  15 +-
 .../commands/CloseContainerCommand.java         |  18 +-
 .../protocol/commands/DeleteBlocksCommand.java  |  18 +-
 .../protocol/commands/RegisteredCommand.java    |  26 +-
 .../protocol/commands/ReregisterCommand.java    |  16 +-
 .../ozone/protocol/commands/SCMCommand.java     |   4 +-
 ...rDatanodeProtocolClientSideTranslatorPB.java |  50 +---
 ...rDatanodeProtocolServerSideTranslatorPB.java |  53 ++--
 .../StorageContainerDatanodeProtocol.proto      | 256 ++++++++---------
 .../ozone/container/common/ScmTestMock.java     |  78 ++----
 .../hdds/scm/container/ContainerMapping.java    |  10 +-
 .../hadoop/hdds/scm/container/Mapping.java      |   6 +-
 .../replication/ContainerSupervisor.java        |  13 +-
 .../container/replication/InProgressPool.java   |  15 +-
 .../hdds/scm/node/HeartbeatQueueItem.java       |  14 +-
 .../hadoop/hdds/scm/node/SCMNodeManager.java    |  58 ++--
 .../hdds/scm/node/SCMNodeStorageStatMap.java    |  14 +-
 .../scm/server/SCMDatanodeProtocolServer.java   | 195 +++++++------
 .../org/apache/hadoop/hdds/scm/TestUtils.java   |  19 +-
 .../hdds/scm/container/MockNodeManager.java     |  26 +-
 .../scm/container/TestContainerMapping.java     |  24 +-
 .../container/closer/TestContainerCloser.java   |  12 +-
 .../hdds/scm/node/TestContainerPlacement.java   |   6 +-
 .../hadoop/hdds/scm/node/TestNodeManager.java   |  83 +++---
 .../scm/node/TestSCMNodeStorageStatMap.java     |  16 +-
 .../ozone/container/common/TestEndPoint.java    | 113 ++------
 .../replication/TestContainerSupervisor.java    | 275 -------------------
 .../ReplicationDatanodeStateManager.java        | 101 -------
 .../testutils/ReplicationNodeManagerMock.java   |  14 +-
 .../ozone/TestStorageContainerManager.java      |  11 +-
 .../apache/hadoop/ozone/scm/TestSCMMetrics.java |  68 ++---
 45 files changed, 706 insertions(+), 1315 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java
index 9355364..af47015 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java
@@ -35,11 +35,11 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMStorageReport;
+    .StorageContainerDatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -854,11 +854,11 @@ public class ContainerManagerImpl implements ContainerManager {
    * @return node report.
    */
   @Override
-  public SCMNodeReport getNodeReport() throws IOException {
+  public NodeReportProto getNodeReport() throws IOException {
     StorageLocationReport[] reports = locationManager.getLocationReport();
-    SCMNodeReport.Builder nrb = SCMNodeReport.newBuilder();
+    NodeReportProto.Builder nrb = NodeReportProto.newBuilder();
     for (int i = 0; i < reports.length; i++) {
-      SCMStorageReport.Builder srb = SCMStorageReport.newBuilder();
+      StorageReportProto.Builder srb = StorageReportProto.newBuilder();
       nrb.addStorageReport(reports[i].getProtoBufMessage());
     }
     return nrb.build();
@@ -891,7 +891,7 @@ public class ContainerManagerImpl implements ContainerManager {
    * @throws IOException
    */
   @Override
-  public ContainerReportsRequestProto getContainerReport() throws IOException {
+  public ContainerReportsProto getContainerReport() throws IOException {
     LOG.debug("Starting container report iteration.");
     // No need for locking since containerMap is a ConcurrentSkipListMap
     // And we can never get the exact state since close might happen
@@ -899,12 +899,8 @@ public class ContainerManagerImpl implements ContainerManager {
     List<ContainerData> containers = containerMap.values().stream()
         .collect(Collectors.toList());
 
-    ContainerReportsRequestProto.Builder crBuilder =
-        ContainerReportsRequestProto.newBuilder();
-
-    // TODO: support delta based container report
-    crBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage())
-        .setType(ContainerReportsRequestProto.reportType.fullReport);
+    ContainerReportsProto.Builder crBuilder =
+        ContainerReportsProto.newBuilder();
 
     for (ContainerData container: containers) {
       long containerId = container.getContainerID();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java
index a5ad6c2..87b9656 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.ozone.container.common.impl;
 
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdds.protocol.proto.
-    StorageContainerDatanodeProtocolProtos.SCMStorageReport;
+    StorageContainerDatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdds.protocol.proto.
     StorageContainerDatanodeProtocolProtos.StorageTypeProto;
 
@@ -137,8 +137,8 @@ public class StorageLocationReport {
    * @return SCMStorageReport
    * @throws IOException In case, the storage type specified is invalid.
    */
-  public SCMStorageReport getProtoBufMessage() throws IOException{
-    SCMStorageReport.Builder srb = SCMStorageReport.newBuilder();
+  public StorageReportProto getProtoBufMessage() throws IOException{
+    StorageReportProto.Builder srb = StorageReportProto.newBuilder();
     return srb.setStorageUuid(getId())
         .setCapacity(getCapacity())
         .setScmUsed(getScmUsed())
@@ -156,7 +156,7 @@ public class StorageLocationReport {
    * @throws IOException in case of invalid storage type
    */
 
-  public static StorageLocationReport getFromProtobuf(SCMStorageReport report)
+  public static StorageLocationReport getFromProtobuf(StorageReportProto report)
       throws IOException {
     StorageLocationReport.Builder builder = StorageLocationReport.newBuilder();
     builder.setId(report.getStorageUuid())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java
index ba70953..49b68dc 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java
@@ -27,9 +27,9 @@ import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 
 import java.io.IOException;
@@ -171,14 +171,14 @@ public interface ContainerManager extends RwLock {
    * Get the Node Report of container storage usage.
    * @return node report.
    */
-  SCMNodeReport getNodeReport() throws IOException;
+  NodeReportProto getNodeReport() throws IOException;
 
   /**
    * Gets container report.
    * @return container report.
    * @throws IOException
    */
-  ContainerReportsRequestProto getContainerReport() throws IOException;
+  ContainerReportsProto getContainerReport() throws IOException;
 
   /**
    * Gets container reports.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/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 a8fe494..d0a4217 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
@@ -21,8 +21,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
-    .CloseContainerHandler;
+import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.CloseContainerCommandHandler;
 import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
     .CommandDispatcher;
 import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
@@ -86,7 +85,7 @@ public class DatanodeStateMachine implements Closeable {
      // When we add new handlers just adding a new handler here should do the
      // trick.
     commandDispatcher = CommandDispatcher.newBuilder()
-        .addHandler(new CloseContainerHandler())
+        .addHandler(new CloseContainerCommandHandler())
         .addHandler(new DeleteBlocksCommandHandler(
             container.getContainerManager(), conf))
         .setConnectionManager(connectionManager)
@@ -131,7 +130,7 @@ public class DatanodeStateMachine implements Closeable {
       try {
         LOG.debug("Executing cycle Number : {}", context.getExecutionCount());
         nextHB.set(Time.monotonicNow() + heartbeatFrequency);
-        context.setReportState(container.getNodeReport());
+        context.setNodeReport(container.getNodeReport());
         context.execute(executorService, heartbeatFrequency,
             TimeUnit.MILLISECONDS);
         now = Time.monotonicNow();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/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 27eb57e..4e3c610 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
@@ -18,7 +18,7 @@ package org.apache.hadoop.ozone.container.common.statemachine;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.ozone.container.common.states.DatanodeState;
 import org.apache.hadoop.ozone.container.common.states.datanode
     .InitDatanodeState;
@@ -52,7 +52,7 @@ public class StateContext {
   private final AtomicLong stateExecutionCount;
   private final Configuration conf;
   private DatanodeStateMachine.DatanodeStates state;
-  private SCMNodeReport nrState;
+  private NodeReportProto dnReport;
 
   /**
    * Constructs a StateContext.
@@ -69,7 +69,7 @@ public class StateContext {
     commandQueue = new LinkedList<>();
     lock = new ReentrantLock();
     stateExecutionCount = new AtomicLong(0);
-    nrState = SCMNodeReport.getDefaultInstance();
+    dnReport = NodeReportProto.getDefaultInstance();
   }
 
   /**
@@ -144,16 +144,16 @@ public class StateContext {
    * Returns the node report of the datanode state context.
    * @return the node report.
    */
-  public SCMNodeReport getNodeReport() {
-    return nrState;
+  public NodeReportProto getNodeReport() {
+    return dnReport;
   }
 
   /**
    * Sets the storage location report of the datanode state context.
-   * @param nrReport - node report
+   * @param nodeReport node report
    */
-  public void setReportState(SCMNodeReport nrReport) {
-    this.nrState = nrReport;
+  public void setNodeReport(NodeReportProto nodeReport) {
+    this.dnReport = nodeReport;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/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
new file mode 100644
index 0000000..e8c602d
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java
@@ -0,0 +1,113 @@
+/**
+ * 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.statemachine.commandhandler;
+
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.CloseContainerCommandProto;
+import org.apache.hadoop.ozone.container.common.statemachine
+    .SCMConnectionManager;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Handler for close container command received from SCM.
+ */
+public class CloseContainerCommandHandler implements CommandHandler {
+  static final Logger LOG =
+      LoggerFactory.getLogger(CloseContainerCommandHandler.class);
+  private int invocationCount;
+  private long totalTime;
+
+  /**
+   * Constructs a ContainerReport handler.
+   */
+  public CloseContainerCommandHandler() {
+  }
+
+  /**
+   * Handles a given SCM command.
+   *
+   * @param command           - SCM Command
+   * @param container         - Ozone Container.
+   * @param context           - Current Context.
+   * @param connectionManager - The SCMs that we are talking to.
+   */
+  @Override
+  public void handle(SCMCommand command, OzoneContainer container,
+      StateContext context, SCMConnectionManager connectionManager) {
+    LOG.debug("Processing Close Container command.");
+    invocationCount++;
+    long startTime = Time.monotonicNow();
+    // TODO: define this as INVALID_CONTAINER_ID in HddsConsts.java (TBA)
+    long containerID = -1;
+    try {
+
+      CloseContainerCommandProto
+          closeContainerProto =
+          CloseContainerCommandProto
+              .parseFrom(command.getProtoBufMessage());
+      containerID = closeContainerProto.getContainerID();
+
+      container.getContainerManager().closeContainer(containerID);
+
+    } catch (Exception e) {
+      LOG.error("Can't close container " + containerID, e);
+    } finally {
+      long endTime = Time.monotonicNow();
+      totalTime += endTime - startTime;
+    }
+  }
+
+  /**
+   * Returns the command type that this command handler handles.
+   *
+   * @return Type
+   */
+  @Override
+  public SCMCommandProto.Type getCommandType() {
+    return SCMCommandProto.Type.closeContainerCommand;
+  }
+
+  /**
+   * Returns number of times this handler has been invoked.
+   *
+   * @return int
+   */
+  @Override
+  public int getInvocationCount() {
+    return invocationCount;
+  }
+
+  /**
+   * Returns the average time this function takes to run.
+   *
+   * @return long
+   */
+  @Override
+  public long getAverageRunTime() {
+    if (invocationCount > 0) {
+      return totalTime / invocationCount;
+    }
+    return 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerHandler.java
deleted file mode 100644
index d8adc7d..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerHandler.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/**
- * 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.statemachine.commandhandler;
-
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCloseContainerCmdResponseProto;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCmdType;
-import org.apache.hadoop.ozone.container.common.statemachine
-    .SCMConnectionManager;
-import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
-import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
-import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
-import org.apache.hadoop.util.Time;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Container Report handler.
- */
-public class CloseContainerHandler implements CommandHandler {
-  static final Logger LOG =
-      LoggerFactory.getLogger(CloseContainerHandler.class);
-  private int invocationCount;
-  private long totalTime;
-
-  /**
-   * Constructs a ContainerReport handler.
-   */
-  public CloseContainerHandler() {
-  }
-
-  /**
-   * Handles a given SCM command.
-   *
-   * @param command           - SCM Command
-   * @param container         - Ozone Container.
-   * @param context           - Current Context.
-   * @param connectionManager - The SCMs that we are talking to.
-   */
-  @Override
-  public void handle(SCMCommand command, OzoneContainer container,
-      StateContext context, SCMConnectionManager connectionManager) {
-    LOG.debug("Processing Close Container command.");
-    invocationCount++;
-    long startTime = Time.monotonicNow();
-    // TODO: define this as INVALID_CONTAINER_ID in HddsConsts.java (TBA)
-    long containerID = -1;
-    try {
-
-      SCMCloseContainerCmdResponseProto
-          closeContainerProto =
-          SCMCloseContainerCmdResponseProto
-              .parseFrom(command.getProtoBufMessage());
-      containerID = closeContainerProto.getContainerID();
-
-      container.getContainerManager().closeContainer(containerID);
-
-    } catch (Exception e) {
-      LOG.error("Can't close container " + containerID, e);
-    } finally {
-      long endTime = Time.monotonicNow();
-      totalTime += endTime - startTime;
-    }
-  }
-
-  /**
-   * Returns the command type that this command handler handles.
-   *
-   * @return Type
-   */
-  @Override
-  public SCMCmdType getCommandType() {
-    return SCMCmdType.closeContainerCommand;
-  }
-
-  /**
-   * Returns number of times this handler has been invoked.
-   *
-   * @return int
-   */
-  @Override
-  public int getInvocationCount() {
-    return invocationCount;
-  }
-
-  /**
-   * Returns the average time this function takes to run.
-   *
-   * @return long
-   */
-  @Override
-  public long getAverageRunTime() {
-    if (invocationCount > 0) {
-      return totalTime / invocationCount;
-    }
-    return 0;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandDispatcher.java
index 40feca3..aedd78f 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandDispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandDispatcher.java
@@ -18,7 +18,8 @@
 package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type;
 import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
 import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
@@ -38,7 +39,7 @@ public final class CommandDispatcher {
   static final Logger LOG =
       LoggerFactory.getLogger(CommandDispatcher.class);
   private final StateContext context;
-  private final Map<SCMCmdType, CommandHandler> handlerMap;
+  private final Map<Type, CommandHandler> handlerMap;
   private final OzoneContainer container;
   private final SCMConnectionManager connectionManager;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandHandler.java
index 13d9f72..60e2dc4 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandHandler.java
@@ -17,8 +17,10 @@
 
 package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
 
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType;
-import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
+import org.apache.hadoop.ozone.container.common.statemachine
+    .SCMConnectionManager;
 import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
@@ -42,7 +44,7 @@ public interface CommandHandler {
    * Returns the command type that this command handler handles.
    * @return Type
    */
-  SCMCmdType getCommandType();
+  SCMCommandProto.Type getCommandType();
 
   /**
    * Returns number of times this handler has been invoked.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/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 5231660..ab69bdc 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
@@ -18,6 +18,8 @@ package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
 
 import com.google.common.primitives.Longs;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
@@ -26,8 +28,6 @@ import org.apache.hadoop.hdds.protocol.proto
     .DeleteBlockTransactionResult;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCmdType;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 import org.apache.hadoop.ozone.container.common.helpers
@@ -73,10 +73,10 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
   @Override
   public void handle(SCMCommand command, OzoneContainer container,
       StateContext context, SCMConnectionManager connectionManager) {
-    if (command.getType() != SCMCmdType.deleteBlocksCommand) {
+    if (command.getType() != SCMCommandProto.Type.deleteBlocksCommand) {
       LOG.warn("Skipping handling command, expected command "
               + "type {} but found {}",
-          SCMCmdType.deleteBlocksCommand, command.getType());
+          SCMCommandProto.Type.deleteBlocksCommand, command.getType());
       return;
     }
     LOG.debug("Processing block deletion command.");
@@ -193,8 +193,8 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
   }
 
   @Override
-  public SCMCmdType getCommandType() {
-    return SCMCmdType.deleteBlocksCommand;
+  public SCMCommandProto.Type getCommandType() {
+    return SCMCommandProto.Type.deleteBlocksCommand;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
index 01b4c72..337cdfb 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
@@ -23,7 +23,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCommandResponseProto;
+    .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
 import org.apache.hadoop.ozone.container.common.helpers
@@ -97,8 +99,13 @@ public class HeartbeatEndpointTask
     try {
       Preconditions.checkState(this.datanodeDetailsProto != null);
 
+      SCMHeartbeatRequestProto request = SCMHeartbeatRequestProto.newBuilder()
+          .setDatanodeDetails(datanodeDetailsProto)
+          .setNodeReport(context.getNodeReport())
+          .build();
+
       SCMHeartbeatResponseProto reponse = rpcEndpoint.getEndPoint()
-          .sendHeartbeat(datanodeDetailsProto, this.context.getNodeReport());
+          .sendHeartbeat(request);
       processResponse(reponse, datanodeDetailsProto);
       rpcEndpoint.setLastSuccessfulHeartbeat(ZonedDateTime.now());
       rpcEndpoint.zeroMissedCount();
@@ -125,13 +132,13 @@ public class HeartbeatEndpointTask
    */
   private void processResponse(SCMHeartbeatResponseProto response,
       final DatanodeDetailsProto datanodeDetails) {
-    for (SCMCommandResponseProto commandResponseProto : response
+    Preconditions.checkState(response.getDatanodeUUID()
+            .equalsIgnoreCase(datanodeDetails.getUuid()),
+        "Unexpected datanode ID in the response.");
+    // Verify the response is indeed for this datanode.
+    for (SCMCommandProto commandResponseProto : response
         .getCommandsList()) {
-      // Verify the response is indeed for this datanode.
-      Preconditions.checkState(commandResponseProto.getDatanodeUUID()
-          .equalsIgnoreCase(datanodeDetails.getUuid()),
-          "Unexpected datanode ID in the response.");
-      switch (commandResponseProto.getCmdType()) {
+      switch (commandResponseProto.getCommandType()) {
       case reregisterCommand:
         if (rpcEndpoint.getState() == EndPointStates.HEARTBEAT) {
           if (LOG.isDebugEnabled()) {
@@ -148,7 +155,8 @@ public class HeartbeatEndpointTask
         break;
       case deleteBlocksCommand:
         DeleteBlocksCommand db = DeleteBlocksCommand
-            .getFromProtobuf(commandResponseProto.getDeleteBlocksProto());
+            .getFromProtobuf(
+                commandResponseProto.getDeleteBlocksCommandProto());
         if (!db.blocksTobeDeleted().isEmpty()) {
           if (LOG.isDebugEnabled()) {
             LOG.debug(DeletedContainerBlocksSummary
@@ -161,7 +169,7 @@ public class HeartbeatEndpointTask
       case closeContainerCommand:
         CloseContainerCommand closeContainer =
             CloseContainerCommand.getFromProtobuf(
-                commandResponseProto.getCloseContainerProto());
+                commandResponseProto.getCloseContainerCommandProto());
         if (LOG.isDebugEnabled()) {
           LOG.debug("Received SCM container close request for container {}",
               closeContainer.getContainerID());
@@ -170,7 +178,7 @@ public class HeartbeatEndpointTask
         break;
       default:
         throw new IllegalArgumentException("Unknown response : "
-            + commandResponseProto.getCmdType().name());
+            + commandResponseProto.getCommandType().name());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/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 77a7084..12b48ab 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
@@ -24,11 +24,11 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.container.common.statemachine
     .EndpointStateMachine;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto;
+    .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -104,11 +104,11 @@ public final class RegisterEndpointTask implements
     rpcEndPoint.lock();
     try {
 
-      ContainerReportsRequestProto contianerReport = datanodeContainerManager
+      ContainerReportsProto contianerReport = datanodeContainerManager
           .getContainerReport();
-      SCMNodeReport nodeReport = datanodeContainerManager.getNodeReport();
+      NodeReportProto nodeReport = datanodeContainerManager.getNodeReport();
       // TODO : Add responses to the command Queue.
-      SCMRegisteredCmdResponseProto response = rpcEndPoint.getEndPoint()
+      SCMRegisteredResponseProto response = rpcEndPoint.getEndPoint()
           .register(datanodeDetails.getProtoBufMessage(), nodeReport,
               contianerReport);
       Preconditions.checkState(UUID.fromString(response.getDatanodeUUID())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/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 6758479..b357fef 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
@@ -19,14 +19,14 @@ package org.apache.hadoop.ozone.container.ozoneimpl;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
@@ -219,7 +219,7 @@ public class OzoneContainer {
   /**
    * Returns node report of container storage usage.
    */
-  public SCMNodeReport getNodeReport() throws IOException {
+  public NodeReportProto getNodeReport() throws IOException {
     return this.manager.getNodeReport();
   }
 
@@ -255,7 +255,7 @@ public class OzoneContainer {
    * @return - container report.
    * @throws IOException
    */
-  public ContainerReportsRequestProto getContainerReport() throws IOException {
+  public ContainerReportsProto getContainerReport() throws IOException {
     return this.manager.getContainerReport();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerDatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerDatanodeProtocol.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerDatanodeProtocol.java
index e2a3bf5..a950a31 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerDatanodeProtocol.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerDatanodeProtocol.java
@@ -19,20 +19,20 @@ package org.apache.hadoop.ozone.protocol;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
 import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
+import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos
     .ContainerBlocksDeletionACKResponseProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsResponseProto;
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto;
+    .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
 import org.apache.hadoop.hdds.protocol.proto
@@ -55,13 +55,12 @@ public interface StorageContainerDatanodeProtocol {
 
   /**
    * Used by data node to send a Heartbeat.
-   * @param datanodeDetails - Datanode Details.
-   * @param nodeReport - node report state
+   * @param heartbeat Heartbeat
    * @return - SCMHeartbeatResponseProto
    * @throws IOException
    */
-  SCMHeartbeatResponseProto sendHeartbeat(DatanodeDetailsProto datanodeDetails,
-      SCMNodeReport nodeReport) throws IOException;
+  SCMHeartbeatResponseProto sendHeartbeat(SCMHeartbeatRequestProto heartbeat)
+      throws IOException;
 
   /**
    * Register Datanode.
@@ -70,20 +69,11 @@ public interface StorageContainerDatanodeProtocol {
    * @param containerReportsRequestProto - Container Reports.
    * @return SCM Command.
    */
-  SCMRegisteredCmdResponseProto register(DatanodeDetailsProto datanodeDetails,
-      SCMNodeReport nodeReport, ContainerReportsRequestProto
+  SCMRegisteredResponseProto register(DatanodeDetailsProto datanodeDetails,
+      NodeReportProto nodeReport, ContainerReportsProto
       containerReportsRequestProto) throws IOException;
 
   /**
-   * Send a container report.
-   * @param reports -- Container report.
-   * @return container reports response.
-   * @throws IOException
-   */
-  ContainerReportsResponseProto sendContainerReport(
-      ContainerReportsRequestProto reports) throws IOException;
-
-  /**
    * Used by datanode to send block deletion ACK to SCM.
    * @param request block deletion transactions.
    * @return block deletion transaction response.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerNodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerNodeProtocol.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerNodeProtocol.java
index 14038fb..790f58a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerNodeProtocol.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/StorageContainerNodeProtocol.java
@@ -18,11 +18,12 @@
 package org.apache.hadoop.ozone.protocol;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
+import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
 
 import java.util.List;
@@ -49,11 +50,11 @@ public interface StorageContainerNodeProtocol {
   /**
    * Register the node if the node finds that it is not registered with any SCM.
    * @param datanodeDetails DatanodeDetails
-   * @param nodeReport SCMNodeReport
+   * @param nodeReport NodeReportProto
    * @return  SCMHeartbeatResponseProto
    */
-  SCMCommand register(DatanodeDetailsProto datanodeDetails, SCMNodeReport
-      nodeReport);
+  RegisteredCommand register(DatanodeDetails datanodeDetails,
+                             NodeReportProto nodeReport);
 
   /**
    * Send heartbeat to indicate the datanode is alive and doing well.
@@ -61,7 +62,7 @@ public interface StorageContainerNodeProtocol {
    * @param nodeReport - node report.
    * @return SCMheartbeat response list
    */
-  List<SCMCommand> sendHeartbeat(DatanodeDetailsProto datanodeDetails,
-      SCMNodeReport nodeReport);
+  List<SCMCommand> sendHeartbeat(DatanodeDetails datanodeDetails,
+      NodeReportProto nodeReport);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CloseContainerCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CloseContainerCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CloseContainerCommand.java
index d1d6488..4f4f82b 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CloseContainerCommand.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CloseContainerCommand.java
@@ -19,18 +19,16 @@ package org.apache.hadoop.ozone.protocol.commands;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCloseContainerCmdResponseProto;
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCmdType;
+    .StorageContainerDatanodeProtocolProtos.CloseContainerCommandProto;
 
-import static org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCmdType.closeContainerCommand;
 
 /**
  * Asks datanode to close a container.
  */
 public class CloseContainerCommand
-    extends SCMCommand<SCMCloseContainerCmdResponseProto> {
+    extends SCMCommand<CloseContainerCommandProto> {
 
   private long containerID;
 
@@ -44,8 +42,8 @@ public class CloseContainerCommand
    * @return Type
    */
   @Override
-  public SCMCmdType getType() {
-    return closeContainerCommand;
+  public SCMCommandProto.Type getType() {
+    return SCMCommandProto.Type.closeContainerCommand;
   }
 
   /**
@@ -58,13 +56,13 @@ public class CloseContainerCommand
     return getProto().toByteArray();
   }
 
-  public SCMCloseContainerCmdResponseProto getProto() {
-    return SCMCloseContainerCmdResponseProto.newBuilder()
+  public CloseContainerCommandProto getProto() {
+    return CloseContainerCommandProto.newBuilder()
         .setContainerID(containerID).build();
   }
 
   public static CloseContainerCommand getFromProtobuf(
-      SCMCloseContainerCmdResponseProto closeContainerProto) {
+      CloseContainerCommandProto closeContainerProto) {
     Preconditions.checkNotNull(closeContainerProto);
     return new CloseContainerCommand(closeContainerProto.getContainerID());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/DeleteBlocksCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/DeleteBlocksCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/DeleteBlocksCommand.java
index a11ca25..4fa33f6 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/DeleteBlocksCommand.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/DeleteBlocksCommand.java
@@ -18,11 +18,11 @@
 package org.apache.hadoop.ozone.protocol.commands;
 
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCmdType;
+    .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMDeleteBlocksCmdResponseProto;
+    .StorageContainerDatanodeProtocolProtos.DeleteBlocksCommandProto;
 
 import java.util.List;
 
@@ -30,7 +30,7 @@ import java.util.List;
  * A SCM command asks a datanode to delete a number of blocks.
  */
 public class DeleteBlocksCommand extends
-    SCMCommand<SCMDeleteBlocksCmdResponseProto> {
+    SCMCommand<DeleteBlocksCommandProto> {
 
   private List<DeletedBlocksTransaction> blocksTobeDeleted;
 
@@ -44,8 +44,8 @@ public class DeleteBlocksCommand extends
   }
 
   @Override
-  public SCMCmdType getType() {
-    return SCMCmdType.deleteBlocksCommand;
+  public SCMCommandProto.Type getType() {
+    return SCMCommandProto.Type.deleteBlocksCommand;
   }
 
   @Override
@@ -54,13 +54,13 @@ public class DeleteBlocksCommand extends
   }
 
   public static DeleteBlocksCommand getFromProtobuf(
-      SCMDeleteBlocksCmdResponseProto deleteBlocksProto) {
+      DeleteBlocksCommandProto deleteBlocksProto) {
     return new DeleteBlocksCommand(deleteBlocksProto
         .getDeletedBlocksTransactionsList());
   }
 
-  public SCMDeleteBlocksCmdResponseProto getProto() {
-    return SCMDeleteBlocksCmdResponseProto.newBuilder()
+  public DeleteBlocksCommandProto getProto() {
+    return DeleteBlocksCommandProto.newBuilder()
         .addAllDeletedBlocksTransactions(blocksTobeDeleted).build();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/RegisteredCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/RegisteredCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/RegisteredCommand.java
index 69f2c18..3a5da72 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/RegisteredCommand.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/RegisteredCommand.java
@@ -19,18 +19,15 @@ package org.apache.hadoop.ozone.protocol.commands;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCmdType;
+    .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto
+    .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto
     .ErrorCode;
 
 /**
  * Response to Datanode Register call.
  */
-public class RegisteredCommand extends
-    SCMCommand<SCMRegisteredCmdResponseProto> {
+public class RegisteredCommand {
   private String datanodeUUID;
   private String clusterID;
   private ErrorCode error;
@@ -60,16 +57,6 @@ public class RegisteredCommand extends
   }
 
   /**
-   * Returns the type of this command.
-   *
-   * @return Type
-   */
-  @Override
-  public SCMCmdType getType() {
-    return SCMCmdType.registeredCommand;
-  }
-
-  /**
    * Returns datanode UUID.
    *
    * @return - Datanode ID.
@@ -117,10 +104,9 @@ public class RegisteredCommand extends
    *
    * @return A protobuf message.
    */
-  @Override
   public byte[] getProtoBufMessage() {
-    SCMRegisteredCmdResponseProto.Builder builder =
-        SCMRegisteredCmdResponseProto.newBuilder()
+    SCMRegisteredResponseProto.Builder builder =
+        SCMRegisteredResponseProto.newBuilder()
             .setClusterID(this.clusterID)
             .setDatanodeUUID(this.datanodeUUID)
             .setErrorCode(this.error);
@@ -157,7 +143,7 @@ public class RegisteredCommand extends
      * @param response - RegisteredCmdResponseProto
      * @return RegisteredCommand
      */
-    public  RegisteredCommand getFromProtobuf(SCMRegisteredCmdResponseProto
+    public  RegisteredCommand getFromProtobuf(SCMRegisteredResponseProto
                                                         response) {
       Preconditions.checkNotNull(response);
       if (response.hasHostname() && response.hasIpAddress()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReregisterCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReregisterCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReregisterCommand.java
index c167d59..953e31a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReregisterCommand.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReregisterCommand.java
@@ -18,18 +18,16 @@
 package org.apache.hadoop.ozone.protocol.commands;
 
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCmdType;
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
 
 import static org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCmdType.reregisterCommand;
-import static org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMReregisterCmdResponseProto;
+    .StorageContainerDatanodeProtocolProtos.ReregisterCommandProto;
 
 /**
  * Informs a datanode to register itself with SCM again.
  */
 public class ReregisterCommand extends
-    SCMCommand<SCMReregisterCmdResponseProto>{
+    SCMCommand<ReregisterCommandProto>{
 
   /**
    * Returns the type of this command.
@@ -37,8 +35,8 @@ public class ReregisterCommand extends
    * @return Type
    */
   @Override
-  public SCMCmdType getType() {
-    return reregisterCommand;
+  public SCMCommandProto.Type getType() {
+    return SCMCommandProto.Type.reregisterCommand;
   }
 
   /**
@@ -51,8 +49,8 @@ public class ReregisterCommand extends
     return getProto().toByteArray();
   }
 
-  public SCMReregisterCmdResponseProto getProto() {
-    return SCMReregisterCmdResponseProto
+  public ReregisterCommandProto getProto() {
+    return ReregisterCommandProto
         .newBuilder()
         .build();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/SCMCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/SCMCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/SCMCommand.java
index 73e4194..35ca802 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/SCMCommand.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/SCMCommand.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.ozone.protocol.commands;
 
 import com.google.protobuf.GeneratedMessage;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCmdType;
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
 
 /**
  * A class that acts as the base class to convert between Java and SCM
@@ -31,7 +31,7 @@ public abstract class SCMCommand<T extends GeneratedMessage> {
    * Returns the type of this command.
    * @return Type
    */
-  public  abstract SCMCmdType getType();
+  public  abstract SCMCommandProto.Type getType();
 
   /**
    * Gets the protobuf message of this object.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolClientSideTranslatorPB.java
index a56c57a..40fe189 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolClientSideTranslatorPB.java
@@ -20,24 +20,23 @@ import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
 import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
+import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos
     .ContainerBlocksDeletionACKResponseProto;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsResponseProto;
+
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMRegisterRequestProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto;
+    .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
 import org.apache.hadoop.hdds.protocol.proto
@@ -123,22 +122,16 @@ public class StorageContainerDatanodeProtocolClientSideTranslatorPB
   /**
    * Send by datanode to SCM.
    *
-   * @param datanodeDetailsProto - Datanode Details
-   * @param nodeReport - node report
+   * @param heartbeat node heartbeat
    * @throws IOException
    */
 
   @Override
   public SCMHeartbeatResponseProto sendHeartbeat(
-      DatanodeDetailsProto datanodeDetailsProto,
-      SCMNodeReport nodeReport) throws IOException {
-    SCMHeartbeatRequestProto.Builder req = SCMHeartbeatRequestProto
-        .newBuilder();
-    req.setDatanodeDetails(datanodeDetailsProto);
-    req.setNodeReport(nodeReport);
+      SCMHeartbeatRequestProto heartbeat) throws IOException {
     final SCMHeartbeatResponseProto resp;
     try {
-      resp = rpcProxy.sendHeartbeat(NULL_RPC_CONTROLLER, req.build());
+      resp = rpcProxy.sendHeartbeat(NULL_RPC_CONTROLLER, heartbeat);
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -154,16 +147,16 @@ public class StorageContainerDatanodeProtocolClientSideTranslatorPB
    * @return SCM Command.
    */
   @Override
-  public SCMRegisteredCmdResponseProto register(
-      DatanodeDetailsProto datanodeDetailsProto, SCMNodeReport nodeReport,
-      ContainerReportsRequestProto containerReportsRequestProto)
+  public SCMRegisteredResponseProto register(
+      DatanodeDetailsProto datanodeDetailsProto, NodeReportProto nodeReport,
+      ContainerReportsProto containerReportsRequestProto)
       throws IOException {
     SCMRegisterRequestProto.Builder req =
         SCMRegisterRequestProto.newBuilder();
     req.setDatanodeDetails(datanodeDetailsProto);
     req.setContainerReport(containerReportsRequestProto);
     req.setNodeReport(nodeReport);
-    final SCMRegisteredCmdResponseProto response;
+    final SCMRegisteredResponseProto response;
     try {
       response = rpcProxy.register(NULL_RPC_CONTROLLER, req.build());
     } catch (ServiceException e) {
@@ -172,25 +165,6 @@ public class StorageContainerDatanodeProtocolClientSideTranslatorPB
     return response;
   }
 
-  /**
-   * Send a container report.
-   *
-   * @param reports -- Container report
-   * @return HeartbeatRespose.nullcommand.
-   * @throws IOException
-   */
-  @Override
-  public ContainerReportsResponseProto sendContainerReport(
-      ContainerReportsRequestProto reports) throws IOException {
-    final ContainerReportsResponseProto resp;
-    try {
-      resp = rpcProxy.sendContainerReport(NULL_RPC_CONTROLLER, reports);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    return resp;
-  }
-
   @Override
   public ContainerBlocksDeletionACKResponseProto sendContainerBlocksDeletionACK(
       ContainerBlocksDeletionACKProto deletedBlocks) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolServerSideTranslatorPB.java
index 07dba57..7e8bd8a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerDatanodeProtocolServerSideTranslatorPB.java
@@ -19,18 +19,22 @@ package org.apache.hadoop.ozone.protocolPB;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos;
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
+    .StorageContainerDatanodeProtocolProtos.SCMRegisterRequestProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos
-    .ContainerBlocksDeletionACKResponseProto;
+    .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
+    .StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsResponseProto;
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos
+    .ContainerBlocksDeletionACKResponseProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
 import org.apache.hadoop.hdds.protocol.proto
@@ -55,9 +59,8 @@ public class StorageContainerDatanodeProtocolServerSideTranslatorPB
   }
 
   @Override
-  public StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto
-      getVersion(RpcController controller,
-      StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto request)
+  public SCMVersionResponseProto getVersion(RpcController controller,
+      SCMVersionRequestProto request)
       throws ServiceException {
     try {
       return impl.getVersion(request);
@@ -67,15 +70,13 @@ public class StorageContainerDatanodeProtocolServerSideTranslatorPB
   }
 
   @Override
-  public StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto
-      register(RpcController controller, StorageContainerDatanodeProtocolProtos
-      .SCMRegisterRequestProto request) throws ServiceException {
+  public SCMRegisteredResponseProto register(RpcController controller,
+      SCMRegisterRequestProto request) throws ServiceException {
     try {
-      ContainerReportsRequestProto containerRequestProto = null;
-      SCMNodeReport scmNodeReport = null;
-      containerRequestProto = request.getContainerReport();
-      scmNodeReport = request.getNodeReport();
-      return impl.register(request.getDatanodeDetails(), scmNodeReport,
+      ContainerReportsProto containerRequestProto = request
+          .getContainerReport();
+      NodeReportProto dnNodeReport = request.getNodeReport();
+      return impl.register(request.getDatanodeDetails(), dnNodeReport,
           containerRequestProto);
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -83,27 +84,15 @@ public class StorageContainerDatanodeProtocolServerSideTranslatorPB
   }
 
   @Override
-  public SCMHeartbeatResponseProto
-      sendHeartbeat(RpcController controller,
+  public SCMHeartbeatResponseProto sendHeartbeat(RpcController controller,
       SCMHeartbeatRequestProto request) throws ServiceException {
     try {
-      return impl.sendHeartbeat(request.getDatanodeDetails(),
-          request.getNodeReport());
+      return impl.sendHeartbeat(request);
     } catch (IOException e) {
       throw new ServiceException(e);
     }
   }
 
-  @Override
-  public ContainerReportsResponseProto sendContainerReport(
-      RpcController controller, ContainerReportsRequestProto request)
-      throws ServiceException {
-    try {
-      return impl.sendContainerReport(request);
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-  }
 
   @Override
   public ContainerBlocksDeletionACKResponseProto sendContainerBlocksDeletionACK(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/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 20e6af8..cc131e0 100644
--- a/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
+++ b/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
@@ -34,81 +34,74 @@ package hadoop.hdds;
 
 import "hdds.proto";
 
+/**
+ * Request for version info of the software stack on the server.
+ */
+message SCMVersionRequestProto {}
 
 /**
-* This message is send by data node to indicate that it is alive or it is
-* registering with the node manager.
+* Generic response that is send to a version request. This allows keys to be
+* added on the fly and protocol to remain stable.
 */
-message SCMHeartbeatRequestProto {
-  required DatanodeDetailsProto datanodeDetails = 1;
-  optional SCMNodeReport nodeReport = 2;
+message SCMVersionResponseProto {
+  required uint32 softwareVersion = 1;
+  repeated hadoop.hdds.KeyValue keys = 2;
 }
 
-/**
-A container report contains the following information.
-*/
-message ContainerInfo {
-  optional string finalhash = 2;
-  optional int64 size = 3;
-  optional int64 used = 4;
-  optional int64 keyCount = 5;
-  // TODO: move the io count to separate message
-  optional int64 readCount = 6;
-  optional int64 writeCount = 7;
-  optional int64 readBytes = 8;
-  optional int64 writeBytes = 9;
-  required int64 containerID = 10;
-  optional hadoop.hdds.LifeCycleState state = 11;
+message SCMRegisterRequestProto {
+  required DatanodeDetailsProto datanodeDetails = 1;
+  required NodeReportProto nodeReport = 2;
+  required ContainerReportsProto containerReport = 3;
 }
 
-// The deleted blocks which are stored in deletedBlock.db of scm.
-// We don't use BlockID because this only contians multiple localIDs
-// of the same containerID.
-message DeletedBlocksTransaction {
-  required int64 txID = 1;
-  required int64 containerID = 2;
-  repeated int64 localID = 3;
-  // the retry time of sending deleting command to datanode.
-  required int32 count = 4;
+/**
+ * Datanode ID returned by the SCM. This is similar to name node
+ * registeration of a datanode.
+ */
+message SCMRegisteredResponseProto {
+  enum ErrorCode {
+    success = 1;
+    errorNodeNotPermitted = 2;
+  }
+  required ErrorCode errorCode = 1;
+  required string datanodeUUID = 2;
+  required string clusterID = 3;
+  optional SCMNodeAddressList addressList = 4;
+  optional string hostname = 5;
+  optional string ipAddress = 6;
 }
 
 /**
-A set of container reports, max count is generally set to
-8192 since that keeps the size of the reports under 1 MB.
+* This message is send by data node to indicate that it is alive or it is
+* registering with the node manager.
 */
-message ContainerReportsRequestProto {
-  enum reportType {
-    fullReport = 0;
-    deltaReport = 1;
-  }
+message SCMHeartbeatRequestProto {
   required DatanodeDetailsProto datanodeDetails = 1;
-  repeated ContainerInfo reports = 2;
-  required reportType type = 3;
+  optional NodeReportProto nodeReport = 2;
+  optional ContainerReportsProto containerReport = 3;
 }
 
-message ContainerReportsResponseProto {
+/*
+ * A group of commands for the datanode to execute
+ */
+message SCMHeartbeatResponseProto {
+  required string datanodeUUID = 1;
+  repeated SCMCommandProto commands = 2;
 }
 
-/**
-* This message is send along with the heart beat to report datanode
-* storage utilization by SCM.
-*/
-message SCMNodeReport {
-  repeated SCMStorageReport storageReport = 1;
+message SCMNodeAddressList {
+  repeated string addressList = 1;
 }
 
 /**
- * Types of recognized storage media.
- */
-enum StorageTypeProto {
-  DISK = 1;
-  SSD = 2;
-  ARCHIVE = 3;
-  RAM_DISK = 4;
-  PROVIDED = 5;
+* This message is send along with the heart beat to report datanode
+* storage utilization to SCM.
+*/
+message NodeReportProto {
+  repeated StorageReportProto storageReport = 1;
 }
 
-message SCMStorageReport {
+message StorageReportProto {
   required string storageUuid = 1;
   required string storageLocation = 2;
   optional uint64 capacity = 3 [default = 0];
@@ -118,107 +111,82 @@ message SCMStorageReport {
   optional bool failed = 7 [default = false];
 }
 
-message SCMRegisterRequestProto {
-  required DatanodeDetailsProto datanodeDetails = 1;
-  required SCMNodeReport nodeReport = 2;
-  required ContainerReportsRequestProto containerReport = 3;
-}
-
-/**
- * Request for version info of the software stack on the server.
- */
-message SCMVersionRequestProto {
-
-}
-
-/**
-* Generic response that is send to a version request. This allows keys to be
-* added on the fly and protocol to remain stable.
-*/
-message SCMVersionResponseProto {
-  required uint32 softwareVersion = 1;
-  repeated hadoop.hdds.KeyValue keys = 2;
-}
-
-message SCMNodeAddressList {
-  repeated string addressList = 1;
-}
-
 /**
- * Datanode ID returned by the SCM. This is similar to name node
- * registeration of a datanode.
+ * Types of recognized storage media.
  */
-message SCMRegisteredCmdResponseProto {
-  enum ErrorCode {
-    success = 1;
-    errorNodeNotPermitted = 2;
-  }
-  required ErrorCode errorCode = 2;
-  required string datanodeUUID = 3;
-  required string clusterID = 4;
-  optional SCMNodeAddressList addressList = 5;
-  optional string hostname = 6;
-  optional string ipAddress = 7;
+enum StorageTypeProto {
+  DISK = 1;
+  SSD = 2;
+  ARCHIVE = 3;
+  RAM_DISK = 4;
+  PROVIDED = 5;
 }
 
 /**
- * SCM informs a datanode to register itself again.
- * With recieving this command, datanode will transit to REGISTER state.
- */
-message SCMReregisterCmdResponseProto {}
-
-/**
-This command tells the data node to send in the container report when possible
+A set of container reports, max count is generally set to
+8192 since that keeps the size of the reports under 1 MB.
 */
-message SendContainerReportProto {
+message ContainerReportsProto {
+  repeated ContainerInfo reports = 2;
 }
 
-/**
-This command asks the datanode to close a specific container.
-*/
-message SCMCloseContainerCmdResponseProto {
-  required int64 containerID = 1;
-}
 
 /**
-Type of commands supported by SCM to datanode protocol.
+A container report contains the following information.
 */
-enum SCMCmdType {
-  versionCommand = 2;
-  registeredCommand = 3;
-  reregisterCommand = 4;
-  deleteBlocksCommand = 5;
-  closeContainerCommand = 6;
+message ContainerInfo {
+  optional string finalhash = 1;
+  optional int64 size = 2;
+  optional int64 used = 3;
+  optional int64 keyCount = 4;
+  // TODO: move the io count to separate message
+  optional int64 readCount = 5;
+  optional int64 writeCount = 6;
+  optional int64 readBytes = 7;
+  optional int64 writeBytes = 8;
+  required int64 containerID = 9;
+  optional hadoop.hdds.LifeCycleState state = 10;
 }
 
 /*
  * These are commands returned by SCM for to the datanode to execute.
  */
-message SCMCommandResponseProto {
-  required SCMCmdType cmdType = 2; // Type of the command
-  optional SCMRegisteredCmdResponseProto registeredProto = 3;
-  optional SCMVersionResponseProto versionProto = 4;
-  optional SCMReregisterCmdResponseProto reregisterProto = 5;
-  optional SCMDeleteBlocksCmdResponseProto deleteBlocksProto = 6;
-  required string datanodeUUID = 7;
-  optional SCMCloseContainerCmdResponseProto closeContainerProto = 8;
+message SCMCommandProto {
+  enum Type {
+    reregisterCommand = 1;
+    deleteBlocksCommand = 2;
+    closeContainerCommand = 3;
+    deleteContainerCommand = 4;
+  }
+  // TODO: once we start using protoc 3.x, refactor this message using "oneof"
+  required Type commandType = 1;
+  optional ReregisterCommandProto reregisterCommandProto = 2;
+  optional DeleteBlocksCommandProto deleteBlocksCommandProto = 3;
+  optional CloseContainerCommandProto closeContainerCommandProto = 4;
+  optional DeleteContainerCommandProto deleteContainerCommandProto = 5;
 }
 
-
-/*
- * A group of commands for the datanode to execute
+/**
+ * SCM informs a datanode to register itself again.
+ * With recieving this command, datanode will transit to REGISTER state.
  */
-message SCMHeartbeatResponseProto {
-  repeated SCMCommandResponseProto commands = 1;
-}
+message ReregisterCommandProto {}
+
 
 // HB response from SCM, contains a list of block deletion transactions.
-message SCMDeleteBlocksCmdResponseProto {
+message DeleteBlocksCommandProto {
   repeated DeletedBlocksTransaction deletedBlocksTransactions = 1;
 }
 
-// SendACK response returned by datanode to SCM, currently empty.
-message ContainerBlocksDeletionACKResponseProto {
+// The deleted blocks which are stored in deletedBlock.db of scm.
+// We don't use BlockID because this only contians multiple localIDs
+// of the same containerID.
+message DeletedBlocksTransaction {
+  required int64 txID = 1;
+  required int64 containerID = 2;
+  repeated int64 localID = 3;
+  // the retry time of sending deleting command to datanode.
+  required int32 count = 4;
 }
 
 // ACK message datanode sent to SCM, contains the result of
@@ -231,6 +199,24 @@ message ContainerBlocksDeletionACKProto {
   repeated DeleteBlockTransactionResult results = 1;
 }
 
+// SendACK response returned by datanode to SCM, currently empty.
+message ContainerBlocksDeletionACKResponseProto {
+}
+
+/**
+This command asks the datanode to close a specific container.
+*/
+message CloseContainerCommandProto {
+  required int64 containerID = 1;
+}
+
+/**
+This command asks the datanode to close a specific container.
+*/
+message DeleteContainerCommandProto {
+  required int64 containerID = 1;
+}
+
 /**
  * Protocol used from a datanode to StorageContainerManager.
  *
@@ -305,7 +291,7 @@ service StorageContainerDatanodeProtocolService {
   /**
   * Registers a data node with SCM.
   */
-  rpc register (SCMRegisterRequestProto) returns (SCMRegisteredCmdResponseProto);
+  rpc register (SCMRegisterRequestProto) returns (SCMRegisteredResponseProto);
 
   /**
    * Send heartbeat from datanode to SCM. HB's under SCM looks more
@@ -315,12 +301,6 @@ service StorageContainerDatanodeProtocolService {
   rpc sendHeartbeat (SCMHeartbeatRequestProto) returns (SCMHeartbeatResponseProto);
 
   /**
-    send container reports sends the container report to SCM. This will
-    return a null command as response.
-  */
-  rpc sendContainerReport(ContainerReportsRequestProto) returns (ContainerReportsResponseProto);
-
-  /**
    * Sends the block deletion ACK to SCM.
    */
   rpc sendContainerBlocksDeletionACK (ContainerBlocksDeletionACKProto) returns (ContainerBlocksDeletionACKResponseProto);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
index c57a366..0ee6321 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
@@ -16,12 +16,12 @@
  */
 package org.apache.hadoop.ozone.container.common;
 
-import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.scm.VersionInfo;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos;
+    .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
 import org.apache.hadoop.hdds.protocol.proto
@@ -30,13 +30,13 @@ import org.apache.hadoop.hdds.protocol.proto
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerInfo;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCommandResponseProto;
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMStorageReport;
+    .StorageContainerDatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol;
 import org.apache.hadoop.ozone.protocol.VersionResponse;
 
@@ -56,7 +56,7 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
   // Map of datanode to containers
   private Map<DatanodeDetails, Map<String, ContainerInfo>> nodeContainers =
       new HashMap();
-  private Map<DatanodeDetails, SCMNodeReport> nodeReports = new HashMap<>();
+  private Map<DatanodeDetails, NodeReportProto> nodeReports = new HashMap<>();
   /**
    * Returns the number of heartbeats made to this class.
    *
@@ -166,20 +166,17 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
   /**
    * Used by data node to send a Heartbeat.
    *
-   * @param datanodeDetailsProto - DatanodeDetailsProto.
-   * @param nodeReport - node report.
+   * @param heartbeat - node heartbeat.
    * @return - SCMHeartbeatResponseProto
    * @throws IOException
    */
   @Override
   public StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto
-      sendHeartbeat(DatanodeDetailsProto datanodeDetailsProto,
-                    SCMNodeReport nodeReport)
-      throws IOException {
+      sendHeartbeat(SCMHeartbeatRequestProto heartbeat) throws IOException {
     rpcCount.incrementAndGet();
     heartbeatCount.incrementAndGet();
     sleepIfNeeded();
-    List<SCMCommandResponseProto>
+    List<SCMCommandProto>
         cmdResponses = new LinkedList<>();
     return SCMHeartbeatResponseProto.newBuilder().addAllCommands(cmdResponses)
         .build();
@@ -193,21 +190,19 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
    */
   @Override
   public StorageContainerDatanodeProtocolProtos
-      .SCMRegisteredCmdResponseProto register(
-          DatanodeDetailsProto datanodeDetailsProto, SCMNodeReport nodeReport,
-          StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto
+      .SCMRegisteredResponseProto register(
+          DatanodeDetailsProto datanodeDetailsProto, NodeReportProto nodeReport,
+          StorageContainerDatanodeProtocolProtos.ContainerReportsProto
               containerReportsRequestProto)
       throws IOException {
     rpcCount.incrementAndGet();
-    sendContainerReport(containerReportsRequestProto);
     updateNodeReport(datanodeDetailsProto, nodeReport);
     sleepIfNeeded();
-    return StorageContainerDatanodeProtocolProtos
-        .SCMRegisteredCmdResponseProto
+    return StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto
         .newBuilder().setClusterID(UUID.randomUUID().toString())
         .setDatanodeUUID(datanodeDetailsProto.getUuid()).setErrorCode(
             StorageContainerDatanodeProtocolProtos
-                .SCMRegisteredCmdResponseProto.ErrorCode.success).build();
+                .SCMRegisteredResponseProto.ErrorCode.success).build();
   }
 
   /**
@@ -216,19 +211,19 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
    * @param nodeReport
    */
   public void updateNodeReport(DatanodeDetailsProto datanodeDetailsProto,
-      SCMNodeReport nodeReport) {
+      NodeReportProto nodeReport) {
     DatanodeDetails datanode = DatanodeDetails.getFromProtoBuf(
         datanodeDetailsProto);
-    SCMNodeReport.Builder datanodeReport = SCMNodeReport.newBuilder();
+    NodeReportProto.Builder nodeReportProto = NodeReportProto.newBuilder();
 
-    List<SCMStorageReport> storageReports =
+    List<StorageReportProto> storageReports =
         nodeReport.getStorageReportList();
 
-    for(SCMStorageReport report : storageReports) {
-      datanodeReport.addStorageReport(report);
+    for(StorageReportProto report : storageReports) {
+      nodeReportProto.addStorageReport(report);
     }
 
-    nodeReports.put(datanode, datanodeReport.build());
+    nodeReports.put(datanode, nodeReportProto.build());
 
   }
 
@@ -254,39 +249,6 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
     return 0;
   }
 
-  /**
-   * Send a container report.
-   *
-   * @param reports -- Container report
-   * @return HeartbeatResponse.nullcommand.
-   * @throws IOException
-   */
-  @Override
-  public StorageContainerDatanodeProtocolProtos.ContainerReportsResponseProto
-      sendContainerReport(StorageContainerDatanodeProtocolProtos
-      .ContainerReportsRequestProto reports) throws IOException {
-    Preconditions.checkNotNull(reports);
-    containerReportsCount.incrementAndGet();
-
-    DatanodeDetails datanode = DatanodeDetails.getFromProtoBuf(
-        reports.getDatanodeDetails());
-    if (reports.getReportsCount() > 0) {
-      Map containers = nodeContainers.get(datanode);
-      if (containers == null) {
-        containers = new LinkedHashMap();
-        nodeContainers.put(datanode, containers);
-      }
-
-      for (StorageContainerDatanodeProtocolProtos.ContainerInfo report:
-          reports.getReportsList()) {
-        containers.put(report.getContainerID(), report);
-      }
-    }
-
-    return StorageContainerDatanodeProtocolProtos
-        .ContainerReportsResponseProto.newBuilder().build();
-  }
-
   @Override
   public ContainerBlocksDeletionACKResponseProto sendContainerBlocksDeletionACK(
       ContainerBlocksDeletionACKProto request) throws IOException {


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


[32/50] [abbrv] hadoop git commit: HDDS-125. Cleanup HDDS CheckStyle issues. Contributed by Anu Engineer.

Posted by ha...@apache.org.
HDDS-125. Cleanup HDDS CheckStyle issues.
Contributed by Anu Engineer.


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

Branch: refs/heads/HDDS-48
Commit: 9502b47bd2a3cf32edae635293169883c2914475
Parents: 17aa40f
Author: Anu Engineer <ae...@apache.org>
Authored: Tue May 29 09:54:06 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Tue May 29 09:54:06 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hdds/scm/block/BlockManagerImpl.java |  1 -
 .../hdds/scm/block/DeletedBlockLogImpl.java     |  2 +-
 .../hdds/scm/container/ContainerMapping.java    |  6 +-
 .../scm/container/ContainerStateManager.java    | 24 +++----
 .../hadoop/hdds/scm/container/Mapping.java      |  9 ++-
 .../hdds/scm/node/SCMNodeStorageStatMXBean.java |  4 +-
 .../hdds/scm/node/SCMNodeStorageStatMap.java    | 19 +++---
 .../hdds/scm/node/StorageReportResult.java      |  8 +--
 .../hdds/scm/node/states/Node2ContainerMap.java |  2 +-
 .../hdds/scm/pipelines/PipelineSelector.java    |  5 +-
 .../scm/server/StorageContainerManager.java     |  3 +-
 .../TestStorageContainerManagerHttpServer.java  |  1 -
 .../hadoop/hdds/scm/block/package-info.java     | 23 +++++++
 .../scm/container/TestContainerMapping.java     | 12 ++--
 .../hdds/scm/container/closer/package-info.java | 22 +++++++
 .../hadoop/hdds/scm/container/package-info.java | 22 +++++++
 .../hdds/scm/container/states/package-info.java | 22 +++++++
 .../hadoop/hdds/scm/node/TestNodeManager.java   | 66 ++++++++++----------
 .../scm/node/TestSCMNodeStorageStatMap.java     | 32 +++++-----
 .../hadoop/hdds/scm/node/package-info.java      | 22 +++++++
 .../ozone/container/common/TestEndPoint.java    |  2 -
 .../ozone/container/common/package-info.java    | 22 +++++++
 .../ozone/container/placement/package-info.java | 22 +++++++
 .../replication/TestContainerSupervisor.java    |  7 ++-
 24 files changed, 263 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/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 5a98e85..d17d6c0 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
@@ -41,7 +41,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
-import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/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 cabcb46..cedc506 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
@@ -190,7 +190,7 @@ public class DeletedBlockLogImpl implements DeletedBlockLog {
     try {
       for(Long txID : txIDs) {
         try {
-          byte [] deleteBlockBytes =
+          byte[] deleteBlockBytes =
               deletedStore.get(Longs.toByteArray(txID));
           if (deleteBlockBytes == null) {
             LOG.warn("Delete txID {} not found", txID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/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 e569874..2d88621 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
@@ -152,7 +152,8 @@ public class ContainerMapping implements Mapping {
     ContainerInfo containerInfo;
     lock.lock();
     try {
-      byte[] containerBytes = containerStore.get(Longs.toByteArray(containerID));
+      byte[] containerBytes = containerStore.get(
+          Longs.toByteArray(containerID));
       if (containerBytes == null) {
         throw new SCMException(
             "Specified key does not exist. key : " + containerID,
@@ -229,7 +230,8 @@ public class ContainerMapping implements Mapping {
           containerStateManager.allocateContainer(
               pipelineSelector, type, replicationFactor, owner);
 
-      byte[] containerIDBytes = Longs.toByteArray(containerInfo.getContainerID());
+      byte[] containerIDBytes = Longs.toByteArray(
+          containerInfo.getContainerID());
       containerStore.put(containerIDBytes, containerInfo.getProtobuf()
               .toByteArray());
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/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 f11a50c..4895b78 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
@@ -230,18 +230,18 @@ public class ContainerStateManager implements Closeable {
    *
    * Container State Flow:
    *
-   * [ALLOCATED]------->[CREATING]--------->[OPEN]---------->[CLOSING]------->[CLOSED]
-   *            (CREATE)     |    (CREATED)       (FINALIZE)          (CLOSE)    |
-   *                         |                                                   |
-   *                         |                                                   |
-   *                         |(TIMEOUT)                                  (DELETE)|
-   *                         |                                                   |
-   *                         +------------------> [DELETING] <-------------------+
-   *                                                   |
-   *                                                   |
-   *                                          (CLEANUP)|
-   *                                                   |
-   *                                               [DELETED]
+   * [ALLOCATED]---->[CREATING]------>[OPEN]-------->[CLOSING]------->[CLOSED]
+   *            (CREATE)     |    (CREATED)       (FINALIZE)     (CLOSE)    |
+   *                         |                                              |
+   *                         |                                              |
+   *                         |(TIMEOUT)                             (DELETE)|
+   *                         |                                              |
+   *                         +-------------> [DELETING] <-------------------+
+   *                                            |
+   *                                            |
+   *                                   (CLEANUP)|
+   *                                            |
+   *                                        [DELETED]
    */
   private void initializeStateMachine() {
     stateMachine.addTransition(LifeCycleState.ALLOCATED,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
index 61dee2b..f560174 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
@@ -45,7 +45,8 @@ public interface Mapping extends Closeable {
    * The max size of the searching range cannot exceed the
    * value of count.
    *
-   * @param startContainerID start containerID, >=0, start searching at the head if 0.
+   * @param startContainerID start containerID, >=0,
+   * start searching at the head if 0.
    * @param count count must be >= 0
    *              Usually the count will be replace with a very big
    *              value instead of being unlimited in case the db is very big.
@@ -53,7 +54,8 @@ public interface Mapping extends Closeable {
    * @return a list of container.
    * @throws IOException
    */
-  List<ContainerInfo> listContainer(long startContainerID, int count) throws IOException;
+  List<ContainerInfo> listContainer(long startContainerID, int count)
+      throws IOException;
 
   /**
    * Allocates a new container for a given keyName and replication factor.
@@ -64,7 +66,8 @@ public interface Mapping extends Closeable {
    * @throws IOException
    */
   ContainerInfo allocateContainer(HddsProtos.ReplicationType type,
-      HddsProtos.ReplicationFactor replicationFactor, String owner) throws IOException;
+      HddsProtos.ReplicationFactor replicationFactor, String owner)
+      throws IOException;
 
   /**
    * Deletes a container from SCM.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMXBean.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMXBean.java
index d81ff0f..32ecbad 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMXBean.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMXBean.java
@@ -31,7 +31,7 @@ import java.util.UUID;
 @InterfaceAudience.Private
 public interface SCMNodeStorageStatMXBean {
   /**
-   * Get the capacity of the dataNode
+   * Get the capacity of the dataNode.
    * @param datanodeID Datanode Id
    * @return long
    */
@@ -52,7 +52,7 @@ public interface SCMNodeStorageStatMXBean {
   long getUsedSpace(UUID datanodeId);
 
   /**
-   * Returns the total capacity of all dataNodes
+   * Returns the total capacity of all dataNodes.
    * @return long
    */
   long getTotalCapacity();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMap.java
index f8ad2af..fa423bb 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMap.java
@@ -56,7 +56,7 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
   // NodeStorageInfo MXBean
   private ObjectName scmNodeStorageInfoBean;
   /**
-   * constructs the scmNodeStorageReportMap object
+   * constructs the scmNodeStorageReportMap object.
    */
   public SCMNodeStorageStatMap(OzoneConfiguration conf) {
     // scmNodeStorageReportMap = new ConcurrentHashMap<>();
@@ -73,6 +73,9 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
             HDDS_DATANODE_STORAGE_UTILIZATION_CRITICAL_THRESHOLD_DEFAULT);
   }
 
+  /**
+   * Enum that Describes what we should do at various thresholds.
+   */
   public enum UtilizationThreshold {
     NORMAL, WARN, CRITICAL;
   }
@@ -107,8 +110,8 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
    * @param datanodeID -- Datanode UUID
    * @param report - set if StorageReports.
    */
-  public void insertNewDatanode(UUID datanodeID, Set<StorageLocationReport> report)
-      throws SCMException {
+  public void insertNewDatanode(UUID datanodeID,
+      Set<StorageLocationReport> report) throws SCMException {
     Preconditions.checkNotNull(report);
     Preconditions.checkState(report.size() != 0);
     Preconditions.checkNotNull(datanodeID);
@@ -142,8 +145,8 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
    * @throws SCMException - if we don't know about this datanode, for new DN
    *                      use insertNewDatanode.
    */
-  public void updateDatanodeMap(UUID datanodeID, Set<StorageLocationReport> report)
-      throws SCMException {
+  public void updateDatanodeMap(UUID datanodeID,
+      Set<StorageLocationReport> report) throws SCMException {
     Preconditions.checkNotNull(datanodeID);
     Preconditions.checkNotNull(report);
     Preconditions.checkState(report.size() != 0);
@@ -301,7 +304,7 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
   }
 
   /**
-   * removes the dataNode from scmNodeStorageReportMap
+   * removes the dataNode from scmNodeStorageReportMap.
    * @param datanodeID
    * @throws SCMException in case the dataNode is not found in the map.
    */
@@ -339,11 +342,11 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
   }
 
   /**
-   * get the scmUsed ratio
+   * get the scmUsed ratio.
    */
   public  double getScmUsedratio(long scmUsed, long capacity) {
     double scmUsedRatio =
-        truncateDecimals (scmUsed / (double) capacity);
+        truncateDecimals(scmUsed / (double) capacity);
     return scmUsedRatio;
   }
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StorageReportResult.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StorageReportResult.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StorageReportResult.java
index 3436e77..0b63ceb 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StorageReportResult.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StorageReportResult.java
@@ -69,14 +69,14 @@ public class StorageReportResult {
     }
 
     public ReportResultBuilder setFullVolumeSet(
-        Set<StorageLocationReport> fullVolumes) {
-      this.fullVolumes = fullVolumes;
+        Set<StorageLocationReport> fullVolumesSet) {
+      this.fullVolumes = fullVolumesSet;
       return this;
     }
 
     public ReportResultBuilder setFailedVolumeSet(
-        Set<StorageLocationReport> failedVolumes) {
-      this.failedVolumes = failedVolumes;
+        Set<StorageLocationReport> failedVolumesSet) {
+      this.failedVolumes = failedVolumesSet;
       return this;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMap.java
index f850e7a..1960604 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMap.java
@@ -92,7 +92,7 @@ public class Node2ContainerMap {
   }
 
   /**
-   * Removes datanode Entry from the map
+   * Removes datanode Entry from the map.
    * @param datanodeID - Datanode ID.
    */
   public void removeDatanode(UUID datanodeID) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/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 d29bb84..2e56043 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
@@ -170,8 +170,9 @@ public class PipelineSelector {
       throws IOException {
     PipelineManager manager = getPipelineManager(replicationType);
     Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
-    LOG.debug("Getting replication pipeline forReplicationType {} : ReplicationFactor {}",
-        replicationType.toString(), replicationFactor.toString());
+    LOG.debug("Getting replication pipeline forReplicationType {} :" +
+            " ReplicationFactor {}", replicationType.toString(),
+        replicationFactor.toString());
     return manager.
         getPipeline(replicationFactor, replicationType);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/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 0fd6843..78f13cb 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
@@ -45,7 +45,6 @@ import org.apache.hadoop.hdds.scm.node.SCMNodeManager;
 import org.apache.hadoop.hdds.server.ServiceRuntimeInfoImpl;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.util.MBeans;
@@ -87,7 +86,7 @@ import static org.apache.hadoop.util.ExitUtil.terminate;
  * create a container, which then can be used to store data.
  */
 @InterfaceAudience.LimitedPrivate({"HDFS", "CBLOCK", "OZONE", "HBASE"})
-public class StorageContainerManager extends ServiceRuntimeInfoImpl
+public final class StorageContainerManager extends ServiceRuntimeInfoImpl
     implements SCMMXBean {
 
   private static final Logger LOG = LoggerFactory

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestStorageContainerManagerHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestStorageContainerManagerHttpServer.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestStorageContainerManagerHttpServer.java
index 0dbb7c1..d9e1425 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestStorageContainerManagerHttpServer.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestStorageContainerManagerHttpServer.java
@@ -39,7 +39,6 @@ import org.junit.runners.Parameterized.Parameters;
 import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.net.MalformedURLException;
 import java.net.URL;
 import java.net.URLConnection;
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/package-info.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/package-info.java
new file mode 100644
index 0000000..a67df69
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ *
+ */
+
+/**
+ * Make checkstyle happy.
+ * */
+package org.apache.hadoop.hdds.scm.block;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/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 a27068bb..f318316 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
@@ -216,8 +216,10 @@ public class TestContainerMapping {
 
     mapping.processContainerReports(crBuilder.build());
 
-    ContainerInfo updatedContainer = mapping.getContainer(info.getContainerID());
-    Assert.assertEquals(100000000L, updatedContainer.getNumberOfKeys());
+    ContainerInfo updatedContainer =
+        mapping.getContainer(info.getContainerID());
+    Assert.assertEquals(100000000L,
+        updatedContainer.getNumberOfKeys());
     Assert.assertEquals(2000000000L, updatedContainer.getUsedBytes());
   }
 
@@ -251,8 +253,10 @@ public class TestContainerMapping {
 
     mapping.processContainerReports(crBuilder.build());
 
-    ContainerInfo updatedContainer = mapping.getContainer(info.getContainerID());
-    Assert.assertEquals(500000000L, updatedContainer.getNumberOfKeys());
+    ContainerInfo updatedContainer =
+        mapping.getContainer(info.getContainerID());
+    Assert.assertEquals(500000000L,
+        updatedContainer.getNumberOfKeys());
     Assert.assertEquals(5368705120L, updatedContainer.getUsedBytes());
     NavigableSet<ContainerID> pendingCloseContainers = mapping.getStateManager()
         .getMatchingContainerIDs(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/package-info.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/package-info.java
new file mode 100644
index 0000000..2f35719
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/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.
+ *
+ */
+/**
+ * Make CheckStyle happy.
+ */
+package org.apache.hadoop.hdds.scm.container.closer;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/package-info.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/package-info.java
new file mode 100644
index 0000000..f93aea6
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/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.
+ *
+ */
+/**
+ * Make CheckStyle Happy.
+ */
+package org.apache.hadoop.hdds.scm.container;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/states/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/states/package-info.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/states/package-info.java
new file mode 100644
index 0000000..795dfc1
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/states/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.
+ *
+ */
+/**
+ * Make CheckStyle Happy.
+ */
+package org.apache.hadoop.hdds.scm.container.states;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java
index 36e796f..de87e50 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java
@@ -510,42 +510,42 @@ public class TestNodeManager {
    * @throws InterruptedException
    * @throws TimeoutException
    */
+  /**
+   * These values are very important. Here is what it means so you don't
+   * have to look it up while reading this code.
+   *
+   *  OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL - This the frequency of the
+   *  HB processing thread that is running in the SCM. This thread must run
+   *  for the SCM  to process the Heartbeats.
+   *
+   *  OZONE_SCM_HEARTBEAT_INTERVAL - This is the frequency at which
+   *  datanodes will send heartbeats to SCM. Please note: This is the only
+   *  config value for node manager that is specified in seconds. We don't
+   *  want SCM heartbeat resolution to be more than in seconds.
+   *  In this test it is not used, but we are forced to set it because we
+   *  have validation code that checks Stale Node interval and Dead Node
+   *  interval is larger than the value of
+   *  OZONE_SCM_HEARTBEAT_INTERVAL.
+   *
+   *  OZONE_SCM_STALENODE_INTERVAL - This is the time that must elapse
+   *  from the last heartbeat for us to mark a node as stale. In this test
+   *  we set that to 3. That is if a node has not heartbeat SCM for last 3
+   *  seconds we will mark it as stale.
+   *
+   *  OZONE_SCM_DEADNODE_INTERVAL - This is the time that must elapse
+   *  from the last heartbeat for a node to be marked dead. We have an
+   *  additional constraint that this must be at least 2 times bigger than
+   *  Stale node Interval.
+   *
+   *  With these we are trying to explore the state of this cluster with
+   *  various timeouts. Each section is commented so that you can keep
+   *  track of the state of the cluster nodes.
+   *
+   */
+
   @Test
   public void testScmClusterIsInExpectedState1() throws IOException,
       InterruptedException, TimeoutException {
-    /**
-     * These values are very important. Here is what it means so you don't
-     * have to look it up while reading this code.
-     *
-     *  OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL - This the frequency of the
-     *  HB processing thread that is running in the SCM. This thread must run
-     *  for the SCM  to process the Heartbeats.
-     *
-     *  OZONE_SCM_HEARTBEAT_INTERVAL - This is the frequency at which
-     *  datanodes will send heartbeats to SCM. Please note: This is the only
-     *  config value for node manager that is specified in seconds. We don't
-     *  want SCM heartbeat resolution to be more than in seconds.
-     *  In this test it is not used, but we are forced to set it because we
-     *  have validation code that checks Stale Node interval and Dead Node
-     *  interval is larger than the value of
-     *  OZONE_SCM_HEARTBEAT_INTERVAL.
-     *
-     *  OZONE_SCM_STALENODE_INTERVAL - This is the time that must elapse
-     *  from the last heartbeat for us to mark a node as stale. In this test
-     *  we set that to 3. That is if a node has not heartbeat SCM for last 3
-     *  seconds we will mark it as stale.
-     *
-     *  OZONE_SCM_DEADNODE_INTERVAL - This is the time that must elapse
-     *  from the last heartbeat for a node to be marked dead. We have an
-     *  additional constraint that this must be at least 2 times bigger than
-     *  Stale node Interval.
-     *
-     *  With these we are trying to explore the state of this cluster with
-     *  various timeouts. Each section is commented so that you can keep
-     *  track of the state of the cluster nodes.
-     *
-     */
-
     OzoneConfiguration conf = getConf();
     conf.setTimeDuration(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL, 100,
         MILLISECONDS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeStorageStatMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeStorageStatMap.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeStorageStatMap.java
index 571de77..b824412 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeStorageStatMap.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeStorageStatMap.java
@@ -42,11 +42,14 @@ import java.util.HashSet;
 import java.io.IOException;
 import java.util.concurrent.ConcurrentHashMap;
 
+/**
+ * Test Node Storage Map.
+ */
 public class TestSCMNodeStorageStatMap {
   private final static int DATANODE_COUNT = 100;
-  final long capacity = 10L * OzoneConsts.GB;
-  final long used = 2L * OzoneConsts.GB;
-  final long remaining = capacity - used;
+  private final long capacity = 10L * OzoneConsts.GB;
+  private final long used = 2L * OzoneConsts.GB;
+  private final long remaining = capacity - used;
   private static OzoneConfiguration conf = new OzoneConfiguration();
   private final Map<UUID, Set<StorageLocationReport>> testData =
       new ConcurrentHashMap<>();
@@ -59,9 +62,10 @@ public class TestSCMNodeStorageStatMap {
       UUID dnId = UUID.randomUUID();
       Set<StorageLocationReport> reportSet = new HashSet<>();
       String path = GenericTestUtils.getTempPath(
-          TestSCMNodeStorageStatMap.class.getSimpleName() + "-" + Integer
-              .toString(dnIndex));
-      StorageLocationReport.Builder builder = StorageLocationReport.newBuilder();
+          TestSCMNodeStorageStatMap.class.getSimpleName() + "-" +
+              Integer.toString(dnIndex));
+      StorageLocationReport.Builder builder =
+          StorageLocationReport.newBuilder();
       builder.setStorageType(StorageType.DISK).setId(dnId.toString())
           .setStorageLocation(path).setScmUsed(used).setRemaining(remaining)
           .setCapacity(capacity).setFailed(false);
@@ -139,12 +143,12 @@ public class TestSCMNodeStorageStatMap {
     String path =
         GenericTestUtils.getRandomizedTempPath().concat("/" + storageId);
     StorageLocationReport report = reportSet.iterator().next();
-    long capacity = report.getCapacity();
-    long used = report.getScmUsed();
-    long remaining = report.getRemaining();
+    long reportCapacity = report.getCapacity();
+    long reportScmUsed = report.getScmUsed();
+    long reportRemaining = report.getRemaining();
     List<SCMStorageReport> reports = TestUtils
-        .createStorageReport(capacity, used, remaining, path, null, storageId,
-            1);
+        .createStorageReport(reportCapacity, reportScmUsed, reportRemaining,
+            path, null, storageId, 1);
     StorageReportResult result =
         map.processNodeReport(key, TestUtils.createNodeReport(reports));
     Assert.assertEquals(result.getStatus(),
@@ -158,7 +162,7 @@ public class TestSCMNodeStorageStatMap {
         SCMNodeStorageStatMap.ReportStatus.ALL_IS_WELL);
 
     reportList.add(TestUtils
-        .createStorageReport(capacity, capacity, 0, path, null,
+        .createStorageReport(reportCapacity, reportCapacity, 0, path, null,
             UUID.randomUUID().toString(), 1).get(0));
     result = map.processNodeReport(key, TestUtils.createNodeReport(reportList));
     Assert.assertEquals(result.getStatus(),
@@ -166,8 +170,8 @@ public class TestSCMNodeStorageStatMap {
     // Mark a disk failed 
     SCMStorageReport srb2 = SCMStorageReport.newBuilder()
         .setStorageUuid(UUID.randomUUID().toString())
-        .setStorageLocation(srb.getStorageLocation()).setScmUsed(capacity)
-        .setCapacity(capacity).setRemaining(0).setFailed(true).build();
+        .setStorageLocation(srb.getStorageLocation()).setScmUsed(reportCapacity)
+        .setCapacity(reportCapacity).setRemaining(0).setFailed(true).build();
     reportList.add(srb2);
     nrb.addAllStorageReport(reportList);
     result = map.processNodeReport(key, nrb.addStorageReport(srb).build());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/package-info.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/package-info.java
new file mode 100644
index 0000000..dfd8397
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/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.
+ *
+ */
+/**
+ * Make CheckStyle Happy.
+ */
+package org.apache.hadoop.hdds.scm.node;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/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 e82dc98..1d92cdc 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
@@ -33,8 +33,6 @@ import org.apache.hadoop.hdds.protocol.proto
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
-import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMStorageReport;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/package-info.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/package-info.java
new file mode 100644
index 0000000..da2ae84
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/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.
+ *
+ */
+/**
+ * Make CheckStyle Happy.
+ */
+package org.apache.hadoop.ozone.container.common;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/placement/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/placement/package-info.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/placement/package-info.java
new file mode 100644
index 0000000..ddd751c
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/placement/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.
+ *
+ */
+/**
+ * Make CheckStyle Happy.
+ */
+package org.apache.hadoop.ozone.container.placement;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9502b47b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerSupervisor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerSupervisor.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerSupervisor.java
index 01f70b1..e197886 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerSupervisor.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerSupervisor.java
@@ -202,8 +202,8 @@ public class TestContainerSupervisor {
       ppool.handleContainerReport(reportsProto);
     }
 
-    clist = datanodeStateManager.getContainerReport(wayOverReplicatedContainerID,
-        ppool.getPool().getPoolName(), 7);
+    clist = datanodeStateManager.getContainerReport(
+        wayOverReplicatedContainerID, ppool.getPool().getPoolName(), 7);
 
     for (ContainerReportsRequestProto reportsProto : clist) {
       ppool.handleContainerReport(reportsProto);
@@ -264,7 +264,8 @@ public class TestContainerSupervisor {
               "PoolNew", 1);
       containerSupervisor.handleContainerReport(clist.get(0));
       GenericTestUtils.waitFor(() ->
-          inProgressLog.getOutput().contains(Long.toString(newContainerID)) && inProgressLog
+          inProgressLog.getOutput()
+              .contains(Long.toString(newContainerID)) && inProgressLog
               .getOutput().contains(id.getUuidString()),
           200, 10 * 1000);
     } finally {


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


[19/50] [abbrv] hadoop git commit: HADOOP-15477. Make unjar in RunJar overrideable

Posted by ha...@apache.org.
HADOOP-15477. Make unjar in RunJar overrideable

Signed-off-by: Akira Ajisaka <aa...@apache.org>


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

Branch: refs/heads/HDDS-48
Commit: d14e26b31fe46fb47a8e99a212c70016fd15a4d9
Parents: 0cf6e87
Author: Johan Gustavsson <jo...@treasure-data.com>
Authored: Mon May 28 17:29:59 2018 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon May 28 17:29:59 2018 +0900

----------------------------------------------------------------------
 .../java/org/apache/hadoop/util/RunJar.java     | 17 ++++++---
 .../java/org/apache/hadoop/util/TestRunJar.java | 37 ++++++++++++++++++--
 .../org/apache/hadoop/streaming/StreamJob.java  |  4 ++-
 3 files changed, 51 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d14e26b3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
index 9dd770c..f1b643c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
@@ -76,7 +76,11 @@ public class RunJar {
    */
   public static final String HADOOP_CLIENT_CLASSLOADER_SYSTEM_CLASSES =
       "HADOOP_CLIENT_CLASSLOADER_SYSTEM_CLASSES";
-
+  /**
+   * Environment key for disabling unjar in client code.
+   */
+  public static final String HADOOP_CLIENT_SKIP_UNJAR =
+      "HADOOP_CLIENT_SKIP_UNJAR";
   /**
    * Buffer size for copy the content of compressed file to new file.
    */
@@ -93,7 +97,7 @@ public class RunJar {
    * @throws IOException if an I/O error has occurred or toDir
    * cannot be created and does not already exist
    */
-  public static void unJar(File jarFile, File toDir) throws IOException {
+  public void unJar(File jarFile, File toDir) throws IOException {
     unJar(jarFile, toDir, MATCH_ANY);
   }
 
@@ -292,8 +296,9 @@ public class RunJar {
           }
         }, SHUTDOWN_HOOK_PRIORITY);
 
-
-    unJar(file, workDir);
+    if (!skipUnjar()) {
+      unJar(file, workDir);
+    }
 
     ClassLoader loader = createClassLoader(file, workDir);
 
@@ -364,6 +369,10 @@ public class RunJar {
     return Boolean.parseBoolean(System.getenv(HADOOP_USE_CLIENT_CLASSLOADER));
   }
 
+  boolean skipUnjar() {
+    return Boolean.parseBoolean(System.getenv(HADOOP_CLIENT_SKIP_UNJAR));
+  }
+
   String getHadoopClasspath() {
     return System.getenv(HADOOP_CLASSPATH);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d14e26b3/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java
index 19485d6..ea07b97 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java
@@ -17,10 +17,14 @@
  */
 package org.apache.hadoop.util;
 
+import static org.apache.hadoop.util.RunJar.MATCH_ANY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
 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 java.io.File;
@@ -99,7 +103,7 @@ public class TestRunJar {
 
     // Unjar everything
     RunJar.unJar(new File(TEST_ROOT_DIR, TEST_JAR_NAME),
-                 unjarDir);
+                 unjarDir, MATCH_ANY);
     assertTrue("foobar unpacked",
                new File(unjarDir, TestRunJar.FOOBAR_TXT).exists());
     assertTrue("foobaz unpacked",
@@ -177,7 +181,7 @@ public class TestRunJar {
 
     // Unjar everything
     RunJar.unJar(new File(TEST_ROOT_DIR, TEST_JAR_NAME),
-            unjarDir);
+            unjarDir, MATCH_ANY);
 
     String failureMessage = "Last modify time was lost during unJar";
     assertEquals(failureMessage, MOCKED_NOW, new File(unjarDir, TestRunJar.FOOBAR_TXT).lastModified());
@@ -221,5 +225,34 @@ public class TestRunJar {
     // run RunJar
     runJar.run(args);
     // it should not throw an exception
+    verify(runJar, times(1)).unJar(any(File.class), any(File.class));
+  }
+
+  @Test
+  public void testClientClassLoaderSkipUnjar() throws Throwable {
+    RunJar runJar = spy(new RunJar());
+    // enable the client classloader
+    when(runJar.useClientClassLoader()).thenReturn(true);
+    // set the system classes and blacklist the test main class and the test
+    // third class so they can be loaded by the application classloader
+    String mainCls = ClassLoaderCheckMain.class.getName();
+    String thirdCls = ClassLoaderCheckThird.class.getName();
+    String systemClasses = "-" + mainCls + "," +
+        "-" + thirdCls + "," +
+        ApplicationClassLoader.SYSTEM_CLASSES_DEFAULT;
+    when(runJar.getSystemClasses()).thenReturn(systemClasses);
+
+    // create the test jar
+    File testJar = JarFinder.makeClassLoaderTestJar(this.getClass(),
+        TEST_ROOT_DIR, TEST_JAR_2_NAME, BUFF_SIZE, mainCls, thirdCls);
+    // form the args
+    String[] args = new String[3];
+    args[0] = testJar.getAbsolutePath();
+    args[1] = mainCls;
+    when(runJar.skipUnjar()).thenReturn(true);
+    // run RunJar
+    runJar.run(args);
+    // it should not throw an exception
+    verify(runJar, times(0)).unJar(any(File.class), any(File.class));
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d14e26b3/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java
index 9b09729..1fe8710 100644
--- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java
+++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java
@@ -72,6 +72,8 @@ import org.apache.hadoop.util.RunJar;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 
+import static org.apache.hadoop.util.RunJar.MATCH_ANY;
+
 /** All the client-side work happens here.
  * (Jar packaging, MapRed job submission and monitoring)
  */
@@ -1006,7 +1008,7 @@ public class StreamJob implements Tool {
     if (jar_ != null && isLocalHadoop()) {
       // getAbs became required when shell and subvm have different working dirs...
       File wd = new File(".").getAbsoluteFile();
-      RunJar.unJar(new File(jar_), wd);
+      RunJar.unJar(new File(jar_), wd, MATCH_ANY);
     }
 
     // if jobConf_ changes must recreate a JobClient


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


[21/50] [abbrv] hadoop git commit: HDFS-13628. Update Archival Storage doc for Provided Storage

Posted by ha...@apache.org.
HDFS-13628. Update Archival Storage doc for Provided Storage

Signed-off-by: Akira Ajisaka <aa...@apache.org>


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

Branch: refs/heads/HDDS-48
Commit: 04757e5864bd4904fd5a59d143fff480814700e4
Parents: 88cbe57
Author: Takanobu Asanuma <ta...@yahoo-corp.jp>
Authored: Mon May 28 19:04:36 2018 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon May 28 19:06:34 2018 +0900

----------------------------------------------------------------------
 .../hadoop-hdfs/src/site/markdown/ArchivalStorage.md             | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/04757e58/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
index ab7975a..3c49cb1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -35,7 +35,7 @@ A new storage type *ARCHIVE*, which has high storage density (petabyte of storag
 
 Another new storage type *RAM\_DISK* is added for supporting writing single replica files in memory.
 
-### Storage Policies: Hot, Warm, Cold, All\_SSD, One\_SSD and Lazy\_Persist
+### Storage Policies: Hot, Warm, Cold, All\_SSD, One\_SSD, Lazy\_Persist and Provided
 
 A new concept of storage policies is introduced in order to allow files to be stored in different storage types according to the storage policy.
 
@@ -47,6 +47,7 @@ We have the following storage policies:
 * **All\_SSD** - for storing all replicas in SSD.
 * **One\_SSD** - for storing one of the replicas in SSD. The remaining replicas are stored in DISK.
 * **Lazy\_Persist** - for writing blocks with single replica in memory. The replica is first written in RAM\_DISK and then it is lazily persisted in DISK.
+* **Provided** - for storing data outside HDFS. See also [HDFS Provided Storage](./HdfsProvidedStorage.html).
 
 More formally, a storage policy consists of the following fields:
 
@@ -68,6 +69,7 @@ The following is a typical storage policy table.
 | 7 | Hot (default) | DISK: *n* | \<none\> | ARCHIVE |
 | 5 | Warm | DISK: 1, ARCHIVE: *n*-1 | ARCHIVE, DISK | ARCHIVE, DISK |
 | 2 | Cold | ARCHIVE: *n* | \<none\> | \<none\> |
+| 1 | Provided | PROVIDED: 1, DISK: *n*-1 | PROVIDED, DISK | PROVIDED, DISK |
 
 Note 1: The Lazy\_Persist policy is useful only for single replica blocks. For blocks with more than one replicas, all the replicas will be written to DISK since writing only one of the replicas to RAM\_DISK does not improve the overall performance.
 


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


[35/50] [abbrv] hadoop git commit: HDDS-81. Moving ContainerReport inside Datanode heartbeat. Contributed by Nanda Kumar.

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationDatanodeStateManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationDatanodeStateManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationDatanodeStateManager.java
deleted file mode 100644
index 50fd18f..0000000
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationDatanodeStateManager.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/**
- * 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.testutils;
-
-import com.google.common.primitives.Longs;
-import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.hadoop.hdds.scm.node.NodeManager;
-import org.apache.hadoop.hdds.scm.node.NodePoolManager;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerInfo;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
-
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Random;
-
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
-    .HEALTHY;
-
-/**
- * This class  manages the state of datanode
- * in conjunction with the node pool and node managers.
- */
-public class ReplicationDatanodeStateManager {
-  private final NodeManager nodeManager;
-  private final NodePoolManager poolManager;
-  private final Random r;
-
-  /**
-   * The datanode state Manager.
-   *
-   * @param nodeManager
-   * @param poolManager
-   */
-  public ReplicationDatanodeStateManager(NodeManager nodeManager,
-      NodePoolManager poolManager) {
-    this.nodeManager = nodeManager;
-    this.poolManager = poolManager;
-    r = new Random();
-  }
-
-  /**
-   * Get Container Report as if it is from a datanode in the cluster.
-   * @param containerID - Container ID.
-   * @param poolName - Pool Name.
-   * @param dataNodeCount - Datanode Count.
-   * @return List of Container Reports.
-   */
-  public List<ContainerReportsRequestProto> getContainerReport(
-      long containerID, String poolName, int dataNodeCount) {
-    List<ContainerReportsRequestProto> containerList = new LinkedList<>();
-    List<DatanodeDetails> nodesInPool = poolManager.getNodes(poolName);
-
-    if (nodesInPool == null) {
-      return containerList;
-    }
-
-    if (nodesInPool.size() < dataNodeCount) {
-      throw new IllegalStateException("Not enough datanodes to create " +
-          "required container reports");
-    }
-
-    while (containerList.size() < dataNodeCount && nodesInPool.size() > 0) {
-      DatanodeDetails id = nodesInPool.get(r.nextInt(nodesInPool.size()));
-      nodesInPool.remove(id);
-      containerID++;
-      // We return container reports only for nodes that are healthy.
-      if (nodeManager.getNodeState(id) == HEALTHY) {
-        ContainerInfo info = ContainerInfo.newBuilder()
-            .setContainerID(containerID)
-            .setFinalhash(DigestUtils.sha256Hex(
-                Longs.toByteArray(containerID)))
-            .setContainerID(containerID)
-            .build();
-        ContainerReportsRequestProto containerReport =
-            ContainerReportsRequestProto.newBuilder().addReports(info)
-            .setDatanodeDetails(id.getProtoBufMessage())
-            .setType(ContainerReportsRequestProto.reportType.fullReport)
-            .build();
-        containerList.add(containerReport);
-      }
-    }
-    return containerList;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java
index 3f814d0..072d821 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java
@@ -24,13 +24,13 @@ import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.hdds.scm.node.NodePoolManager;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
 import org.apache.hadoop.ozone.protocol.VersionResponse;
+import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
 import org.mockito.Mockito;
 
@@ -277,12 +277,12 @@ public class ReplicationNodeManagerMock implements NodeManager {
    * Register the node if the node finds that it is not registered with any SCM.
    *
    * @param dd DatanodeDetailsProto
-   * @param nodeReport SCMNodeReport
+   * @param nodeReport NodeReportProto
    * @return SCMHeartbeatResponseProto
    */
   @Override
-  public SCMCommand register(HddsProtos.DatanodeDetailsProto dd,
-                             SCMNodeReport nodeReport) {
+  public RegisteredCommand register(DatanodeDetails dd,
+                                    NodeReportProto nodeReport) {
     return null;
   }
 
@@ -294,8 +294,8 @@ public class ReplicationNodeManagerMock implements NodeManager {
    * @return SCMheartbeat response list
    */
   @Override
-  public List<SCMCommand> sendHeartbeat(HddsProtos.DatanodeDetailsProto dd,
-      SCMNodeReport nodeReport) {
+  public List<SCMCommand> sendHeartbeat(DatanodeDetails dd,
+      NodeReportProto nodeReport) {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
index a0d41a8..0c1d8f2 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
@@ -32,8 +32,10 @@ import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager.StartupOption;
 import org.apache.hadoop.hdds.scm.block.DeletedBlockLog;
@@ -302,12 +304,11 @@ public class TestStorageContainerManager {
       NodeManager nodeManager = cluster.getStorageContainerManager()
           .getScmNodeManager();
       List<SCMCommand> commands = nodeManager.sendHeartbeat(
-          nodeManager.getNodes(NodeState.HEALTHY).get(0).getProtoBufMessage(),
-          null);
+          nodeManager.getNodes(NodeState.HEALTHY).get(0), null);
 
       if (commands != null) {
         for (SCMCommand cmd : commands) {
-          if (cmd.getType() == SCMCmdType.deleteBlocksCommand) {
+          if (cmd.getType() == SCMCommandProto.Type.deleteBlocksCommand) {
             List<DeletedBlocksTransaction> deletedTXs =
                 ((DeleteBlocksCommand) cmd).blocksTobeDeleted();
             return deletedTXs != null && deletedTXs.size() == limitSize;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMetrics.java
index 1d19bb3..1dbe760 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMetrics.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMetrics.java
@@ -32,8 +32,10 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerReport;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.hdds.scm.container.placement.metrics.ContainerStat;
 import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMMetrics;
 import org.apache.hadoop.hdds.scm.node.SCMNodeManager;
@@ -75,11 +77,11 @@ public class TestSCMMetrics {
       ContainerStat stat = new ContainerStat(size, used, keyCount, readBytes,
           writeBytes, readCount, writeCount);
       StorageContainerManager scmManager = cluster.getStorageContainerManager();
-
-      ContainerReportsRequestProto request = createContainerReport(numReport,
-          stat, null);
-      String fstDatanodeUuid = request.getDatanodeDetails().getUuid();
-      scmManager.getDatanodeProtocolServer().sendContainerReport(request);
+      DatanodeDetails fstDatanodeDetails = TestUtils.getDatanodeDetails();
+      ContainerReportsProto request = createContainerReport(numReport, stat);
+      String fstDatanodeUuid = fstDatanodeDetails.getUuidString();
+      scmManager.getDatanodeProtocolServer().processContainerReports(
+          fstDatanodeDetails, request);
 
       // verify container stat metrics
       MetricsRecordBuilder scmMetrics = getMetrics(SCMMetrics.SOURCE_NAME);
@@ -100,9 +102,11 @@ public class TestSCMMetrics {
           getLongGauge("LastContainerReportWriteCount", scmMetrics));
 
       // add one new report
-      request = createContainerReport(1, stat, null);
-      String sndDatanodeUuid = request.getDatanodeDetails().getUuid();
-      scmManager.getDatanodeProtocolServer().sendContainerReport(request);
+      DatanodeDetails sndDatanodeDetails = TestUtils.getDatanodeDetails();
+      request = createContainerReport(1, stat);
+      String sndDatanodeUuid = sndDatanodeDetails.getUuidString();
+      scmManager.getDatanodeProtocolServer().processContainerReports(
+          sndDatanodeDetails, request);
 
       scmMetrics = getMetrics(SCMMetrics.SOURCE_NAME);
       assertEquals(size * (numReport + 1),
@@ -124,12 +128,12 @@ public class TestSCMMetrics {
       // Re-send reports but with different value for validating
       // the aggregation.
       stat = new ContainerStat(100, 50, 3, 50, 60, 5, 6);
-      scmManager.getDatanodeProtocolServer().sendContainerReport(
-          createContainerReport(1, stat, fstDatanodeUuid));
+      scmManager.getDatanodeProtocolServer().processContainerReports(
+          fstDatanodeDetails, createContainerReport(1, stat));
 
       stat = new ContainerStat(1, 1, 1, 1, 1, 1, 1);
-      scmManager.getDatanodeProtocolServer().sendContainerReport(
-          createContainerReport(1, stat, sndDatanodeUuid));
+      scmManager.getDatanodeProtocolServer().processContainerReports(
+          sndDatanodeDetails, createContainerReport(1, stat));
 
       // the global container metrics value should be updated
       scmMetrics = getMetrics(SCMMetrics.SOURCE_NAME);
@@ -170,11 +174,11 @@ public class TestSCMMetrics {
           writeBytes, readCount, writeCount);
       StorageContainerManager scmManager = cluster.getStorageContainerManager();
 
-      String datanodeUuid = cluster.getHddsDatanodes().get(0)
-          .getDatanodeDetails().getUuidString();
-      ContainerReportsRequestProto request = createContainerReport(numReport,
-          stat, datanodeUuid);
-      scmManager.getDatanodeProtocolServer().sendContainerReport(request);
+      DatanodeDetails datanodeDetails = cluster.getHddsDatanodes().get(0)
+          .getDatanodeDetails();
+      ContainerReportsProto request = createContainerReport(numReport, stat);
+      scmManager.getDatanodeProtocolServer().processContainerReports(
+          datanodeDetails, request);
 
       MetricsRecordBuilder scmMetrics = getMetrics(SCMMetrics.SOURCE_NAME);
       assertEquals(size * numReport,
@@ -216,11 +220,11 @@ public class TestSCMMetrics {
     }
   }
 
-  private ContainerReportsRequestProto createContainerReport(int numReport,
-      ContainerStat stat, String datanodeUuid) {
-    StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto.Builder
+  private ContainerReportsProto createContainerReport(int numReport,
+      ContainerStat stat) {
+    StorageContainerDatanodeProtocolProtos.ContainerReportsProto.Builder
         reportsBuilder = StorageContainerDatanodeProtocolProtos
-        .ContainerReportsRequestProto.newBuilder();
+        .ContainerReportsProto.newBuilder();
 
     for (int i = 0; i < numReport; i++) {
       ContainerReport report = new ContainerReport(
@@ -234,24 +238,6 @@ public class TestSCMMetrics {
       report.setWriteBytes(stat.getWriteBytes().get());
       reportsBuilder.addReports(report.getProtoBufMessage());
     }
-
-    DatanodeDetails datanodeDetails;
-    if (datanodeUuid == null) {
-      datanodeDetails = TestUtils.getDatanodeDetails();
-    } else {
-      datanodeDetails = DatanodeDetails.newBuilder()
-          .setUuid(datanodeUuid)
-          .setIpAddress("127.0.0.1")
-          .setHostName("localhost")
-          .setContainerPort(0)
-          .setRatisPort(0)
-          .setOzoneRestPort(0)
-          .build();
-    }
-
-    reportsBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage());
-    reportsBuilder.setType(StorageContainerDatanodeProtocolProtos
-        .ContainerReportsRequestProto.reportType.fullReport);
     return reportsBuilder.build();
   }
 }


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


[23/50] [abbrv] hadoop git commit: YARN-4781. Support intra-queue preemption for fairness ordering policy. Contributed by Eric Payne.

Posted by ha...@apache.org.
YARN-4781. Support intra-queue preemption for fairness ordering policy. Contributed by Eric Payne.


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

Branch: refs/heads/HDDS-48
Commit: 7c343669baf660df3b70d58987d6e68aec54d6fa
Parents: 61df174
Author: Sunil G <su...@apache.org>
Authored: Mon May 28 16:32:53 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Mon May 28 16:32:53 2018 +0530

----------------------------------------------------------------------
 .../FifoIntraQueuePreemptionPlugin.java         |  37 ++-
 .../capacity/IntraQueueCandidatesSelector.java  |  40 +++
 .../monitor/capacity/TempAppPerPartition.java   |   9 +
 .../AbstractComparatorOrderingPolicy.java       |   2 -
 ...alCapacityPreemptionPolicyMockFramework.java |  12 +-
 ...yPreemptionPolicyIntraQueueFairOrdering.java | 276 +++++++++++++++++++
 6 files changed, 366 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c343669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoIntraQueuePreemptionPlugin.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/monitor/capacity/FifoIntraQueuePreemptionPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoIntraQueuePreemptionPlugin.java
index 40f333f..12c178c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoIntraQueuePreemptionPlugin.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoIntraQueuePreemptionPlugin.java
@@ -34,6 +34,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.IntraQueueCandidatesSelector.TAFairOrderingComparator;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.IntraQueueCandidatesSelector.TAPriorityComparator;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.IntraQueuePreemptionOrderPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -41,6 +42,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FairOrderingPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
@@ -263,8 +266,17 @@ public class FifoIntraQueuePreemptionPlugin
       Resource queueReassignableResource,
       PriorityQueue<TempAppPerPartition> orderedByPriority) {
 
-    Comparator<TempAppPerPartition> reverseComp = Collections
-        .reverseOrder(new TAPriorityComparator());
+    Comparator<TempAppPerPartition> reverseComp;
+    OrderingPolicy<FiCaSchedulerApp> queueOrderingPolicy =
+        tq.leafQueue.getOrderingPolicy();
+    if (queueOrderingPolicy instanceof FairOrderingPolicy
+        && (context.getIntraQueuePreemptionOrderPolicy()
+            == IntraQueuePreemptionOrderPolicy.USERLIMIT_FIRST)) {
+      reverseComp = Collections.reverseOrder(
+          new TAFairOrderingComparator(this.rc, clusterResource));
+    } else {
+      reverseComp = Collections.reverseOrder(new TAPriorityComparator());
+    }
     TreeSet<TempAppPerPartition> orderedApps = new TreeSet<>(reverseComp);
 
     String partition = tq.partition;
@@ -355,7 +367,16 @@ public class FifoIntraQueuePreemptionPlugin
       TempQueuePerPartition tq, Collection<FiCaSchedulerApp> apps,
       Resource clusterResource,
       Map<String, Resource> perUserAMUsed) {
-    TAPriorityComparator taComparator = new TAPriorityComparator();
+    Comparator<TempAppPerPartition> taComparator;
+    OrderingPolicy<FiCaSchedulerApp> orderingPolicy =
+        tq.leafQueue.getOrderingPolicy();
+    if (orderingPolicy instanceof FairOrderingPolicy
+        && (context.getIntraQueuePreemptionOrderPolicy()
+            == IntraQueuePreemptionOrderPolicy.USERLIMIT_FIRST)) {
+      taComparator = new TAFairOrderingComparator(this.rc, clusterResource);
+    } else {
+       taComparator = new TAPriorityComparator();
+    }
     PriorityQueue<TempAppPerPartition> orderedByPriority = new PriorityQueue<>(
         100, taComparator);
 
@@ -393,13 +414,12 @@ public class FifoIntraQueuePreemptionPlugin
       // Set ideal allocation of app as 0.
       tmpApp.idealAssigned = Resources.createResource(0, 0);
 
-      orderedByPriority.add(tmpApp);
-
       // Create a TempUserPerPartition structure to hold more information
       // regarding each user's entities such as UserLimit etc. This could
       // be kept in a user to TempUserPerPartition map for further reference.
       String userName = app.getUser();
-      if (!usersPerPartition.containsKey(userName)) {
+      TempUserPerPartition tmpUser = usersPerPartition.get(userName);
+      if (tmpUser == null) {
         ResourceUsage userResourceUsage = tq.leafQueue.getUser(userName)
             .getResourceUsage();
 
@@ -409,7 +429,7 @@ public class FifoIntraQueuePreemptionPlugin
         amUsed = (userSpecificAmUsed == null)
             ? Resources.none() : userSpecificAmUsed;
 
-        TempUserPerPartition tmpUser = new TempUserPerPartition(
+        tmpUser = new TempUserPerPartition(
             tq.leafQueue.getUser(userName), tq.queueName,
             Resources.clone(userResourceUsage.getUsed(partition)),
             Resources.clone(amUsed),
@@ -432,7 +452,10 @@ public class FifoIntraQueuePreemptionPlugin
         tmpUser.idealAssigned = Resources.createResource(0, 0);
         tq.addUserPerPartition(userName, tmpUser);
       }
+      tmpApp.setTempUserPerPartition(tmpUser);
+      orderedByPriority.add(tmpApp);
     }
+
     return orderedByPriority;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c343669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/IntraQueueCandidatesSelector.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/monitor/capacity/IntraQueueCandidatesSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/IntraQueueCandidatesSelector.java
index a91fac7..8ab9507 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/IntraQueueCandidatesSelector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/IntraQueueCandidatesSelector.java
@@ -28,6 +28,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsMana
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.AbstractComparatorOrderingPolicy;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import java.io.Serializable;
@@ -64,6 +66,44 @@ public class IntraQueueCandidatesSelector extends PreemptionCandidatesSelector {
     }
   }
 
+  /*
+   * Order first by amount used from least to most. Then order from oldest to
+   * youngest if amount used is the same.
+   */
+  static class TAFairOrderingComparator
+      implements Comparator<TempAppPerPartition> {
+
+    private ResourceCalculator rc;
+    private Resource clusterRes;
+
+    TAFairOrderingComparator(ResourceCalculator rc, Resource clusterRes) {
+      this.rc = rc;
+      this.clusterRes = clusterRes;
+    }
+
+    @Override
+    public int compare(TempAppPerPartition ta1, TempAppPerPartition ta2) {
+      if (ta1.getUser().equals(ta2.getUser())) {
+        AbstractComparatorOrderingPolicy<FiCaSchedulerApp> acop =
+            (AbstractComparatorOrderingPolicy<FiCaSchedulerApp>)
+            ta1.getFiCaSchedulerApp().getCSLeafQueue().getOrderingPolicy();
+        return acop.getComparator()
+                  .compare(ta1.getFiCaSchedulerApp(), ta2.getFiCaSchedulerApp());
+      } else {
+        Resource usedByUser1 = ta1.getTempUserPerPartition().getUsedDeductAM();
+        Resource usedByUser2 = ta2.getTempUserPerPartition().getUsedDeductAM();
+        if (Resources.equals(usedByUser1, usedByUser2)) {
+          return ta1.getApplicationId().compareTo(ta2.getApplicationId());
+        }
+        if (Resources.lessThan(rc, clusterRes, usedByUser1, usedByUser2)) {
+          return -1;
+        } else {
+          return 1;
+        }
+      }
+    }
+  }
+
   IntraQueuePreemptionComputePlugin fifoPreemptionComputePlugin = null;
   final CapacitySchedulerPreemptionContext context;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c343669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempAppPerPartition.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/monitor/capacity/TempAppPerPartition.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempAppPerPartition.java
index e9a934b..05d8096 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempAppPerPartition.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempAppPerPartition.java
@@ -34,6 +34,7 @@ public class TempAppPerPartition extends AbstractPreemptionEntity {
   // Following fields are settled and used by candidate selection policies
   private final int priority;
   private final ApplicationId applicationId;
+  private TempUserPerPartition tempUser;
 
   FiCaSchedulerApp app;
 
@@ -102,4 +103,12 @@ public class TempAppPerPartition extends AbstractPreemptionEntity {
       Resources.subtractFrom(getActuallyToBePreempted(), toBeDeduct);
     }
   }
+
+  public void setTempUserPerPartition(TempUserPerPartition tu) {
+    tempUser = tu;
+  }
+
+  public TempUserPerPartition getTempUserPerPartition() {
+    return tempUser;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c343669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.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/policy/AbstractComparatorOrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.java
index b7cb1bf..09dd3bf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/AbstractComparatorOrderingPolicy.java
@@ -26,7 +26,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.*;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
-import com.google.common.annotations.VisibleForTesting;
 
 
 /**
@@ -89,7 +88,6 @@ public abstract class AbstractComparatorOrderingPolicy<S extends SchedulableEnti
     }
   }
 
-  @VisibleForTesting
   public Comparator<SchedulableEntity> getComparator() {
     return comparator; 
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c343669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.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/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java
index a972584..64b56fb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicyMockFramework.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preempti
 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.ContainerPreemptEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FairOrderingPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
@@ -64,6 +65,7 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.mockito.ArgumentMatcher;
+import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -337,9 +339,11 @@ public class ProportionalCapacityPreemptionPolicyMockFramework {
           .thenReturn(pendingForDefaultPartition);
 
       // need to set pending resource in resource usage as well
-      ResourceUsage ru = new ResourceUsage();
+      ResourceUsage ru = Mockito.spy(new ResourceUsage());
       ru.setUsed(label, used);
+      when(ru.getCachedUsed(anyString())).thenReturn(used);
       when(app.getAppAttemptResourceUsage()).thenReturn(ru);
+      when(app.getSchedulingResourceUsage()).thenReturn(ru);
 
       start = end + 1;
     }
@@ -637,6 +641,12 @@ public class ProportionalCapacityPreemptionPolicyMockFramework {
         when(leafQueue.getApplications()).thenReturn(apps);
         when(leafQueue.getAllApplications()).thenReturn(apps);
         OrderingPolicy<FiCaSchedulerApp> so = mock(OrderingPolicy.class);
+        String opName = conf.get(CapacitySchedulerConfiguration.PREFIX
+            + CapacitySchedulerConfiguration.ROOT + "." + getQueueName(q)
+            + ".ordering-policy", "fifo");
+        if (opName.equals("fair")) {
+          so = Mockito.spy(new FairOrderingPolicy<FiCaSchedulerApp>());
+        }
         when(so.getPreemptionIterator()).thenAnswer(new Answer() {
           public Object answer(InvocationOnMock invocation) {
             return apps.descendingIterator();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c343669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueFairOrdering.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/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueFairOrdering.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueFairOrdering.java
new file mode 100644
index 0000000..1678651
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueFairOrdering.java
@@ -0,0 +1,276 @@
+/**
+ * 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.server.resourcemanager.monitor.capacity;
+
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.junit.Before;
+import org.junit.Test;
+
+/*
+ * Test class for testing intra-queue preemption when the fair ordering policy
+ * is enabled on a capacity queue.
+ */
+public class TestProportionalCapacityPreemptionPolicyIntraQueueFairOrdering
+    extends ProportionalCapacityPreemptionPolicyMockFramework {
+  @Before
+  public void setup() {
+    super.setup();
+    conf.setBoolean(
+        CapacitySchedulerConfiguration.INTRAQUEUE_PREEMPTION_ENABLED, true);
+    policy = new ProportionalCapacityPreemptionPolicy(rmContext, cs, mClock);
+  }
+
+  /*
+   * When the capacity scheduler fair ordering policy is enabled, preempt first
+   * from the application owned by the user that is the farthest over their
+   * user limit.
+   */
+  @Test
+  public void testIntraQueuePreemptionFairOrderingPolicyEnabledOneAppPerUser()
+      throws IOException {
+    // Enable FairOrderingPolicy for yarn.scheduler.capacity.root.a
+    conf.set(CapacitySchedulerConfiguration.PREFIX
+        + CapacitySchedulerConfiguration.ROOT + ".a.ordering-policy", "fair");
+    // Make sure all containers will be preempted in a single round.
+    conf.setFloat(CapacitySchedulerConfiguration.
+        INTRAQUEUE_PREEMPTION_MAX_ALLOWABLE_LIMIT,
+        (float) 1.0);
+
+    String labelsConfig = "=100,true;";
+    String nodesConfig = // n1 has no label
+        "n1= res=100";
+    String queuesConfig =
+        // guaranteed,max,used,pending,reserved
+        "root(=[100 100 100 1 0]);" + // root
+            "-a(=[100 100 100 1 0])"; // a
+
+    // user1/app1 has 60 resources in queue a
+    // user2/app2 has 40 resources in queue a
+    // user3/app3 is requesting 20 resources in queue a
+    // With 3 users, preemptable user limit should be around 35 resources each.
+    // With FairOrderingPolicy enabled on queue a, all 20 resources should be
+    // preempted from app1
+    String appsConfig =
+    // queueName\t(prio,resource,host,expression,#repeat,reserved,pending,user)
+        "a\t" // app1, user1 in a
+            + "(1,1,n1,,60,false,0,user1);" +
+            "a\t" // app2, user2 in a
+            + "(1,1,n1,,40,false,0,user2);" +
+            "a\t" // app3, user3 in a
+            + "(1,1,n1,,0,false,20,user3)"
+            ;
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    verify(mDisp, times(20)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(1))));
+  }
+
+  /*
+   * When the capacity scheduler fifo ordering policy is enabled, preempt first
+   * from the youngest application until reduced to user limit, then preempt
+   * from next youngest app.
+   */
+  @Test
+  public void testIntraQueuePreemptionFifoOrderingPolicyEnabled()
+      throws IOException {
+    // Enable FifoOrderingPolicy for yarn.scheduler.capacity.root.a
+    conf.set(CapacitySchedulerConfiguration.PREFIX
+        + CapacitySchedulerConfiguration.ROOT + ".a.ordering-policy", "fifo");
+    // Make sure all containers will be preempted in a single round.
+    conf.setFloat(CapacitySchedulerConfiguration.
+        INTRAQUEUE_PREEMPTION_MAX_ALLOWABLE_LIMIT,
+        (float) 1.0);
+
+    String labelsConfig = "=100,true;";
+    String nodesConfig = // n1 has no label
+        "n1= res=100";
+    String queuesConfig =
+        // guaranteed,max,used,pending,reserved
+        "root(=[100 100 100 1 0]);" + // root
+            "-a(=[100 100 100 1 0])"; // a
+
+    // user1/app1 has 60 resources in queue a
+    // user2/app2 has 40 resources in queue a
+    // user3/app3 is requesting 20 resources in queue a
+    // With 3 users, preemptable user limit should be around 35 resources each.
+    // With FifoOrderingPolicy enabled on queue a, the first 5 should come from
+    // the youngest app, app2, until app2 is reduced to the user limit of 35.
+    String appsConfig =
+    // queueName\t(prio,resource,host,expression,#repeat,reserved,pending,user)
+        "a\t" // app1, user1 in a
+            + "(1,1,n1,,60,false,0,user1);" +
+            "a\t" // app2, user2 in a
+            + "(1,1,n1,,40,false,0,user2);" +
+            "a\t" // app3, user3 in a
+            + "(1,1,n1,,0,false,5,user3)"
+            ;
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    verify(mDisp, times(5)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(2))));
+
+    // user1/app1 has 60 resources in queue a
+    // user2/app2 has 35 resources in queue a
+    // user3/app3 has 5 resources and is requesting 15 resources in queue a
+    // With 3 users, preemptable user limit should be around 35 resources each.
+    // The next 15 should come from app1 even though app2 is younger since app2
+    // has already been reduced to its user limit.
+    appsConfig =
+    // queueName\t(prio,resource,host,expression,#repeat,reserved,pending,user)
+        "a\t" // app1, user1 in a
+            + "(1,1,n1,,60,false,0,user1);" +
+            "a\t" // app2, user2 in a
+            + "(1,1,n1,,35,false,0,user2);" +
+            "a\t" // app3, user3 in a
+            + "(1,1,n1,,5,false,15,user3)"
+            ;
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    verify(mDisp, times(15)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(1))));
+  }
+
+  /*
+   * When the capacity scheduler fair ordering policy is enabled, preempt first
+   * from the youngest application from the user that is the farthest over their
+   * user limit.
+   */
+  @Test
+  public void testIntraQueuePreemptionFairOrderingPolicyMulitipleAppsPerUser()
+      throws IOException {
+    // Enable FairOrderingPolicy for yarn.scheduler.capacity.root.a
+    conf.set(CapacitySchedulerConfiguration.PREFIX
+        + CapacitySchedulerConfiguration.ROOT + ".a.ordering-policy", "fair");
+    // Make sure all containers will be preempted in a single round.
+    conf.setFloat(CapacitySchedulerConfiguration.
+        INTRAQUEUE_PREEMPTION_MAX_ALLOWABLE_LIMIT,
+        (float) 1.0);
+
+    String labelsConfig = "=100,true;";
+    String nodesConfig = // n1 has no label
+        "n1= res=100";
+    String queuesConfig =
+        // guaranteed,max,used,pending,reserved
+        "root(=[100 100 100 1 0]);" + // root
+            "-a(=[100 100 100 1 0])"; // a
+
+    // user1/app1 has 35 resources in queue a
+    // user1/app2 has 25 resources in queue a
+    // user2/app3 has 40 resources in queue a
+    // user3/app4 is requesting 20 resources in queue a
+    // With 3 users, preemptable user limit should be around 35 resources each.
+    // With FairOrderingPolicy enabled on queue a, all 20 resources should be
+    // preempted from app1 since it's the most over served app from the most
+    // over served user
+    String appsConfig =
+    // queueName\t(prio,resource,host,expression,#repeat,reserved,pending,user)
+        "a\t" // app1 and app2, user1 in a
+            + "(1,1,n1,,35,false,0,user1);" +
+            "a\t"
+            + "(1,1,n1,,25,false,0,user1);" +
+            "a\t" // app3, user2 in a
+            + "(1,1,n1,,40,false,0,user2);" +
+            "a\t" // app4, user3 in a
+            + "(1,1,n1,,0,false,20,user3)"
+            ;
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    verify(mDisp, times(20)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(1))));
+  }
+
+  /*
+   * When the capacity scheduler fifo ordering policy is enabled and a user has
+   * multiple apps, preempt first from the youngest application.
+   */
+  @Test
+  public void testIntraQueuePreemptionFifoOrderingPolicyMultipleAppsPerUser()
+      throws IOException {
+    // Enable FifoOrderingPolicy for yarn.scheduler.capacity.root.a
+    conf.set(CapacitySchedulerConfiguration.PREFIX
+        + CapacitySchedulerConfiguration.ROOT + ".a.ordering-policy", "fifo");
+    // Make sure all containers will be preempted in a single round.
+    conf.setFloat(CapacitySchedulerConfiguration.
+        INTRAQUEUE_PREEMPTION_MAX_ALLOWABLE_LIMIT,
+        (float) 1.0);
+
+    String labelsConfig = "=100,true;";
+    String nodesConfig = // n1 has no label
+        "n1= res=100";
+    String queuesConfig =
+        // guaranteed,max,used,pending,reserved
+        "root(=[100 100 100 1 0]);" + // root
+            "-a(=[100 100 100 1 0])"; // a
+
+    // user1/app1 has 40 resources in queue a
+    // user1/app2 has 20 resources in queue a
+    // user3/app3 has 40 resources in queue a
+    // user4/app4 is requesting 20 resources in queue a
+    // With 3 users, preemptable user limit should be around 35 resources each.
+    String appsConfig =
+    // queueName\t(prio,resource,host,expression,#repeat,reserved,pending,user)
+        "a\t" // app1, user1 in a
+            + "(1,1,n1,,40,false,0,user1);" +
+        "a\t" // app2, user1 in a
+            + "(1,1,n1,,20,false,0,user1);" +
+        "a\t" // app3, user3 in a
+            + "(1,1,n1,,40,false,0,user3);" +
+        "a\t" // app4, user4 in a
+            + "(1,1,n1,,0,false,25,user4)"
+            ;
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    policy.editSchedule();
+
+    // app3 is the younges and also over its user limit. 5 should be preempted
+    // from app3 until it comes down to user3's user limit.
+    verify(mDisp, times(5)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(3))));
+
+    // User1's app2 is its youngest. 19 should be preempted from app2, leaving
+    // only the AM
+    verify(mDisp, times(19)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(2))));
+
+    // Preempt the remaining resource from User1's oldest app1.
+    verify(mDisp, times(1)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(1))));
+  }
+}


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


[25/50] [abbrv] hadoop git commit: HDFS-13591. TestDFSShell#testSetrepLow fails on Windows. Contributed by Anbang Hu.

Posted by ha...@apache.org.
HDFS-13591. TestDFSShell#testSetrepLow fails on Windows. Contributed by Anbang Hu.


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

Branch: refs/heads/HDDS-48
Commit: 9dbf4f01665d5480a70395a24519cbab5d4db0c5
Parents: 91d7c74
Author: Inigo Goiri <in...@apache.org>
Authored: Mon May 28 16:34:02 2018 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Mon May 28 16:34:02 2018 -0700

----------------------------------------------------------------------
 .../test/java/org/apache/hadoop/hdfs/TestDFSShell.java    | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9dbf4f01/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 e82863a..c352dc9 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
@@ -2829,11 +2829,11 @@ public class TestDFSShell {
         System.setErr(origErr);
       }
 
-      assertEquals("Error message is not the expected error message",
-          "setrep: Requested replication factor of 1 is less than "
-              + "the required minimum of 2 for /tmp/TestDFSShell-"
-              + "testSetrepLow/testFileForSetrepLow\n",
-          bao.toString());
+      assertTrue("Error message is not the expected error message"
+          + bao.toString(), bao.toString().startsWith(
+              "setrep: Requested replication factor of 1 is less than "
+                  + "the required minimum of 2 for /tmp/TestDFSShell-"
+                  + "testSetrepLow/testFileForSetrepLow"));
     } finally {
       shell.close();
       cluster.shutdown();


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


[11/50] [abbrv] hadoop git commit: HADOOP-15473. Configure serialFilter in KeyProvider to avoid UnrecoverableKeyException caused by JDK-8189997. Contributed by Gabor Bota.

Posted by ha...@apache.org.
HADOOP-15473. Configure serialFilter in KeyProvider to avoid UnrecoverableKeyException caused by JDK-8189997. 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/02322de3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/02322de3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/02322de3

Branch: refs/heads/HDDS-48
Commit: 02322de3f95ba78a22c057037ef61aa3ab1d3824
Parents: 8d5509c
Author: Xiao Chen <xi...@apache.org>
Authored: Fri May 25 09:08:15 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Fri May 25 09:10:51 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/crypto/key/KeyProvider.java   | 18 +++++++++++++++
 .../fs/CommonConfigurationKeysPublic.java       |  7 ++++++
 .../src/main/resources/core-default.xml         | 23 ++++++++++++++++++++
 3 files changed, 48 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/02322de3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
index 5d670e5..050540b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
@@ -42,6 +42,8 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 
 import javax.crypto.KeyGenerator;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_JCEKS_KEY_SERIALFILTER;
+
 /**
  * A provider of secret key material for Hadoop applications. Provides an
  * abstraction to separate key storage from users of encryption. It
@@ -61,6 +63,14 @@ public abstract class KeyProvider {
       CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_DEFAULT_BITLENGTH_KEY;
   public static final int DEFAULT_BITLENGTH = CommonConfigurationKeysPublic.
       HADOOP_SECURITY_KEY_DEFAULT_BITLENGTH_DEFAULT;
+  public static final String JCEKS_KEY_SERIALFILTER_DEFAULT =
+      "java.lang.Enum;"
+          + "java.security.KeyRep;"
+          + "java.security.KeyRep$Type;"
+          + "javax.crypto.spec.SecretKeySpec;"
+          + "org.apache.hadoop.crypto.key.JavaKeyStoreProvider$KeyMetadata;"
+          + "!*";
+  public static final String JCEKS_KEY_SERIAL_FILTER = "jceks.key.serialFilter";
 
   private final Configuration conf;
 
@@ -394,6 +404,14 @@ public abstract class KeyProvider {
    */
   public KeyProvider(Configuration conf) {
     this.conf = new Configuration(conf);
+    // Added for HADOOP-15473. Configured serialFilter property fixes
+    // java.security.UnrecoverableKeyException in JDK 8u171.
+    if(System.getProperty(JCEKS_KEY_SERIAL_FILTER) == null) {
+      String serialFilter =
+          conf.get(HADOOP_SECURITY_CRYPTO_JCEKS_KEY_SERIALFILTER,
+              JCEKS_KEY_SERIALFILTER_DEFAULT);
+      System.setProperty(JCEKS_KEY_SERIAL_FILTER, serialFilter);
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02322de3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index 8837cfb..9e0ba20 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -662,6 +662,13 @@ public class CommonConfigurationKeysPublic {
    * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
    * core-default.xml</a>
    */
+  public static final String HADOOP_SECURITY_CRYPTO_JCEKS_KEY_SERIALFILTER =
+      "hadoop.security.crypto.jceks.key.serialfilter";
+  /**
+   * @see
+   * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
+   * core-default.xml</a>
+   */
   public static final String HADOOP_SECURITY_CRYPTO_BUFFER_SIZE_KEY = 
     "hadoop.security.crypto.buffer.size";
   /** Defalt value for HADOOP_SECURITY_CRYPTO_BUFFER_SIZE_KEY */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02322de3/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index fad2985..9564587 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -2487,6 +2487,29 @@
 </property>
 
 <property>
+  <name>hadoop.security.crypto.jceks.key.serialfilter</name>
+  <description>
+    Enhanced KeyStore Mechanisms in JDK 8u171 introduced jceks.key.serialFilter.
+    If jceks.key.serialFilter is configured, the JCEKS KeyStore uses it during
+    the deserialization of the encrypted Key object stored inside a
+    SecretKeyEntry.
+    If jceks.key.serialFilter is not configured it will cause an error when
+    recovering keystore file in KeyProviderFactory when recovering key from
+    keystore file using JDK 8u171 or newer. The filter pattern uses the same
+    format as jdk.serialFilter.
+
+    The value of this property will be used as the following:
+    1. The value of jceks.key.serialFilter system property takes precedence
+    over the value of this property.
+    2. In the absence of jceks.key.serialFilter system property the value of
+    this property will be set as the value of jceks.key.serialFilter.
+    3. If the value of this property and jceks.key.serialFilter system
+    property has not been set, org.apache.hadoop.crypto.key.KeyProvider
+    sets a default value for jceks.key.serialFilter.
+  </description>
+</property>
+
+<property>
   <name>hadoop.security.crypto.buffer.size</name>
   <value>8192</value>
   <description>


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


[41/50] [abbrv] hadoop git commit: HADOOP-15480 AbstractS3GuardToolTestBase.testDiffCommand fails when using dynamo (Gabor Bota)

Posted by ha...@apache.org.
HADOOP-15480 AbstractS3GuardToolTestBase.testDiffCommand fails when using dynamo (Gabor Bota)


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

Branch: refs/heads/HDDS-48
Commit: 5f6769f7964ff002b6c04a95893b5baeb424b6db
Parents: 135941e
Author: Aaron Fabbri <fa...@apache.org>
Authored: Tue May 29 19:20:22 2018 -0700
Committer: Aaron Fabbri <fa...@apache.org>
Committed: Tue May 29 19:20:22 2018 -0700

----------------------------------------------------------------------
 .../s3guard/AbstractS3GuardToolTestBase.java    | 37 +++++++++++++-------
 .../s3a/s3guard/ITestS3GuardToolDynamoDB.java   |  5 ---
 .../fs/s3a/s3guard/ITestS3GuardToolLocal.java   |  5 ---
 3 files changed, 25 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f6769f7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
index 2b43810..7d75f52 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
@@ -25,6 +25,7 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.PrintStream;
+import java.net.URI;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Set;
@@ -32,6 +33,8 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.util.StopWatch;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.FileSystem;
 import org.junit.Assume;
 import org.junit.Test;
 
@@ -48,6 +51,8 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.StringUtils;
 
+import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_METASTORE_NULL;
+import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
 import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.E_BAD_STATE;
 import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -65,6 +70,7 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
   private static final int PRUNE_MAX_AGE_SECS = 2;
 
   private MetadataStore ms;
+  private S3AFileSystem rawFs;
 
   protected static void expectResult(int expected,
       String message,
@@ -129,28 +135,34 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
     return ms;
   }
 
-  protected abstract MetadataStore newMetadataStore();
-
   @Override
   public void setup() throws Exception {
     super.setup();
     S3ATestUtils.assumeS3GuardState(true, getConfiguration());
-    ms = newMetadataStore();
-    ms.initialize(getFileSystem());
+    ms = getFileSystem().getMetadataStore();
+
+    // Also create a "raw" fs without any MetadataStore configured
+    Configuration conf = new Configuration(getConfiguration());
+    conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL);
+    URI fsUri = getFileSystem().getUri();
+    rawFs = (S3AFileSystem) FileSystem.newInstance(fsUri, conf);
   }
 
   @Override
   public void teardown() throws Exception {
     super.teardown();
     IOUtils.cleanupWithLogger(LOG, ms);
+    IOUtils.closeStream(rawFs);
   }
 
   protected void mkdirs(Path path, boolean onS3, boolean onMetadataStore)
       throws IOException {
+    Preconditions.checkArgument(onS3 || onMetadataStore);
+    // getFileSystem() returns an fs with MetadataStore configured
+    S3AFileSystem fs = onMetadataStore ? getFileSystem() : rawFs;
     if (onS3) {
-      getFileSystem().mkdirs(path);
-    }
-    if (onMetadataStore) {
+      fs.mkdirs(path);
+    } else if (onMetadataStore) {
       S3AFileStatus status = new S3AFileStatus(true, path, OWNER);
       ms.put(new PathMetadata(status));
     }
@@ -178,13 +190,14 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
    */
   protected void createFile(Path path, boolean onS3, boolean onMetadataStore)
       throws IOException {
+    Preconditions.checkArgument(onS3 || onMetadataStore);
+    // getFileSystem() returns an fs with MetadataStore configured
+    S3AFileSystem fs = onMetadataStore ? getFileSystem() : rawFs;
     if (onS3) {
-      ContractTestUtils.touch(getFileSystem(), path);
-    }
-
-    if (onMetadataStore) {
+      ContractTestUtils.touch(fs, path);
+    } else if (onMetadataStore) {
       S3AFileStatus status = new S3AFileStatus(100L, System.currentTimeMillis(),
-          getFileSystem().qualify(path), 512L, "hdfs");
+          fs.qualify(path), 512L, "hdfs");
       putFile(ms, status);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f6769f7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
index 821bba5..1a59bf1 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
@@ -48,11 +48,6 @@ import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.*;
 public class ITestS3GuardToolDynamoDB extends AbstractS3GuardToolTestBase {
 
   @Override
-  protected MetadataStore newMetadataStore() {
-    return new DynamoDBMetadataStore();
-  }
-
-  @Override
   public void setup() throws Exception {
     super.setup();
     Assume.assumeTrue("Test only applies when DynamoDB is used for S3Guard",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f6769f7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
index a40c7a5..f5c4b03 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
@@ -52,11 +52,6 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
   private static final String[] ABORT_FORCE_OPTIONS = new String[] {"-abort",
       "-force", "-verbose"};
 
-  @Override
-  protected MetadataStore newMetadataStore() {
-    return new LocalMetadataStore();
-  }
-
   @Test
   public void testImportCommand() throws Exception {
     S3AFileSystem fs = getFileSystem();


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


[47/50] [abbrv] hadoop git commit: YARN-8368. yarn app start cli should print applicationId. Contributed by Rohith Sharma K S

Posted by ha...@apache.org.
YARN-8368. yarn app start cli should print applicationId. 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/96eefcc8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/96eefcc8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/96eefcc8

Branch: refs/heads/HDDS-48
Commit: 96eefcc84aacc4cc82ad7e3e72c5bdad56f4a7b7
Parents: 47c31ff
Author: Billie Rinaldi <bi...@apache.org>
Authored: Wed May 30 12:37:01 2018 -0700
Committer: Billie Rinaldi <bi...@apache.org>
Committed: Wed May 30 12:37:43 2018 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/service/webapp/ApiServer.java   | 28 +++++++++++---------
 .../hadoop/yarn/service/ServiceClientTest.java  | 18 ++++++++++++-
 .../yarn/service/client/ServiceClient.java      |  2 ++
 3 files changed, 35 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/96eefcc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
index 46c9abe..578273c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
@@ -641,20 +641,24 @@ public class ApiServer {
   private Response startService(String appName,
       final UserGroupInformation ugi) throws IOException,
       InterruptedException {
-    ugi.doAs(new PrivilegedExceptionAction<Void>() {
-      @Override
-      public Void run() throws YarnException, IOException {
-        ServiceClient sc = getServiceClient();
-        sc.init(YARN_CONFIG);
-        sc.start();
-        sc.actionStart(appName);
-        sc.close();
-        return null;
-      }
-    });
+    ApplicationId appId =
+        ugi.doAs(new PrivilegedExceptionAction<ApplicationId>() {
+          @Override public ApplicationId run()
+              throws YarnException, IOException {
+            ServiceClient sc = getServiceClient();
+            sc.init(YARN_CONFIG);
+            sc.start();
+            sc.actionStart(appName);
+            ApplicationId appId = sc.getAppId(appName);
+            sc.close();
+            return appId;
+          }
+        });
     LOG.info("Successfully started service " + appName);
     ServiceStatus status = new ServiceStatus();
-    status.setDiagnostics("Service " + appName + " is successfully started.");
+    status.setDiagnostics(
+        "Service " + appName + " is successfully started with ApplicationId: "
+            + appId);
     status.setState(ServiceState.ACCEPTED);
     return formatResponse(Status.OK, status);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96eefcc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/ServiceClientTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/ServiceClientTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/ServiceClientTest.java
index 75b9486..81be750 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/ServiceClientTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/ServiceClientTest.java
@@ -34,8 +34,10 @@ import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
 
@@ -50,6 +52,8 @@ public class ServiceClientTest extends ServiceClient {
   private Service goodServiceStatus = buildLiveGoodService();
   private boolean initialized;
   private Set<String> expectedInstances = new HashSet<>();
+  private Map<String, ApplicationId> serviceAppId = new HashMap<>();
+
 
   public ServiceClientTest() {
     super();
@@ -83,7 +87,10 @@ public class ServiceClientTest extends ServiceClient {
   public ApplicationId actionCreate(Service service) throws IOException {
     ServiceApiUtil.validateAndResolveService(service,
         new SliderFileSystem(conf), getConfig());
-    return ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    serviceAppId.put(service.getName(), appId);
+    return appId;
   }
 
   @Override
@@ -99,6 +106,9 @@ public class ServiceClientTest extends ServiceClient {
   public int actionStart(String serviceName)
       throws YarnException, IOException {
     if (serviceName != null && serviceName.equals("jenkins")) {
+      ApplicationId appId =
+          ApplicationId.newInstance(System.currentTimeMillis(), 1);
+      serviceAppId.put(serviceName, appId);
       return EXIT_SUCCESS;
     } else {
       throw new ApplicationNotFoundException("");
@@ -207,4 +217,10 @@ public class ServiceClientTest extends ServiceClient {
     comp.setContainers(containers);
     return service;
   }
+
+  @Override
+  public synchronized ApplicationId getAppId(String serviceName)
+      throws IOException, YarnException {
+    return serviceAppId.get(serviceName);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96eefcc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.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/client/ServiceClient.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/client/ServiceClient.java
index 0ab3322..e86ecbc 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/client/ServiceClient.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/client/ServiceClient.java
@@ -978,6 +978,8 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
       // see if it is actually running and bail out;
       verifyNoLiveAppInRM(serviceName, "start");
       ApplicationId appId = submitApp(service);
+      cachedAppInfo.put(serviceName, new AppInfo(appId, service
+          .getKerberosPrincipal().getPrincipalName()));
       service.setId(appId.toString());
       // write app definition on to hdfs
       Path appJson = ServiceApiUtil.writeAppDefinition(fs, appDir, service);


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


[26/50] [abbrv] hadoop git commit: HADOOP-15498. TestHadoopArchiveLogs (#testGenerateScript, #testPrepareWorkingDir) fails on Windows. Contributed by Anbang Hu.

Posted by ha...@apache.org.
HADOOP-15498. TestHadoopArchiveLogs (#testGenerateScript, #testPrepareWorkingDir) fails on Windows. Contributed by Anbang Hu.


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

Branch: refs/heads/HDDS-48
Commit: 8fdc993a993728c65084d7dc3ac469059cb1f603
Parents: 9dbf4f0
Author: Inigo Goiri <in...@apache.org>
Authored: Mon May 28 16:45:42 2018 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Mon May 28 16:45:42 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/tools/TestHadoopArchiveLogs.java  | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fdc993a/hadoop-tools/hadoop-archive-logs/src/test/java/org/apache/hadoop/tools/TestHadoopArchiveLogs.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-archive-logs/src/test/java/org/apache/hadoop/tools/TestHadoopArchiveLogs.java b/hadoop-tools/hadoop-archive-logs/src/test/java/org/apache/hadoop/tools/TestHadoopArchiveLogs.java
index 2ddd4c5..a1b662c 100644
--- a/hadoop-tools/hadoop-archive-logs/src/test/java/org/apache/hadoop/tools/TestHadoopArchiveLogs.java
+++ b/hadoop-tools/hadoop-archive-logs/src/test/java/org/apache/hadoop/tools/TestHadoopArchiveLogs.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
@@ -278,7 +279,7 @@ public class TestHadoopArchiveLogs {
     hal.generateScript(localScript);
     Assert.assertTrue(localScript.exists());
     String script = IOUtils.toString(localScript.toURI());
-    String[] lines = script.split(System.lineSeparator());
+    String[] lines = script.split("\n");
     Assert.assertEquals(22, lines.length);
     Assert.assertEquals("#!/bin/bash", lines[0]);
     Assert.assertEquals("set -e", lines[1]);
@@ -368,7 +369,8 @@ public class TestHadoopArchiveLogs {
     Assert.assertTrue(dirPrepared);
     Assert.assertTrue(fs.exists(workingDir));
     Assert.assertEquals(
-        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL, true),
+        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL,
+            !Shell.WINDOWS),
         fs.getFileStatus(workingDir).getPermission());
     // Throw a file in the dir
     Path dummyFile = new Path(workingDir, "dummy.txt");
@@ -381,7 +383,8 @@ public class TestHadoopArchiveLogs {
     Assert.assertTrue(fs.exists(workingDir));
     Assert.assertTrue(fs.exists(dummyFile));
     Assert.assertEquals(
-        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL, true),
+        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL,
+            !Shell.WINDOWS),
         fs.getFileStatus(workingDir).getPermission());
     // -force is true and the dir exists, so it will recreate it and the dummy
     // won't exist anymore
@@ -390,7 +393,8 @@ public class TestHadoopArchiveLogs {
     Assert.assertTrue(dirPrepared);
     Assert.assertTrue(fs.exists(workingDir));
     Assert.assertEquals(
-        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL, true),
+        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL,
+            !Shell.WINDOWS),
         fs.getFileStatus(workingDir).getPermission());
     Assert.assertFalse(fs.exists(dummyFile));
   }


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


[05/50] [abbrv] hadoop git commit: HDDS-80. Remove SendContainerCommand from SCM. Contributed by Nanda Kumar.

Posted by ha...@apache.org.
HDDS-80. Remove SendContainerCommand from SCM. 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/2d19e7d0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2d19e7d0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2d19e7d0

Branch: refs/heads/HDDS-48
Commit: 2d19e7d08f031341078a36fee74860c58de02993
Parents: c9b63de
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Thu May 24 11:10:30 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Thu May 24 11:10:30 2018 -0700

----------------------------------------------------------------------
 .../statemachine/DatanodeStateMachine.java      |   3 -
 .../commandhandler/ContainerReportHandler.java  | 114 -------------------
 .../states/endpoint/HeartbeatEndpointTask.java  |   5 -
 .../protocol/commands/SendContainerCommand.java |  80 -------------
 .../StorageContainerDatanodeProtocol.proto      |  16 ++-
 .../container/replication/InProgressPool.java   |  57 ----------
 .../scm/server/SCMDatanodeProtocolServer.java   |   7 --
 7 files changed, 7 insertions(+), 275 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d19e7d0/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 a16bfdc..a8fe494 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
@@ -26,8 +26,6 @@ import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
 import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
     .CommandDispatcher;
 import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
-    .ContainerReportHandler;
-import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
     .DeleteBlocksCommandHandler;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
@@ -88,7 +86,6 @@ public class DatanodeStateMachine implements Closeable {
      // When we add new handlers just adding a new handler here should do the
      // trick.
     commandDispatcher = CommandDispatcher.newBuilder()
-        .addHandler(new ContainerReportHandler())
         .addHandler(new CloseContainerHandler())
         .addHandler(new DeleteBlocksCommandHandler(
             container.getContainerManager(), conf))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d19e7d0/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ContainerReportHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ContainerReportHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ContainerReportHandler.java
deleted file mode 100644
index fbea290..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ContainerReportHandler.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/**
- * 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.statemachine.commandhandler;
-
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCmdType;
-import org.apache.hadoop.ozone.container.common.statemachine
-    .EndpointStateMachine;
-import org.apache.hadoop.ozone.container.common.statemachine
-    .SCMConnectionManager;
-import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
-import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
-import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
-import org.apache.hadoop.util.Time;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/**
- * Container Report handler.
- */
-public class ContainerReportHandler implements CommandHandler {
-  static final Logger LOG =
-      LoggerFactory.getLogger(ContainerReportHandler.class);
-  private int invocationCount;
-  private long totalTime;
-
-  /**
-   * Constructs a ContainerReport handler.
-   */
-  public ContainerReportHandler() {
-  }
-
-  /**
-   * Handles a given SCM command.
-   *
-   * @param command - SCM Command
-   * @param container - Ozone Container.
-   * @param context - Current Context.
-   * @param connectionManager - The SCMs that we are talking to.
-   */
-  @Override
-  public void handle(SCMCommand command, OzoneContainer container,
-      StateContext context, SCMConnectionManager connectionManager) {
-    LOG.debug("Processing Container Report.");
-    invocationCount++;
-    long startTime = Time.monotonicNow();
-    try {
-      ContainerReportsRequestProto containerReport =
-          container.getContainerReport();
-
-      // TODO : We send this report to all SCMs.Check if it is enough only to
-      // send to the leader once we have RAFT enabled SCMs.
-      for (EndpointStateMachine endPoint : connectionManager.getValues()) {
-        endPoint.getEndPoint().sendContainerReport(containerReport);
-      }
-    } catch (IOException ex) {
-      LOG.error("Unable to process the Container Report command.", ex);
-    } finally {
-      long endTime = Time.monotonicNow();
-      totalTime += endTime - startTime;
-    }
-  }
-
-  /**
-   * Returns the command type that this command handler handles.
-   *
-   * @return Type
-   */
-  @Override
-  public SCMCmdType getCommandType() {
-    return SCMCmdType.sendContainerReport;
-  }
-
-  /**
-   * Returns number of times this handler has been invoked.
-   *
-   * @return int
-   */
-  @Override
-  public int getInvocationCount() {
-    return invocationCount;
-  }
-
-  /**
-   * Returns the average time this function takes to run.
-   *
-   * @return long
-   */
-  @Override
-  public long getAverageRunTime() {
-    if (invocationCount > 0) {
-      return totalTime / invocationCount;
-    }
-    return 0;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d19e7d0/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
index 2f1db39..01b4c72 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.ozone.container.common.statemachine
 import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
 import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
 import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand;
-import org.apache.hadoop.ozone.protocol.commands.SendContainerCommand;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -133,10 +132,6 @@ public class HeartbeatEndpointTask
           .equalsIgnoreCase(datanodeDetails.getUuid()),
           "Unexpected datanode ID in the response.");
       switch (commandResponseProto.getCmdType()) {
-      case sendContainerReport:
-        this.context.addCommand(SendContainerCommand.getFromProtobuf(
-            commandResponseProto.getSendReport()));
-        break;
       case reregisterCommand:
         if (rpcEndpoint.getState() == EndPointStates.HEARTBEAT) {
           if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d19e7d0/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/SendContainerCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/SendContainerCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/SendContainerCommand.java
deleted file mode 100644
index 8431752..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/SendContainerCommand.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/**
- * 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.protocol.commands;
-
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCmdType;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SendContainerReportProto;
-
-/**
- * Allows a Datanode to send in the container report.
- */
-public class SendContainerCommand extends SCMCommand<SendContainerReportProto> {
-  /**
-   * Returns a NullCommand class from NullCommandResponse Proto.
-   * @param unused  - unused
-   * @return NullCommand
-   */
-  public static SendContainerCommand getFromProtobuf(
-      final SendContainerReportProto unused) {
-    return new SendContainerCommand();
-  }
-
-  /**
-   * returns a new builder.
-   * @return Builder
-   */
-  public static SendContainerCommand.Builder newBuilder() {
-    return new SendContainerCommand.Builder();
-  }
-
-  /**
-   * Returns the type of this command.
-   *
-   * @return Type
-   */
-  @Override
-  public SCMCmdType getType() {
-    return SCMCmdType.sendContainerReport;
-  }
-
-  /**
-   * Gets the protobuf message of this object.
-   *
-   * @return A protobuf message.
-   */
-  @Override
-  public byte[] getProtoBufMessage() {
-    return SendContainerReportProto.newBuilder().build().toByteArray();
-  }
-
-  /**
-   * A Builder class this is the standard pattern we are using for all commands.
-   */
-  public static class Builder {
-    /**
-     * Return a null command.
-     * @return - NullCommand.
-     */
-    public SendContainerCommand build() {
-      return new SendContainerCommand();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d19e7d0/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 91070b3..20e6af8 100644
--- a/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
+++ b/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
@@ -186,10 +186,9 @@ Type of commands supported by SCM to datanode protocol.
 enum SCMCmdType {
   versionCommand = 2;
   registeredCommand = 3;
-  sendContainerReport = 4;
-  reregisterCommand = 5;
-  deleteBlocksCommand = 6;
-  closeContainerCommand = 7;
+  reregisterCommand = 4;
+  deleteBlocksCommand = 5;
+  closeContainerCommand = 6;
 }
 
 /*
@@ -199,11 +198,10 @@ message SCMCommandResponseProto {
   required SCMCmdType cmdType = 2; // Type of the command
   optional SCMRegisteredCmdResponseProto registeredProto = 3;
   optional SCMVersionResponseProto versionProto = 4;
-  optional SendContainerReportProto sendReport = 5;
-  optional SCMReregisterCmdResponseProto reregisterProto = 6;
-  optional SCMDeleteBlocksCmdResponseProto deleteBlocksProto = 7;
-  required string datanodeUUID = 8;
-  optional SCMCloseContainerCmdResponseProto closeContainerProto = 9;
+  optional SCMReregisterCmdResponseProto reregisterProto = 5;
+  optional SCMDeleteBlocksCmdResponseProto deleteBlocksProto = 6;
+  required string datanodeUUID = 7;
+  optional SCMCloseContainerCmdResponseProto closeContainerProto = 8;
 }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d19e7d0/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java
index af878bf..c444e90 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java
@@ -21,12 +21,10 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.hdds.scm.node.NodePoolManager;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerInfo;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
-import org.apache.hadoop.ozone.protocol.commands.SendContainerCommand;
 import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -36,19 +34,10 @@ import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
-import static com.google.common.util.concurrent.Uninterruptibles
-    .sleepUninterruptibly;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
-    .HEALTHY;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
-    .INVALID;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE;
-
 /**
  * These are pools that are actively checking for replication status of the
  * containers.
@@ -177,57 +166,11 @@ public final class InProgressPool {
     nodeProcessed = new AtomicInteger(0);
     containerProcessedCount = new AtomicInteger(0);
     nodeCount = new AtomicInteger(0);
-    /*
-       Ask each datanode to send us commands.
-     */
-    SendContainerCommand cmd = SendContainerCommand.newBuilder().build();
-    for (DatanodeDetails dd : datanodeDetailsList) {
-      NodeState currentState = getNodestate(dd);
-      if (currentState == HEALTHY || currentState == STALE) {
-        nodeCount.incrementAndGet();
-        // Queue commands to all datanodes in this pool to send us container
-        // report. Since we ignore dead nodes, it is possible that we would have
-        // over replicated the container if the node comes back.
-        nodeManager.addDatanodeCommand(dd.getUuid(), cmd);
-      }
-    }
     this.status = ProgressStatus.InProgress;
     this.getPool().setLastProcessedTime(Time.monotonicNow());
   }
 
   /**
-   * Gets the node state.
-   *
-   * @param datanode - datanode information.
-   * @return NodeState.
-   */
-  private NodeState getNodestate(DatanodeDetails datanode) {
-    NodeState  currentState = INVALID;
-    int maxTry = 100;
-    // We need to loop to make sure that we will retry if we get
-    // node state unknown. This can lead to infinite loop if we send
-    // in unknown node ID. So max try count is used to prevent it.
-
-    int currentTry = 0;
-    while (currentState == INVALID && currentTry < maxTry) {
-      // Retry to make sure that we deal with the case of node state not
-      // known.
-      currentState = nodeManager.getNodeState(datanode);
-      currentTry++;
-      if (currentState == INVALID) {
-        // Sleep to make sure that this is not a tight loop.
-        sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
-      }
-    }
-    if (currentState == INVALID) {
-      LOG.error("Not able to determine the state of Node: {}, Exceeded max " +
-          "try and node manager returns INVALID state. This indicates we " +
-          "are dealing with a node that we don't know about.", datanode);
-    }
-    return currentState;
-  }
-
-  /**
    * Queues a container Report for handling. This is done in a worker thread
    * since decoding a container report might be compute intensive . We don't
    * want to block since we have asked for bunch of container reports

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d19e7d0/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 58b8c82..6e5b7de 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
@@ -31,7 +31,6 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolPro
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SendContainerReportProto;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMReregisterCmdResponseProto;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandResponseProto;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType;
@@ -46,7 +45,6 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolPro
 
 import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.versionCommand;
 import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.registeredCommand;
-import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.sendContainerReport;
 import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.reregisterCommand;
 import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.deleteBlocksCommand;
 import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.closeContainerCommand;
@@ -318,11 +316,6 @@ public class SCMDatanodeProtocolServer implements
           .setCmdType(versionCommand)
           .setVersionProto(SCMVersionResponseProto.getDefaultInstance())
           .build();
-    case sendContainerReport:
-      return builder
-          .setCmdType(sendContainerReport)
-          .setSendReport(SendContainerReportProto.getDefaultInstance())
-          .build();
     case reregisterCommand:
       return builder
           .setCmdType(reregisterCommand)


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


[12/50] [abbrv] hadoop git commit: HDFS-13618. Fix TestDataNodeFaultInjector test failures on Windows. Contributed by Xiao Liang.

Posted by ha...@apache.org.
HDFS-13618. Fix TestDataNodeFaultInjector test failures on Windows. Contributed by Xiao Liang.


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

Branch: refs/heads/HDDS-48
Commit: 1e0d4b1c283fb98a95c60a1723f594befb3c18a9
Parents: 02322de
Author: Inigo Goiri <in...@apache.org>
Authored: Fri May 25 09:10:32 2018 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Fri May 25 09:14:28 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/datanode/TestDataNodeFaultInjector.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e0d4b1c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFaultInjector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFaultInjector.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFaultInjector.java
index 1507844..4afacd9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFaultInjector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFaultInjector.java
@@ -118,7 +118,7 @@ public class TestDataNodeFaultInjector {
       final MetricsDataNodeFaultInjector mdnFaultInjector) throws Exception {
 
     final Path baseDir = new Path(
-        PathUtils.getTestDir(getClass()).getAbsolutePath(),
+        PathUtils.getTestDir(getClass()).getPath(),
         GenericTestUtils.getMethodName());
     final DataNodeFaultInjector oldDnInjector = DataNodeFaultInjector.get();
     DataNodeFaultInjector.set(mdnFaultInjector);


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


[28/50] [abbrv] hadoop git commit: YARN-8338. TimelineService V1.5 doesn't come up after HADOOP-15406. Contributed by Vinod Kumar Vavilapalli

Posted by ha...@apache.org.
YARN-8338. TimelineService V1.5 doesn't come up after HADOOP-15406. Contributed by Vinod Kumar Vavilapalli


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

Branch: refs/heads/HDDS-48
Commit: 31ab960f4f931df273481927b897388895d803ba
Parents: 438ef49
Author: Jason Lowe <jl...@apache.org>
Authored: Tue May 29 11:00:30 2018 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Tue May 29 11:00:30 2018 -0500

----------------------------------------------------------------------
 hadoop-project/pom.xml                                          | 5 +++++
 .../hadoop-yarn-server-applicationhistoryservice/pom.xml        | 5 +++++
 2 files changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ab960f/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 73c3f5b..59a9bd2 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -1144,6 +1144,11 @@
         <version>1.8.5</version>
       </dependency>
       <dependency>
+        <groupId>org.objenesis</groupId>
+        <artifactId>objenesis</artifactId>
+        <version>1.0</version>
+      </dependency>
+      <dependency>
         <groupId>org.mock-server</groupId>
         <artifactId>mockserver-netty</artifactId>
         <version>3.9.2</version>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ab960f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
index f310518..0527095 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
@@ -155,6 +155,11 @@
       <artifactId>leveldbjni-all</artifactId>
     </dependency>
 
+    <dependency>
+      <groupId>org.objenesis</groupId>
+      <artifactId>objenesis</artifactId>
+    </dependency>
+
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <dependency>
       <groupId>org.apache.hadoop</groupId>


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


[34/50] [abbrv] hadoop git commit: YARN-8329. Docker client configuration can still be set incorrectly. Contributed by Shane Kumpf

Posted by ha...@apache.org.
YARN-8329. Docker client configuration can still be set incorrectly. Contributed by Shane Kumpf


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

Branch: refs/heads/HDDS-48
Commit: 4827e9a9085b306bc379cb6e0b1fe4b92326edcd
Parents: e3236a9
Author: Jason Lowe <jl...@apache.org>
Authored: Tue May 29 14:43:17 2018 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Tue May 29 14:43:17 2018 -0500

----------------------------------------------------------------------
 .../yarn/util/DockerClientConfigHandler.java    | 23 +++++++++++---------
 .../security/TestDockerClientConfigHandler.java |  4 ++--
 .../runtime/DockerLinuxContainerRuntime.java    |  7 +++---
 3 files changed, 19 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4827e9a9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/DockerClientConfigHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/DockerClientConfigHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/DockerClientConfigHandler.java
index 5522cf4..8ec4deb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/DockerClientConfigHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/DockerClientConfigHandler.java
@@ -154,14 +154,15 @@ public final class DockerClientConfigHandler {
    * @param outConfigFile the File to write the Docker client configuration to.
    * @param credentials the populated Credentials object.
    * @throws IOException if the write fails.
+   * @return true if a Docker credential is found in the supplied credentials.
    */
-  public static void writeDockerCredentialsToPath(File outConfigFile,
+  public static boolean writeDockerCredentialsToPath(File outConfigFile,
       Credentials credentials) throws IOException {
-    ObjectMapper mapper = new ObjectMapper();
-    ObjectNode rootNode = mapper.createObjectNode();
-    ObjectNode registryUrlNode = mapper.createObjectNode();
     boolean foundDockerCred = false;
     if (credentials.numberOfTokens() > 0) {
+      ObjectMapper mapper = new ObjectMapper();
+      ObjectNode rootNode = mapper.createObjectNode();
+      ObjectNode registryUrlNode = mapper.createObjectNode();
       for (Token<? extends TokenIdentifier> tk : credentials.getAllTokens()) {
         if (tk.getKind().equals(DockerCredentialTokenIdentifier.KIND)) {
           foundDockerCred = true;
@@ -176,12 +177,14 @@ public final class DockerClientConfigHandler {
           }
         }
       }
+      if (foundDockerCred) {
+        rootNode.put(CONFIG_AUTHS_KEY, registryUrlNode);
+        String json = mapper.writerWithDefaultPrettyPrinter()
+            .writeValueAsString(rootNode);
+        FileUtils.writeStringToFile(
+            outConfigFile, json, StandardCharsets.UTF_8);
+      }
     }
-    if (foundDockerCred) {
-      rootNode.put(CONFIG_AUTHS_KEY, registryUrlNode);
-      String json =
-          mapper.writerWithDefaultPrettyPrinter().writeValueAsString(rootNode);
-      FileUtils.writeStringToFile(outConfigFile, json, StandardCharsets.UTF_8);
-    }
+    return foundDockerCred;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4827e9a9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestDockerClientConfigHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestDockerClientConfigHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestDockerClientConfigHandler.java
index c4cbe45..cfe5a45 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestDockerClientConfigHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestDockerClientConfigHandler.java
@@ -116,8 +116,8 @@ public class TestDockerClientConfigHandler {
     Credentials credentials =
         DockerClientConfigHandler.readCredentialsFromConfigFile(
             new Path(file.toURI()), conf, APPLICATION_ID);
-    DockerClientConfigHandler.writeDockerCredentialsToPath(outFile,
-        credentials);
+    assertTrue(DockerClientConfigHandler.writeDockerCredentialsToPath(outFile,
+        credentials));
     assertTrue(outFile.exists());
     String fileContents = FileUtils.readFileToString(outFile);
     assertTrue(fileContents.contains("auths"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4827e9a9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.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/linux/runtime/DockerLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
index 5e2233b..fc095d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
@@ -1299,14 +1299,15 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
                   .getParent();
           File dockerConfigPath = new File(nmPrivateDir + "/config.json");
           try {
-            DockerClientConfigHandler
-                .writeDockerCredentialsToPath(dockerConfigPath, credentials);
+            if (DockerClientConfigHandler
+                .writeDockerCredentialsToPath(dockerConfigPath, credentials)) {
+              dockerRunCommand.setClientConfigDir(dockerConfigPath.getParent());
+            }
           } catch (IOException e) {
             throw new ContainerExecutionException(
                 "Unable to write Docker client credentials to "
                     + dockerConfigPath);
           }
-          dockerRunCommand.setClientConfigDir(dockerConfigPath.getParent());
         }
       }
     }


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


[46/50] [abbrv] hadoop git commit: HDFS-13629. Some tests in TestDiskBalancerCommand fail on Windows due to MiniDFSCluster path conflict and improper path usage. Contributed by Anbang Hu.

Posted by ha...@apache.org.
HDFS-13629. Some tests in TestDiskBalancerCommand fail on Windows due to MiniDFSCluster path conflict and improper path usage. Contributed by Anbang Hu.


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

Branch: refs/heads/HDDS-48
Commit: 47c31ff16b452d47afc6ffc1cf936ac2de9b788d
Parents: 8197b9b
Author: Inigo Goiri <in...@apache.org>
Authored: Wed May 30 10:22:04 2018 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed May 30 10:22:04 2018 -0700

----------------------------------------------------------------------
 .../server/diskbalancer/DiskBalancerTestUtil.java    |  5 ++++-
 .../command/TestDiskBalancerCommand.java             | 15 +++++++--------
 2 files changed, 11 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/47c31ff1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
index bd8dbce..fef9c63 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 
 import org.slf4j.Logger;
@@ -46,6 +47,7 @@ import org.slf4j.LoggerFactory;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.Random;
 import java.util.UUID;
@@ -307,7 +309,8 @@ public class DiskBalancerTestUtil {
         "need to specify capacities for two storages.");
 
     // Write a file and restart the cluster
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+    File basedir = new File(GenericTestUtils.getRandomizedTempPath());
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, basedir)
         .numDataNodes(numDatanodes)
         .storageCapacities(storageCapacities)
         .storageTypes(new StorageType[]{StorageType.DISK, StorageType.DISK})

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47c31ff1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
index 8266c1f..dee2a90 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
@@ -615,15 +615,15 @@ public class TestDiskBalancerCommand {
     assertThat(
         outputs.get(3),
         is(allOf(containsString("DISK"),
-            containsString(cluster.getInstanceStorageDir(0, 0)
-                .getAbsolutePath()),
+            containsString(new Path(cluster.getInstanceStorageDir(0, 0)
+                .getAbsolutePath()).toString()),
             containsString("0.00"),
             containsString("1.00"))));
     assertThat(
         outputs.get(4),
         is(allOf(containsString("DISK"),
-            containsString(cluster.getInstanceStorageDir(0, 1)
-                .getAbsolutePath()),
+            containsString(new Path(cluster.getInstanceStorageDir(0, 1)
+                .getAbsolutePath()).toString()),
             containsString("0.00"),
             containsString("1.00"))));
   }
@@ -719,9 +719,7 @@ public class TestDiskBalancerCommand {
   @Test
   public void testPrintFullPathOfPlan()
       throws Exception {
-    final Path parent = new Path(
-        PathUtils.getTestPath(getClass()),
-        GenericTestUtils.getMethodName());
+    String parent = GenericTestUtils.getRandomizedTempPath();
 
     MiniDFSCluster miniCluster = null;
     try {
@@ -815,7 +813,8 @@ public class TestDiskBalancerCommand {
     Configuration conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
     final int numDatanodes = 2;
-    MiniDFSCluster miniDFSCluster = new MiniDFSCluster.Builder(conf)
+    File basedir = new File(GenericTestUtils.getRandomizedTempPath());
+    MiniDFSCluster miniDFSCluster = new MiniDFSCluster.Builder(conf, basedir)
         .numDataNodes(numDatanodes).build();
     try {
       miniDFSCluster.waitActive();


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


[07/50] [abbrv] hadoop git commit: YARN-8357. Fixed NPE when YARN service is saved and not deployed. Contributed by Chandni Singh

Posted by ha...@apache.org.
YARN-8357.  Fixed NPE when YARN service is saved and not deployed.
            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/d9852eb5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d9852eb5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d9852eb5

Branch: refs/heads/HDDS-48
Commit: d9852eb5897a25323ab0302c2c0decb61d310e5e
Parents: 7ff5a40
Author: Eric Yang <ey...@apache.org>
Authored: Thu May 24 16:32:13 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Thu May 24 16:32:13 2018 -0400

----------------------------------------------------------------------
 .../java/org/apache/hadoop/yarn/service/client/ServiceClient.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d9852eb5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.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/client/ServiceClient.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/client/ServiceClient.java
index 93a74e3..0ab3322 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/client/ServiceClient.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/client/ServiceClient.java
@@ -1198,6 +1198,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     ServiceApiUtil.validateNameFormat(serviceName, getConfig());
     Service appSpec = new Service();
     appSpec.setName(serviceName);
+    appSpec.setState(ServiceState.STOPPED);
     ApplicationId currentAppId = getAppId(serviceName);
     if (currentAppId == null) {
       LOG.info("Service {} does not have an application ID", serviceName);


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


[27/50] [abbrv] hadoop git commit: HADOOP-15455. Incorrect debug message in KMSACL#hasAccess. Contributed by Yuen-Kuei Hsueh.

Posted by ha...@apache.org.
HADOOP-15455. Incorrect debug message in KMSACL#hasAccess. Contributed by Yuen-Kuei Hsueh.


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

Branch: refs/heads/HDDS-48
Commit: 438ef4951a38171f193eaf2631da31d0f4bc3c62
Parents: 8fdc993
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Mon May 28 17:32:32 2018 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Mon May 28 17:32:32 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/438ef495/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java
index b02f34e..17faec2 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java
@@ -247,9 +247,9 @@ public class KMSACLs implements Runnable, KeyACLs {
         if (blacklist == null) {
           LOG.debug("No blacklist for {}", type.toString());
         } else if (access) {
-          LOG.debug("user is in {}" , blacklist.getAclString());
-        } else {
           LOG.debug("user is not in {}" , blacklist.getAclString());
+        } else {
+          LOG.debug("user is in {}" , blacklist.getAclString());
         }
       }
     }


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


[24/50] [abbrv] hadoop git commit: HDFS-13627. TestErasureCodingExerciseAPIs fails on Windows. Contributed by Anbang Hu.

Posted by ha...@apache.org.
HDFS-13627. TestErasureCodingExerciseAPIs fails on Windows. Contributed by Anbang Hu.


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

Branch: refs/heads/HDDS-48
Commit: 91d7c74e6aa4850922f68bab490b585443e4fccb
Parents: 7c34366
Author: Inigo Goiri <in...@apache.org>
Authored: Mon May 28 10:26:47 2018 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Mon May 28 10:26:47 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/TestErasureCodingExerciseAPIs.java   | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/91d7c74e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingExerciseAPIs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingExerciseAPIs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingExerciseAPIs.java
index 4335527..c63ba34 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingExerciseAPIs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingExerciseAPIs.java
@@ -40,6 +40,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.DataOutputStream;
+import java.io.File;
 import java.io.IOException;
 import java.nio.file.Paths;
 import java.security.NoSuchAlgorithmException;
@@ -91,8 +92,10 @@ public class TestErasureCodingExerciseAPIs {
     // Set up java key store
     String testRootDir = Paths.get(new FileSystemTestHelper().getTestRootDir())
         .toString();
+    Path targetFile = new Path(new File(testRootDir).getAbsolutePath(),
+        "test.jks");
     String keyProviderURI = JavaKeyStoreProvider.SCHEME_NAME + "://file"
-        + new Path(testRootDir, "test.jks").toUri();
+        + targetFile.toUri();
     conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
         keyProviderURI);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY,


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


[02/50] [abbrv] hadoop git commit: HDDS-45. Removal of old OzoneRestClient. Contributed by Lokesh Jain.

Posted by ha...@apache.org.
HDDS-45. Removal of old OzoneRestClient. 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/774daa8d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/774daa8d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/774daa8d

Branch: refs/heads/HDDS-48
Commit: 774daa8d532f9eeee1fe8e342a8da2cfa65a8629
Parents: c05b5d4
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Thu May 24 15:53:42 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Thu May 24 15:53:42 2018 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hdds/scm/XceiverClient.java   |  22 +-
 .../hadoop/ozone/web/client/OzoneBucket.java    | 646 ---------------
 .../hadoop/ozone/web/client/OzoneKey.java       |  44 -
 .../ozone/web/client/OzoneRestClient.java       | 804 -------------------
 .../hadoop/ozone/web/client/OzoneVolume.java    | 584 --------------
 .../hadoop/ozone/web/client/package-info.java   |  34 -
 .../hadoop/ozone/MiniOzoneClusterImpl.java      |   3 +-
 .../apache/hadoop/ozone/RatisTestHelper.java    |  14 +-
 .../ozone/web/TestOzoneRestWithMiniCluster.java | 207 ++---
 .../hadoop/ozone/web/client/TestBuckets.java    | 193 +++--
 .../ozone/web/client/TestBucketsRatis.java      |  15 +-
 .../hadoop/ozone/web/client/TestKeys.java       | 286 ++++---
 .../hadoop/ozone/web/client/TestKeysRatis.java  |  29 +-
 .../hadoop/ozone/web/client/TestVolume.java     | 285 +++----
 .../ozone/web/client/TestVolumeRatis.java       |  29 +-
 15 files changed, 548 insertions(+), 2647 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java
index 6d33cd4..42e02f9 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java
@@ -54,6 +54,7 @@ public class XceiverClient extends XceiverClientSpi {
   private Bootstrap b;
   private EventLoopGroup group;
   private final Semaphore semaphore;
+  private boolean closed = false;
 
   /**
    * Constructs a client that can communicate with the Container framework on
@@ -74,6 +75,10 @@ public class XceiverClient extends XceiverClientSpi {
 
   @Override
   public void connect() throws Exception {
+    if (closed) {
+      throw new IOException("This channel is not connected.");
+    }
+
     if (channel != null && channel.isActive()) {
       throw new IOException("This client is already connected to a host.");
     }
@@ -97,6 +102,18 @@ public class XceiverClient extends XceiverClientSpi {
     channel = b.connect(leader.getHostName(), port).sync().channel();
   }
 
+  public void reconnect() throws IOException {
+    try {
+      connect();
+      if (channel == null || !channel.isActive()) {
+        throw new IOException("This channel is not connected.");
+      }
+    } catch (Exception e) {
+      LOG.error("Error while connecting: ", e);
+      throw new IOException(e);
+    }
+  }
+
   /**
    * Returns if the exceiver client connects to a server.
    *
@@ -109,6 +126,7 @@ public class XceiverClient extends XceiverClientSpi {
 
   @Override
   public void close() {
+    closed = true;
     if (group != null) {
       group.shutdownGracefully().awaitUninterruptibly();
     }
@@ -124,7 +142,7 @@ public class XceiverClient extends XceiverClientSpi {
       ContainerProtos.ContainerCommandRequestProto request) throws IOException {
     try {
       if ((channel == null) || (!channel.isActive())) {
-        throw new IOException("This channel is not connected.");
+        reconnect();
       }
       XceiverClientHandler handler =
           channel.pipeline().get(XceiverClientHandler.class);
@@ -160,7 +178,7 @@ public class XceiverClient extends XceiverClientSpi {
       sendCommandAsync(ContainerProtos.ContainerCommandRequestProto request)
       throws IOException, ExecutionException, InterruptedException {
     if ((channel == null) || (!channel.isActive())) {
-      throw new IOException("This channel is not connected.");
+      reconnect();
     }
     XceiverClientHandler handler =
         channel.pipeline().get(XceiverClientHandler.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
deleted file mode 100644
index 3183d03..0000000
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneBucket.java
+++ /dev/null
@@ -1,646 +0,0 @@
-/*
- * 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.web.client;
-
-import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
-import org.apache.hadoop.io.IOUtils;
-
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
-import org.apache.hadoop.ozone.client.rest.headers.Header;
-import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.web.response.BucketInfo;
-import org.apache.hadoop.ozone.web.response.KeyInfo;
-import org.apache.hadoop.ozone.web.response.ListKeys;
-
-import static org.apache.hadoop.hdds.server.ServerUtils.releaseConnection;
-import org.apache.http.HttpEntity;
-import org.apache.http.HttpResponse;
-import org.apache.http.client.methods.HttpDelete;
-import org.apache.http.client.methods.HttpGet;
-import org.apache.http.client.methods.HttpPut;
-import org.apache.http.client.utils.URIBuilder;
-import org.apache.http.entity.ContentType;
-import org.apache.http.entity.FileEntity;
-import org.apache.http.entity.InputStreamEntity;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.impl.client.HttpClientBuilder;
-import org.apache.http.util.EntityUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Strings;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.file.Path;
-import java.util.LinkedList;
-import java.util.List;
-
-import static java.net.HttpURLConnection.HTTP_CREATED;
-import static java.net.HttpURLConnection.HTTP_OK;
-import static org.apache.hadoop.ozone.web.utils.OzoneUtils.ENCODING;
-import static org.apache.hadoop.ozone.web.utils.OzoneUtils.ENCODING_NAME;
-
-/**
- * A Bucket class the represents an Ozone Bucket.
- */
-public class OzoneBucket {
-  static final Logger LOG = LoggerFactory.getLogger(OzoneBucket.class);
-
-  private BucketInfo bucketInfo;
-  private OzoneVolume volume;
-
-  /**
-   * Constructor for bucket.
-   *
-   * @param info   - BucketInfo
-   * @param volume - OzoneVolume Object that contains this bucket
-   */
-  public OzoneBucket(BucketInfo info, OzoneVolume volume) {
-    this.bucketInfo = info;
-    this.volume = volume;
-  }
-
-  /**
-   * Gets bucket Info.
-   *
-   * @return BucketInfo
-   */
-  public BucketInfo getBucketInfo() {
-    return bucketInfo;
-  }
-
-  /**
-   * Sets Bucket Info.
-   *
-   * @param bucketInfo BucketInfo
-   */
-  public void setBucketInfo(BucketInfo bucketInfo) {
-    this.bucketInfo = bucketInfo;
-  }
-
-  /**
-   * Returns the parent volume class.
-   *
-   * @return - OzoneVolume
-   */
-  OzoneVolume getVolume() {
-    return volume;
-  }
-
-  /**
-   * Returns bucket name.
-   *
-   * @return Bucket Name
-   */
-  public String getBucketName() {
-    return bucketInfo.getBucketName();
-  }
-
-  /**
-   * Returns the Acls on the bucket.
-   *
-   * @return - Acls
-   */
-  public List<OzoneAcl> getAcls() {
-    return bucketInfo.getAcls();
-  }
-
-  /**
-   * Return versioning info on the bucket - Enabled or disabled.
-   *
-   * @return - Version Enum
-   */
-  public OzoneConsts.Versioning getVersioning() {
-    return bucketInfo.getVersioning();
-  }
-
-  /**
-   * Gets the Storage class for the bucket.
-   *
-   * @return Storage Class Enum
-   */
-  public StorageType getStorageType() {
-    return bucketInfo.getStorageType();
-  }
-
-  /**
-   * Gets the creation time of the bucket.
-   *
-   * @return String
-   */
-  public String getCreatedOn() {
-    return bucketInfo.getCreatedOn();
-  }
-
-  /**
-   * Puts an Object in Ozone bucket.
-   *
-   * @param keyName - Name of the key
-   * @param data    - Data that you want to put
-   * @throws OzoneException
-   */
-  public void putKey(String keyName, String data) throws OzoneException {
-    if ((keyName == null) || keyName.isEmpty()) {
-      throw new OzoneClientException("Invalid key Name.");
-    }
-
-    if (data == null) {
-      throw new OzoneClientException("Invalid data.");
-    }
-
-    HttpPut putRequest = null;
-    InputStream is = null;
-    try (CloseableHttpClient httpClient = HddsClientUtils.newHttpClient()) {
-      URIBuilder builder = new URIBuilder(volume.getClient().getEndPointURI());
-      builder.setPath("/" + getVolume().getVolumeName() + "/" + getBucketName()
-          + "/" + keyName).build();
-
-      putRequest = getVolume().getClient().getHttpPut(builder.toString());
-
-      is = new ByteArrayInputStream(data.getBytes(ENCODING));
-      putRequest.setEntity(new InputStreamEntity(is, data.length()));
-      is.mark(data.length());
-      try {
-        putRequest.setHeader(Header.CONTENT_MD5, DigestUtils.md5Hex(is));
-      } finally {
-        is.reset();
-      }
-      executePutKey(putRequest, httpClient);
-    } catch (IOException | URISyntaxException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      IOUtils.closeStream(is);
-      releaseConnection(putRequest);
-    }
-  }
-
-  /**
-   * Puts an Object in Ozone Bucket.
-   *
-   * @param dataFile - File from which you want the data to be put. Key Name
-   *                 will same as the file name, devoid of any path.
-   * @throws OzoneException
-   */
-  public void putKey(File dataFile) throws OzoneException {
-    if (dataFile == null) {
-      throw new OzoneClientException("Invalid file object.");
-    }
-    String keyName = dataFile.getName();
-    putKey(keyName, dataFile);
-  }
-
-  /**
-   * Puts a Key in Ozone Bucket.
-   *
-   * @param keyName - Name of the Key
-   * @param file    - Stream that gets read to be put into Ozone.
-   * @throws OzoneException
-   */
-  public void putKey(String keyName, File file)
-      throws OzoneException {
-
-    if ((keyName == null) || keyName.isEmpty()) {
-      throw new OzoneClientException("Invalid key Name");
-    }
-
-    if (file == null) {
-      throw new OzoneClientException("Invalid data stream");
-    }
-
-    HttpPut putRequest = null;
-    FileInputStream fis = null;
-    try (CloseableHttpClient httpClient = HddsClientUtils.newHttpClient()) {
-      URIBuilder builder = new URIBuilder(volume.getClient().getEndPointURI());
-      builder.setPath("/" + getVolume().getVolumeName() + "/" + getBucketName()
-          + "/" + keyName).build();
-
-      putRequest = getVolume().getClient().getHttpPut(builder.toString());
-
-      FileEntity fileEntity = new FileEntity(file, ContentType
-          .APPLICATION_OCTET_STREAM);
-      putRequest.setEntity(fileEntity);
-
-      fis = new FileInputStream(file);
-      putRequest.setHeader(Header.CONTENT_MD5, DigestUtils.md5Hex(fis));
-      executePutKey(putRequest, httpClient);
-
-    } catch (IOException | URISyntaxException ex) {
-      final OzoneClientException orce = new OzoneClientException(
-          "Failed to putKey: keyName=" + keyName + ", file=" + file);
-      orce.initCause(ex);
-      LOG.trace("", orce);
-      throw orce;
-    } finally {
-      IOUtils.closeStream(fis);
-      releaseConnection(putRequest);
-    }
-  }
-
-  /**
-   * executePutKey executes the Put request against the Ozone Server.
-   *
-   * @param putRequest - Http Put Request
-   * @param httpClient - httpClient
-   * @throws OzoneException
-   * @throws IOException
-   */
-  public static void executePutKey(HttpPut putRequest,
-      CloseableHttpClient httpClient) throws OzoneException, IOException {
-    HttpEntity entity = null;
-    try {
-      HttpResponse response = httpClient.execute(putRequest);
-      int errorCode = response.getStatusLine().getStatusCode();
-      entity = response.getEntity();
-
-      if ((errorCode == HTTP_OK) || (errorCode == HTTP_CREATED)) {
-        return;
-      }
-
-      if (entity == null) {
-        throw new OzoneClientException("Unexpected null in http payload");
-      }
-
-      throw OzoneException.parse(EntityUtils.toString(entity));
-    } finally {
-      if (entity != null) {
-        EntityUtils.consumeQuietly(entity);
-      }
-    }
-  }
-
-  /**
-   * Gets a key from the Ozone server and writes to the file pointed by the
-   * downloadTo PAth.
-   *
-   * @param keyName    - Key Name in Ozone.
-   * @param downloadTo File Name to download the Key's Data to
-   */
-  public void getKey(String keyName, Path downloadTo) throws OzoneException {
-
-    if ((keyName == null) || keyName.isEmpty()) {
-      throw new OzoneClientException("Invalid key Name");
-    }
-
-    if (downloadTo == null) {
-      throw new OzoneClientException("Invalid download path");
-    }
-
-    FileOutputStream outPutFile = null;
-    HttpGet getRequest = null;
-    try (CloseableHttpClient httpClient = HddsClientUtils.newHttpClient()) {
-      outPutFile = new FileOutputStream(downloadTo.toFile());
-
-      URIBuilder builder = new URIBuilder(volume.getClient().getEndPointURI());
-      builder.setPath("/" + getVolume().getVolumeName() + "/" + getBucketName()
-          + "/" + keyName).build();
-
-      getRequest = getVolume().getClient().getHttpGet(builder.toString());
-      executeGetKey(getRequest, httpClient, outPutFile);
-      outPutFile.flush();
-    } catch (IOException | URISyntaxException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      IOUtils.closeStream(outPutFile);
-      releaseConnection(getRequest);
-    }
-  }
-
-  /**
-   * Returns the data part of the key as a string.
-   *
-   * @param keyName - KeyName to get
-   * @return String - Data
-   * @throws OzoneException
-   */
-  public String getKey(String keyName) throws OzoneException {
-
-    if ((keyName == null) || keyName.isEmpty()) {
-      throw new OzoneClientException("Invalid key Name");
-    }
-
-    HttpGet getRequest = null;
-    ByteArrayOutputStream outPutStream = null;
-    try (CloseableHttpClient httpClient = HddsClientUtils.newHttpClient()) {
-      outPutStream = new ByteArrayOutputStream();
-
-      URIBuilder builder = new URIBuilder(volume.getClient().getEndPointURI());
-
-      builder.setPath("/" + getVolume().getVolumeName() + "/" + getBucketName()
-          + "/" + keyName).build();
-
-      getRequest = getVolume().getClient().getHttpGet(builder.toString());
-      executeGetKey(getRequest, httpClient, outPutStream);
-      return outPutStream.toString(ENCODING_NAME);
-    } catch (IOException | URISyntaxException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      IOUtils.closeStream(outPutStream);
-      releaseConnection(getRequest);
-    }
-
-  }
-
-  /**
-   * Executes get key and returns the data.
-   *
-   * @param getRequest - http Get Request
-   * @param httpClient - Client
-   * @param stream     - Stream to write data to.
-   * @throws IOException
-   * @throws OzoneException
-   */
-  public static void executeGetKey(HttpGet getRequest,
-      CloseableHttpClient httpClient, OutputStream stream)
-      throws IOException, OzoneException {
-
-    HttpEntity entity = null;
-    try {
-
-      HttpResponse response = httpClient.execute(getRequest);
-      int errorCode = response.getStatusLine().getStatusCode();
-      entity = response.getEntity();
-
-      if (errorCode == HTTP_OK) {
-        entity.writeTo(stream);
-        return;
-      }
-
-      if (entity == null) {
-        throw new OzoneClientException("Unexpected null in http payload");
-      }
-
-      throw OzoneException.parse(EntityUtils.toString(entity));
-    } finally {
-      if (entity != null) {
-        EntityUtils.consumeQuietly(entity);
-      }
-    }
-  }
-
-  /**
-   * Deletes a key in this bucket.
-   *
-   * @param keyName - Name of the Key
-   * @throws OzoneException
-   */
-  public void deleteKey(String keyName) throws OzoneException {
-
-    if ((keyName == null) || keyName.isEmpty()) {
-      throw new OzoneClientException("Invalid key Name");
-    }
-
-    HttpDelete deleteRequest = null;
-    try (CloseableHttpClient httpClient = HddsClientUtils.newHttpClient()) {
-      URIBuilder builder = new URIBuilder(volume.getClient().getEndPointURI());
-      builder.setPath("/" + getVolume().getVolumeName() + "/" + getBucketName()
-          + "/" + keyName).build();
-
-      deleteRequest = getVolume()
-          .getClient().getHttpDelete(builder.toString());
-      executeDeleteKey(deleteRequest, httpClient);
-    } catch (IOException | URISyntaxException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      releaseConnection(deleteRequest);
-    }
-  }
-
-  /**
-   * Executes deleteKey.
-   *
-   * @param deleteRequest - http Delete Request
-   * @param httpClient    - Client
-   * @throws IOException
-   * @throws OzoneException
-   */
-  private void executeDeleteKey(HttpDelete deleteRequest,
-      CloseableHttpClient httpClient)
-      throws IOException, OzoneException {
-
-    HttpEntity entity = null;
-    try {
-
-      HttpResponse response = httpClient.execute(deleteRequest);
-      int errorCode = response.getStatusLine().getStatusCode();
-      entity = response.getEntity();
-
-      if (errorCode == HTTP_OK) {
-        return;
-      }
-
-      if (entity == null) {
-        throw new OzoneClientException("Unexpected null in http payload");
-      }
-
-      throw OzoneException.parse(EntityUtils.toString(entity));
-    } finally {
-      if (entity != null) {
-        EntityUtils.consumeQuietly(entity);
-      }
-    }
-  }
-
-  /**
-   * List all keys in a bucket.
-   *
-   * @param resultLength The max length of listing result.
-   * @param previousKey The key from where listing should start,
-   *                    this key is excluded in the result.
-   * @param prefix The prefix that return list keys start with.
-   * @return List of OzoneKeys
-   * @throws OzoneException
-   */
-  public List<OzoneKey> listKeys(String resultLength, String previousKey,
-      String prefix) throws OzoneException {
-    HttpGet getRequest = null;
-    try (CloseableHttpClient httpClient = HddsClientUtils.newHttpClient()) {
-      OzoneRestClient client = getVolume().getClient();
-      URIBuilder builder = new URIBuilder(volume.getClient().getEndPointURI());
-      builder.setPath("/" + getVolume().getVolumeName() + "/" + getBucketName())
-          .build();
-
-      if (!Strings.isNullOrEmpty(resultLength)) {
-        builder.addParameter(Header.OZONE_LIST_QUERY_MAXKEYS, resultLength);
-      }
-
-      if (!Strings.isNullOrEmpty(previousKey)) {
-        builder.addParameter(Header.OZONE_LIST_QUERY_PREVKEY, previousKey);
-      }
-
-      if (!Strings.isNullOrEmpty(prefix)) {
-        builder.addParameter(Header.OZONE_LIST_QUERY_PREFIX, prefix);
-      }
-
-      final String uri = builder.toString();
-      getRequest = client.getHttpGet(uri);
-      LOG.trace("listKeys URI={}", uri);
-      return executeListKeys(getRequest, httpClient);
-
-    } catch (IOException | URISyntaxException e) {
-      throw new OzoneClientException(e.getMessage(), e);
-    } finally {
-      releaseConnection(getRequest);
-    }
-  }
-
-  /**
-   * List keys in a bucket with the provided prefix, with paging results.
-   *
-   * @param prefix The prefix of the object keys
-   * @param maxResult max size per response
-   * @param prevKey the previous key for paging
-   */
-  public List<OzoneKey> listKeys(String prefix, int maxResult, String prevKey)
-      throws OzoneException {
-    HttpGet getRequest = null;
-    try {
-      final URI uri =  new URIBuilder(volume.getClient().getEndPointURI())
-          .setPath(OzoneConsts.KSM_KEY_PREFIX + getVolume().getVolumeName() +
-              OzoneConsts.KSM_KEY_PREFIX + getBucketName())
-          .setParameter(Header.OZONE_LIST_QUERY_PREFIX, prefix)
-          .setParameter(Header.OZONE_LIST_QUERY_MAXKEYS,
-              String.valueOf(maxResult))
-          .setParameter(Header.OZONE_LIST_QUERY_PREVKEY, prevKey)
-          .build();
-      final OzoneRestClient client = getVolume().getClient();
-      getRequest = client.getHttpGet(uri.toString());
-      return executeListKeys(getRequest, HttpClientBuilder.create().build());
-    } catch (IOException | URISyntaxException e) {
-      throw new OzoneClientException(e.getMessage());
-    } finally {
-      releaseConnection(getRequest);
-    }
-  }
-
-  /**
-   * Execute list Key.
-   *
-   * @param getRequest - HttpGet
-   * @param httpClient - HttpClient
-   * @return List<OzoneKey>
-   * @throws IOException
-   * @throws OzoneException
-   */
-  public static List<OzoneKey> executeListKeys(HttpGet getRequest,
-      CloseableHttpClient httpClient) throws IOException, OzoneException {
-    HttpEntity entity = null;
-    List<OzoneKey> ozoneKeyList = new LinkedList<OzoneKey>();
-    try {
-      HttpResponse response = httpClient.execute(getRequest);
-      int errorCode = response.getStatusLine().getStatusCode();
-
-      entity = response.getEntity();
-
-      if (entity == null) {
-        throw new OzoneClientException("Unexpected null in http payload");
-      }
-      if (errorCode == HTTP_OK) {
-        String temp = EntityUtils.toString(entity);
-        ListKeys keyList = ListKeys.parse(temp);
-
-        for (KeyInfo info : keyList.getKeyList()) {
-          ozoneKeyList.add(new OzoneKey(info));
-        }
-        return ozoneKeyList;
-
-      } else {
-        throw OzoneException.parse(EntityUtils.toString(entity));
-      }
-    } finally {
-      if (entity != null) {
-        EntityUtils.consumeQuietly(entity);
-      }
-    }
-  }
-
-  /**
-   * Get info of the specified key.
-   */
-  public OzoneKey getKeyInfo(String keyName) throws OzoneException {
-    if ((keyName == null) || keyName.isEmpty()) {
-      throw new OzoneClientException(
-          "Unable to get key info, key name is null or empty");
-    }
-
-    HttpGet getRequest = null;
-    try (CloseableHttpClient httpClient = HddsClientUtils.newHttpClient()) {
-      OzoneRestClient client = getVolume().getClient();
-      URIBuilder builder = new URIBuilder(volume.getClient().getEndPointURI());
-      builder
-          .setPath("/" + getVolume().getVolumeName() + "/" + getBucketName()
-              + "/" + keyName)
-          .setParameter(Header.OZONE_INFO_QUERY_TAG,
-              Header.OZONE_INFO_QUERY_KEY)
-          .build();
-
-      getRequest = client.getHttpGet(builder.toString());
-      return executeGetKeyInfo(getRequest, httpClient);
-    } catch (IOException | URISyntaxException e) {
-      throw new OzoneClientException(e.getMessage(), e);
-    } finally {
-      releaseConnection(getRequest);
-    }
-  }
-
-  /**
-   * Execute get Key info.
-   *
-   * @param getRequest - HttpGet
-   * @param httpClient - HttpClient
-   * @return List<OzoneKey>
-   * @throws IOException
-   * @throws OzoneException
-   */
-  private OzoneKey executeGetKeyInfo(HttpGet getRequest,
-      CloseableHttpClient httpClient) throws IOException, OzoneException {
-    HttpEntity entity = null;
-    try {
-      HttpResponse response = httpClient.execute(getRequest);
-      int errorCode = response.getStatusLine().getStatusCode();
-      entity = response.getEntity();
-      if (entity == null) {
-        throw new OzoneClientException("Unexpected null in http payload");
-      }
-
-      if (errorCode == HTTP_OK) {
-        OzoneKey key = new OzoneKey(
-            KeyInfo.parse(EntityUtils.toString(entity)));
-        return key;
-      }
-      throw OzoneException.parse(EntityUtils.toString(entity));
-    } finally {
-      if (entity != null) {
-        EntityUtils.consumeQuietly(entity);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneKey.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneKey.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneKey.java
deleted file mode 100644
index 5a3a0c4..0000000
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneKey.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.web.client;
-
-import org.apache.hadoop.ozone.web.response.KeyInfo;
-
-/**
- * Client side representation of an ozone Key.
- */
-public class OzoneKey {
-  private KeyInfo keyInfo;
-
-  /**
-   * Constructor for Ozone Key.
-   * @param keyInfo - Key Info
-   */
-  public OzoneKey(KeyInfo keyInfo) {
-    this.keyInfo = keyInfo;
-  }
-
-  /**
-   * Returns Key Info.
-   * @return Object Info
-   */
-  public KeyInfo getObjectInfo() {
-    return keyInfo;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneRestClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneRestClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneRestClient.java
deleted file mode 100644
index 8373f67..0000000
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneRestClient.java
+++ /dev/null
@@ -1,804 +0,0 @@
-/*
- * 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.web.client;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Strings;
-
-import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
-import org.apache.hadoop.ozone.client.rest.headers.Header;
-import org.apache.hadoop.ozone.web.response.ListVolumes;
-import org.apache.hadoop.ozone.web.response.VolumeInfo;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
-import org.apache.hadoop.util.Time;
-
-import static org.apache.hadoop.hdds.server.ServerUtils.releaseConnection;
-import org.apache.http.HttpEntity;
-import org.apache.http.HttpResponse;
-import org.apache.http.client.methods.HttpDelete;
-import org.apache.http.client.methods.HttpGet;
-import org.apache.http.client.methods.HttpPost;
-import org.apache.http.client.methods.HttpPut;
-import org.apache.http.client.methods.HttpRequestBase;
-import org.apache.http.client.utils.URIBuilder;
-import org.apache.http.entity.ContentType;
-import org.apache.http.entity.FileEntity;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.util.EntityUtils;
-
-import javax.ws.rs.core.HttpHeaders;
-import java.io.Closeable;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.file.Path;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Locale;
-
-import static java.net.HttpURLConnection.HTTP_CREATED;
-import static java.net.HttpURLConnection.HTTP_OK;
-
-/**
- * Ozone client that connects to an Ozone server. Please note that this class is
- * not  thread safe.
- */
-public class OzoneRestClient implements Closeable {
-  private URI endPointURI;
-  private String userAuth;
-
-  /**
-   * Constructor for OzoneRestClient.
-   */
-  public OzoneRestClient() {
-  }
-
-  /**
-   * Constructor for OzoneRestClient.
-   */
-  public OzoneRestClient(String ozoneURI)
-      throws OzoneException, URISyntaxException {
-    setEndPoint(ozoneURI);
-  }
-
-  /**
-   * Constructor for OzoneRestClient.
-   */
-  public OzoneRestClient(String ozoneURI, String userAuth)
-      throws OzoneException, URISyntaxException {
-    setEndPoint(ozoneURI);
-    setUserAuth(userAuth);
-  }
-
-  /**
-   * Returns the end Point.
-   *
-   * @return String
-   */
-  public URI getEndPointURI() {
-    return endPointURI;
-  }
-
-  /**
-   * Sets the End Point info using an URI.
-   *
-   * @param endPointURI - URI
-   * @throws OzoneException
-   */
-  public void setEndPointURI(URI endPointURI) throws OzoneException {
-    if ((endPointURI == null) || (endPointURI.toString().isEmpty())) {
-      throw new OzoneClientException("Invalid ozone URI");
-    }
-    this.endPointURI = endPointURI;
-  }
-
-  /**
-   * Set endPoint.
-   *
-   * @param clusterFQDN - cluster FQDN.
-   */
-  public void setEndPoint(String clusterFQDN) throws
-      OzoneException, URISyntaxException {
-    setEndPointURI(new URI(clusterFQDN));
-  }
-
-  /**
-   * Get user Auth String.
-   *
-   * @return - User Auth String
-   */
-  public String getUserAuth() {
-    return this.userAuth;
-  }
-
-  /**
-   * Set User Auth.
-   *
-   * @param userAuth - User Auth String
-   */
-  public void setUserAuth(String userAuth) {
-    this.userAuth = userAuth;
-  }
-
-  /**
-   * create volume.
-   *
-   * @param volumeName - volume name 3 - 63 chars, small letters.
-   * @param onBehalfOf - The user on behalf we are making the call for
-   * @param quota      - Quota's are specified in a specific format. it is
-   *                   integer(MB|GB|TB), for example 100TB.
-   * @throws OzoneClientException
-   */
-  public OzoneVolume createVolume(String volumeName, String onBehalfOf,
-                                  String quota) throws OzoneException {
-    HttpPost httpPost = null;
-    try (CloseableHttpClient httpClient = newHttpClient()) {
-      OzoneUtils.verifyResourceName(volumeName);
-
-      URIBuilder builder = new URIBuilder(endPointURI);
-      builder.setPath("/" + volumeName);
-      if (quota != null) {
-        builder.setParameter(Header.OZONE_QUOTA_QUERY_TAG, quota);
-      }
-
-      httpPost = getHttpPost(onBehalfOf, builder.build().toString());
-      executeCreateVolume(httpPost, httpClient);
-      return getVolume(volumeName);
-    } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      releaseConnection(httpPost);
-    }
-  }
-
-  /**
-   * Returns information about an existing Volume. if the Volume does not exist,
-   * or if the user does not have access rights OzoneException is thrown
-   *
-   * @param volumeName - volume name 3 - 63 chars, small letters.
-   * @return OzoneVolume Ozone Client Volume Class.
-   * @throws OzoneException
-   */
-  public OzoneVolume getVolume(String volumeName) throws OzoneException {
-    HttpGet httpGet = null;
-    try (CloseableHttpClient httpClient = newHttpClient()) {
-      OzoneUtils.verifyResourceName(volumeName);
-      URIBuilder builder = new URIBuilder(endPointURI);
-      builder.setPath("/" + volumeName)
-          .setParameter(Header.OZONE_INFO_QUERY_TAG,
-              Header.OZONE_INFO_QUERY_VOLUME)
-          .build();
-
-      httpGet = getHttpGet(builder.toString());
-      return executeInfoVolume(httpGet, httpClient);
-    } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      releaseConnection(httpGet);
-    }
-  }
-
-  /**
-   * List all the volumes owned by the user or Owned by the user specified in
-   * the behalf of string.
-   *
-   * @param onBehalfOf
-   *  User Name of the user if it is not the caller. for example,
-   *  an admin wants to list some other users volumes.
-   * @param prefix
-   *   Return only volumes that match this prefix.
-   * @param maxKeys
-   *   Maximum number of results to return, if the result set
-   *   is smaller than requested size, it means that list is
-   *   complete.
-   * @param previousVolume
-   *   The previous volume name.
-   * @return List of Volumes
-   * @throws OzoneException
-   */
-  public List<OzoneVolume> listVolumes(String onBehalfOf, String prefix,
-      int maxKeys, String previousVolume) throws OzoneException {
-    HttpGet httpGet = null;
-    try (CloseableHttpClient httpClient = newHttpClient()) {
-      URIBuilder builder = new URIBuilder(endPointURI);
-      if (!Strings.isNullOrEmpty(prefix)) {
-        builder.addParameter(Header.OZONE_LIST_QUERY_PREFIX, prefix);
-      }
-
-      if (maxKeys > 0) {
-        builder.addParameter(Header.OZONE_LIST_QUERY_MAXKEYS, Integer
-            .toString(maxKeys));
-      }
-
-      if (!Strings.isNullOrEmpty(previousVolume)) {
-        builder.addParameter(Header.OZONE_LIST_QUERY_PREVKEY,
-            previousVolume);
-      }
-
-      builder.setPath("/").build();
-
-      httpGet = getHttpGet(builder.toString());
-      if (onBehalfOf != null) {
-        httpGet.addHeader(Header.OZONE_USER, onBehalfOf);
-      }
-      return executeListVolume(httpGet, httpClient);
-    } catch (IOException | URISyntaxException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      releaseConnection(httpGet);
-    }
-  }
-
-  /**
-   * List all the volumes owned by the user or Owned by the user specified in
-   * the behalf of string.
-   *
-   * @param onBehalfOf - User Name of the user if it is not the caller. for
-   *                   example, an admin wants to list some other users
-   *                   volumes.
-   * @param prefix     - Return only volumes that match this prefix.
-   * @param maxKeys    - Maximum number of results to return, if the result set
-   *                   is smaller than requested size, it means that list is
-   *                   complete.
-   * @param prevKey    - The last key that client got, server will continue
-   *                   returning results from that point.
-   * @return List of Volumes
-   * @throws OzoneException
-   */
-  public List<OzoneVolume> listVolumes(String onBehalfOf, String prefix,
-      int maxKeys, OzoneVolume prevKey) throws OzoneException {
-    String volumeName = null;
-
-    if (prevKey != null) {
-      volumeName = prevKey.getVolumeName();
-    }
-
-    return listVolumes(onBehalfOf, prefix, maxKeys, volumeName);
-  }
-
-  /**
-   * List volumes of the current user or if onBehalfof is not null lists volume
-   * owned by that user. You need admin privilege to read other users volume
-   * lists.
-   *
-   * @param onBehalfOf - Name of the user you want to get volume list
-   * @return - Volume list.
-   * @throws OzoneException
-   */
-  public List<OzoneVolume> listVolumes(String onBehalfOf)
-      throws OzoneException {
-    return listVolumes(onBehalfOf, null,
-        Integer.parseInt(Header.OZONE_DEFAULT_LIST_SIZE), StringUtils.EMPTY);
-  }
-
-  /**
-   * List all volumes in a cluster. This can be invoked only by an Admin.
-   *
-   * @param prefix  - Returns only volumes that match this prefix.
-   * @param maxKeys - Maximum niumber of keys to return
-   * @param prevKey - Last Ozone Volume from the last Iteration.
-   * @return List of Volumes
-   * @throws OzoneException
-   */
-  public List<OzoneVolume> listAllVolumes(String prefix, int maxKeys,
-      OzoneVolume prevKey) throws OzoneException {
-    HttpGet httpGet = null;
-    try (CloseableHttpClient httpClient = newHttpClient()) {
-      URIBuilder builder = new URIBuilder(endPointURI);
-      if (prefix != null) {
-        builder.addParameter(Header.OZONE_LIST_QUERY_PREFIX, prefix);
-      }
-
-      if (maxKeys > 0) {
-        builder.addParameter(Header.OZONE_LIST_QUERY_MAXKEYS, Integer
-            .toString(maxKeys));
-      }
-
-      if (prevKey != null) {
-        builder.addParameter(Header.OZONE_LIST_QUERY_PREVKEY,
-            prevKey.getOwnerName()+ "/" + prevKey.getVolumeName());
-      }
-
-      builder.addParameter(Header.OZONE_LIST_QUERY_ROOTSCAN, "true");
-      builder.setPath("/").build();
-      httpGet = getHttpGet(builder.toString());
-      return executeListVolume(httpGet, httpClient);
-
-    } catch (IOException | URISyntaxException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      releaseConnection(httpGet);
-    }
-  }
-
-    /**
-     * delete a given volume.
-     *
-     * @param volumeName - volume to be deleted.
-     * @throws OzoneException - Ozone Exception
-     */
-  public void deleteVolume(String volumeName) throws OzoneException {
-    HttpDelete httpDelete = null;
-    try (CloseableHttpClient httpClient = newHttpClient()) {
-      OzoneUtils.verifyResourceName(volumeName);
-      URIBuilder builder = new URIBuilder(endPointURI);
-      builder.setPath("/" + volumeName).build();
-
-      httpDelete = getHttpDelete(builder.toString());
-      executeDeleteVolume(httpDelete, httpClient);
-    } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      releaseConnection(httpDelete);
-    }
-  }
-
-  /**
-   * Sets the Volume Owner.
-   *
-   * @param volumeName - Volume Name
-   * @param newOwner   - New Owner Name
-   * @throws OzoneException
-   */
-  public void setVolumeOwner(String volumeName, String newOwner)
-      throws OzoneException {
-    HttpPut putRequest = null;
-    if (newOwner == null || newOwner.isEmpty()) {
-      throw new OzoneClientException("Invalid new owner name");
-    }
-    try (CloseableHttpClient httpClient = newHttpClient()) {
-      OzoneUtils.verifyResourceName(volumeName);
-      URIBuilder builder = new URIBuilder(endPointURI);
-      builder.setPath("/" + volumeName).build();
-
-      putRequest = getHttpPut(builder.toString());
-      putRequest.addHeader(Header.OZONE_USER, newOwner);
-      executePutVolume(putRequest, httpClient);
-
-    } catch (URISyntaxException | IllegalArgumentException | IOException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      releaseConnection(putRequest);
-    }
-  }
-
-  /**
-   * Sets the Volume Quota. Quota's are specified in a specific format. it is
-   * <integer>|(MB|GB|TB. for example 100TB.
-   * <p>
-   * To Remove a quota you can specify Header.OZONE_QUOTA_REMOVE
-   *
-   * @param volumeName - volume name
-   * @param quota      - Quota String or  Header.OZONE_QUOTA_REMOVE
-   * @throws OzoneException
-   */
-  public void setVolumeQuota(String volumeName, String quota)
-      throws OzoneException {
-    if (quota == null || quota.isEmpty()) {
-      throw new OzoneClientException("Invalid quota");
-    }
-    HttpPut putRequest = null;
-    try (CloseableHttpClient httpClient = newHttpClient()) {
-      OzoneUtils.verifyResourceName(volumeName);
-      URIBuilder builder = new URIBuilder(endPointURI);
-      builder.setPath("/" + volumeName)
-          .setParameter(Header.OZONE_QUOTA_QUERY_TAG, quota)
-          .build();
-
-      putRequest = getHttpPut(builder.toString());
-      executePutVolume(putRequest, httpClient);
-
-    } catch (URISyntaxException | IllegalArgumentException | IOException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      releaseConnection(putRequest);
-    }
-  }
-
-  /**
-   * Sends the create Volume request to the server.
-   *
-   * @param httppost   - http post class
-   * @param httpClient - httpClient
-   * @throws IOException    -
-   * @throws OzoneException
-   */
-  private void executeCreateVolume(HttpPost httppost,
-      final CloseableHttpClient httpClient)
-      throws IOException, OzoneException {
-    HttpEntity entity = null;
-    try {
-      HttpResponse response = httpClient.execute(httppost);
-      int errorCode = response.getStatusLine().getStatusCode();
-      entity = response.getEntity();
-
-      if ((errorCode == HTTP_OK) || (errorCode == HTTP_CREATED)) {
-        return;
-      }
-
-      if (entity != null) {
-        throw OzoneException.parse(EntityUtils.toString(entity));
-      } else {
-        throw new OzoneClientException("Unexpected null in http payload");
-      }
-    } finally {
-      if (entity != null) {
-        EntityUtils.consume(entity);
-      }
-    }
-  }
-
-  /**
-   * Sends the create Volume request to the server.
-   *
-   * @param httpGet - httpGet
-   * @return OzoneVolume
-   * @throws IOException    -
-   * @throws OzoneException
-   */
-  private OzoneVolume executeInfoVolume(HttpGet httpGet,
-      final CloseableHttpClient httpClient)
-      throws IOException, OzoneException {
-    HttpEntity entity = null;
-    try {
-      HttpResponse response = httpClient.execute(httpGet);
-      int errorCode = response.getStatusLine().getStatusCode();
-
-      entity = response.getEntity();
-      if (entity == null) {
-        throw new OzoneClientException("Unexpected null in http payload");
-      }
-
-      if (errorCode == HTTP_OK) {
-        OzoneVolume volume = new OzoneVolume(this);
-        volume.setVolumeInfo(EntityUtils.toString(entity));
-        return volume;
-      } else {
-        throw OzoneException.parse(EntityUtils.toString(entity));
-      }
-    } finally {
-      if (entity != null) {
-        EntityUtils.consumeQuietly(entity);
-      }
-    }
-  }
-
-  /**
-   * Sends update volume requests to the server.
-   *
-   * @param putRequest http request
-   * @throws IOException
-   * @throws OzoneException
-   */
-  private void executePutVolume(HttpPut putRequest,
-      final CloseableHttpClient httpClient)
-      throws IOException, OzoneException {
-    HttpEntity entity = null;
-    try {
-      HttpResponse response = httpClient.execute(putRequest);
-      int errorCode = response.getStatusLine().getStatusCode();
-      entity = response.getEntity();
-      if (errorCode != HTTP_OK) {
-        throw OzoneException.parse(EntityUtils.toString(entity));
-      }
-    } finally {
-      if (entity != null) {
-        EntityUtils.consume(entity);
-      }
-    }
-  }
-
-  /**
-   * List Volumes.
-   *
-   * @param httpGet - httpGet
-   * @return OzoneVolume
-   * @throws IOException    -
-   * @throws OzoneException
-   */
-  private List<OzoneVolume> executeListVolume(HttpGet httpGet,
-      final CloseableHttpClient httpClient)
-      throws IOException, OzoneException {
-    HttpEntity entity = null;
-    List<OzoneVolume> volList = new LinkedList<>();
-    try {
-      HttpResponse response = httpClient.execute(httpGet);
-      int errorCode = response.getStatusLine().getStatusCode();
-      entity = response.getEntity();
-
-      if (entity == null) {
-        throw new OzoneClientException("Unexpected null in http payload");
-      }
-
-      String temp = EntityUtils.toString(entity);
-      if (errorCode == HTTP_OK) {
-        ListVolumes listVolumes =
-            ListVolumes.parse(temp);
-
-        for (VolumeInfo info : listVolumes.getVolumes()) {
-          volList.add(new OzoneVolume(info, this));
-        }
-        return volList;
-
-      } else {
-        throw OzoneException.parse(EntityUtils.toString(entity));
-      }
-    } finally {
-      if (entity != null) {
-        EntityUtils.consumeQuietly(entity);
-      }
-    }
-  }
-
-  /**
-   * Delete Volume.
-   *
-   * @param httpDelete - Http Delete Request
-   * @throws IOException
-   * @throws OzoneException
-   */
-  private void executeDeleteVolume(HttpDelete httpDelete,
-      final CloseableHttpClient httpClient)
-      throws IOException, OzoneException {
-    HttpEntity entity = null;
-    try {
-      HttpResponse response = httpClient.execute(httpDelete);
-      int errorCode = response.getStatusLine().getStatusCode();
-      entity = response.getEntity();
-
-      if (errorCode != HTTP_OK) {
-        throw OzoneException.parse(EntityUtils.toString(entity));
-      }
-    } finally {
-      if (entity != null) {
-        EntityUtils.consumeQuietly(entity);
-      }
-    }
-  }
-
-  /**
-   * Puts a Key in Ozone Bucket.
-   *
-   * @param volumeName - Name of the Volume
-   * @param bucketName - Name of the Bucket
-   * @param keyName - Name of the Key
-   * @param file    - Stream that gets read to be put into Ozone.
-   * @throws OzoneException
-   */
-  public void putKey(String volumeName, String bucketName, String keyName,
-      File file) throws OzoneException {
-    OzoneUtils.verifyResourceName(volumeName);
-    OzoneUtils.verifyResourceName(bucketName);
-
-    if (StringUtils.isEmpty(keyName)) {
-      throw new OzoneClientException("Invalid key Name");
-    }
-
-    if (file == null) {
-      throw new OzoneClientException("Invalid data stream");
-    }
-
-    HttpPut putRequest = null;
-    FileInputStream fis = null;
-    try (CloseableHttpClient httpClient = HddsClientUtils.newHttpClient()) {
-      URIBuilder builder = new URIBuilder(getEndPointURI());
-      builder.setPath("/" + volumeName + "/" + bucketName + "/" + keyName)
-          .build();
-
-      putRequest = getHttpPut(builder.toString());
-
-      FileEntity fileEntity = new FileEntity(file, ContentType
-          .APPLICATION_OCTET_STREAM);
-      putRequest.setEntity(fileEntity);
-
-      fis = new FileInputStream(file);
-      putRequest.setHeader(Header.CONTENT_MD5, DigestUtils.md5Hex(fis));
-      OzoneBucket.executePutKey(putRequest, httpClient);
-    } catch (IOException | URISyntaxException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      IOUtils.closeStream(fis);
-      releaseConnection(putRequest);
-    }
-  }
-
-  /**
-   * Gets a key from the Ozone server and writes to the file pointed by the
-   * downloadTo Path.
-   *
-   * @param volumeName - Volume Name in Ozone.
-   * @param bucketName - Bucket Name in Ozone.
-   * @param keyName - Key Name in Ozone.
-   * @param downloadTo File Name to download the Key's Data to
-   */
-  public void getKey(String volumeName, String bucketName, String keyName,
-      Path downloadTo) throws OzoneException {
-    OzoneUtils.verifyResourceName(volumeName);
-    OzoneUtils.verifyResourceName(bucketName);
-
-    if (StringUtils.isEmpty(keyName)) {
-      throw new OzoneClientException("Invalid key Name");
-    }
-
-    if (downloadTo == null) {
-      throw new OzoneClientException("Invalid download path");
-    }
-
-    FileOutputStream outPutFile = null;
-    HttpGet getRequest = null;
-    try (CloseableHttpClient httpClient = HddsClientUtils.newHttpClient()) {
-      outPutFile = new FileOutputStream(downloadTo.toFile());
-
-      URIBuilder builder = new URIBuilder(getEndPointURI());
-      builder.setPath("/" + volumeName + "/" + bucketName + "/" + keyName)
-          .build();
-
-      getRequest = getHttpGet(builder.toString());
-      OzoneBucket.executeGetKey(getRequest, httpClient, outPutFile);
-      outPutFile.flush();
-    } catch (IOException | URISyntaxException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      IOUtils.closeStream(outPutFile);
-      releaseConnection(getRequest);
-    }
-  }
-
-  /**
-   * List all keys in the given bucket.
-   *
-   * @param volumeName - Volume name
-   * @param bucketName - Bucket name
-   * @param resultLength The max length of listing result.
-   * @param previousKey The key from where listing should start,
-   *                    this key is excluded in the result.
-   * @param prefix The prefix that return list keys start with.
-   *
-   * @return List of OzoneKeys
-   */
-  public List<OzoneKey> listKeys(String volumeName, String bucketName,
-      String resultLength, String previousKey, String prefix)
-      throws OzoneException {
-    OzoneUtils.verifyResourceName(volumeName);
-    OzoneUtils.verifyResourceName(bucketName);
-
-    HttpGet getRequest = null;
-    try (CloseableHttpClient httpClient = HddsClientUtils.newHttpClient()) {
-      URIBuilder builder = new URIBuilder(getEndPointURI());
-      builder.setPath("/" + volumeName + "/" + bucketName).build();
-
-      if (!Strings.isNullOrEmpty(resultLength)) {
-        builder.addParameter(Header.OZONE_LIST_QUERY_MAXKEYS, resultLength);
-      }
-
-      if (!Strings.isNullOrEmpty(previousKey)) {
-        builder.addParameter(Header.OZONE_LIST_QUERY_PREVKEY, previousKey);
-      }
-
-      if (!Strings.isNullOrEmpty(prefix)) {
-        builder.addParameter(Header.OZONE_LIST_QUERY_PREFIX, prefix);
-      }
-
-      getRequest = getHttpGet(builder.toString());
-      return OzoneBucket.executeListKeys(getRequest, httpClient);
-    } catch (IOException | URISyntaxException e) {
-      throw new OzoneClientException(e.getMessage(), e);
-    } finally {
-      releaseConnection(getRequest);
-    }
-  }
-
-  /**
-   * Returns a standard HttpPost Object to use for ozone post requests.
-   *
-   * @param onBehalfOf - If the use is being made on behalf of user, that user
-   * @param uriString  - UriString
-   * @return HttpPost
-   */
-  public HttpPost getHttpPost(String onBehalfOf, String uriString) {
-    HttpPost httpPost = new HttpPost(uriString);
-    addOzoneHeaders(httpPost);
-    if (onBehalfOf != null) {
-      httpPost.addHeader(Header.OZONE_USER, onBehalfOf);
-    }
-    return httpPost;
-  }
-
-  /**
-   * Returns a standard HttpGet Object to use for ozone Get requests.
-   *
-   * @param uriString - The full Uri String
-   * @return HttpGet
-   */
-  public HttpGet getHttpGet(String uriString) {
-    HttpGet httpGet = new HttpGet(uriString);
-    addOzoneHeaders(httpGet);
-    return httpGet;
-  }
-
-  /**
-   * Returns httpDelete.
-   *
-   * @param uriString - uri
-   * @return HttpDelete
-   */
-  public HttpDelete getHttpDelete(String uriString) {
-    HttpDelete httpDel = new HttpDelete(uriString);
-    addOzoneHeaders(httpDel);
-    return httpDel;
-  }
-
-  /**
-   * returns an HttpPut Object.
-   *
-   * @param uriString - Uri
-   * @return HttpPut
-   */
-  public HttpPut getHttpPut(String uriString) {
-    HttpPut httpPut = new HttpPut(uriString);
-    addOzoneHeaders(httpPut);
-    return httpPut;
-  }
-
-  /**
-   * Add Ozone Headers.
-   *
-   * @param httpRequest - Http Request
-   */
-  private void addOzoneHeaders(HttpRequestBase httpRequest) {
-    SimpleDateFormat format =
-        new SimpleDateFormat("EEE, dd MMM yyyy HH:mm:ss ZZZ", Locale.US);
-
-    httpRequest.addHeader(Header.OZONE_VERSION_HEADER,
-        Header.OZONE_V1_VERSION_HEADER);
-    httpRequest.addHeader(HttpHeaders.DATE,
-        format.format(new Date(Time.monotonicNow())));
-    if (getUserAuth() != null) {
-      httpRequest.addHeader(HttpHeaders.AUTHORIZATION,
-          Header.OZONE_SIMPLE_AUTHENTICATION_SCHEME + " " +
-              getUserAuth());
-    }
-  }
-
-  /**
-   * Closes this stream and releases any system resources associated with it. If
-   * the stream is already closed then invoking this method has no effect.
-   *
-   * @throws IOException if an I/O error occurs
-   */
-  @Override
-  public void close() throws IOException {
-    // TODO : Currently we create a new HTTP client. We should switch
-    // This to a Pool and cleanup the pool here.
-  }
-
-  @VisibleForTesting
-  public CloseableHttpClient newHttpClient() {
-    return HddsClientUtils.newHttpClient();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneVolume.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneVolume.java
deleted file mode 100644
index 9d3831c..0000000
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/OzoneVolume.java
+++ /dev/null
@@ -1,584 +0,0 @@
-/*
- * 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.web.client;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Strings;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
-import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
-import org.apache.hadoop.ozone.client.rest.headers.Header;
-import org.apache.hadoop.ozone.web.request.OzoneQuota;
-import org.apache.hadoop.ozone.web.response.BucketInfo;
-import org.apache.hadoop.ozone.web.response.ListBuckets;
-import org.apache.hadoop.ozone.web.response.VolumeInfo;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
-
-import static org.apache.hadoop.hdds.server.ServerUtils.releaseConnection;
-import org.apache.http.HttpEntity;
-import org.apache.http.HttpResponse;
-import org.apache.http.client.methods.HttpDelete;
-import org.apache.http.client.methods.HttpGet;
-import org.apache.http.client.methods.HttpPost;
-import org.apache.http.client.methods.HttpPut;
-import org.apache.http.client.utils.URIBuilder;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.util.EntityUtils;
-
-import java.io.IOException;
-import java.net.URISyntaxException;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import static java.net.HttpURLConnection.HTTP_CREATED;
-import static java.net.HttpURLConnection.HTTP_OK;
-
-/**
- * Ozone Volume Class.
- */
-public class OzoneVolume {
-  private VolumeInfo volumeInfo;
-  private Map<String, String> headerMap;
-  private final OzoneRestClient client;
-
-  /**
-   * Constructor for OzoneVolume.
-   */
-  public OzoneVolume(OzoneRestClient client) {
-    this.client = client;
-    this.headerMap = new HashMap<>();
-  }
-
-  /**
-   * Constructor for OzoneVolume.
-   *
-   * @param volInfo - volume Info.
-   * @param client  Client
-   */
-  public OzoneVolume(VolumeInfo volInfo, OzoneRestClient client) {
-    this.volumeInfo = volInfo;
-    this.client = client;
-  }
-
-  /**
-   * Returns a Json String of this class.
-   * @return String
-   * @throws IOException
-   */
-  public String getJsonString() throws IOException {
-    return volumeInfo.toJsonString();
-  }
-
-  /**
-   * sets the Volume Info.
-   *
-   * @param volInfoString - Volume Info String
-   */
-  public void setVolumeInfo(String volInfoString) throws IOException {
-    this.volumeInfo = VolumeInfo.parse(volInfoString);
-  }
-
-  /**
-   * @return the volume info.
-   */
-  public VolumeInfo getVolumeInfo() {
-    return this.volumeInfo;
-  }
-
-  /**
-   * Returns volume Name.
-   *
-   * @return Volume Name.
-   */
-  public String getVolumeName() {
-    return this.volumeInfo.getVolumeName();
-  }
-
-  /**
-   * Get created by.
-   *
-   * @return String
-   */
-  public String getCreatedby() {
-    return this.volumeInfo.getCreatedBy();
-  }
-
-  /**
-   * returns the Owner name.
-   *
-   * @return String
-   */
-  public String getOwnerName() {
-    return this.volumeInfo.getOwner().getName();
-  }
-
-  /**
-   * Returns Quota Info.
-   *
-   * @return Quota
-   */
-  public OzoneQuota getQuota() {
-    return volumeInfo.getQuota();
-  }
-
-  /**
-   * Returns creation time of Volume.
-   *
-   * @return String
-   */
-  public String getCreatedOn() {
-    return volumeInfo.getCreatedOn();
-  }
-
-  /**
-   * Returns a Http header from the Last Volume related call.
-   *
-   * @param headerName - Name of the header
-   * @return - Header Value
-   */
-  public String getHeader(String headerName) {
-    return headerMap.get(headerName);
-  }
-
-  /**
-   * Gets the Client, this is used by Bucket and Key Classes.
-   *
-   * @return - Ozone Client
-   */
-  OzoneRestClient getClient() {
-    return client;
-  }
-
-  /**
-   * Create Bucket - Creates a bucket under a given volume.
-   *
-   * @param bucketName - Bucket Name
-   * @param acls - Acls - User Acls
-   * @param storageType - Storage Class
-   * @param versioning - enable versioning support on a bucket.
-   *
-   *
-   * @return - a Ozone Bucket Object
-   */
-  public OzoneBucket createBucket(String bucketName, String[] acls,
-                                  StorageType storageType,
-                                  OzoneConsts.Versioning versioning)
-      throws OzoneException {
-
-    HttpPost httpPost = null;
-    try (CloseableHttpClient httpClient = newHttpClient()) {
-      OzoneUtils.verifyResourceName(bucketName);
-      URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
-      builder.setPath("/" + getVolumeName() + "/" + bucketName).build();
-
-      httpPost = client.getHttpPost(null, builder.toString());
-      if (acls != null) {
-        for (String acl : acls) {
-          httpPost
-              .addHeader(Header.OZONE_ACLS, Header.OZONE_ACL_ADD + " " + acl);
-        }
-      }
-
-      httpPost.addHeader(Header.OZONE_STORAGE_TYPE, storageType.toString());
-      httpPost.addHeader(Header.OZONE_BUCKET_VERSIONING, versioning.toString());
-      executeCreateBucket(httpPost, httpClient);
-      return getBucket(bucketName);
-    } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      releaseConnection(httpPost);
-    }
-  }
-
-  /**
-   * Create Bucket.
-   *
-   * @param bucketName - bucket name
-   * @param acls - acls
-   * @param storageType - storage class
-   *
-   * @throws OzoneException
-   */
-  public OzoneBucket createBucket(String bucketName, String[] acls,
-                                  StorageType storageType)
-      throws OzoneException {
-    return createBucket(bucketName, acls, storageType,
-        OzoneConsts.Versioning.DISABLED);
-  }
-
-  /**
-   * Create Bucket.
-   *
-   * @param bucketName - bucket name
-   * @param acls - acls
-   *
-   * @throws OzoneException
-   */
-  public OzoneBucket createBucket(String bucketName, String[] acls)
-      throws OzoneException {
-    return createBucket(bucketName, acls, StorageType.DEFAULT,
-        OzoneConsts.Versioning.DISABLED);
-  }
-
-
-  /**
-   * Create Bucket.
-   *
-   * @param bucketName - bucket name
-   *
-   * @throws OzoneException
-   */
-  public OzoneBucket createBucket(String bucketName) throws OzoneException {
-    return createBucket(bucketName, null,  StorageType.DEFAULT,
-        OzoneConsts.Versioning.DISABLED);
-  }
-
-
-  /**
-   * execute a Create Bucket Request against Ozone server.
-   *
-   * @param httppost - httpPost
-   *
-   * @throws IOException
-   * @throws OzoneException
-   */
-  private void executeCreateBucket(HttpPost httppost,
-      CloseableHttpClient httpClient)
-      throws IOException, OzoneException {
-    HttpEntity entity = null;
-    try {
-      HttpResponse response = httpClient.execute(httppost);
-      int errorCode = response.getStatusLine().getStatusCode();
-      entity = response.getEntity();
-      if ((errorCode == HTTP_OK) || (errorCode == HTTP_CREATED)) {
-        return;
-      }
-
-      if (entity != null) {
-        throw OzoneException.parse(EntityUtils.toString(entity));
-      } else {
-        throw new OzoneClientException("Unexpected null in http payload");
-      }
-    } finally {
-      if (entity != null) {
-        EntityUtils.consumeQuietly(entity);
-      }
-    }
-  }
-
-  /**
-   * Adds Acls to an existing bucket.
-   *
-   * @param bucketName - Name of the bucket
-   * @param acls - Acls
-   *
-   * @throws OzoneException
-   */
-  public void addAcls(String bucketName, String[] acls) throws OzoneException {
-    HttpPut putRequest = null;
-    try (CloseableHttpClient httpClient = newHttpClient()) {
-      OzoneUtils.verifyResourceName(bucketName);
-      URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
-      builder.setPath("/" + getVolumeName() + "/" + bucketName).build();
-      putRequest = client.getHttpPut(builder.toString());
-
-      for (String acl : acls) {
-        putRequest
-            .addHeader(Header.OZONE_ACLS, Header.OZONE_ACL_ADD + " " + acl);
-      }
-      executePutBucket(putRequest, httpClient);
-    } catch (URISyntaxException | IOException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      releaseConnection(putRequest);
-    }
-  }
-
-  /**
-   * Removes ACLs from a bucket.
-   *
-   * @param bucketName - Bucket Name
-   * @param acls - Acls to be removed
-   *
-   * @throws OzoneException
-   */
-  public void removeAcls(String bucketName, String[] acls)
-      throws OzoneException {
-    HttpPut putRequest = null;
-    try (CloseableHttpClient httpClient = newHttpClient()) {
-      OzoneUtils.verifyResourceName(bucketName);
-      URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
-      builder.setPath("/" + getVolumeName() + "/" + bucketName).build();
-      putRequest = client.getHttpPut(builder.toString());
-
-      for (String acl : acls) {
-        putRequest
-            .addHeader(Header.OZONE_ACLS, Header.OZONE_ACL_REMOVE + " " + acl);
-      }
-      executePutBucket(putRequest, httpClient);
-    } catch (URISyntaxException | IOException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      releaseConnection(putRequest);
-    }
-  }
-
-  /**
-   * Returns information about an existing bucket.
-   *
-   * @param bucketName - BucketName
-   *
-   * @return OZoneBucket
-   */
-  public OzoneBucket getBucket(String bucketName) throws OzoneException {
-    HttpGet getRequest = null;
-    try (CloseableHttpClient httpClient = newHttpClient()) {
-      OzoneUtils.verifyResourceName(bucketName);
-      URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
-      builder.setPath("/" + getVolumeName() + "/" + bucketName)
-        .setParameter(Header.OZONE_INFO_QUERY_TAG,
-            Header.OZONE_INFO_QUERY_BUCKET).build();
-      getRequest = client.getHttpGet(builder.toString());
-      return executeInfoBucket(getRequest, httpClient);
-
-    } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      releaseConnection(getRequest);
-    }
-  }
-
-
-  /**
-   * Execute the info bucket call.
-   *
-   * @param getRequest - httpGet Request
-   * @param httpClient - Http Client
-   *
-   * @return OzoneBucket
-   *
-   * @throws IOException
-   * @throws OzoneException
-   */
-  private OzoneBucket executeInfoBucket(HttpGet getRequest,
-      CloseableHttpClient httpClient)
-      throws IOException, OzoneException {
-    HttpEntity entity = null;
-    try {
-      HttpResponse response = httpClient.execute(getRequest);
-      int errorCode = response.getStatusLine().getStatusCode();
-      entity = response.getEntity();
-      if (entity == null) {
-        throw new OzoneClientException("Unexpected null in http payload");
-      }
-      if ((errorCode == HTTP_OK) || (errorCode == HTTP_CREATED)) {
-        OzoneBucket bucket =
-            new OzoneBucket(BucketInfo.parse(EntityUtils.toString(entity)),
-                this);
-        return bucket;
-      }
-      throw OzoneException.parse(EntityUtils.toString(entity));
-    } finally {
-      if (entity != null) {
-        EntityUtils.consumeQuietly(entity);
-      }
-    }
-  }
-
-  /**
-   * Execute the put bucket call.
-   *
-   * @param putRequest - http put request
-   * @param httpClient - Http Client
-   *
-   * @return OzoneBucket
-   *
-   * @throws IOException
-   * @throws OzoneException
-   */
-  private void executePutBucket(HttpPut putRequest,
-      CloseableHttpClient httpClient)
-      throws IOException, OzoneException {
-    HttpEntity entity = null;
-    try {
-      HttpResponse response = httpClient.execute(putRequest);
-      int errorCode = response.getStatusLine().getStatusCode();
-      entity = response.getEntity();
-
-      if (errorCode == HTTP_OK) {
-        return;
-      }
-
-      if (entity != null) {
-        throw OzoneException.parse(EntityUtils.toString(entity));
-      }
-
-      throw new OzoneClientException("Unexpected null in http result");
-    } finally {
-      if (entity != null) {
-        EntityUtils.consumeQuietly(entity);
-      }
-    }
-  }
-
-  /**
-   * Gets a list of buckets on this volume.
-   *
-   * @return - List of buckets
-   *
-   * @throws OzoneException
-   */
-  public List<OzoneBucket> listBuckets(String resultLength,
-      String previousBucket, String prefix) throws OzoneException {
-    HttpGet getRequest = null;
-    try (CloseableHttpClient httpClient = newHttpClient()) {
-      URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
-      builder.setPath("/" + getVolumeName()).build();
-      if (!Strings.isNullOrEmpty(resultLength)) {
-        builder.addParameter(Header.OZONE_LIST_QUERY_MAXKEYS, resultLength);
-      }
-      if (!Strings.isNullOrEmpty(previousBucket)) {
-        builder.addParameter(Header.OZONE_LIST_QUERY_PREVKEY, previousBucket);
-      }
-      if (!Strings.isNullOrEmpty(prefix)) {
-        builder.addParameter(Header.OZONE_LIST_QUERY_PREFIX, prefix);
-      }
-
-      getRequest = client.getHttpGet(builder.toString());
-      return executeListBuckets(getRequest, httpClient);
-
-    } catch (IOException | URISyntaxException e) {
-      throw new OzoneClientException(e.getMessage(), e);
-    } finally {
-      releaseConnection(getRequest);
-    }
-  }
-
-  /**
-   * executes the List Bucket Call.
-   *
-   * @param getRequest - http Request
-   * @param httpClient - http Client
-   *
-   * @return List of OzoneBuckets
-   *
-   * @throws IOException
-   * @throws OzoneException
-   */
-  private List<OzoneBucket> executeListBuckets(HttpGet getRequest,
-      CloseableHttpClient httpClient)
-      throws IOException, OzoneException {
-    HttpEntity entity = null;
-    List<OzoneBucket> ozoneBucketList = new LinkedList<OzoneBucket>();
-    try {
-      HttpResponse response = httpClient.execute(getRequest);
-      int errorCode = response.getStatusLine().getStatusCode();
-
-      entity = response.getEntity();
-
-      if (entity == null) {
-        throw new OzoneClientException("Unexpected null in http payload");
-      }
-      if (errorCode == HTTP_OK) {
-        ListBuckets bucketList =
-            ListBuckets.parse(EntityUtils.toString(entity));
-
-        for (BucketInfo info : bucketList.getBuckets()) {
-          ozoneBucketList.add(new OzoneBucket(info, this));
-        }
-        return ozoneBucketList;
-
-      } else {
-        throw OzoneException.parse(EntityUtils.toString(entity));
-      }
-    } finally {
-      if (entity != null) {
-        EntityUtils.consumeQuietly(entity);
-      }
-    }
-  }
-
-  /**
-   * Delete an empty bucket.
-   *
-   * @param bucketName - Name of the bucket to delete
-   *
-   * @throws OzoneException
-   */
-  public void deleteBucket(String bucketName) throws OzoneException {
-    HttpDelete delRequest = null;
-    try (CloseableHttpClient httpClient = newHttpClient()) {
-      OzoneUtils.verifyResourceName(bucketName);
-      URIBuilder builder = new URIBuilder(getClient().getEndPointURI());
-      builder.setPath("/" + getVolumeName() + "/" + bucketName).build();
-
-      delRequest = client.getHttpDelete(builder.toString());
-      executeDeleteBucket(delRequest, httpClient);
-
-    } catch (IOException | URISyntaxException | IllegalArgumentException ex) {
-      throw new OzoneClientException(ex.getMessage(), ex);
-    } finally {
-      releaseConnection(delRequest);
-    }
-  }
-
-  /**
-   * Executes delete bucket call.
-   *
-   * @param delRequest - Delete Request
-   * @param httpClient - Http Client
-7   *
-   * @throws IOException
-   * @throws OzoneException
-   */
-  private void executeDeleteBucket(HttpDelete delRequest,
-      CloseableHttpClient httpClient)
-      throws IOException, OzoneException {
-    HttpEntity entity = null;
-    try {
-      HttpResponse response = httpClient.execute(delRequest);
-      int errorCode = response.getStatusLine().getStatusCode();
-      entity = response.getEntity();
-
-      if (errorCode == HTTP_OK) {
-        return;
-      }
-
-      if (entity == null) {
-        throw new OzoneClientException("Unexpected null in http payload.");
-      }
-
-      throw OzoneException.parse(EntityUtils.toString(entity));
-
-    } finally {
-      if (entity != null) {
-        EntityUtils.consumeQuietly(entity);
-      }
-    }
-  }
-
-  @VisibleForTesting
-  public CloseableHttpClient newHttpClient() {
-    return HddsClientUtils.newHttpClient();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/package-info.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/package-info.java
deleted file mode 100644
index 046568b..0000000
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/web/client/package-info.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.
- */
-
-
-/**
- * Ozone client library is a java client for the Ozone
- * Object Store.
- */
-package org.apache.hadoop.ozone.web.client;
-
-/**
- This library is  a simple Ozone REST Library.
-
- This library is a very *minimal* client written for tests and
- command line utils that work against Ozone. It does not have
- things like thread-pools and support for extended security models yet.
-
- OzoneClients return OzoneVolumes and OzoneVolumes return OzoneBuckets.
- **/

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
index 9936815..ad8b016 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
 import org.apache.hadoop.ozone.ksm.KeySpaceManager;
 import org.apache.hadoop.hdds.scm.server.SCMStorage;
 import org.apache.hadoop.ozone.ksm.KSMStorage;
-import org.apache.hadoop.ozone.web.client.OzoneRestClient;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.protocolPB
     .StorageContainerLocationProtocolClientSideTranslatorPB;
@@ -167,7 +166,7 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
   }
 
   /**
-   * Creates an {@link OzoneRestClient} connected to this cluster's REST
+   * Creates an {@link OzoneClient} connected to this cluster's REST
    * service. Callers take ownership of the client and must close it when done.
    *
    * @return OzoneRestClient connected to this cluster's REST service

http://git-wip-us.apache.org/repos/asf/hadoop/blob/774daa8d/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
index 9aefe9a..1a35c50 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
@@ -20,8 +20,9 @@ package org.apache.hadoop.ozone;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
-import org.apache.hadoop.ozone.web.client.OzoneRestClient;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.ratis.rpc.RpcType;
 import org.apache.ratis.rpc.SupportedRpcType;
@@ -65,9 +66,9 @@ public interface RatisTestHelper {
       return cluster;
     }
 
-    public OzoneRestClient newOzoneRestClient()
-        throws OzoneException, URISyntaxException {
-      return RatisTestHelper.newOzoneRestClient(getDatanodeOzoneRestPort());
+    public ClientProtocol newOzoneClient()
+        throws OzoneException, URISyntaxException, IOException {
+      return new RpcClient(conf);
     }
 
     @Override
@@ -102,9 +103,4 @@ public interface RatisTestHelper {
         .setNumDatanodes(numDatanodes).build();
     return cluster;
   }
-
-  static OzoneRestClient newOzoneRestClient(int port)
-      throws OzoneException, URISyntaxException {
-    return new OzoneRestClient("http://localhost:" + port);
-  }
 }


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


[09/50] [abbrv] hadoop git commit: HADOOP-15494. TestRawLocalFileSystemContract fails on Windows. Contributed by Anbang Hu.

Posted by ha...@apache.org.
HADOOP-15494. TestRawLocalFileSystemContract fails on Windows.
Contributed by Anbang Hu.


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

Branch: refs/heads/HDDS-48
Commit: bddfe796f2f992fc1dcc8a1dd44d64ff2b3c9cf4
Parents: 86bc642
Author: Steve Loughran <st...@apache.org>
Authored: Fri May 25 11:12:47 2018 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Fri May 25 11:12:47 2018 +0100

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bddfe796/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java
index ebf9ea7..908e330 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java
@@ -42,7 +42,7 @@ public class TestRawLocalFileSystemContract extends FileSystemContractBaseTest {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestRawLocalFileSystemContract.class);
   private final static Path TEST_BASE_DIR =
-      new Path(GenericTestUtils.getTempPath(""));
+      new Path(GenericTestUtils.getRandomizedTestDir().getAbsolutePath());
 
   @Before
   public void setUp() throws Exception {


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


[03/50] [abbrv] hadoop git commit: YARN-6919. Add default volume mount list. Contributed by Eric Badger

Posted by ha...@apache.org.
YARN-6919. Add default volume mount list. Contributed by Eric Badger


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

Branch: refs/heads/HDDS-48
Commit: 1388de18ad51434569589a8f5b0b05c38fe02ab3
Parents: 774daa8
Author: Shane Kumpf <sk...@apache.org>
Authored: Thu May 24 09:30:39 2018 -0600
Committer: Shane Kumpf <sk...@apache.org>
Committed: Thu May 24 09:30:39 2018 -0600

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  10 ++
 .../src/main/resources/yarn-default.xml         |  14 ++
 .../runtime/DockerLinuxContainerRuntime.java    |  38 +++++
 .../runtime/TestDockerContainerRuntime.java     | 138 +++++++++++++++++++
 4 files changed, 200 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1388de18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 004a59f..f7f82f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2002,6 +2002,16 @@ public class YarnConfiguration extends Configuration {
    */
   public static final int DEFAULT_NM_DOCKER_STOP_GRACE_PERIOD = 10;
 
+  /** The default list of read-only mounts to be bind-mounted into all
+   *  Docker containers that use DockerContainerRuntime. */
+  public static final String NM_DOCKER_DEFAULT_RO_MOUNTS =
+      DOCKER_CONTAINER_RUNTIME_PREFIX + "default-ro-mounts";
+
+  /** The default list of read-write mounts to be bind-mounted into all
+   *  Docker containers that use DockerContainerRuntime. */
+  public static final String NM_DOCKER_DEFAULT_RW_MOUNTS =
+      DOCKER_CONTAINER_RUNTIME_PREFIX + "default-rw-mounts";
+
   /** The mode in which the Java Container Sandbox should run detailed by
    *  the JavaSandboxLinuxContainerRuntime. */
   public static final String YARN_CONTAINER_SANDBOX =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1388de18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index c82474c..b0ffc48 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -1811,6 +1811,20 @@
   </property>
 
   <property>
+    <description>The default list of read-only mounts to be bind-mounted
+      into all Docker containers that use DockerContainerRuntime.</description>
+    <name>yarn.nodemanager.runtime.linux.docker.default-ro-mounts</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>The default list of read-write mounts to be bind-mounted
+      into all Docker containers that use DockerContainerRuntime.</description>
+    <name>yarn.nodemanager.runtime.linux.docker.default-rw-mounts</name>
+    <value></value>
+  </property>
+
+  <property>
     <description>The mode in which the Java Container Sandbox should run detailed by
       the JavaSandboxLinuxContainerRuntime.</description>
     <name>yarn.nodemanager.runtime.linux.sandbox-mode</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1388de18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.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/linux/runtime/DockerLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
index e131e9d..5e2233b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
@@ -229,6 +229,8 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
   private Set<String> capabilities;
   private boolean delayedRemovalAllowed;
   private int dockerStopGracePeriod;
+  private Set<String> defaultROMounts = new HashSet<>();
+  private Set<String> defaultRWMounts = new HashSet<>();
 
   /**
    * Return whether the given environment variables indicate that the operation
@@ -291,6 +293,8 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     this.conf = conf;
     dockerClient = new DockerClient(conf);
     allowedNetworks.clear();
+    defaultROMounts.clear();
+    defaultRWMounts.clear();
     allowedNetworks.addAll(Arrays.asList(
         conf.getTrimmedStrings(
             YarnConfiguration.NM_DOCKER_ALLOWED_CONTAINER_NETWORKS,
@@ -336,6 +340,14 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     dockerStopGracePeriod = conf.getInt(
         YarnConfiguration.NM_DOCKER_STOP_GRACE_PERIOD,
         YarnConfiguration.DEFAULT_NM_DOCKER_STOP_GRACE_PERIOD);
+
+    defaultROMounts.addAll(Arrays.asList(
+        conf.getTrimmedStrings(
+        YarnConfiguration.NM_DOCKER_DEFAULT_RO_MOUNTS)));
+
+    defaultRWMounts.addAll(Arrays.asList(
+        conf.getTrimmedStrings(
+        YarnConfiguration.NM_DOCKER_DEFAULT_RW_MOUNTS)));
   }
 
   private Set<String> getDockerCapabilitiesFromConf() throws
@@ -829,6 +841,32 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       }
     }
 
+    if(defaultROMounts != null && !defaultROMounts.isEmpty()) {
+      for (String mount : defaultROMounts) {
+        String[] dir = StringUtils.split(mount, ':');
+        if (dir.length != 2) {
+          throw new ContainerExecutionException("Invalid mount : " +
+              mount);
+        }
+        String src = dir[0];
+        String dst = dir[1];
+        runCommand.addReadOnlyMountLocation(src, dst);
+      }
+    }
+
+    if(defaultRWMounts != null && !defaultRWMounts.isEmpty()) {
+      for (String mount : defaultRWMounts) {
+        String[] dir = StringUtils.split(mount, ':');
+        if (dir.length != 2) {
+          throw new ContainerExecutionException("Invalid mount : " +
+              mount);
+        }
+        String src = dir[0];
+        String dst = dir[1];
+        runCommand.addReadWriteMountLocation(src, dst);
+      }
+    }
+
     if (allowHostPidNamespace(container)) {
       runCommand.setPidNamespace("host");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1388de18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.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/containermanager/linux/runtime/TestDockerContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
index ef21ef0..b6de366 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
@@ -82,6 +82,8 @@ import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_DOCKER_DEFAULT_RO_MOUNTS;
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_DOCKER_DEFAULT_RW_MOUNTS;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.APPID;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.APPLICATION_LOCAL_DIRS;
@@ -1332,6 +1334,142 @@ public class TestDockerContainerRuntime {
   }
 
   @Test
+  public void testDefaultROMounts()
+      throws ContainerExecutionException, PrivilegedOperationException,
+      IOException {
+    conf.setStrings(NM_DOCKER_DEFAULT_RO_MOUNTS,
+        "/tmp/foo:/tmp/foo,/tmp/bar:/tmp/bar");
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor, mockCGroupsHandler);
+    runtime.initialize(conf, nmContext);
+
+    runtime.launchContainer(builder.build());
+    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
+    List<String> args = op.getArguments();
+    String dockerCommandFile = args.get(11);
+
+    List<String> dockerCommands = Files.readAllLines(
+        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+
+    int expected = 14;
+    int counter = 0;
+    Assert.assertEquals(expected, dockerCommands.size());
+    Assert.assertEquals("[docker-command-execution]",
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  cap-add=SYS_CHROOT,NET_BIND_SERVICE",
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  cap-drop=ALL", dockerCommands.get(counter++));
+    Assert.assertEquals("  detach=true", dockerCommands.get(counter++));
+    Assert.assertEquals("  docker-command=run", dockerCommands.get(counter++));
+    Assert.assertEquals("  group-add=" + String.join(",", groups),
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  image=busybox:latest",
+        dockerCommands.get(counter++));
+    Assert.assertEquals(
+        "  launch-command=bash,/test_container_work_dir/launch_container.sh",
+        dockerCommands.get(counter++));
+    Assert.assertEquals(
+        "  name=container_e11_1518975676334_14532816_01_000001",
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  net=host", dockerCommands.get(counter++));
+    Assert.assertEquals("  ro-mounts=/test_filecache_dir:/test_filecache_dir,"
+        + "/test_user_filecache_dir:/test_user_filecache_dir,"
+        + "/tmp/foo:/tmp/foo,/tmp/bar:/tmp/bar",
+        dockerCommands.get(counter++));
+    Assert.assertEquals(
+        "  rw-mounts=/test_container_log_dir:/test_container_log_dir,"
+        + "/test_application_local_dir:/test_application_local_dir",
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  user=" + uidGidPair, dockerCommands.get(counter++));
+    Assert.assertEquals("  workdir=/test_container_work_dir",
+        dockerCommands.get(counter));
+  }
+
+  @Test
+  public void testDefaultROMountsInvalid() throws ContainerExecutionException {
+    conf.setStrings(NM_DOCKER_DEFAULT_RO_MOUNTS,
+        "source,target");
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor, mockCGroupsHandler);
+    runtime.initialize(conf, nmContext);
+
+    try {
+      runtime.launchContainer(builder.build());
+      Assert.fail("Expected a launch container failure due to invalid mount.");
+    } catch (ContainerExecutionException e) {
+      LOG.info("Caught expected exception : " + e);
+    }
+  }
+
+  @Test
+  public void testDefaultRWMounts()
+      throws ContainerExecutionException, PrivilegedOperationException,
+      IOException {
+    conf.setStrings(NM_DOCKER_DEFAULT_RW_MOUNTS,
+        "/tmp/foo:/tmp/foo,/tmp/bar:/tmp/bar");
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor, mockCGroupsHandler);
+    runtime.initialize(conf, nmContext);
+
+    runtime.launchContainer(builder.build());
+    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
+    List<String> args = op.getArguments();
+    String dockerCommandFile = args.get(11);
+
+    List<String> dockerCommands = Files.readAllLines(
+        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+
+    int expected = 14;
+    int counter = 0;
+    Assert.assertEquals(expected, dockerCommands.size());
+    Assert.assertEquals("[docker-command-execution]",
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  cap-add=SYS_CHROOT,NET_BIND_SERVICE",
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  cap-drop=ALL", dockerCommands.get(counter++));
+    Assert.assertEquals("  detach=true", dockerCommands.get(counter++));
+    Assert.assertEquals("  docker-command=run", dockerCommands.get(counter++));
+    Assert.assertEquals("  group-add=" + String.join(",", groups),
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  image=busybox:latest",
+        dockerCommands.get(counter++));
+    Assert.assertEquals(
+        "  launch-command=bash,/test_container_work_dir/launch_container.sh",
+        dockerCommands.get(counter++));
+    Assert.assertEquals(
+        "  name=container_e11_1518975676334_14532816_01_000001",
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  net=host", dockerCommands.get(counter++));
+    Assert.assertEquals("  ro-mounts=/test_filecache_dir:/test_filecache_dir,"
+        + "/test_user_filecache_dir:/test_user_filecache_dir",
+        dockerCommands.get(counter++));
+    Assert.assertEquals(
+        "  rw-mounts=/test_container_log_dir:/test_container_log_dir,"
+        + "/test_application_local_dir:/test_application_local_dir,"
+        + "/tmp/foo:/tmp/foo,/tmp/bar:/tmp/bar",
+        dockerCommands.get(counter++));
+    Assert.assertEquals("  user=" + uidGidPair, dockerCommands.get(counter++));
+    Assert.assertEquals("  workdir=/test_container_work_dir",
+        dockerCommands.get(counter));
+  }
+
+  @Test
+  public void testDefaultRWMountsInvalid() throws ContainerExecutionException {
+    conf.setStrings(NM_DOCKER_DEFAULT_RW_MOUNTS,
+        "source,target");
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor, mockCGroupsHandler);
+    runtime.initialize(conf, nmContext);
+
+    try {
+      runtime.launchContainer(builder.build());
+      Assert.fail("Expected a launch container failure due to invalid mount.");
+    } catch (ContainerExecutionException e) {
+      LOG.info("Caught expected exception : " + e);
+    }
+  }
+
+  @Test
   public void testContainerLivelinessCheck()
       throws ContainerExecutionException, PrivilegedOperationException {
 


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


[06/50] [abbrv] hadoop git commit: YARN-8316. Improved diagnostic message for ATS unavailability for YARN Service. Contributed by Billie Rinaldi

Posted by ha...@apache.org.
YARN-8316.  Improved diagnostic message for ATS unavailability for YARN Service.
            Contributed by Billie Rinaldi


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

Branch: refs/heads/HDDS-48
Commit: 7ff5a40218241ad2380595175a493794129a7402
Parents: 2d19e7d
Author: Eric Yang <ey...@apache.org>
Authored: Thu May 24 16:26:02 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Thu May 24 16:26:02 2018 -0400

----------------------------------------------------------------------
 .../org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java   | 2 +-
 .../org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java   | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ff5a402/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
index 072e606..1ceb462 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
@@ -400,7 +400,7 @@ public class YarnClientImpl extends YarnClient {
             + e.getMessage());
         return null;
       }
-      throw e;
+      throw new IOException(e);
     } catch (NoClassDefFoundError e) {
       NoClassDefFoundError wrappedError = new NoClassDefFoundError(
           e.getMessage() + ". It appears that the timeline client "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ff5a402/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
index b84b49c..70ff47b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
@@ -1159,7 +1159,7 @@ public class TestYarnClient extends ParameterizedSchedulerTestBase {
       TimelineClient createTimelineClient() throws IOException, YarnException {
         timelineClient = mock(TimelineClient.class);
         when(timelineClient.getDelegationToken(any(String.class)))
-          .thenThrow(new IOException("Best effort test exception"));
+          .thenThrow(new RuntimeException("Best effort test exception"));
         return timelineClient;
       }
     });
@@ -1175,7 +1175,7 @@ public class TestYarnClient extends ParameterizedSchedulerTestBase {
       client.serviceInit(conf);
       client.getTimelineDelegationToken();
       Assert.fail("Get delegation token should have thrown an exception");
-    } catch (Exception e) {
+    } catch (IOException e) {
       // Success
     }
   }


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


[38/50] [abbrv] hadoop git commit: HADOOP-14946 S3Guard testPruneCommandCLI can fail. Contributed by Gabor Bota.

Posted by ha...@apache.org.
HADOOP-14946 S3Guard testPruneCommandCLI can fail. 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/30284d02
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/30284d02
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/30284d02

Branch: refs/heads/HDDS-48
Commit: 30284d020d36c502dad5bdbae61ec48e9dfe9f8c
Parents: 201440b
Author: Aaron Fabbri <fa...@apache.org>
Authored: Tue May 29 13:38:15 2018 -0700
Committer: Aaron Fabbri <fa...@apache.org>
Committed: Tue May 29 13:38:15 2018 -0700

----------------------------------------------------------------------
 .../s3guard/AbstractS3GuardToolTestBase.java    | 52 +++++++++++++++++---
 1 file changed, 44 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/30284d02/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
index 4381749..2b43810 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
@@ -31,6 +31,7 @@ import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.hadoop.util.StopWatch;
 import org.junit.Assume;
 import org.junit.Test;
 
@@ -61,6 +62,8 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
   protected static final String S3A_THIS_BUCKET_DOES_NOT_EXIST
       = "s3a://this-bucket-does-not-exist-00000000000";
 
+  private static final int PRUNE_MAX_AGE_SECS = 2;
+
   private MetadataStore ms;
 
   protected static void expectResult(int expected,
@@ -186,24 +189,57 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
     }
   }
 
+  /**
+   * Attempt to test prune() with sleep() without having flaky tests
+   * when things run slowly. Test is basically:
+   * 1. Set max path age to X seconds
+   * 2. Create some files (which writes entries to MetadataStore)
+   * 3. Sleep X+2 seconds (all files from above are now "stale")
+   * 4. Create some other files (these are "fresh").
+   * 5. Run prune on MetadataStore.
+   * 6. Assert that only files that were created before the sleep() were pruned.
+   *
+   * Problem is: #6 can fail if X seconds elapse between steps 4 and 5, since
+   * the newer files also become stale and get pruned.  This is easy to
+   * reproduce by running all integration tests in parallel with a ton of
+   * threads, or anything else that slows down execution a lot.
+   *
+   * Solution: Keep track of time elapsed between #4 and #5, and if it
+   * exceeds X, just print a warn() message instead of failing.
+   *
+   * @param cmdConf configuration for command
+   * @param parent path
+   * @param args command args
+   * @throws Exception
+   */
   private void testPruneCommand(Configuration cmdConf, Path parent,
       String...args) throws Exception {
     Path keepParent = path("prune-cli-keep");
+    StopWatch timer = new StopWatch();
     try {
-      getFileSystem().mkdirs(parent);
-      getFileSystem().mkdirs(keepParent);
-
       S3GuardTool.Prune cmd = new S3GuardTool.Prune(cmdConf);
       cmd.setMetadataStore(ms);
 
+      getFileSystem().mkdirs(parent);
+      getFileSystem().mkdirs(keepParent);
       createFile(new Path(parent, "stale"), true, true);
       createFile(new Path(keepParent, "stale-to-keep"), true, true);
-      Thread.sleep(TimeUnit.SECONDS.toMillis(2));
+
+      Thread.sleep(TimeUnit.SECONDS.toMillis(PRUNE_MAX_AGE_SECS + 2));
+
+      timer.start();
       createFile(new Path(parent, "fresh"), true, true);
 
       assertMetastoreListingCount(parent, "Children count before pruning", 2);
       exec(cmd, args);
-      assertMetastoreListingCount(parent, "Pruned children count", 1);
+      long msecElapsed = timer.now(TimeUnit.MILLISECONDS);
+      if (msecElapsed >= PRUNE_MAX_AGE_SECS * 1000) {
+        LOG.warn("Skipping an assertion: Test running too slowly ({} msec)",
+            msecElapsed);
+      } else {
+        assertMetastoreListingCount(parent, "Pruned children count remaining",
+            1);
+      }
       assertMetastoreListingCount(keepParent,
           "This child should have been kept (prefix restriction).", 1);
     } finally {
@@ -224,13 +260,14 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
   public void testPruneCommandCLI() throws Exception {
     Path testPath = path("testPruneCommandCLI");
     testPruneCommand(getFileSystem().getConf(), testPath,
-        "prune", "-seconds", "1", testPath.toString());
+        "prune", "-seconds", String.valueOf(PRUNE_MAX_AGE_SECS),
+        testPath.toString());
   }
 
   @Test
   public void testPruneCommandConf() throws Exception {
     getConfiguration().setLong(Constants.S3GUARD_CLI_PRUNE_AGE,
-        TimeUnit.SECONDS.toMillis(1));
+        TimeUnit.SECONDS.toMillis(PRUNE_MAX_AGE_SECS));
     Path testPath = path("testPruneCommandConf");
     testPruneCommand(getConfiguration(), testPath,
         "prune", testPath.toString());
@@ -286,7 +323,6 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
    * Execute a command, returning the buffer if the command actually completes.
    * If an exception is raised the output is logged instead.
    * @param cmd command
-   * @param buf buffer to use for tool output (not SLF4J output)
    * @param args argument list
    * @throws Exception on any failure
    */


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


[31/50] [abbrv] hadoop git commit: YARN-8369. Javadoc build failed due to 'bad use of >'. (Takanobu Asanuma via wangda)

Posted by ha...@apache.org.
YARN-8369. Javadoc build failed due to 'bad use of >'. (Takanobu Asanuma via wangda)

Change-Id: I79a42154e8f86ab1c3cc939b3745024b8eebe5f4


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

Branch: refs/heads/HDDS-48
Commit: 17aa40f669f197d43387d67dc00040d14cd00948
Parents: 3061bfc
Author: Wangda Tan <wa...@apache.org>
Authored: Tue May 29 09:27:36 2018 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Tue May 29 09:27:36 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/yarn/util/resource/ResourceCalculator.java | 4 ++--
 .../monitor/capacity/CapacitySchedulerPreemptionUtils.java   | 8 ++++----
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/17aa40f6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
index 51078cd..27394f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
@@ -260,10 +260,10 @@ public abstract class ResourceCalculator {
 
   /**
    * Check if resource has any major resource types (which are all NodeManagers
-   * included) has a >0 value.
+   * included) has a {@literal >} 0 value.
    *
    * @param resource resource
-   * @return returns true if any resource is >0
+   * @return returns true if any resource is {@literal >} 0
    */
   public abstract boolean isAnyMajorResourceAboveZero(Resource resource);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/17aa40f6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.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/monitor/capacity/CapacitySchedulerPreemptionUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
index 5396d61..690eb02 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
@@ -136,12 +136,12 @@ public class CapacitySchedulerPreemptionUtils {
    * @param conservativeDRF
    *          should we do conservativeDRF preemption or not.
    *          When true:
-   *            stop preempt container when any major resource type <= 0 for to-
-   *            preempt.
+   *            stop preempt container when any major resource type
+   *            {@literal <=} 0 for to-preempt.
    *            This is default preemption behavior of intra-queue preemption
    *          When false:
-   *            stop preempt container when: all major resource type <= 0 for
-   *            to-preempt.
+   *            stop preempt container when: all major resource type
+   *            {@literal <=} 0 for to-preempt.
    *            This is default preemption behavior of inter-queue preemption
    * @return should we preempt rmContainer. If we should, deduct from
    *         <code>resourceToObtainByPartition</code>


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


[33/50] [abbrv] hadoop git commit: Additional check when unpacking archives. Contributed by Wilfred Spiegelenburg.

Posted by ha...@apache.org.
Additional check when unpacking archives. Contributed by Wilfred Spiegelenburg.


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

Branch: refs/heads/HDDS-48
Commit: e3236a9680709de7a95ffbc11b20e1bdc95a8605
Parents: 9502b47
Author: Kihwal Lee <ki...@apache.org>
Authored: Tue May 29 14:15:12 2018 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Tue May 29 14:15:12 2018 -0500

----------------------------------------------------------------------
 .../java/org/apache/hadoop/util/RunJar.java     | 10 +++++
 .../java/org/apache/hadoop/util/TestRunJar.java | 42 ++++++++++++++++++++
 2 files changed, 52 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3236a96/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
index f1b643c..4c94dbc 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
@@ -117,12 +117,17 @@ public class RunJar {
       throws IOException {
     try (JarInputStream jar = new JarInputStream(inputStream)) {
       int numOfFailedLastModifiedSet = 0;
+      String targetDirPath = toDir.getCanonicalPath() + File.separator;
       for (JarEntry entry = jar.getNextJarEntry();
            entry != null;
            entry = jar.getNextJarEntry()) {
         if (!entry.isDirectory() &&
             unpackRegex.matcher(entry.getName()).matches()) {
           File file = new File(toDir, entry.getName());
+          if (!file.getCanonicalPath().startsWith(targetDirPath)) {
+            throw new IOException("expanding " + entry.getName()
+                + " would create file outside of " + toDir);
+          }
           ensureDirectory(file.getParentFile());
           try (OutputStream out = new FileOutputStream(file)) {
             IOUtils.copyBytes(jar, out, BUFFER_SIZE);
@@ -182,6 +187,7 @@ public class RunJar {
       throws IOException {
     try (JarFile jar = new JarFile(jarFile)) {
       int numOfFailedLastModifiedSet = 0;
+      String targetDirPath = toDir.getCanonicalPath() + File.separator;
       Enumeration<JarEntry> entries = jar.entries();
       while (entries.hasMoreElements()) {
         final JarEntry entry = entries.nextElement();
@@ -189,6 +195,10 @@ public class RunJar {
             unpackRegex.matcher(entry.getName()).matches()) {
           try (InputStream in = jar.getInputStream(entry)) {
             File file = new File(toDir, entry.getName());
+            if (!file.getCanonicalPath().startsWith(targetDirPath)) {
+              throw new IOException("expanding " + entry.getName()
+                  + " would create file outside of " + toDir);
+            }
             ensureDirectory(file.getParentFile());
             try (OutputStream out = new FileOutputStream(file)) {
               IOUtils.copyBytes(in, out, BUFFER_SIZE);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3236a96/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java
index ea07b97..a8c27d4 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java
@@ -21,6 +21,7 @@ import static org.apache.hadoop.util.RunJar.MATCH_ANY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
@@ -32,6 +33,7 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
 import java.util.Random;
 import java.util.jar.JarEntry;
 import java.util.jar.JarOutputStream;
@@ -255,4 +257,44 @@ public class TestRunJar {
     // it should not throw an exception
     verify(runJar, times(0)).unJar(any(File.class), any(File.class));
   }
+
+  @Test
+  public void testUnJar2() throws IOException {
+    // make a simple zip
+    File jarFile = new File(TEST_ROOT_DIR, TEST_JAR_NAME);
+    JarOutputStream jstream =
+        new JarOutputStream(new FileOutputStream(jarFile));
+    JarEntry je = new JarEntry("META-INF/MANIFEST.MF");
+    byte[] data = "Manifest-Version: 1.0\nCreated-By: 1.8.0_1 (Manual)"
+        .getBytes(StandardCharsets.UTF_8);
+    je.setSize(data.length);
+    jstream.putNextEntry(je);
+    jstream.write(data);
+    jstream.closeEntry();
+    je = new JarEntry("../outside.path");
+    data = "any data here".getBytes(StandardCharsets.UTF_8);
+    je.setSize(data.length);
+    jstream.putNextEntry(je);
+    jstream.write(data);
+    jstream.closeEntry();
+    jstream.close();
+
+    File unjarDir = getUnjarDir("unjar-path");
+
+    // Unjar everything
+    try {
+      RunJar.unJar(jarFile, unjarDir, MATCH_ANY);
+      fail("unJar should throw IOException.");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains(
+          "would create file outside of", e);
+    }
+    try {
+      RunJar.unJar(new FileInputStream(jarFile), unjarDir, MATCH_ANY);
+      fail("unJar should throw IOException.");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains(
+          "would create file outside of", e);
+    }
+  }
 }
\ 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


[04/50] [abbrv] hadoop git commit: HDFS-13611. Unsafe use of Text as a ConcurrentHashMap key in PBHelperClient.

Posted by ha...@apache.org.
HDFS-13611. Unsafe use of Text as a ConcurrentHashMap key in PBHelperClient.


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

Branch: refs/heads/HDDS-48
Commit: c9b63deb533274ca8ef4939f6cd13f728a067f7b
Parents: 1388de1
Author: Andrew Wang <wa...@apache.org>
Authored: Thu May 24 09:56:23 2018 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu May 24 09:56:23 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9b63deb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index 579ac43..490ccb4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -247,7 +247,7 @@ public class PBHelperClient {
     ByteString value = fixedByteStringCache.get(key);
     if (value == null) {
       value = ByteString.copyFromUtf8(key.toString());
-      fixedByteStringCache.put(key, value);
+      fixedByteStringCache.put(new Text(key.copyBytes()), value);
     }
     return value;
   }


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


[43/50] [abbrv] hadoop git commit: HDDS-88. Create separate message structure to represent ports in DatanodeDetails. Contributed by Nanda Kumar.

Posted by ha...@apache.org.
HDDS-88. Create separate message structure to represent ports in DatanodeDetails.
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/3b34148c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3b34148c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3b34148c

Branch: refs/heads/HDDS-48
Commit: 3b34148c4f7380d201de59c4a1870b597649248f
Parents: b24098b
Author: Anu Engineer <ae...@apache.org>
Authored: Wed May 30 08:52:07 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Wed May 30 08:52:07 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdds/scm/XceiverClient.java   |   2 +-
 .../hadoop/hdds/scm/XceiverClientGrpc.java      |   2 +-
 .../hadoop/hdds/protocol/DatanodeDetails.java   | 219 +++++++++++--------
 .../main/java/org/apache/ratis/RatisHelper.java |   6 +-
 hadoop-hdds/common/src/main/proto/hdds.proto    |  10 +-
 .../common/transport/server/XceiverServer.java  |   3 +-
 .../transport/server/XceiverServerGrpc.java     |   3 +-
 .../server/ratis/XceiverServerRatis.java        |   3 +-
 .../common/TestDatanodeStateMachine.java        |  16 +-
 .../org/apache/hadoop/hdds/scm/TestUtils.java   |  12 +-
 .../hdds/scm/block/TestDeletedBlockLog.java     |  18 +-
 .../hadoop/ozone/MiniOzoneClusterImpl.java      |  17 +-
 .../apache/hadoop/ozone/RatisTestHelper.java    |   3 +-
 .../hadoop/ozone/TestMiniOzoneCluster.java      |   8 +-
 .../ozone/container/ContainerTestHelper.java    |  12 +-
 .../container/metrics/TestContainerMetrics.java |   3 +-
 .../container/ozoneimpl/TestOzoneContainer.java |  11 +-
 .../container/server/TestContainerServer.java   |   8 +-
 .../ksm/TestKeySpaceManagerRestInterface.java   |   5 +-
 .../ozone/web/TestDistributedOzoneVolumes.java  |   4 +-
 .../hadoop/ozone/web/TestLocalOzoneVolumes.java |   4 +-
 .../hadoop/ozone/web/TestOzoneWebAccess.java    |   4 +-
 .../ozone/web/client/TestOzoneClient.java       |   4 +-
 .../hadoop/ozone/web/client/TestVolume.java     |   2 -
 .../ozone/web/client/TestVolumeRatis.java       |   4 +-
 .../ozone/web/OzoneHddsDatanodeService.java     |   5 +-
 .../hadoop/ozone/ksm/KeySpaceManager.java       |   4 +-
 .../hadoop/ozone/genesis/GenesisUtil.java       |  12 +-
 .../org/apache/hadoop/ozone/scm/cli/SQLCLI.java |   6 +-
 .../hadoop/fs/ozone/TestOzoneFSInputStream.java |   3 +-
 30 files changed, 260 insertions(+), 153 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java
index 42e02f9..709f0dc 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java
@@ -93,7 +93,7 @@ public class XceiverClient extends XceiverClientSpi {
 
     // read port from the data node, on failure use default configured
     // port.
-    int port = leader.getContainerPort();
+    int port = leader.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
     if (port == 0) {
       port = config.getInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
           OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
index 84790e8..c787024 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
@@ -80,7 +80,7 @@ public class XceiverClientGrpc extends XceiverClientSpi {
 
     // read port from the data node, on failure use default configured
     // port.
-    int port = leader.getContainerPort();
+    int port = leader.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
     if (port == 0) {
       port = config.getInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
           OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java
index b2fa291..c373e22 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java
@@ -23,6 +23,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.UUID;
 
 /**
@@ -42,9 +44,7 @@ public final class DatanodeDetails implements Comparable<DatanodeDetails> {
 
   private String ipAddress;
   private String hostName;
-  private Integer containerPort;
-  private Integer ratisPort;
-  private Integer ozoneRestPort;
+  private List<Port> ports;
 
 
   /**
@@ -53,18 +53,14 @@ public final class DatanodeDetails implements Comparable<DatanodeDetails> {
    * @param uuid DataNode's UUID
    * @param ipAddress IP Address of this DataNode
    * @param hostName DataNode's hostname
-   * @param containerPort Container Port
-   * @param ratisPort Ratis Port
-   * @param ozoneRestPort Rest Port
+   * @param ports Ports used by the DataNode
    */
   private DatanodeDetails(String uuid, String ipAddress, String hostName,
-      Integer containerPort, Integer ratisPort, Integer ozoneRestPort) {
+      List<Port> ports) {
     this.uuid = UUID.fromString(uuid);
     this.ipAddress = ipAddress;
     this.hostName = hostName;
-    this.containerPort = containerPort;
-    this.ratisPort = ratisPort;
-    this.ozoneRestPort = ozoneRestPort;
+    this.ports = ports;
   }
 
   /**
@@ -122,54 +118,40 @@ public final class DatanodeDetails implements Comparable<DatanodeDetails> {
   }
 
   /**
-   * Sets the Container Port.
-   * @param port ContainerPort
-   */
-  public void setContainerPort(int port) {
-    containerPort = port;
-  }
-
-  /**
-   * Returns standalone container Port.
+   * Sets a DataNode Port.
    *
-   * @return Container Port
+   * @param port DataNode port
    */
-  public int getContainerPort() {
-    return containerPort;
+  public void setPort(Port port) {
+    // If the port is already in the list remove it first and add the
+    // new/updated port value.
+    ports.remove(port);
+    ports.add(port);
   }
 
   /**
-   * Sets Ratis Port.
-   * @param port RatisPort
-   */
-  public void setRatisPort(int port) {
-    ratisPort = port;
-  }
-
-
-  /**
-   * Returns Ratis Port.
-   * @return Ratis Port
-   */
-  public int getRatisPort() {
-    return ratisPort;
-  }
-
-
-  /**
-   * Sets OzoneRestPort.
-   * @param port OzoneRestPort
+   * Returns all the Ports used by DataNode.
+   *
+   * @return DataNode Ports
    */
-  public void setOzoneRestPort(int port) {
-    ozoneRestPort = port;
+  public List<Port> getPorts() {
+    return ports;
   }
 
   /**
-   * Returns Ozone Rest Port.
-   * @return OzoneRestPort
+   * Given the name returns port number, null if the asked port is not found.
+   *
+   * @param name Name of the port
+   *
+   * @return Port
    */
-  public int getOzoneRestPort() {
-    return ozoneRestPort;
+  public Port getPort(Port.Name name) {
+    for (Port port : ports) {
+      if (port.getName().equals(name)) {
+        return port;
+      }
+    }
+    return null;
   }
 
   /**
@@ -188,14 +170,9 @@ public final class DatanodeDetails implements Comparable<DatanodeDetails> {
     if (datanodeDetailsProto.hasHostName()) {
       builder.setHostName(datanodeDetailsProto.getHostName());
     }
-    if (datanodeDetailsProto.hasContainerPort()) {
-      builder.setContainerPort(datanodeDetailsProto.getContainerPort());
-    }
-    if (datanodeDetailsProto.hasRatisPort()) {
-      builder.setRatisPort(datanodeDetailsProto.getRatisPort());
-    }
-    if (datanodeDetailsProto.hasOzoneRestPort()) {
-      builder.setOzoneRestPort(datanodeDetailsProto.getOzoneRestPort());
+    for (HddsProtos.Port port : datanodeDetailsProto.getPortsList()) {
+      builder.addPort(newPort(
+          Port.Name.valueOf(port.getName().toUpperCase()), port.getValue()));
     }
     return builder.build();
   }
@@ -214,14 +191,11 @@ public final class DatanodeDetails implements Comparable<DatanodeDetails> {
     if (hostName != null) {
       builder.setHostName(hostName);
     }
-    if (containerPort != null) {
-      builder.setContainerPort(containerPort);
-    }
-    if (ratisPort != null) {
-      builder.setRatisPort(ratisPort);
-    }
-    if (ozoneRestPort != null) {
-      builder.setOzoneRestPort(ozoneRestPort);
+    for (Port port : ports) {
+      builder.addPorts(HddsProtos.Port.newBuilder()
+          .setName(port.getName().toString())
+          .setValue(port.getValue())
+          .build());
     }
     return builder.build();
   }
@@ -268,9 +242,15 @@ public final class DatanodeDetails implements Comparable<DatanodeDetails> {
     private String id;
     private String ipAddress;
     private String hostName;
-    private Integer containerPort;
-    private Integer ratisPort;
-    private Integer ozoneRestPort;
+    private List<Port> ports;
+
+    /**
+     * Default private constructor. To create Builder instance use
+     * DatanodeDetails#newBuilder.
+     */
+    private Builder() {
+      ports = new ArrayList<>();
+    }
 
     /**
      * Sets the DatanodeUuid.
@@ -304,50 +284,111 @@ public final class DatanodeDetails implements Comparable<DatanodeDetails> {
       this.hostName = host;
       return this;
     }
+
     /**
-     * Sets the ContainerPort.
+     * Adds a DataNode Port.
+     *
+     * @param port DataNode port
      *
-     * @param port ContainerPort
      * @return DatanodeDetails.Builder
      */
-    public Builder setContainerPort(Integer port) {
-      this.containerPort = port;
+    public Builder addPort(Port port) {
+      this.ports.add(port);
       return this;
     }
 
     /**
-     * Sets the RatisPort.
+     * Builds and returns DatanodeDetails instance.
      *
-     * @param port RatisPort
-     * @return DatanodeDetails.Builder
+     * @return DatanodeDetails
      */
-    public Builder setRatisPort(Integer port) {
-      this.ratisPort = port;
-      return this;
+    public DatanodeDetails build() {
+      Preconditions.checkNotNull(id);
+      return new DatanodeDetails(id, ipAddress, hostName, ports);
     }
 
+  }
+
+  /**
+   * Constructs a new Port with name and value.
+   *
+   * @param name Name of the port
+   * @param value Port number
+   *
+   * @return {@code Port} instance
+   */
+  public static Port newPort(Port.Name name, Integer value) {
+    return new Port(name, value);
+  }
+
+  /**
+   * Container to hold DataNode Port details.
+   */
+  public static class Port {
+
+    /**
+     * Ports that are supported in DataNode.
+     */
+    public enum Name {
+      STANDALONE, RATIS, REST
+    }
+
+    private Name name;
+    private Integer value;
+
     /**
-     * Sets the OzoneRestPort.
+     * Private constructor for constructing Port object. Use
+     * DatanodeDetails#newPort to create a new Port object.
      *
-     * @param port OzoneRestPort
-     * @return DatanodeDetails.Builder
+     * @param name
+     * @param value
      */
-    public Builder setOzoneRestPort(Integer port) {
-      this.ozoneRestPort = port;
-      return this;
+    private Port(Name name, Integer value) {
+      this.name = name;
+      this.value = value;
     }
 
     /**
-     * Builds and returns DatanodeDetails instance.
+     * Returns the name of the port.
      *
-     * @return DatanodeDetails
+     * @return Port name
      */
-    public DatanodeDetails build() {
-      Preconditions.checkNotNull(id);
-      return new DatanodeDetails(id, ipAddress, hostName, containerPort,
-          ratisPort, ozoneRestPort);
+    public Name getName() {
+      return name;
+    }
+
+    /**
+     * Returns the port number.
+     *
+     * @return Port number
+     */
+    public Integer getValue() {
+      return value;
+    }
+
+    @Override
+    public int hashCode() {
+      return name.hashCode();
     }
 
+    /**
+     * Ports are considered equal if they have the same name.
+     *
+     * @param anObject
+     *          The object to compare this {@code Port} against
+     * @return {@code true} if the given object represents a {@code Port}
+               and has the same name, {@code false} otherwise
+     */
+    @Override
+    public boolean equals(Object anObject) {
+      if (this == anObject) {
+        return true;
+      }
+      if (anObject instanceof Port) {
+        return name.equals(((Port) anObject).name);
+      }
+      return false;
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-hdds/common/src/main/java/org/apache/ratis/RatisHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/ratis/RatisHelper.java b/hadoop-hdds/common/src/main/java/org/apache/ratis/RatisHelper.java
index 3a55831..20356b3 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/ratis/RatisHelper.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/ratis/RatisHelper.java
@@ -48,11 +48,13 @@ public interface RatisHelper {
   Logger LOG = LoggerFactory.getLogger(RatisHelper.class);
 
   static String toRaftPeerIdString(DatanodeDetails id) {
-    return id.getUuidString() + "_" + id.getRatisPort();
+    return id.getUuidString() + "_" +
+        id.getPort(DatanodeDetails.Port.Name.RATIS);
   }
 
   static String toRaftPeerAddressString(DatanodeDetails id) {
-    return id.getIpAddress() + ":" + id.getRatisPort();
+    return id.getIpAddress() + ":" +
+        id.getPort(DatanodeDetails.Port.Name.RATIS);
   }
 
   static RaftPeerId toRaftPeerId(DatanodeDetails id) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-hdds/common/src/main/proto/hdds.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/proto/hdds.proto b/hadoop-hdds/common/src/main/proto/hdds.proto
index 6ea5727..f834c73 100644
--- a/hadoop-hdds/common/src/main/proto/hdds.proto
+++ b/hadoop-hdds/common/src/main/proto/hdds.proto
@@ -29,13 +29,15 @@ option java_generate_equals_and_hash = true;
 package hadoop.hdds;
 
 message DatanodeDetailsProto {
-    // TODO: make the port as a seperate proto message and use it here
     required string uuid = 1;  // UUID assigned to the Datanode.
     required string ipAddress = 2;     // IP address
     required string hostName = 3;      // hostname
-    optional uint32 containerPort = 4 [default = 0];  // Ozone stand_alone protocol
-    optional uint32 ratisPort = 5 [default = 0];      //Ozone ratis port
-    optional uint32 ozoneRestPort = 6 [default = 0];
+    repeated Port ports = 4;
+}
+
+message Port {
+    required string name = 1;
+    required uint32 value = 2;
 }
 
 message PipelineChannel {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServer.java
index 7105fd7..455df49 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServer.java
@@ -80,7 +80,8 @@ public final class XceiverServer implements XceiverServerSpi {
             + "fallback to use default port {}", this.port, e);
       }
     }
-    datanodeDetails.setContainerPort(port);
+    datanodeDetails.setPort(
+        DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, port));
     this.storageContainer = dispatcher;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
index 30a2f87..550fe41 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
@@ -71,7 +71,8 @@ public final class XceiverServerGrpc implements XceiverServerSpi {
             + "fallback to use default port {}", this.port, e);
       }
     }
-    datanodeDetails.setContainerPort(port);
+    datanodeDetails.setPort(
+        DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, port));
     server = ((NettyServerBuilder) ServerBuilder.forPort(port))
         .maxMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE)
         .addService(new GrpcXceiverService(dispatcher))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
index 46def09..33c25ea 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
@@ -203,7 +203,8 @@ public final class XceiverServerRatis implements XceiverServerSpi {
             + "fallback to use default port {}", localPort, e);
       }
     }
-    datanodeDetails.setRatisPort(localPort);
+    datanodeDetails.setPort(
+        DatanodeDetails.newPort(DatanodeDetails.Port.Name.RATIS, localPort));
     return new XceiverServerRatis(datanodeDetails, localPort, storageDir,
         dispatcher, ozoneConf);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
index ee82c57..ece7545 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
@@ -209,8 +209,10 @@ public class TestDatanodeStateMachine {
         conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID));
     idPath.delete();
     DatanodeDetails datanodeDetails = getNewDatanodeDetails();
-    datanodeDetails.setContainerPort(
+    DatanodeDetails.Port port = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.STANDALONE,
         OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
+    datanodeDetails.setPort(port);
     ContainerUtils.writeDatanodeDetailsTo(datanodeDetails, idPath);
 
     try (DatanodeStateMachine stateMachine =
@@ -360,13 +362,19 @@ public class TestDatanodeStateMachine {
   }
 
   private DatanodeDetails getNewDatanodeDetails() {
+    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);
     return DatanodeDetails.newBuilder()
         .setUuid(UUID.randomUUID().toString())
         .setHostName("localhost")
         .setIpAddress("127.0.0.1")
-        .setContainerPort(0)
-        .setRatisPort(0)
-        .setOzoneRestPort(0)
+        .addPort(containerPort)
+        .addPort(ratisPort)
+        .addPort(restPort)
         .build();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
index b8036d7..7568bf3 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
@@ -124,13 +124,19 @@ public final class TestUtils {
             .nextInt(256) + "." + random.nextInt(256);
 
     String hostName = uuid;
+    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)
         .setHostName("localhost")
         .setIpAddress(ipAddress)
-        .setContainerPort(0)
-        .setRatisPort(0)
-        .setOzoneRestPort(0);
+        .addPort(containerPort)
+        .addPort(ratisPort)
+        .addPort(restPort);
     return builder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/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 8c12806..adb212a 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
@@ -265,21 +265,27 @@ public class TestDeletedBlockLog {
 
     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")
-        .setContainerPort(0)
-        .setRatisPort(0)
-        .setOzoneRestPort(0)
+        .addPort(containerPort)
+        .addPort(ratisPort)
+        .addPort(restPort)
         .build();
     DatanodeDetails dnId2 = DatanodeDetails.newBuilder()
         .setUuid(UUID.randomUUID().toString())
         .setIpAddress("127.0.0.1")
         .setHostName("localhost")
-        .setContainerPort(0)
-        .setRatisPort(0)
-        .setOzoneRestPort(0)
+        .addPort(containerPort)
+        .addPort(ratisPort)
+        .addPort(restPort)
         .build();
     Mapping mappingService = mock(ContainerMapping.class);
     // Creates {TXNum} TX in the log.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
index ad8b016..f0bfef1 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
@@ -24,6 +24,7 @@ import java.util.Optional;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ipc.Client;
@@ -219,13 +220,15 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
     datanodeService.stop();
     datanodeService.join();
     // ensure same ports are used across restarts.
-    Configuration config = datanodeService.getConf();
-    int currentPort = datanodeService.getDatanodeDetails().getContainerPort();
-    config.setInt(DFS_CONTAINER_IPC_PORT, currentPort);
-    config.setBoolean(DFS_CONTAINER_IPC_RANDOM_PORT, false);
-    int ratisPort = datanodeService.getDatanodeDetails().getRatisPort();
-    config.setInt(DFS_CONTAINER_RATIS_IPC_PORT, ratisPort);
-    config.setBoolean(DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, false);
+    Configuration conf = datanodeService.getConf();
+    int currentPort = datanodeService.getDatanodeDetails()
+        .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
+    conf.setInt(DFS_CONTAINER_IPC_PORT, currentPort);
+    conf.setBoolean(DFS_CONTAINER_IPC_RANDOM_PORT, false);
+    int ratisPort = datanodeService.getDatanodeDetails()
+        .getPort(DatanodeDetails.Port.Name.RATIS).getValue();
+    conf.setInt(DFS_CONTAINER_RATIS_IPC_PORT, ratisPort);
+    conf.setBoolean(DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, false);
     datanodeService.start(null);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
index 1a35c50..fce9e77 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.ratis.rpc.RpcType;
@@ -78,7 +79,7 @@ public interface RatisTestHelper {
 
     public int getDatanodeOzoneRestPort() {
       return cluster.getHddsDatanodes().get(0).getDatanodeDetails()
-          .getOzoneRestPort();
+          .getPort(DatanodeDetails.Port.Name.REST).getValue();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java
index 29238cf..0254984 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachin
 import org.apache.hadoop.ozone.container.ozoneimpl.TestOzoneContainer;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.TestUtils;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.hdds.scm.XceiverClient;
 import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineChannel;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
@@ -44,6 +43,7 @@ import java.io.IOException;
 import java.util.HashSet;
 import java.util.List;
 
+import static org.apache.hadoop.hdds.protocol.DatanodeDetails.Port;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_DIRS;
@@ -114,9 +114,9 @@ public class TestMiniOzoneCluster {
     DatanodeDetails id1 = TestUtils.getDatanodeDetails();
     DatanodeDetails id2 = TestUtils.getDatanodeDetails();
     DatanodeDetails id3 = TestUtils.getDatanodeDetails();
-    id1.setContainerPort(1);
-    id2.setContainerPort(2);
-    id3.setContainerPort(3);
+    id1.setPort(DatanodeDetails.newPort(Port.Name.STANDALONE, 1));
+    id2.setPort(DatanodeDetails.newPort(Port.Name.STANDALONE, 2));
+    id3.setPort(DatanodeDetails.newPort(Port.Name.STANDALONE, 3));
 
     // Write a single ID to the file and read it out
     File validIdsFile = new File(WRITE_TMP, "valid-values.id");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/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 d2a6434..7046132 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
@@ -94,13 +94,19 @@ public final class ContainerTestHelper {
   public static DatanodeDetails createDatanodeDetails() throws IOException {
     ServerSocket socket = new ServerSocket(0);
     int port = socket.getLocalPort();
+    DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.STANDALONE, port);
+    DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.RATIS, port);
+    DatanodeDetails.Port restPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.REST, port);
     DatanodeDetails datanodeDetails = DatanodeDetails.newBuilder()
         .setUuid(UUID.randomUUID().toString())
         .setIpAddress(socket.getInetAddress().getHostAddress())
         .setHostName(socket.getInetAddress().getHostName())
-        .setContainerPort(port)
-        .setRatisPort(port)
-        .setOzoneRestPort(port)
+        .addPort(containerPort)
+        .addPort(ratisPort)
+        .addPort(restPort)
         .build();
 
     socket.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/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 2921be2..ccad6f8 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
@@ -65,7 +65,8 @@ public class TestContainerMetrics {
           .createSingleNodePipeline();
       OzoneConfiguration conf = new OzoneConfiguration();
       conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
-          pipeline.getLeader().getContainerPort());
+          pipeline.getLeader()
+              .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
       conf.setInt(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY,
           interval);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/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 513974a..67a8160 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
@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.container.ozoneimpl;
 
 import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -62,8 +63,8 @@ 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.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
-          pipeline.getLeader().getContainerPort());
+      conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, pipeline.getLeader()
+              .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
       conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, false);
       container = new OzoneContainer(TestUtils.getDatanodeDetails(), conf);
       container.start();
@@ -101,7 +102,8 @@ public class TestOzoneContainer {
       Pipeline pipeline =
           ContainerTestHelper.createSingleNodePipeline();
       conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
-          pipeline.getLeader().getContainerPort());
+          pipeline.getLeader()
+              .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
 
       cluster = MiniOzoneCluster.newBuilder(conf)
           .setRandomContainerPort(false)
@@ -527,7 +529,8 @@ public class TestOzoneContainer {
     Pipeline pipeline =
         ContainerTestHelper.createSingleNodePipeline();
     conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
-        pipeline.getLeader().getContainerPort());
+        pipeline.getLeader()
+            .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
 
     // This client talks to ozone container via datanode.
     return new XceiverClient(pipeline, conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/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 eb170ea..d4c572f 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
@@ -103,7 +103,8 @@ public class TestContainerServer {
     DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
     runTestClientServer(1,
         (pipeline, conf) -> conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
-            pipeline.getLeader().getContainerPort()),
+            pipeline.getLeader()
+                .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue()),
         XceiverClient::new,
         (dn, conf) -> new XceiverServer(datanodeDetails, conf,
             new TestContainerDispatcher()),
@@ -130,7 +131,7 @@ public class TestContainerServer {
   static XceiverServerRatis newXceiverServerRatis(
       DatanodeDetails dn, OzoneConfiguration conf) throws IOException {
     conf.setInt(OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_PORT,
-        dn.getRatisPort());
+        dn.getPort(DatanodeDetails.Port.Name.RATIS).getValue());
     final String dir = TEST_DIR + dn.getUuid();
     conf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR, dir);
 
@@ -208,7 +209,8 @@ public class TestContainerServer {
       Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline();
       OzoneConfiguration conf = new OzoneConfiguration();
       conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
-          pipeline.getLeader().getContainerPort());
+          pipeline.getLeader()
+              .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
 
       Dispatcher dispatcher =
               new Dispatcher(mock(ContainerManager.class), conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManagerRestInterface.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManagerRestInterface.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManagerRestInterface.java
index 2fb70f9..feb83d3 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManagerRestInterface.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManagerRestInterface.java
@@ -118,8 +118,9 @@ public class TestKeySpaceManagerRestInterface {
       switch (type) {
       case HTTP:
       case HTTPS:
-        Assert.assertEquals(datanodeDetails.getOzoneRestPort(),
-            (int) ports.get(type));
+        Assert.assertEquals(
+            datanodeDetails.getPort(DatanodeDetails.Port.Name.REST).getValue(),
+            ports.get(type));
         break;
       default:
         // KSM only sends Datanode's info port details

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestDistributedOzoneVolumes.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestDistributedOzoneVolumes.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestDistributedOzoneVolumes.java
index 1015ae1..0e61391 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestDistributedOzoneVolumes.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestDistributedOzoneVolumes.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.ozone.web;
 
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -67,7 +68,8 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper {
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
+        .getDatanodeDetails()
+        .getPort(DatanodeDetails.Port.Name.REST).getValue();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestLocalOzoneVolumes.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestLocalOzoneVolumes.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestLocalOzoneVolumes.java
index 922587e..441f771 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestLocalOzoneVolumes.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestLocalOzoneVolumes.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.ozone.web;
 
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.TestOzoneHelper;
@@ -70,7 +71,8 @@ public class TestLocalOzoneVolumes extends TestOzoneHelper {
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
+        .getDatanodeDetails().getPort(
+            DatanodeDetails.Port.Name.REST).getValue();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
index 6c32f07..c014a60 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.web;
 
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -78,7 +79,8 @@ public class TestOzoneWebAccess {
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
+        .getDatanodeDetails().getPort(
+            DatanodeDetails.Port.Name.REST).getValue();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestOzoneClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestOzoneClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestOzoneClient.java
index 627826e..86de8df 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestOzoneClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestOzoneClient.java
@@ -43,6 +43,7 @@ import io.netty.handler.codec.http.LastHttpContent;
 import io.netty.handler.logging.LogLevel;
 import io.netty.handler.logging.LoggingHandler;
 
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -96,7 +97,8 @@ public class TestOzoneClient {
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     int port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
+        .getDatanodeDetails()
+        .getPort(DatanodeDetails.Port.Name.REST).getValue();
     endpoint = String.format("http://localhost:%d", port);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
index f8c7eec..a510430 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
@@ -81,8 +81,6 @@ public class TestVolume {
 
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
-    final int port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
 
     client = new RpcClient(conf);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
index bc4ba25..dcb4030 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.web.client;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -63,7 +64,8 @@ public class TestVolumeRatis {
     cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3).build();
     cluster.waitForClusterToBeReady();
     final int port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
+        .getDatanodeDetails()
+        .getPort(DatanodeDetails.Port.Name.REST).getValue();
 
     client = new RpcClient(conf);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/OzoneHddsDatanodeService.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/OzoneHddsDatanodeService.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/OzoneHddsDatanodeService.java
index 2283ba6..87b1e21 100644
--- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/OzoneHddsDatanodeService.java
+++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/OzoneHddsDatanodeService.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.web;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
 import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.web.netty.ObjectStoreRestHttpServer;
@@ -51,8 +52,10 @@ public class OzoneHddsDatanodeService implements ServicePlugin {
         objectStoreRestHttpServer = new ObjectStoreRestHttpServer(
             conf, null, handler);
         objectStoreRestHttpServer.start();
-        hddsDatanodeService.getDatanodeDetails().setOzoneRestPort(
+        DatanodeDetails.Port restPort = DatanodeDetails.newPort(
+            DatanodeDetails.Port.Name.REST,
             objectStoreRestHttpServer.getHttpAddress().getPort());
+        hddsDatanodeService.getDatanodeDetails().setPort(restPort);
 
       } catch (IOException e) {
         throw new RuntimeException("Can't start the Object Store Rest server",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java
index d0f0c9b..dc8fc91 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java
@@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.BlockingService;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -896,7 +897,8 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl
 
       dnServiceInfoBuilder.addServicePort(ServicePort.newBuilder()
           .setType(ServicePort.Type.HTTP)
-          .setValue(datanode.getOzoneRestPort())
+          .setValue(DatanodeDetails.getFromProtoBuf(datanode)
+              .getPort(DatanodeDetails.Port.Name.REST).getValue())
           .build());
 
       services.add(dnServiceInfoBuilder.build());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java
index 611b62d..7f864ae 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java
@@ -78,13 +78,19 @@ public final class GenesisUtil {
         random.nextInt(256) + "." + random.nextInt(256) + "." + random
             .nextInt(256) + "." + random.nextInt(256);
 
+    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)
         .setHostName("localhost")
         .setIpAddress(ipAddress)
-        .setContainerPort(0)
-        .setRatisPort(0)
-        .setOzoneRestPort(0);
+        .addPort(containerPort)
+        .addPort(ratisPort)
+        .addPort(restPort);
     return builder.build();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java
index 028b1fc..d4ac994 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java
@@ -530,7 +530,8 @@ public class SQLCLI  extends Configured implements Tool {
         // but this seems a bit cleaner.
         String ipAddr = dd.getIpAddress();
         String hostName = dd.getHostName();
-        int containerPort = dd.getContainerPort();
+        int containerPort = DatanodeDetails.getFromProtoBuf(dd)
+            .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
         String insertMachineInfo = String.format(
             INSERT_DATANODE_INFO, hostName, uuid, ipAddr, containerPort);
         executeSQL(conn, insertMachineInfo);
@@ -598,7 +599,8 @@ public class SQLCLI  extends Configured implements Tool {
     String insertDatanodeDetails = String
         .format(INSERT_DATANODE_INFO, datanodeDetails.getHostName(),
             datanodeDetails.getUuidString(), datanodeDetails.getIpAddress(),
-            datanodeDetails.getContainerPort());
+            datanodeDetails.getPort(DatanodeDetails.Port.Name.STANDALONE)
+                .getValue());
     executeSQL(conn, insertDatanodeDetails);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b34148c/hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java b/hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
index 4d8c9d6..b82c4a1 100644
--- a/hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
+++ b/hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
@@ -90,7 +90,8 @@ public class TestOzoneFSInputStream {
     // Fetch the host and port for File System init
     DatanodeDetails datanodeDetails = cluster.getHddsDatanodes().get(0)
         .getDatanodeDetails();
-    int port = datanodeDetails.getOzoneRestPort();
+    int port = datanodeDetails
+        .getPort(DatanodeDetails.Port.Name.REST).getValue();
     String host = datanodeDetails.getHostName();
 
     // Set the fs.defaultFS and start the filesystem


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


[36/50] [abbrv] hadoop git commit: HDDS-81. Moving ContainerReport inside Datanode heartbeat. Contributed by Nanda Kumar.

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/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 2d88621..f5fe46a 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
@@ -20,6 +20,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.primitives.Longs;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.closer.ContainerCloser;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
@@ -33,7 +34,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.lease.Lease;
 import org.apache.hadoop.ozone.lease.LeaseException;
@@ -368,11 +369,12 @@ public class ContainerMapping implements Mapping {
    * @param reports Container report
    */
   @Override
-  public void processContainerReports(ContainerReportsRequestProto reports)
+  public void processContainerReports(DatanodeDetails datanodeDetails,
+                                      ContainerReportsProto reports)
       throws IOException {
     List<StorageContainerDatanodeProtocolProtos.ContainerInfo>
         containerInfos = reports.getReportsList();
-    containerSupervisor.handleContainerReport(reports);
+    containerSupervisor.handleContainerReport(datanodeDetails, reports);
     for (StorageContainerDatanodeProtocolProtos.ContainerInfo datanodeState :
         containerInfos) {
       byte[] dbKey = Longs.toByteArray(datanodeState.getContainerID());
@@ -402,7 +404,7 @@ public class ContainerMapping implements Mapping {
           // Container not found in our container db.
           LOG.error("Error while processing container report from datanode :" +
                   " {}, for container: {}, reason: container doesn't exist in" +
-                  "container database.", reports.getDatanodeDetails(),
+                  "container database.", datanodeDetails,
               datanodeState.getContainerID());
         }
       } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
index f560174..ee8e344 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
@@ -16,10 +16,11 @@
  */
 package org.apache.hadoop.hdds.scm.container;
 
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 
 import java.io.Closeable;
 import java.io.IOException;
@@ -98,7 +99,8 @@ public interface Mapping extends Closeable {
    *
    * @param reports Container report
    */
-  void processContainerReports(ContainerReportsRequestProto reports)
+  void processContainerReports(DatanodeDetails datanodeDetails,
+                               ContainerReportsProto reports)
       throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ContainerSupervisor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ContainerSupervisor.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ContainerSupervisor.java
index c14303f..5bd0574 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ContainerSupervisor.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ContainerSupervisor.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.hdds.scm.node.NodePoolManager;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.concurrent.HadoopExecutors;
 import org.apache.hadoop.util.concurrent.HadoopThreadPoolExecutor;
@@ -295,24 +295,21 @@ public class ContainerSupervisor implements Closeable {
    * @param containerReport  -- Container report for a specific container from
    * a datanode.
    */
-  public void handleContainerReport(
-      ContainerReportsRequestProto containerReport) {
-    DatanodeDetails datanodeDetails = DatanodeDetails.getFromProtoBuf(
-        containerReport.getDatanodeDetails());
+  public void handleContainerReport(DatanodeDetails datanodeDetails,
+      ContainerReportsProto containerReport) {
     inProgressPoolListLock.readLock().lock();
     try {
       String poolName = poolManager.getNodePool(datanodeDetails);
       for (InProgressPool ppool : inProgressPoolList) {
         if (ppool.getPoolName().equalsIgnoreCase(poolName)) {
-          ppool.handleContainerReport(containerReport);
+          ppool.handleContainerReport(datanodeDetails, containerReport);
           return;
         }
       }
       // TODO: Decide if we can do anything else with this report.
       LOG.debug("Discarding the container report for pool {}. " +
               "That pool is not currently in the pool reconciliation process." +
-              " Container Name: {}", poolName,
-          containerReport.getDatanodeDetails());
+              " Container Name: {}", poolName, datanodeDetails);
     } catch (SCMException e) {
       LOG.warn("Skipping processing container report from datanode {}, "
               + "cause: failed to get the corresponding node pool",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java
index c444e90..4b54731 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerInfo;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -178,21 +178,20 @@ public final class InProgressPool {
    *
    * @param containerReport - ContainerReport
    */
-  public void handleContainerReport(
-      ContainerReportsRequestProto containerReport) {
+  public void handleContainerReport(DatanodeDetails datanodeDetails,
+      ContainerReportsProto containerReport) {
     if (status == ProgressStatus.InProgress) {
-      executorService.submit(processContainerReport(containerReport));
+      executorService.submit(processContainerReport(datanodeDetails,
+          containerReport));
     } else {
       LOG.debug("Cannot handle container report when the pool is in {} status.",
           status);
     }
   }
 
-  private Runnable processContainerReport(
-      ContainerReportsRequestProto reports) {
+  private Runnable processContainerReport(DatanodeDetails datanodeDetails,
+      ContainerReportsProto reports) {
     return () -> {
-      DatanodeDetails datanodeDetails =
-          DatanodeDetails.getFromProtoBuf(reports.getDatanodeDetails());
       if (processedNodeSet.computeIfAbsent(datanodeDetails.getUuid(),
           (k) -> true)) {
         nodeProcessed.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/HeartbeatQueueItem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/HeartbeatQueueItem.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/HeartbeatQueueItem.java
index 05a9fc3..04658bd 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/HeartbeatQueueItem.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/HeartbeatQueueItem.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hdds.scm.node;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 
 import static org.apache.hadoop.util.Time.monotonicNow;
 
@@ -31,7 +31,7 @@ import static org.apache.hadoop.util.Time.monotonicNow;
 public class HeartbeatQueueItem {
   private DatanodeDetails datanodeDetails;
   private long recvTimestamp;
-  private SCMNodeReport nodeReport;
+  private NodeReportProto nodeReport;
 
   /**
    *
@@ -40,7 +40,7 @@ public class HeartbeatQueueItem {
    * @param nodeReport - node report associated with the heartbeat if any.
    */
   HeartbeatQueueItem(DatanodeDetails datanodeDetails, long recvTimestamp,
-      SCMNodeReport nodeReport) {
+      NodeReportProto nodeReport) {
     this.datanodeDetails = datanodeDetails;
     this.recvTimestamp = recvTimestamp;
     this.nodeReport = nodeReport;
@@ -56,7 +56,7 @@ public class HeartbeatQueueItem {
   /**
    * @return node report.
    */
-  public SCMNodeReport getNodeReport() {
+  public NodeReportProto getNodeReport() {
     return nodeReport;
   }
 
@@ -72,7 +72,7 @@ public class HeartbeatQueueItem {
    */
   public static class Builder {
     private DatanodeDetails datanodeDetails;
-    private SCMNodeReport nodeReport;
+    private NodeReportProto nodeReport;
     private long recvTimestamp = monotonicNow();
 
     public Builder setDatanodeDetails(DatanodeDetails dnDetails) {
@@ -80,8 +80,8 @@ public class HeartbeatQueueItem {
       return this;
     }
 
-    public Builder setNodeReport(SCMNodeReport scmNodeReport) {
-      this.nodeReport = scmNodeReport;
+    public Builder setNodeReport(NodeReportProto report) {
+      this.nodeReport = report;
       return this;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
index 353a069..b339fb7 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
@@ -28,15 +28,14 @@ import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto
+    .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto
     .ErrorCode;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMStorageReport;
+    .StorageContainerDatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
 import org.apache.hadoop.ipc.Server;
@@ -592,7 +591,7 @@ public class SCMNodeManager
 
     DatanodeDetails datanodeDetails = hbItem.getDatanodeDetails();
     UUID datanodeUuid = datanodeDetails.getUuid();
-    SCMNodeReport nodeReport = hbItem.getNodeReport();
+    NodeReportProto nodeReport = hbItem.getNodeReport();
     long recvTimestamp = hbItem.getRecvTimestamp();
     long processTimestamp = Time.monotonicNow();
     if (LOG.isTraceEnabled()) {
@@ -637,7 +636,7 @@ public class SCMNodeManager
         new ReregisterCommand());
   }
 
-  private void updateNodeStat(UUID dnId, SCMNodeReport nodeReport) {
+  private void updateNodeStat(UUID dnId, NodeReportProto nodeReport) {
     SCMNodeStat stat = nodeStats.get(dnId);
     if (stat == null) {
       LOG.debug("SCM updateNodeStat based on heartbeat from previous" +
@@ -649,8 +648,9 @@ public class SCMNodeManager
       long totalCapacity = 0;
       long totalRemaining = 0;
       long totalScmUsed = 0;
-      List<SCMStorageReport> storageReports = nodeReport.getStorageReportList();
-      for (SCMStorageReport report : storageReports) {
+      List<StorageReportProto> storageReports = nodeReport
+          .getStorageReportList();
+      for (StorageReportProto report : storageReports) {
         totalCapacity += report.getCapacity();
         totalRemaining +=  report.getRemaining();
         totalScmUsed+= report.getScmUsed();
@@ -710,7 +710,7 @@ public class SCMNodeManager
    * Register the node if the node finds that it is not registered with any
    * SCM.
    *
-   * @param datanodeDetailsProto - Send datanodeDetails with Node info.
+   * @param datanodeDetails - Send datanodeDetails with Node info.
    *                   This function generates and assigns new datanode ID
    *                   for the datanode. This allows SCM to be run independent
    *                   of Namenode if required.
@@ -719,13 +719,11 @@ public class SCMNodeManager
    * @return SCMHeartbeatResponseProto
    */
   @Override
-  public SCMCommand register(DatanodeDetailsProto datanodeDetailsProto,
-                             SCMNodeReport nodeReport) {
+  public RegisteredCommand register(
+      DatanodeDetails datanodeDetails, NodeReportProto nodeReport) {
 
     String hostname = null;
     String ip = null;
-    DatanodeDetails datanodeDetails = DatanodeDetails.getFromProtoBuf(
-        datanodeDetailsProto);
     InetAddress dnAddress = Server.getRemoteIp();
     if (dnAddress != null) {
       // Mostly called inside an RPC, update ip and peer hostname
@@ -734,7 +732,7 @@ public class SCMNodeManager
       datanodeDetails.setHostName(hostname);
       datanodeDetails.setIpAddress(ip);
     }
-    SCMCommand responseCommand = verifyDatanodeUUID(datanodeDetails);
+    RegisteredCommand responseCommand = verifyDatanodeUUID(datanodeDetails);
     if (responseCommand != null) {
       return responseCommand;
     }
@@ -785,7 +783,8 @@ public class SCMNodeManager
    * @param datanodeDetails - Datanode Details.
    * @return SCMCommand
    */
-  private SCMCommand verifyDatanodeUUID(DatanodeDetails datanodeDetails) {
+  private RegisteredCommand verifyDatanodeUUID(
+      DatanodeDetails datanodeDetails) {
     if (datanodeDetails.getUuid() != null &&
         nodes.containsKey(datanodeDetails.getUuid())) {
       LOG.trace("Datanode is already registered. Datanode: {}",
@@ -802,34 +801,23 @@ public class SCMNodeManager
   /**
    * Send heartbeat to indicate the datanode is alive and doing well.
    *
-   * @param datanodeDetailsProto - DatanodeDetailsProto.
+   * @param datanodeDetails - DatanodeDetailsProto.
    * @param nodeReport - node report.
    * @return SCMheartbeat response.
    * @throws IOException
    */
   @Override
   public List<SCMCommand> sendHeartbeat(
-      DatanodeDetailsProto datanodeDetailsProto, SCMNodeReport nodeReport) {
+      DatanodeDetails datanodeDetails, NodeReportProto nodeReport) {
 
-    Preconditions.checkNotNull(datanodeDetailsProto, "Heartbeat is missing " +
+    Preconditions.checkNotNull(datanodeDetails, "Heartbeat is missing " +
         "DatanodeDetails.");
-    DatanodeDetails datanodeDetails = DatanodeDetails
-        .getFromProtoBuf(datanodeDetailsProto);
-    // Checking for NULL to make sure that we don't get
-    // an exception from ConcurrentList.
-    // This could be a problem in tests, if this function is invoked via
-    // protobuf, transport layer will guarantee that this is not null.
-    if (datanodeDetails != null) {
-      heartbeatQueue.add(
-          new HeartbeatQueueItem.Builder()
-              .setDatanodeDetails(datanodeDetails)
-              .setNodeReport(nodeReport)
-              .build());
-      return commandQueue.getCommand(datanodeDetails.getUuid());
-    } else {
-      LOG.error("Datanode ID in heartbeat is null");
-    }
-    return null;
+    heartbeatQueue.add(
+        new HeartbeatQueueItem.Builder()
+            .setDatanodeDetails(datanodeDetails)
+            .setNodeReport(nodeReport)
+            .build());
+    return commandQueue.getCommand(datanodeDetails.getUuid());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMap.java
index fa423bb..6ea83df 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMap.java
@@ -23,7 +23,7 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.protocol.proto.
-    StorageContainerDatanodeProtocolProtos.SCMStorageReport;
+    StorageContainerDatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -33,7 +33,11 @@ import org.slf4j.LoggerFactory;
 
 import javax.management.ObjectName;
 import java.io.IOException;
-import java.util.*;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.stream.Collectors;
 
@@ -159,7 +163,7 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
   }
 
   public StorageReportResult processNodeReport(UUID datanodeID,
-      StorageContainerDatanodeProtocolProtos.SCMNodeReport nodeReport)
+      StorageContainerDatanodeProtocolProtos.NodeReportProto nodeReport)
       throws IOException {
     Preconditions.checkNotNull(datanodeID);
     Preconditions.checkNotNull(nodeReport);
@@ -170,9 +174,9 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
     Set<StorageLocationReport> storagReportSet = new HashSet<>();
     Set<StorageLocationReport> fullVolumeSet = new HashSet<>();
     Set<StorageLocationReport> failedVolumeSet = new HashSet<>();
-    List<SCMStorageReport>
+    List<StorageReportProto>
         storageReports = nodeReport.getStorageReportList();
-    for (SCMStorageReport report : storageReports) {
+    for (StorageReportProto report : storageReports) {
       StorageLocationReport storageReport =
           StorageLocationReport.getFromProtobuf(report);
       storagReportSet.add(storageReport);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/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 6e5b7de..1b1645d 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
@@ -25,29 +25,47 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.BlockingService;
 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.protocol.proto.StorageContainerDatanodeProtocolProtos;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMReregisterCmdResponseProto;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandResponseProto;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsResponseProto;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKResponseProto;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto.DeleteBlockTransactionResult;
+    .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ReregisterCommandProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos
+    .ContainerBlocksDeletionACKResponseProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos
+    .ContainerBlocksDeletionACKProto.DeleteBlockTransactionResult;
+
 
+import static org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto
+    .Type.closeContainerCommand;
+import static org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto
+    .Type.deleteBlocksCommand;
+import static org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto
+    .Type.reregisterCommand;
 
-import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.versionCommand;
-import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.registeredCommand;
-import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.reregisterCommand;
-import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.deleteBlocksCommand;
-import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.closeContainerCommand;
 
 
 import org.apache.hadoop.hdds.scm.HddsServerUtil;
@@ -150,96 +168,81 @@ public class SCMDatanodeProtocolServer implements
 
   @Override
   public SCMHeartbeatResponseProto sendHeartbeat(
-      HddsProtos.DatanodeDetailsProto datanodeDetails,
-      StorageContainerDatanodeProtocolProtos.SCMNodeReport nodeReport)
+      SCMHeartbeatRequestProto heartbeat)
       throws IOException {
+    // TODO: Add a heartbeat dispatcher.
+    DatanodeDetails datanodeDetails = DatanodeDetails
+        .getFromProtoBuf(heartbeat.getDatanodeDetails());
+    NodeReportProto nodeReport = heartbeat.getNodeReport();
     List<SCMCommand> commands =
         scm.getScmNodeManager().sendHeartbeat(datanodeDetails, nodeReport);
-    List<SCMCommandResponseProto> cmdResponses = new LinkedList<>();
+    List<SCMCommandProto> cmdResponses = new LinkedList<>();
     for (SCMCommand cmd : commands) {
-      cmdResponses.add(getCommandResponse(cmd, datanodeDetails.getUuid()));
+      cmdResponses.add(getCommandResponse(cmd));
     }
     return SCMHeartbeatResponseProto.newBuilder()
+        .setDatanodeUUID(datanodeDetails.getUuidString())
         .addAllCommands(cmdResponses).build();
   }
 
   @Override
-  public SCMRegisteredCmdResponseProto register(
-      HddsProtos.DatanodeDetailsProto datanodeDetails, SCMNodeReport nodeReport,
-      ContainerReportsRequestProto containerReportsRequestProto)
+  public SCMRegisteredResponseProto register(
+      HddsProtos.DatanodeDetailsProto datanodeDetailsProto,
+      NodeReportProto nodeReport,
+      ContainerReportsProto containerReportsProto)
       throws IOException {
+    DatanodeDetails datanodeDetails = DatanodeDetails
+        .getFromProtoBuf(datanodeDetailsProto);
     // TODO : Return the list of Nodes that forms the SCM HA.
-    RegisteredCommand registeredCommand = (RegisteredCommand) scm
-        .getScmNodeManager().register(datanodeDetails, nodeReport);
-    SCMCmdType type = registeredCommand.getType();
-    if (type == SCMCmdType.registeredCommand && registeredCommand.getError()
-        == SCMRegisteredCmdResponseProto.ErrorCode.success) {
-      scm.getScmContainerManager().processContainerReports(
-          containerReportsRequestProto);
+    RegisteredCommand registeredCommand = scm.getScmNodeManager()
+        .register(datanodeDetails, nodeReport);
+    if (registeredCommand.getError()
+        == SCMRegisteredResponseProto.ErrorCode.success) {
+      scm.getScmContainerManager().processContainerReports(datanodeDetails,
+          containerReportsProto);
     }
     return getRegisteredResponse(registeredCommand);
   }
 
   @VisibleForTesting
-  public static SCMRegisteredCmdResponseProto getRegisteredResponse(
-        SCMCommand cmd) {
-    Preconditions.checkState(cmd.getClass() == RegisteredCommand.class);
-    RegisteredCommand rCmd = (RegisteredCommand) cmd;
-    SCMCmdType type = cmd.getType();
-    if (type != SCMCmdType.registeredCommand) {
-      throw new IllegalArgumentException(
-          "Registered command is not well " + "formed. Internal Error.");
-    }
-    return SCMRegisteredCmdResponseProto.newBuilder()
+  public static SCMRegisteredResponseProto getRegisteredResponse(
+      RegisteredCommand cmd) {
+    return SCMRegisteredResponseProto.newBuilder()
         // TODO : Fix this later when we have multiple SCM support.
         // .setAddressList(addressList)
-        .setErrorCode(rCmd.getError())
-        .setClusterID(rCmd.getClusterID())
-        .setDatanodeUUID(rCmd.getDatanodeUUID())
+        .setErrorCode(cmd.getError())
+        .setClusterID(cmd.getClusterID())
+        .setDatanodeUUID(cmd.getDatanodeUUID())
         .build();
   }
 
-  @Override
-  public ContainerReportsResponseProto sendContainerReport(
-      ContainerReportsRequestProto reports)
+  public void processContainerReports(DatanodeDetails datanodeDetails,
+                                      ContainerReportsProto reports)
       throws IOException {
-    updateContainerReportMetrics(reports);
-
+    updateContainerReportMetrics(datanodeDetails, reports);
     // should we process container reports async?
-    scm.getScmContainerManager().processContainerReports(reports);
-    return ContainerReportsResponseProto.newBuilder().build();
+    scm.getScmContainerManager()
+        .processContainerReports(datanodeDetails, reports);
   }
 
-  private void updateContainerReportMetrics(
-      ContainerReportsRequestProto reports) {
-    ContainerStat newStat = null;
-    // TODO: We should update the logic once incremental container report
-    // type is supported.
-    if (reports
-        .getType() == StorageContainerDatanodeProtocolProtos
-        .ContainerReportsRequestProto.reportType.fullReport) {
-      newStat = new ContainerStat();
-      for (StorageContainerDatanodeProtocolProtos.ContainerInfo info : reports
-          .getReportsList()) {
-        newStat.add(new ContainerStat(info.getSize(), info.getUsed(),
-            info.getKeyCount(), info.getReadBytes(), info.getWriteBytes(),
-            info.getReadCount(), info.getWriteCount()));
-      }
-
-      // update container metrics
-      StorageContainerManager.getMetrics().setLastContainerStat(newStat);
+  private void updateContainerReportMetrics(DatanodeDetails datanodeDetails,
+                                            ContainerReportsProto reports) {
+    ContainerStat newStat = new ContainerStat();
+    for (StorageContainerDatanodeProtocolProtos.ContainerInfo info : reports
+        .getReportsList()) {
+      newStat.add(new ContainerStat(info.getSize(), info.getUsed(),
+          info.getKeyCount(), info.getReadBytes(), info.getWriteBytes(),
+          info.getReadCount(), info.getWriteCount()));
     }
+    // update container metrics
+    StorageContainerManager.getMetrics().setLastContainerStat(newStat);
 
     // Update container stat entry, this will trigger a removal operation if it
     // exists in cache.
-    synchronized (scm.getContainerReportCache()) {
-      String datanodeUuid = reports.getDatanodeDetails().getUuid();
-      if (datanodeUuid != null && newStat != null) {
-        scm.getContainerReportCache().put(datanodeUuid, newStat);
-        // update global view container metrics
-        StorageContainerManager.getMetrics().incrContainerStat(newStat);
-      }
-    }
+    String datanodeUuid = datanodeDetails.getUuidString();
+    scm.getContainerReportCache().put(datanodeUuid, newStat);
+    // update global view container metrics
+    StorageContainerManager.getMetrics().incrContainerStat(newStat);
   }
 
 
@@ -298,28 +301,15 @@ public class SCMDatanodeProtocolServer implements
    * @throws IOException
    */
   @VisibleForTesting
-  public StorageContainerDatanodeProtocolProtos.SCMCommandResponseProto
-      getCommandResponse(
-      SCMCommand cmd, final String datanodeID) throws IOException {
-    SCMCmdType type = cmd.getType();
-    SCMCommandResponseProto.Builder builder =
-        SCMCommandResponseProto.newBuilder().setDatanodeUUID(datanodeID);
-    switch (type) {
-    case registeredCommand:
-      return builder
-          .setCmdType(registeredCommand)
-          .setRegisteredProto(SCMRegisteredCmdResponseProto
-              .getDefaultInstance())
-          .build();
-    case versionCommand:
-      return builder
-          .setCmdType(versionCommand)
-          .setVersionProto(SCMVersionResponseProto.getDefaultInstance())
-          .build();
+  public SCMCommandProto getCommandResponse(SCMCommand cmd)
+      throws IOException {
+    SCMCommandProto.Builder builder =
+        SCMCommandProto.newBuilder();
+    switch (cmd.getType()) {
     case reregisterCommand:
       return builder
-          .setCmdType(reregisterCommand)
-          .setReregisterProto(SCMReregisterCmdResponseProto
+          .setCommandType(reregisterCommand)
+          .setReregisterCommandProto(ReregisterCommandProto
               .getDefaultInstance())
           .build();
     case deleteBlocksCommand:
@@ -335,13 +325,14 @@ public class SCMDatanodeProtocolServer implements
               .collect(Collectors.toList());
       scm.getScmBlockManager().getDeletedBlockLog().incrementCount(txs);
       return builder
-          .setCmdType(deleteBlocksCommand)
-          .setDeleteBlocksProto(((DeleteBlocksCommand) cmd).getProto())
+          .setCommandType(deleteBlocksCommand)
+          .setDeleteBlocksCommandProto(((DeleteBlocksCommand) cmd).getProto())
           .build();
     case closeContainerCommand:
       return builder
-          .setCmdType(closeContainerCommand)
-          .setCloseContainerProto(((CloseContainerCommand) cmd).getProto())
+          .setCommandType(closeContainerCommand)
+          .setCloseContainerCommandProto(
+              ((CloseContainerCommand) cmd).getProto())
           .build();
     default:
       throw new IllegalArgumentException("Not implemented");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
index 5cf0a92..b8036d7 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
@@ -18,9 +18,9 @@ package org.apache.hadoop.hdds.scm;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.protocol
-    .proto.StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    .proto.StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.hdds.protocol.proto
-        .StorageContainerDatanodeProtocolProtos.SCMStorageReport;
+        .StorageContainerDatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.StorageTypeProto;
 import org.apache.hadoop.hdds.scm.node.SCMNodeManager;
@@ -53,16 +53,17 @@ public final class TestUtils {
   public static DatanodeDetails getDatanodeDetails(SCMNodeManager nodeManager,
       String uuid) {
     DatanodeDetails datanodeDetails = getDatanodeDetails(uuid);
-    nodeManager.register(datanodeDetails.getProtoBufMessage(), null);
+    nodeManager.register(datanodeDetails, null);
     return datanodeDetails;
   }
 
   /**
    * Create Node Report object.
-   * @return SCMNodeReport
+   * @return NodeReportProto
    */
-  public static SCMNodeReport createNodeReport(List<SCMStorageReport> reports) {
-    SCMNodeReport.Builder nodeReport = SCMNodeReport.newBuilder();
+  public static NodeReportProto createNodeReport(
+      List<StorageReportProto> reports) {
+    NodeReportProto.Builder nodeReport = NodeReportProto.newBuilder();
     nodeReport.addAllStorageReport(reports);
     return nodeReport.build();
   }
@@ -71,14 +72,14 @@ public final class TestUtils {
    * Create SCM Storage Report object.
    * @return list of SCMStorageReport
    */
-  public static List<SCMStorageReport> createStorageReport(long capacity,
+  public static List<StorageReportProto> createStorageReport(long capacity,
       long used, long remaining, String path, StorageTypeProto type, String id,
       int count) {
-    List<SCMStorageReport> reportList = new ArrayList<>();
+    List<StorageReportProto> reportList = new ArrayList<>();
     for (int i = 0; i < count; i++) {
       Preconditions.checkNotNull(path);
       Preconditions.checkNotNull(id);
-      SCMStorageReport.Builder srb = SCMStorageReport.newBuilder();
+      StorageReportProto.Builder srb = StorageReportProto.newBuilder();
       srb.setStorageUuid(id).setStorageLocation(path).setCapacity(capacity)
           .setScmUsed(used).setRemaining(remaining);
       StorageTypeProto storageTypeProto =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java
index a46d7ba..8c59462 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java
@@ -24,13 +24,14 @@ import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMStorageReport;
+    .StorageContainerDatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.protocol.VersionResponse;
+import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
 import org.assertj.core.util.Preconditions;
 import org.mockito.Mockito;
@@ -370,13 +371,13 @@ public class MockNodeManager implements NodeManager {
    * Register the node if the node finds that it is not registered with any
    * SCM.
    *
-   * @param datanodeDetails DatanodeDetailsProto
-   * @param nodeReport SCMNodeReport
+   * @param datanodeDetails DatanodeDetails
+   * @param nodeReport NodeReportProto
    * @return SCMHeartbeatResponseProto
    */
   @Override
-  public SCMCommand register(HddsProtos.DatanodeDetailsProto datanodeDetails,
-                             SCMNodeReport nodeReport) {
+  public RegisteredCommand register(DatanodeDetails datanodeDetails,
+      NodeReportProto nodeReport) {
     return null;
   }
 
@@ -388,9 +389,8 @@ public class MockNodeManager implements NodeManager {
    * @return SCMheartbeat response list
    */
   @Override
-  public List<SCMCommand> sendHeartbeat(
-      HddsProtos.DatanodeDetailsProto datanodeDetails,
-      SCMNodeReport nodeReport) {
+  public List<SCMCommand> sendHeartbeat(DatanodeDetails datanodeDetails,
+      NodeReportProto nodeReport) {
     if ((datanodeDetails != null) && (nodeReport != null) && (nodeReport
         .getStorageReportCount() > 0)) {
       SCMNodeStat stat = this.nodeMetricMap.get(datanodeDetails.getUuid());
@@ -398,8 +398,9 @@ public class MockNodeManager implements NodeManager {
       long totalCapacity = 0L;
       long totalRemaining = 0L;
       long totalScmUsed = 0L;
-      List<SCMStorageReport> storageReports = nodeReport.getStorageReportList();
-      for (SCMStorageReport report : storageReports) {
+      List<StorageReportProto> storageReports = nodeReport
+          .getStorageReportList();
+      for (StorageReportProto report : storageReports) {
         totalCapacity += report.getCapacity();
         totalRemaining += report.getRemaining();
         totalScmUsed += report.getScmUsed();
@@ -407,8 +408,7 @@ public class MockNodeManager implements NodeManager {
       aggregateStat.subtract(stat);
       stat.set(totalCapacity, totalScmUsed, totalRemaining);
       aggregateStat.add(stat);
-      nodeMetricMap.put(DatanodeDetails
-          .getFromProtoBuf(datanodeDetails).getUuid(), stat);
+      nodeMetricMap.put(datanodeDetails.getUuid(), stat);
 
     }
     return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/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 f318316..ba2ab64 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
@@ -28,7 +28,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.common.SCMTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -191,8 +191,6 @@ public class TestContainerMapping {
   public void testFullContainerReport() throws IOException {
     ContainerInfo info = createContainer();
     DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
-    ContainerReportsRequestProto.reportType reportType =
-        ContainerReportsRequestProto.reportType.fullReport;
     List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports =
         new ArrayList<>();
     StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder =
@@ -209,12 +207,11 @@ public class TestContainerMapping {
 
     reports.add(ciBuilder.build());
 
-    ContainerReportsRequestProto.Builder crBuilder =
-        ContainerReportsRequestProto.newBuilder();
-    crBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage())
-        .setType(reportType).addAllReports(reports);
+    ContainerReportsProto.Builder crBuilder = ContainerReportsProto
+        .newBuilder();
+    crBuilder.addAllReports(reports);
 
-    mapping.processContainerReports(crBuilder.build());
+    mapping.processContainerReports(datanodeDetails, crBuilder.build());
 
     ContainerInfo updatedContainer =
         mapping.getContainer(info.getContainerID());
@@ -227,8 +224,6 @@ public class TestContainerMapping {
   public void testContainerCloseWithContainerReport() throws IOException {
     ContainerInfo info = createContainer();
     DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
-    ContainerReportsRequestProto.reportType reportType =
-        ContainerReportsRequestProto.reportType.fullReport;
     List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports =
         new ArrayList<>();
 
@@ -246,12 +241,11 @@ public class TestContainerMapping {
 
     reports.add(ciBuilder.build());
 
-    ContainerReportsRequestProto.Builder crBuilder =
-        ContainerReportsRequestProto.newBuilder();
-    crBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage())
-        .setType(reportType).addAllReports(reports);
+    ContainerReportsProto.Builder crBuilder =
+        ContainerReportsProto.newBuilder();
+    crBuilder.addAllReports(reports);
 
-    mapping.processContainerReports(crBuilder.build());
+    mapping.processContainerReports(datanodeDetails, crBuilder.build());
 
     ContainerInfo updatedContainer =
         mapping.getContainer(info.getContainerID());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/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 15ecbad..0a3efda 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
@@ -30,7 +30,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.common.SCMTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -199,9 +199,8 @@ public class TestContainerCloser {
 
   private void sendContainerReport(ContainerInfo info, long used) throws
       IOException {
-    ContainerReportsRequestProto.Builder
-        reports =  ContainerReportsRequestProto.newBuilder();
-    reports.setType(ContainerReportsRequestProto.reportType.fullReport);
+    ContainerReportsProto.Builder
+        reports =  ContainerReportsProto.newBuilder();
 
     StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder =
         StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder();
@@ -214,9 +213,8 @@ public class TestContainerCloser {
         .setWriteCount(100000000L)
         .setReadBytes(2000000000L)
         .setWriteBytes(2000000000L);
-    reports.setDatanodeDetails(
-        TestUtils.getDatanodeDetails().getProtoBufMessage());
     reports.addReports(ciBuilder);
-    mapping.processContainerReports(reports.build());
+    mapping.processContainerReports(TestUtils.getDatanodeDetails(),
+        reports.build());
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/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 09b6cd1..5ad28f6 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
@@ -33,7 +33,7 @@ import org.apache.hadoop.hdds.scm.container.placement.algorithms
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMStorageReport;
+    .StorageContainerDatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -133,9 +133,9 @@ public class TestContainerPlacement {
       for (DatanodeDetails datanodeDetails : datanodes) {
         String id = UUID.randomUUID().toString();
         String path = testDir.getAbsolutePath() + "/" + id;
-        List<SCMStorageReport> reports = TestUtils
+        List<StorageReportProto> reports = TestUtils
             .createStorageReport(capacity, used, remaining, path, null, id, 1);
-        nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
+        nodeManager.sendHeartbeat(datanodeDetails,
             TestUtils.createNodeReport(reports));
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java
index de87e50..2b04d6b 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.hdds.scm.node;
 
 import com.google.common.base.Supplier;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.TestUtils;
 import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
@@ -26,7 +28,7 @@ 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.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMStorageReport;
+    .StorageContainerDatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -63,8 +65,6 @@ import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DEAD;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
     .HEALTHY;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE;
-import static org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCmdType;
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.core.StringStartsWith.startsWith;
 import static org.junit.Assert.assertEquals;
@@ -144,7 +144,7 @@ public class TestNodeManager {
       for (int x = 0; x < nodeManager.getMinimumChillModeNodes(); x++) {
         DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(
             nodeManager);
-        nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
+        nodeManager.sendHeartbeat(datanodeDetails,
             null);
       }
 
@@ -191,8 +191,8 @@ public class TestNodeManager {
 
       // Need 100 nodes to come out of chill mode, only one node is sending HB.
       nodeManager.setMinimumChillModeNodes(100);
-      nodeManager.sendHeartbeat(TestUtils.getDatanodeDetails(nodeManager)
-          .getProtoBufMessage(), null);
+      nodeManager.sendHeartbeat(TestUtils.getDatanodeDetails(nodeManager),
+          null);
       GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
           100, 4 * 1000);
       assertFalse("Not enough heartbeat, Node manager should have" +
@@ -219,7 +219,7 @@ public class TestNodeManager {
 
       // Send 10 heartbeat from same node, and assert we never leave chill mode.
       for (int x = 0; x < 10; x++) {
-        nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
+        nodeManager.sendHeartbeat(datanodeDetails,
             null);
       }
 
@@ -250,7 +250,7 @@ public class TestNodeManager {
     nodeManager.close();
 
     // These should never be processed.
-    nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
+    nodeManager.sendHeartbeat(datanodeDetails,
         null);
 
     // Let us just wait for 2 seconds to prove that HBs are not processed.
@@ -274,13 +274,13 @@ public class TestNodeManager {
     DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
     String dnId = datanodeDetails.getUuidString();
     String storagePath = testDir.getAbsolutePath() + "/" + dnId;
-    List<SCMStorageReport> reports =
+    List<StorageReportProto> reports =
         TestUtils.createStorageReport(100, 10, 90, storagePath, null, dnId, 1);
     try (SCMNodeManager nodemanager = createNodeManager(conf)) {
-      nodemanager.register(datanodeDetails.getProtoBufMessage(),
+      nodemanager.register(datanodeDetails,
           TestUtils.createNodeReport(reports));
       List<SCMCommand> command = nodemanager.sendHeartbeat(
-          datanodeDetails.getProtoBufMessage(), null);
+          datanodeDetails, null);
       Assert.assertTrue(nodemanager.getAllNodes().contains(datanodeDetails));
       Assert.assertTrue("On regular HB calls, SCM responses a "
           + "datanode with an empty command list", command.isEmpty());
@@ -298,10 +298,10 @@ public class TestNodeManager {
         GenericTestUtils.waitFor(new Supplier<Boolean>() {
           @Override public Boolean get() {
             List<SCMCommand> command =
-                nodemanager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
+                nodemanager.sendHeartbeat(datanodeDetails,
                     null);
             return command.size() == 1 && command.get(0).getType()
-                .equals(SCMCmdType.reregisterCommand);
+                .equals(SCMCommandProto.Type.reregisterCommand);
           }
         }, 100, 3 * 1000);
       } catch (TimeoutException e) {
@@ -330,7 +330,7 @@ public class TestNodeManager {
       for (int x = 0; x < count; x++) {
         DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(
             nodeManager);
-        nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
+        nodeManager.sendHeartbeat(datanodeDetails,
             null);
       }
       GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
@@ -422,19 +422,19 @@ public class TestNodeManager {
       DatanodeDetails staleNode = TestUtils.getDatanodeDetails(nodeManager);
 
       // Heartbeat once
-      nodeManager.sendHeartbeat(staleNode.getProtoBufMessage(),
+      nodeManager.sendHeartbeat(staleNode,
           null);
 
       // Heartbeat all other nodes.
       for (DatanodeDetails dn : nodeList) {
-        nodeManager.sendHeartbeat(dn.getProtoBufMessage(), null);
+        nodeManager.sendHeartbeat(dn, null);
       }
 
       // Wait for 2 seconds .. and heartbeat good nodes again.
       Thread.sleep(2 * 1000);
 
       for (DatanodeDetails dn : nodeList) {
-        nodeManager.sendHeartbeat(dn.getProtoBufMessage(), null);
+        nodeManager.sendHeartbeat(dn, null);
       }
 
       // Wait for 2 seconds, wait a total of 4 seconds to make sure that the
@@ -451,7 +451,7 @@ public class TestNodeManager {
 
       // heartbeat good nodes again.
       for (DatanodeDetails dn : nodeList) {
-        nodeManager.sendHeartbeat(dn.getProtoBufMessage(), null);
+        nodeManager.sendHeartbeat(dn, null);
       }
 
       //  6 seconds is the dead window for this test , so we wait a total of
@@ -565,11 +565,11 @@ public class TestNodeManager {
       DatanodeDetails deadNode =
           TestUtils.getDatanodeDetails(nodeManager);
       nodeManager.sendHeartbeat(
-          healthyNode.getProtoBufMessage(), null);
+          healthyNode, null);
       nodeManager.sendHeartbeat(
-          staleNode.getProtoBufMessage(), null);
+          staleNode, null);
       nodeManager.sendHeartbeat(
-          deadNode.getProtoBufMessage(), null);
+          deadNode, null);
 
       // Sleep so that heartbeat processing thread gets to run.
       Thread.sleep(500);
@@ -596,15 +596,15 @@ public class TestNodeManager {
        */
 
       nodeManager.sendHeartbeat(
-          healthyNode.getProtoBufMessage(), null);
+          healthyNode, null);
       nodeManager.sendHeartbeat(
-          staleNode.getProtoBufMessage(), null);
+          staleNode, null);
       nodeManager.sendHeartbeat(
-          deadNode.getProtoBufMessage(), null);
+          deadNode, null);
 
       Thread.sleep(1500);
       nodeManager.sendHeartbeat(
-          healthyNode.getProtoBufMessage(), null);
+          healthyNode, null);
       Thread.sleep(2 * 1000);
       assertEquals(1, nodeManager.getNodeCount(HEALTHY));
 
@@ -625,12 +625,12 @@ public class TestNodeManager {
        */
 
       nodeManager.sendHeartbeat(
-          healthyNode.getProtoBufMessage(), null);
+          healthyNode, null);
       nodeManager.sendHeartbeat(
-          staleNode.getProtoBufMessage(), null);
+          staleNode, null);
       Thread.sleep(1500);
       nodeManager.sendHeartbeat(
-          healthyNode.getProtoBufMessage(), null);
+          healthyNode, null);
       Thread.sleep(2 * 1000);
 
       // 3.5 seconds have elapsed for stale node, so it moves into Stale.
@@ -664,11 +664,11 @@ public class TestNodeManager {
        * back all the nodes in healthy state.
        */
       nodeManager.sendHeartbeat(
-          healthyNode.getProtoBufMessage(), null);
+          healthyNode, null);
       nodeManager.sendHeartbeat(
-          staleNode.getProtoBufMessage(), null);
+          staleNode, null);
       nodeManager.sendHeartbeat(
-          deadNode.getProtoBufMessage(), null);
+          deadNode, null);
       Thread.sleep(500);
       //Assert all nodes are healthy.
       assertEquals(3, nodeManager.getAllNodes().size());
@@ -689,7 +689,7 @@ public class TestNodeManager {
                                 int sleepDuration) throws InterruptedException {
     while (!Thread.currentThread().isInterrupted()) {
       for (DatanodeDetails dn : list) {
-        manager.sendHeartbeat(dn.getProtoBufMessage(), null);
+        manager.sendHeartbeat(dn, null);
       }
       Thread.sleep(sleepDuration);
     }
@@ -775,7 +775,7 @@ public class TestNodeManager {
       // No Thread just one time HBs the node manager, so that these will be
       // marked as dead nodes eventually.
       for (DatanodeDetails dn : deadNodeList) {
-        nodeManager.sendHeartbeat(dn.getProtoBufMessage(), null);
+        nodeManager.sendHeartbeat(dn, null);
       }
 
 
@@ -940,7 +940,7 @@ public class TestNodeManager {
       DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(
           nodeManager);
       nodeManager.sendHeartbeat(
-          datanodeDetails.getProtoBufMessage(), null);
+          datanodeDetails, null);
       String status = nodeManager.getChillModeStatus();
       Assert.assertThat(status, containsString("Still in chill " +
           "mode, waiting on nodes to report in."));
@@ -967,8 +967,7 @@ public class TestNodeManager {
       // Assert that node manager force enter cannot be overridden by nodes HBs.
       for (int x = 0; x < 20; x++) {
         DatanodeDetails datanode = TestUtils.getDatanodeDetails(nodeManager);
-        nodeManager.sendHeartbeat(datanode.getProtoBufMessage(),
-            null);
+        nodeManager.sendHeartbeat(datanode, null);
       }
 
       Thread.sleep(500);
@@ -1009,10 +1008,10 @@ public class TestNodeManager {
         String dnId = datanodeDetails.getUuidString();
         long free = capacity - used;
         String storagePath = testDir.getAbsolutePath() + "/" + dnId;
-        List<SCMStorageReport> reports = TestUtils
+        List<StorageReportProto> reports = TestUtils
             .createStorageReport(capacity, used, free, storagePath,
                 null, dnId, 1);
-        nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
+        nodeManager.sendHeartbeat(datanodeDetails,
             TestUtils.createNodeReport(reports));
       }
       GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
@@ -1058,11 +1057,11 @@ public class TestNodeManager {
         long scmUsed = x * usedPerHeartbeat;
         long remaining = capacity - scmUsed;
         String storagePath = testDir.getAbsolutePath() + "/" + dnId;
-        List<SCMStorageReport> reports = TestUtils
+        List<StorageReportProto> reports = TestUtils
             .createStorageReport(capacity, scmUsed, remaining, storagePath,
                 null, dnId, 1);
 
-        nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
+        nodeManager.sendHeartbeat(datanodeDetails,
             TestUtils.createNodeReport(reports));
         Thread.sleep(100);
       }
@@ -1140,10 +1139,10 @@ public class TestNodeManager {
 
       // Send a new report to bring the dead node back to healthy
       String storagePath = testDir.getAbsolutePath() + "/" + dnId;
-      List<SCMStorageReport> reports = TestUtils
+      List<StorageReportProto> reports = TestUtils
           .createStorageReport(capacity, expectedScmUsed, expectedRemaining,
               storagePath, null, dnId, 1);
-      nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
+      nodeManager.sendHeartbeat(datanodeDetails,
           TestUtils.createNodeReport(reports));
 
       // Wait up to 5 seconds so that the dead node becomes healthy

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeStorageStatMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeStorageStatMap.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeStorageStatMap.java
index b824412..072dee7 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeStorageStatMap.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeStorageStatMap.java
@@ -21,9 +21,9 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.protocol.proto.
-    StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+    StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.hdds.protocol.proto.
-    StorageContainerDatanodeProtocolProtos.SCMStorageReport;
+    StorageContainerDatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdds.scm.TestUtils;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -134,7 +134,7 @@ public class TestSCMNodeStorageStatMap {
   @Test
   public void testProcessNodeReportCheckOneNode() throws IOException {
     UUID key = getFirstKey();
-    List<SCMStorageReport> reportList = new ArrayList<>();
+    List<StorageReportProto> reportList = new ArrayList<>();
     Set<StorageLocationReport> reportSet = testData.get(key);
     SCMNodeStorageStatMap map = new SCMNodeStorageStatMap(conf);
     map.insertNewDatanode(key, reportSet);
@@ -146,16 +146,16 @@ public class TestSCMNodeStorageStatMap {
     long reportCapacity = report.getCapacity();
     long reportScmUsed = report.getScmUsed();
     long reportRemaining = report.getRemaining();
-    List<SCMStorageReport> reports = TestUtils
+    List<StorageReportProto> reports = TestUtils
         .createStorageReport(reportCapacity, reportScmUsed, reportRemaining,
             path, null, storageId, 1);
     StorageReportResult result =
         map.processNodeReport(key, TestUtils.createNodeReport(reports));
     Assert.assertEquals(result.getStatus(),
         SCMNodeStorageStatMap.ReportStatus.ALL_IS_WELL);
-    StorageContainerDatanodeProtocolProtos.SCMNodeReport.Builder nrb =
-        SCMNodeReport.newBuilder();
-    SCMStorageReport srb = reportSet.iterator().next().getProtoBufMessage();
+    StorageContainerDatanodeProtocolProtos.NodeReportProto.Builder nrb =
+        NodeReportProto.newBuilder();
+    StorageReportProto srb = reportSet.iterator().next().getProtoBufMessage();
     reportList.add(srb);
     result = map.processNodeReport(key, TestUtils.createNodeReport(reportList));
     Assert.assertEquals(result.getStatus(),
@@ -168,7 +168,7 @@ public class TestSCMNodeStorageStatMap {
     Assert.assertEquals(result.getStatus(),
         SCMNodeStorageStatMap.ReportStatus.STORAGE_OUT_OF_SPACE);
     // Mark a disk failed 
-    SCMStorageReport srb2 = SCMStorageReport.newBuilder()
+    StorageReportProto srb2 = StorageReportProto.newBuilder()
         .setStorageUuid(UUID.randomUUID().toString())
         .setStorageLocation(srb.getStorageLocation()).setScmUsed(reportCapacity)
         .setCapacity(reportCapacity).setRemaining(0).setFailed(true).build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/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 1d92cdc..34779da 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
@@ -20,22 +20,21 @@ import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.scm.TestUtils;
 import org.apache.hadoop.hdds.scm.VersionInfo;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos;
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsResponseProto;
+    .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto;
+    .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMStorageReport;
+    .StorageContainerDatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
 import org.apache.hadoop.ipc.RPC;
@@ -200,7 +199,7 @@ public class TestEndPoint {
     DatanodeDetails nodeToRegister = getDatanodeDetails();
     try (EndpointStateMachine rpcEndPoint = createEndpoint(
         SCMTestUtils.getConf(), serverAddress, 1000)) {
-      SCMRegisteredCmdResponseProto responseProto = rpcEndPoint.getEndPoint()
+      SCMRegisteredResponseProto responseProto = rpcEndPoint.getEndPoint()
           .register(nodeToRegister.getProtoBufMessage(), TestUtils
                   .createNodeReport(
                       getStorageReports(nodeToRegister.getUuidString())),
@@ -215,7 +214,7 @@ public class TestEndPoint {
     }
   }
 
-  private List<SCMStorageReport> getStorageReports(String id) {
+  private List<StorageReportProto> getStorageReports(String id) {
     String storagePath = testDir.getAbsolutePath() + "/" + id;
     return TestUtils.createStorageReport(100, 10, 90, storagePath, null, id, 1);
   }
@@ -293,9 +292,14 @@ public class TestEndPoint {
              createEndpoint(SCMTestUtils.getConf(),
                  serverAddress, 1000)) {
       String storageId = UUID.randomUUID().toString();
+      SCMHeartbeatRequestProto request = SCMHeartbeatRequestProto.newBuilder()
+          .setDatanodeDetails(dataNode.getProtoBufMessage())
+          .setNodeReport(TestUtils.createNodeReport(
+              getStorageReports(storageId)))
+          .build();
+
       SCMHeartbeatResponseProto responseProto = rpcEndPoint.getEndPoint()
-          .sendHeartbeat(dataNode.getProtoBufMessage(),
-              TestUtils.createNodeReport(getStorageReports(storageId)));
+          .sendHeartbeat(request);
       Assert.assertNotNull(responseProto);
       Assert.assertEquals(0, responseProto.getCommandsCount());
     }
@@ -361,86 +365,11 @@ public class TestEndPoint {
         lessThanOrEqualTo(rpcTimeout + tolerance));
   }
 
-  /**
-   * Returns a new container report.
-   * @return
-   */
-  ContainerReport getRandomContainerReport() {
-    return new ContainerReport(RandomUtils.nextLong(),
-        DigestUtils.sha256Hex("Random"));
-  }
-
-  /**
-   * Creates dummy container reports.
-   * @param count - The number of closed containers to create.
-   * @return ContainerReportsProto
-   */
-  StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto
-      createDummyContainerReports(int count) {
-    StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto.Builder
-        reportsBuilder = StorageContainerDatanodeProtocolProtos
-        .ContainerReportsRequestProto.newBuilder();
-    for (int x = 0; x < count; x++) {
-      reportsBuilder.addReports(getRandomContainerReport()
-          .getProtoBufMessage());
-    }
-    reportsBuilder.setDatanodeDetails(getDatanodeDetails()
-        .getProtoBufMessage());
-    reportsBuilder.setType(StorageContainerDatanodeProtocolProtos
-        .ContainerReportsRequestProto.reportType.fullReport);
-    return reportsBuilder.build();
-  }
-
-  /**
-   * Tests that rpcEndpoint sendContainerReport works as expected.
-   * @throws Exception
-   */
-  @Test
-  public void testContainerReportSend() throws Exception {
-    final int count = 1000;
-    scmServerImpl.reset();
-    try (EndpointStateMachine rpcEndPoint =
-             createEndpoint(SCMTestUtils.getConf(),
-                 serverAddress, 1000)) {
-      ContainerReportsResponseProto responseProto = rpcEndPoint
-          .getEndPoint().sendContainerReport(createDummyContainerReports(
-              count));
-      Assert.assertNotNull(responseProto);
-    }
-    Assert.assertEquals(1, scmServerImpl.getContainerReportsCount());
-    Assert.assertEquals(count, scmServerImpl.getContainerCount());
-  }
-
-
-  /**
-   * Tests that rpcEndpoint sendContainerReport works as expected.
-   * @throws Exception
-   */
-  @Test
-  public void testContainerReport() throws Exception {
-    final int count = 1000;
-    scmServerImpl.reset();
-    try (EndpointStateMachine rpcEndPoint =
-             createEndpoint(SCMTestUtils.getConf(),
-                 serverAddress, 1000)) {
-      ContainerReportsResponseProto responseProto = rpcEndPoint
-          .getEndPoint().sendContainerReport(createContainerReport(count,
-              null));
-      Assert.assertNotNull(responseProto);
-    }
-    Assert.assertEquals(1, scmServerImpl.getContainerReportsCount());
-    Assert.assertEquals(count, scmServerImpl.getContainerCount());
-    final long expectedKeyCount = count * 1000;
-    Assert.assertEquals(expectedKeyCount, scmServerImpl.getKeyCount());
-    final long expectedBytesUsed = count * OzoneConsts.GB * 2;
-    Assert.assertEquals(expectedBytesUsed, scmServerImpl.getBytesUsed());
-  }
-
-  private ContainerReportsRequestProto createContainerReport(
+  private ContainerReportsProto createContainerReport(
       int count, DatanodeDetails datanodeDetails) {
-    StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto.Builder
+    StorageContainerDatanodeProtocolProtos.ContainerReportsProto.Builder
         reportsBuilder = StorageContainerDatanodeProtocolProtos
-        .ContainerReportsRequestProto.newBuilder();
+        .ContainerReportsProto.newBuilder();
     for (int x = 0; x < count; x++) {
       long containerID = RandomUtils.nextLong();
       ContainerReport report = new ContainerReport(containerID,
@@ -455,14 +384,6 @@ public class TestEndPoint {
 
       reportsBuilder.addReports(report.getProtoBufMessage());
     }
-    if(datanodeDetails == null) {
-      reportsBuilder.setDatanodeDetails(getDatanodeDetails()
-          .getProtoBufMessage());
-    } else {
-      reportsBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage());
-    }
-    reportsBuilder.setType(StorageContainerDatanodeProtocolProtos
-        .ContainerReportsRequestProto.reportType.fullReport);
     return reportsBuilder.build();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/201440b9/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerSupervisor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerSupervisor.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerSupervisor.java
deleted file mode 100644
index e197886..0000000
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerSupervisor.java
+++ /dev/null
@@ -1,275 +0,0 @@
-/**
- * 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.replication;
-
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hdds.scm.TestUtils;
-import org.apache.hadoop.hdds.scm.container.replication.ContainerSupervisor;
-import org.apache.hadoop.hdds.scm.container.replication.InProgressPool;
-import org.apache.hadoop.hdds.scm.node.CommandQueue;
-import org.apache.hadoop.hdds.scm.node.NodeManager;
-import org.apache.hadoop.hdds.scm.node.NodePoolManager;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
-import org.apache.hadoop.ozone.container.common.SCMTestUtils;
-import org.apache.hadoop.ozone.container.testutils
-    .ReplicationDatanodeStateManager;
-import org.apache.hadoop.ozone.container.testutils.ReplicationNodeManagerMock;
-import org.apache.hadoop.ozone.container.testutils
-    .ReplicationNodePoolManagerMock;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.event.Level;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CONTAINER_REPORTS_WAIT_TIMEOUT;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CONTAINER_REPORT_PROCESSING_INTERVAL;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
-    .HEALTHY;
-import static org.apache.ratis.shaded.com.google.common.util.concurrent
-    .Uninterruptibles.sleepUninterruptibly;
-
-/**
- * Tests for the container manager.
- */
-public class TestContainerSupervisor {
-  final static String POOL_NAME_TEMPLATE = "Pool%d";
-  static final int MAX_DATANODES = 72;
-  static final int POOL_SIZE = 24;
-  static final int POOL_COUNT = 3;
-  private LogCapturer logCapturer = LogCapturer.captureLogs(
-      LogFactory.getLog(ContainerSupervisor.class));
-  private List<DatanodeDetails> datanodes = new LinkedList<>();
-  private NodeManager nodeManager;
-  private NodePoolManager poolManager;
-  private CommandQueue commandQueue;
-  private ContainerSupervisor containerSupervisor;
-  private ReplicationDatanodeStateManager datanodeStateManager;
-
-  @After
-  public void tearDown() throws Exception {
-    logCapturer.stopCapturing();
-    GenericTestUtils.setLogLevel(ContainerSupervisor.LOG, Level.INFO);
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    GenericTestUtils.setLogLevel(ContainerSupervisor.LOG, Level.DEBUG);
-    Map<DatanodeDetails, NodeState> nodeStateMap = new HashMap<>();
-    // We are setting up 3 pools with 24 nodes each in this cluster.
-    // First we create 72 Datanodes.
-    for (int x = 0; x < MAX_DATANODES; x++) {
-      DatanodeDetails datanode = TestUtils.getDatanodeDetails();
-      datanodes.add(datanode);
-      nodeStateMap.put(datanode, HEALTHY);
-    }
-
-    commandQueue = new CommandQueue();
-
-    // All nodes in this cluster are healthy for time being.
-    nodeManager = new ReplicationNodeManagerMock(nodeStateMap, commandQueue);
-    poolManager = new ReplicationNodePoolManagerMock();
-
-
-    Assert.assertEquals("Max datanodes should be equal to POOL_SIZE * " +
-        "POOL_COUNT", POOL_COUNT * POOL_SIZE, MAX_DATANODES);
-
-    // Start from 1 instead of zero so we can multiply and get the node index.
-    for (int y = 1; y <= POOL_COUNT; y++) {
-      String poolName = String.format(POOL_NAME_TEMPLATE, y);
-      for (int z = 0; z < POOL_SIZE; z++) {
-        DatanodeDetails id = datanodes.get(y * z);
-        poolManager.addNode(poolName, id);
-      }
-    }
-    OzoneConfiguration config = SCMTestUtils.getOzoneConf();
-    config.setTimeDuration(OZONE_SCM_CONTAINER_REPORTS_WAIT_TIMEOUT, 2,
-        TimeUnit.SECONDS);
-    config.setTimeDuration(OZONE_SCM_CONTAINER_REPORT_PROCESSING_INTERVAL, 1,
-        TimeUnit.SECONDS);
-    containerSupervisor = new ContainerSupervisor(config,
-        nodeManager, poolManager);
-    datanodeStateManager = new ReplicationDatanodeStateManager(nodeManager,
-        poolManager);
-    // Sleep for one second to make sure all threads get time to run.
-    sleepUninterruptibly(1, TimeUnit.SECONDS);
-  }
-
-  @Test
-  /**
-   * Asserts that at least one pool is picked up for processing.
-   */
-  public void testAssertPoolsAreProcessed() {
-    // This asserts that replication manager has started processing at least
-    // one pool.
-    Assert.assertTrue(containerSupervisor.getInProgressPoolCount() > 0);
-
-    // Since all datanodes are flagged as healthy in this test, for each
-    // datanode we must have queued a command.
-    Assert.assertEquals("Commands are in queue :",
-        POOL_SIZE * containerSupervisor.getInProgressPoolCount(),
-        commandQueue.getCommandsInQueue());
-  }
-
-  @Test
-  /**
-   * This test sends container reports for 2 containers to a pool in progress.
-   * Asserts that we are able to find a container with single replica and do
-   * not find container with 3 replicas.
-   */
-  public void testDetectSingleContainerReplica() throws TimeoutException,
-      InterruptedException {
-    long singleNodeContainerID = 9001;
-    long threeNodeContainerID = 9003;
-    InProgressPool ppool = containerSupervisor.getInProcessPoolList().get(0);
-    // Only single datanode reporting that "SingleNodeContainer" exists.
-    List<ContainerReportsRequestProto> clist =
-        datanodeStateManager.getContainerReport(singleNodeContainerID,
-            ppool.getPool().getPoolName(), 1);
-    ppool.handleContainerReport(clist.get(0));
-
-    // Three nodes are going to report that ThreeNodeContainer  exists.
-    clist = datanodeStateManager.getContainerReport(threeNodeContainerID,
-        ppool.getPool().getPoolName(), 3);
-
-    for (ContainerReportsRequestProto reportsProto : clist) {
-      ppool.handleContainerReport(reportsProto);
-    }
-    GenericTestUtils.waitFor(() -> ppool.getContainerProcessedCount() == 4,
-        200, 1000);
-    ppool.setDoneProcessing();
-
-    List<Map.Entry<Long, Integer>> containers = ppool.filterContainer(p -> p
-        .getValue() == 1);
-    Assert.assertEquals(singleNodeContainerID,
-        containers.get(0).getKey().longValue());
-    int count = containers.get(0).getValue();
-    Assert.assertEquals(1L, count);
-  }
-
-  @Test
-  /**
-   * We create three containers, Normal,OveReplicated and WayOverReplicated
-   * containers. This test asserts that we are able to find the
-   * over replicated containers.
-   */
-  public void testDetectOverReplica() throws TimeoutException,
-      InterruptedException {
-    long normalContainerID = 9000;
-    long overReplicatedContainerID = 9001;
-    long wayOverReplicatedContainerID = 9002;
-    InProgressPool ppool = containerSupervisor.getInProcessPoolList().get(0);
-
-    List<ContainerReportsRequestProto> clist =
-        datanodeStateManager.getContainerReport(normalContainerID,
-            ppool.getPool().getPoolName(), 3);
-    ppool.handleContainerReport(clist.get(0));
-
-    clist = datanodeStateManager.getContainerReport(overReplicatedContainerID,
-        ppool.getPool().getPoolName(), 4);
-
-    for (ContainerReportsRequestProto reportsProto : clist) {
-      ppool.handleContainerReport(reportsProto);
-    }
-
-    clist = datanodeStateManager.getContainerReport(
-        wayOverReplicatedContainerID, ppool.getPool().getPoolName(), 7);
-
-    for (ContainerReportsRequestProto reportsProto : clist) {
-      ppool.handleContainerReport(reportsProto);
-    }
-
-    // We ignore container reports from the same datanodes.
-    // it is possible that these each of these containers get placed
-    // on same datanodes, so allowing for 4 duplicates in the set of 14.
-    GenericTestUtils.waitFor(() -> ppool.getContainerProcessedCount() > 10,
-        200, 1000);
-    ppool.setDoneProcessing();
-
-    List<Map.Entry<Long, Integer>> containers = ppool.filterContainer(p -> p
-        .getValue() > 3);
-    Assert.assertEquals(2, containers.size());
-  }
-
-  @Test
-  /**
-   * This test verifies that all pools are picked up for replica processing.
-   *
-   */
-  public void testAllPoolsAreProcessed() throws TimeoutException,
-      InterruptedException {
-    // Verify that we saw all three pools being picked up for processing.
-    GenericTestUtils.waitFor(() -> containerSupervisor.getPoolProcessCount()
-        >= 3, 200, 15 * 1000);
-    Assert.assertTrue(logCapturer.getOutput().contains("Pool1") &&
-        logCapturer.getOutput().contains("Pool2") &&
-        logCapturer.getOutput().contains("Pool3"));
-  }
-
-  @Test
-  /**
-   * Adds a new pool and tests that we are able to pick up that new pool for
-   * processing as well as handle container reports for datanodes in that pool.
-   * @throws TimeoutException
-   * @throws InterruptedException
-   */
-  public void testAddingNewPoolWorks()
-      throws TimeoutException, InterruptedException, IOException {
-    LogCapturer inProgressLog = LogCapturer.captureLogs(
-        LogFactory.getLog(InProgressPool.class));
-    GenericTestUtils.setLogLevel(InProgressPool.LOG, Level.DEBUG);
-    try {
-      DatanodeDetails id = TestUtils.getDatanodeDetails();
-      ((ReplicationNodeManagerMock) (nodeManager)).addNode(id, HEALTHY);
-      poolManager.addNode("PoolNew", id);
-      GenericTestUtils.waitFor(() ->
-              logCapturer.getOutput().contains("PoolNew"),
-          200, 15 * 1000);
-
-      long newContainerID = 7001;
-      // Assert that we are able to send a container report to this new
-      // pool and datanode.
-      List<ContainerReportsRequestProto> clist =
-          datanodeStateManager.getContainerReport(newContainerID,
-              "PoolNew", 1);
-      containerSupervisor.handleContainerReport(clist.get(0));
-      GenericTestUtils.waitFor(() ->
-          inProgressLog.getOutput()
-              .contains(Long.toString(newContainerID)) && inProgressLog
-              .getOutput().contains(id.getUuidString()),
-          200, 10 * 1000);
-    } finally {
-      inProgressLog.stopCapturing();
-    }
-  }
-}


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


[42/50] [abbrv] hadoop git commit: HDFS-13626. Fix incorrect username when deny the setOwner operation. Contributed by Zsolt Venczel.

Posted by ha...@apache.org.
HDFS-13626. Fix incorrect username when deny the setOwner operation. Contributed by Zsolt Venczel.


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

Branch: refs/heads/HDDS-48
Commit: b24098bc8ffe976d662acabc168e20eac8cc8460
Parents: 5f6769f
Author: Yiqun Lin <yq...@apache.org>
Authored: Wed May 30 16:52:21 2018 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Wed May 30 16:52:21 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hdfs/server/namenode/FSDirAttrOp.java       |  4 ++--
 .../org/apache/hadoop/security/TestPermission.java     | 13 ++++++++-----
 2 files changed, 10 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b24098bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 406fe80..1dbee96 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -82,12 +82,12 @@ public class FSDirAttrOp {
       fsd.checkOwner(pc, iip);
       if (!pc.isSuperUser()) {
         if (username != null && !pc.getUser().equals(username)) {
-          throw new AccessControlException("User " + username
+          throw new AccessControlException("User " + pc.getUser()
               + " is not a super user (non-super user cannot change owner).");
         }
         if (group != null && !pc.isMemberOfGroup(group)) {
           throw new AccessControlException(
-              "User " + username + " does not belong to " + group);
+              "User " + pc.getUser() + " does not belong to " + group);
         }
       }
       unprotectedSetOwner(fsd, iip, username, group);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b24098bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java
index 813ac5a..388e7f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java
@@ -337,7 +337,8 @@ public class TestPermission {
       fail("Expect ACE when a non-super user tries to change a file to a " +
           "group where the user does not belong.");
     } catch (AccessControlException e) {
-      assertThat(e.getMessage(), startsWith("User null does not belong to"));
+      assertThat(e.getMessage(), startsWith("User " +
+          userfs.getFileStatus(file).getOwner() + " does not belong to"));
     }
   }
 
@@ -371,8 +372,9 @@ public class TestPermission {
       userfs.setOwner(file, NOUSER, null);
       fail("Expect ACE when a non-super user tries to change owner");
     } catch (AccessControlException e) {
-      assertThat(e.getMessage(), startsWith("User " + NOUSER
-          + " is not a super user (non-super user cannot change owner)"));
+      assertThat(e.getMessage(), startsWith("User " +
+          userfs.getFileStatus(file).getOwner() +
+          " is not a super user (non-super user cannot change owner)"));
     }
   }
 
@@ -397,8 +399,9 @@ public class TestPermission {
       fail("Expect ACE or FNFE when a non-super user tries to change owner " +
           "for a non-existent file");
     } catch (AccessControlException e) {
-      assertThat(e.getMessage(), startsWith("User " + NOUSER
-          + " is not a super user (non-super user cannot change owner)"));
+      assertThat(e.getMessage(), startsWith("User " +
+          userfs.getFileStatus(file).getOwner() +
+          " is not a super user (non-super user cannot change owner)"));
     } catch (FileNotFoundException e) {
     }
   }


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


[17/50] [abbrv] hadoop git commit: YARN-8213. Add Capacity Scheduler performance metrics. (Weiwei Yang via wangda)

Posted by ha...@apache.org.
YARN-8213. Add Capacity Scheduler performance metrics. (Weiwei Yang via wangda)

Change-Id: Ieea6f3eeb83c90cd74233fea896f0fcd0f325d5f


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

Branch: refs/heads/HDDS-48
Commit: f24c842d52e166e8566337ef93c96438f1c870d8
Parents: 8605a38
Author: Wangda Tan <wa...@apache.org>
Authored: Fri May 25 21:53:20 2018 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri May 25 21:53:20 2018 -0700

----------------------------------------------------------------------
 .../server/resourcemanager/ResourceManager.java |   1 +
 .../scheduler/AbstractYarnScheduler.java        |   5 +
 .../scheduler/ResourceScheduler.java            |   5 +
 .../scheduler/capacity/CapacityScheduler.java   |  31 ++++-
 .../capacity/CapacitySchedulerMetrics.java      | 119 +++++++++++++++++++
 .../TestCapacitySchedulerMetrics.java           | 110 +++++++++++++++++
 6 files changed, 269 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24c842d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.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/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index 05745ec..c533111 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -1216,6 +1216,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
   void reinitialize(boolean initialize) {
     ClusterMetrics.destroy();
     QueueMetrics.clearQueueMetrics();
+    getResourceScheduler().resetSchedulerMetrics();
     if (initialize) {
       resetRMContext();
       createAndInitActiveServices(true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24c842d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.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/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index b2747f7..18c7b4e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -1464,4 +1464,9 @@ public abstract class AbstractYarnScheduler
       SchedulingRequest schedulingRequest, SchedulerNode schedulerNode) {
     return false;
   }
+
+  @Override
+  public void resetSchedulerMetrics() {
+    // reset scheduler metrics
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24c842d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceScheduler.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/ResourceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceScheduler.java
index 5a56ac7..dcb6edd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceScheduler.java
@@ -71,4 +71,9 @@ public interface ResourceScheduler extends YarnScheduler, Recoverable {
    */
   boolean attemptAllocationOnNode(SchedulerApplicationAttempt appAttempt,
       SchedulingRequest schedulingRequest, SchedulerNode schedulerNode);
+
+  /**
+   * Reset scheduler metrics.
+   */
+  void resetSchedulerMetrics();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24c842d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.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/CapacityScheduler.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/CapacityScheduler.java
index 162d3bb..1c9bf6b 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/CapacityScheduler.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/CapacityScheduler.java
@@ -1252,6 +1252,7 @@ public class CapacityScheduler extends
 
   @Override
   protected void nodeUpdate(RMNode rmNode) {
+    long begin = System.nanoTime();
     try {
       readLock.lock();
       setLastNodeUpdateTime(Time.now());
@@ -1279,6 +1280,9 @@ public class CapacityScheduler extends
         writeLock.unlock();
       }
     }
+
+    long latency = System.nanoTime() - begin;
+    CapacitySchedulerMetrics.getMetrics().addNodeUpdate(latency);
   }
 
   /**
@@ -1643,17 +1647,28 @@ public class CapacityScheduler extends
       return null;
     }
 
+    long startTime = System.nanoTime();
+
     // Backward compatible way to make sure previous behavior which allocation
     // driven by node heartbeat works.
     FiCaSchedulerNode node = CandidateNodeSetUtils.getSingleNode(candidates);
 
     // We have two different logics to handle allocation on single node / multi
     // nodes.
+    CSAssignment assignment;
     if (null != node) {
-      return allocateContainerOnSingleNode(candidates, node, withNodeHeartbeat);
+      assignment = allocateContainerOnSingleNode(candidates,
+          node, withNodeHeartbeat);
     } else{
-      return allocateContainersOnMultiNodes(candidates);
+      assignment = allocateContainersOnMultiNodes(candidates);
+    }
+
+    if (assignment != null && assignment.getAssignmentInformation() != null
+        && assignment.getAssignmentInformation().getNumAllocations() > 0) {
+      long allocateTime = System.nanoTime() - startTime;
+      CapacitySchedulerMetrics.getMetrics().addAllocate(allocateTime);
     }
+    return assignment;
   }
 
   @Override
@@ -2806,6 +2821,7 @@ public class CapacityScheduler extends
   @Override
   public boolean tryCommit(Resource cluster, ResourceCommitRequest r,
       boolean updatePending) {
+    long commitStart = System.nanoTime();
     ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request =
         (ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode>) r;
 
@@ -2844,9 +2860,15 @@ public class CapacityScheduler extends
       if (app != null && attemptId.equals(app.getApplicationAttemptId())) {
         if (app.accept(cluster, request, updatePending)
             && app.apply(cluster, request, updatePending)) {
+          long commitSuccess = System.nanoTime() - commitStart;
+          CapacitySchedulerMetrics.getMetrics()
+              .addCommitSuccess(commitSuccess);
           LOG.info("Allocation proposal accepted");
           isSuccess = true;
         } else{
+          long commitFailed = System.nanoTime() - commitStart;
+          CapacitySchedulerMetrics.getMetrics()
+              .addCommitFailure(commitFailed);
           LOG.info("Failed to accept allocation proposal");
         }
 
@@ -3029,4 +3051,9 @@ public class CapacityScheduler extends
     }
     return autoCreatedLeafQueue;
   }
+
+  @Override
+  public void resetSchedulerMetrics() {
+    CapacitySchedulerMetrics.destroy();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24c842d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerMetrics.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/CapacitySchedulerMetrics.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/CapacitySchedulerMetrics.java
new file mode 100644
index 0000000..5f8988b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerMetrics.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
+ *
+ *     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.server.resourcemanager.scheduler.capacity;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableRate;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.metrics2.lib.Interns.info;
+
+/**
+ * Metrics for capacity scheduler.
+ */
+@InterfaceAudience.Private
+@Metrics(context="yarn")
+public class CapacitySchedulerMetrics {
+
+  private static AtomicBoolean isInitialized = new AtomicBoolean(false);
+
+  private static final MetricsInfo RECORD_INFO =
+      info("CapacitySchedulerMetrics",
+          "Metrics for the Yarn Capacity Scheduler");
+
+  @Metric("Scheduler allocate containers") MutableRate allocate;
+  @Metric("Scheduler commit success") MutableRate commitSuccess;
+  @Metric("Scheduler commit failure") MutableRate commitFailure;
+  @Metric("Scheduler node update") MutableRate nodeUpdate;
+
+  private static volatile CapacitySchedulerMetrics INSTANCE = null;
+  private static MetricsRegistry registry;
+
+  public static CapacitySchedulerMetrics getMetrics() {
+    if(!isInitialized.get()){
+      synchronized (CapacitySchedulerMetrics.class) {
+        if(INSTANCE == null){
+          INSTANCE = new CapacitySchedulerMetrics();
+          registerMetrics();
+          isInitialized.set(true);
+        }
+      }
+    }
+    return INSTANCE;
+  }
+
+  private static void registerMetrics() {
+    registry = new MetricsRegistry(RECORD_INFO);
+    registry.tag(RECORD_INFO, "ResourceManager");
+    MetricsSystem ms = DefaultMetricsSystem.instance();
+    if (ms != null) {
+      ms.register("CapacitySchedulerMetrics",
+          "Metrics for the Yarn Capacity Scheduler", INSTANCE);
+    }
+  }
+
+  @VisibleForTesting
+  public synchronized static void destroy() {
+    isInitialized.set(false);
+    INSTANCE = null;
+    MetricsSystem ms = DefaultMetricsSystem.instance();
+    if (ms != null) {
+      ms.unregisterSource("CapacitySchedulerMetrics");
+    }
+  }
+
+  public void addAllocate(long latency) {
+    this.allocate.add(latency);
+  }
+
+  public void addCommitSuccess(long latency) {
+    this.commitSuccess.add(latency);
+  }
+
+  public void addCommitFailure(long latency) {
+    this.commitFailure.add(latency);
+  }
+
+  public void addNodeUpdate(long latency) {
+    this.nodeUpdate.add(latency);
+  }
+
+  @VisibleForTesting
+  public long getNumOfNodeUpdate() {
+    return this.nodeUpdate.lastStat().numSamples();
+  }
+
+  @VisibleForTesting
+  public long getNumOfAllocates() {
+    return this.allocate.lastStat().numSamples();
+  }
+
+  @VisibleForTesting
+  public long getNumOfCommitSuccess() {
+    return this.commitSuccess.lastStat().numSamples();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24c842d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestCapacitySchedulerMetrics.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/TestCapacitySchedulerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestCapacitySchedulerMetrics.java
new file mode 100644
index 0000000..eaa966a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestCapacitySchedulerMetrics.java
@@ -0,0 +1,110 @@
+/**
+ * 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.server.resourcemanager;
+
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+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.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerMetrics;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Test class for CS metrics.
+ */
+public class TestCapacitySchedulerMetrics {
+
+  private MockRM rm;
+
+  @Test
+  public void testCSMetrics() throws Exception {
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.setBoolean(
+        CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_ENABLE, true);
+
+    RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
+    mgr.init(conf);
+    rm = new MockRM(conf) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm.getRMContext().setNodeLabelManager(mgr);
+    rm.start();
+
+    MockNM nm1 = rm.registerNode("host1:1234", 2048);
+    MockNM nm2 = rm.registerNode("host2:1234", 2048);
+    nm1.nodeHeartbeat(true);
+    nm2.nodeHeartbeat(true);
+
+    CapacitySchedulerMetrics csMetrics = CapacitySchedulerMetrics.getMetrics();
+    Assert.assertNotNull(csMetrics);
+    try {
+      GenericTestUtils.waitFor(()
+          -> csMetrics.getNumOfNodeUpdate() == 2, 100, 3000);
+    } catch(TimeoutException e) {
+      Assert.fail("CS metrics not updated on node-update events.");
+    }
+
+    Assert.assertEquals(0, csMetrics.getNumOfAllocates());
+    Assert.assertEquals(0, csMetrics.getNumOfCommitSuccess());
+
+    RMApp rmApp = rm.submitApp(1024, "app", "user", null, false,
+        "default", 1, null, null, false);
+    MockAM am = MockRM.launchAMWhenAsyncSchedulingEnabled(rmApp, rm);
+    am.registerAppAttempt();
+    am.allocate("*", 1024, 1, new ArrayList<>());
+
+    nm1.nodeHeartbeat(true);
+    nm2.nodeHeartbeat(true);
+
+    // Verify HB metrics updated
+    try {
+      GenericTestUtils.waitFor(()
+          -> csMetrics.getNumOfNodeUpdate() == 4, 100, 3000);
+    } catch(TimeoutException e) {
+      Assert.fail("CS metrics not updated on node-update events.");
+    }
+
+    // For async mode, the number of alloc might be bigger than 1
+    Assert.assertTrue(csMetrics.getNumOfAllocates() > 0);
+    // But there will be only 2 successful commit (1 AM + 1 task)
+    Assert.assertEquals(2, csMetrics.getNumOfCommitSuccess());
+  }
+
+  @After
+  public void tearDown() {
+    if (rm != null) {
+      rm.stop();
+    }
+  }
+}


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


[15/50] [abbrv] hadoop git commit: HDDS-113. Rest and Rpc Client should verify resource name using HddsClientUtils. Contributed by Lokesh Jain.

Posted by ha...@apache.org.
HDDS-113. Rest and Rpc Client should verify resource name using HddsClientUtils.
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/2a9652e6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2a9652e6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2a9652e6

Branch: refs/heads/HDDS-48
Commit: 2a9652e69650973f6158b60ff131215827738db6
Parents: 13d2528
Author: Anu Engineer <ae...@apache.org>
Authored: Fri May 25 15:40:46 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri May 25 15:45:50 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hdds/scm/client/HddsClientUtils.java | 23 +++++++++
 .../apache/hadoop/ozone/client/ObjectStore.java |  9 ----
 .../apache/hadoop/ozone/client/OzoneBucket.java | 24 +--------
 .../apache/hadoop/ozone/client/OzoneVolume.java | 18 +------
 .../hadoop/ozone/client/rest/RestClient.java    | 52 ++++++++------------
 .../hadoop/ozone/client/rpc/RpcClient.java      | 46 +++++++----------
 6 files changed, 64 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a9652e6/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/HddsClientUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/HddsClientUtils.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/HddsClientUtils.java
index bc5f8d6..a6813eb 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/HddsClientUtils.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/HddsClientUtils.java
@@ -170,6 +170,29 @@ public final class HddsClientUtils {
   }
 
   /**
+   * verifies that bucket / volume name is a valid DNS name.
+   *
+   * @param resourceNames Array of bucket / volume names to be verified.
+   */
+  public static void verifyResourceName(String... resourceNames) {
+    for (String resourceName : resourceNames) {
+      HddsClientUtils.verifyResourceName(resourceName);
+    }
+  }
+
+  /**
+   * Checks that object parameters passed as reference is not null.
+   *
+   * @param references Array of object references to be checked.
+   * @param <T>
+   */
+  public static <T> void checkNotNull(T... references) {
+    for (T ref: references) {
+      Preconditions.checkNotNull(ref);
+    }
+  }
+
+  /**
    * Returns the cache value to be used for list calls.
    * @param conf Configuration object
    * @return list cache size

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a9652e6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
index d8b3011..c5f0689 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
@@ -63,8 +63,6 @@ public class ObjectStore {
    * @throws IOException
    */
   public void createVolume(String volumeName) throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    HddsClientUtils.verifyResourceName(volumeName);
     proxy.createVolume(volumeName);
   }
 
@@ -76,9 +74,6 @@ public class ObjectStore {
    */
   public void createVolume(String volumeName, VolumeArgs volumeArgs)
       throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(volumeArgs);
-    HddsClientUtils.verifyResourceName(volumeName);
     proxy.createVolume(volumeName, volumeArgs);
   }
 
@@ -89,8 +84,6 @@ public class ObjectStore {
    * @throws IOException
    */
   public OzoneVolume getVolume(String volumeName) throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    HddsClientUtils.verifyResourceName(volumeName);
     OzoneVolume volume = proxy.getVolumeDetails(volumeName);
     return volume;
   }
@@ -150,8 +143,6 @@ public class ObjectStore {
    * @throws IOException
    */
   public void deleteVolume(String volumeName) throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    HddsClientUtils.verifyResourceName(volumeName);
     proxy.deleteVolume(volumeName);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a9652e6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
index 5df0254..2f3cff6 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
@@ -104,6 +104,7 @@ public class OzoneBucket {
                      String volumeName, String bucketName,
                      List<OzoneAcl> acls, StorageType storageType,
                      Boolean versioning, long creationTime) {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
     this.proxy = proxy;
     this.volumeName = volumeName;
     this.name = bucketName;
@@ -180,8 +181,6 @@ public class OzoneBucket {
    * @throws IOException
    */
   public void addAcls(List<OzoneAcl> addAcls) throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(addAcls);
     proxy.addBucketAcls(volumeName, name, addAcls);
     addAcls.stream().filter(acl -> !acls.contains(acl)).forEach(
         acls::add);
@@ -193,8 +192,6 @@ public class OzoneBucket {
    * @throws IOException
    */
   public void removeAcls(List<OzoneAcl> removeAcls) throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(removeAcls);
     proxy.removeBucketAcls(volumeName, name, removeAcls);
     acls.removeAll(removeAcls);
   }
@@ -205,8 +202,6 @@ public class OzoneBucket {
    * @throws IOException
    */
   public void setStorageType(StorageType newStorageType) throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(newStorageType);
     proxy.setBucketStorageType(volumeName, name, newStorageType);
     storageType = newStorageType;
   }
@@ -217,8 +212,6 @@ public class OzoneBucket {
    * @throws IOException
    */
   public void setVersioning(Boolean newVersioning) throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(newVersioning);
     proxy.setBucketVersioning(volumeName, name, newVersioning);
     versioning = newVersioning;
   }
@@ -233,8 +226,6 @@ public class OzoneBucket {
    */
   public OzoneOutputStream createKey(String key, long size)
       throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(key);
     return createKey(key, size, defaultReplicationType, defaultReplication);
   }
 
@@ -251,10 +242,6 @@ public class OzoneBucket {
                                      ReplicationType type,
                                      ReplicationFactor factor)
       throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(key);
-    Preconditions.checkNotNull(type);
-    Preconditions.checkNotNull(factor);
     return proxy.createKey(volumeName, name, key, size, type, factor);
   }
 
@@ -265,8 +252,6 @@ public class OzoneBucket {
    * @throws IOException
    */
   public OzoneInputStream readKey(String key) throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(key);
     return proxy.getKey(volumeName, name, key);
   }
 
@@ -277,8 +262,6 @@ public class OzoneBucket {
    * @throws IOException
    */
   public OzoneKey getKey(String key) throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(key);
     return proxy.getKeyDetails(volumeName, name, key);
   }
 
@@ -314,16 +297,11 @@ public class OzoneBucket {
    * @throws IOException
    */
   public void deleteKey(String key) throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(key);
     proxy.deleteKey(volumeName, name, key);
   }
 
   public void renameKey(String fromKeyName, String toKeyName)
       throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(fromKeyName);
-    Preconditions.checkNotNull(toKeyName);
     proxy.renameKey(volumeName, name, fromKeyName, toKeyName);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a9652e6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
index 4601f1a..77f882a 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
@@ -83,6 +83,7 @@ public class OzoneVolume {
   public OzoneVolume(Configuration conf, ClientProtocol proxy, String name,
                      String admin, String owner, long quotaInBytes,
                      long creationTime, List<OzoneAcl> acls) {
+    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
     this.proxy = proxy;
     this.name = name;
     this.admin = admin;
@@ -153,8 +154,6 @@ public class OzoneVolume {
    * @throws IOException
    */
   public void setOwner(String owner) throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(owner);
     proxy.setVolumeOwner(name, owner);
     this.owner = owner;
   }
@@ -165,8 +164,6 @@ public class OzoneVolume {
    * @throws IOException
    */
   public void setQuota(OzoneQuota  quota) throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(quota);
     proxy.setVolumeQuota(name, quota);
     this.quotaInBytes = quota.sizeInBytes();
   }
@@ -178,9 +175,6 @@ public class OzoneVolume {
    */
   public void createBucket(String bucketName)
       throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(bucketName);
-    HddsClientUtils.verifyResourceName(bucketName);
     proxy.createBucket(name, bucketName);
   }
 
@@ -192,10 +186,6 @@ public class OzoneVolume {
    */
   public void createBucket(String bucketName, BucketArgs bucketArgs)
       throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(bucketName);
-    Preconditions.checkNotNull(bucketArgs);
-    HddsClientUtils.verifyResourceName(bucketName);
     proxy.createBucket(name, bucketName, bucketArgs);
   }
 
@@ -206,9 +196,6 @@ public class OzoneVolume {
    * @throws IOException
    */
   public OzoneBucket getBucket(String bucketName) throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(bucketName);
-    HddsClientUtils.verifyResourceName(bucketName);
     OzoneBucket bucket = proxy.getBucketDetails(name, bucketName);
     return bucket;
   }
@@ -246,9 +233,6 @@ public class OzoneVolume {
    * @throws IOException
    */
   public void deleteBucket(String bucketName) throws IOException {
-    Preconditions.checkNotNull(proxy, "Client proxy is not set.");
-    Preconditions.checkNotNull(bucketName);
-    HddsClientUtils.verifyResourceName(bucketName);
     proxy.deleteBucket(name, bucketName);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a9652e6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
index ac71abe..1169820 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
@@ -211,7 +211,8 @@ public class RestClient implements ClientProtocol {
   public void createVolume(String volumeName, VolumeArgs volArgs)
       throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
+      HddsClientUtils.verifyResourceName(volumeName);
+      Preconditions.checkNotNull(volArgs);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
       String owner = volArgs.getOwner() == null ?
           ugi.getUserName() : volArgs.getOwner();
@@ -256,7 +257,7 @@ public class RestClient implements ClientProtocol {
   public void setVolumeOwner(String volumeName, String owner)
       throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
+      HddsClientUtils.verifyResourceName(volumeName);
       Preconditions.checkNotNull(owner);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
       builder.setPath(PATH_SEPARATOR + volumeName);
@@ -273,7 +274,7 @@ public class RestClient implements ClientProtocol {
   public void setVolumeQuota(String volumeName, OzoneQuota quota)
       throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
+      HddsClientUtils.verifyResourceName(volumeName);
       Preconditions.checkNotNull(quota);
       String quotaString = quota.toString();
       URIBuilder builder = new URIBuilder(ozoneRestUri);
@@ -291,7 +292,7 @@ public class RestClient implements ClientProtocol {
   public OzoneVolume getVolumeDetails(String volumeName)
       throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
+      HddsClientUtils.verifyResourceName(volumeName);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
       builder.setPath(PATH_SEPARATOR + volumeName);
       builder.setParameter(Header.OZONE_INFO_QUERY_TAG,
@@ -326,7 +327,7 @@ public class RestClient implements ClientProtocol {
   @Override
   public void deleteVolume(String volumeName) throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
+      HddsClientUtils.verifyResourceName(volumeName);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
       builder.setPath(PATH_SEPARATOR + volumeName);
       HttpDelete httpDelete = new HttpDelete(builder.build());
@@ -362,8 +363,7 @@ public class RestClient implements ClientProtocol {
       String volumeName, String bucketName, BucketArgs bucketArgs)
       throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
-      Preconditions.checkNotNull(bucketName);
+      HddsClientUtils.verifyResourceName(volumeName, bucketName);
       Preconditions.checkNotNull(bucketArgs);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
       OzoneConsts.Versioning versioning = OzoneConsts.Versioning.DISABLED;
@@ -404,8 +404,7 @@ public class RestClient implements ClientProtocol {
       String volumeName, String bucketName, List<OzoneAcl> addAcls)
       throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
-      Preconditions.checkNotNull(bucketName);
+      HddsClientUtils.verifyResourceName(volumeName, bucketName);
       Preconditions.checkNotNull(addAcls);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
 
@@ -429,8 +428,7 @@ public class RestClient implements ClientProtocol {
       String volumeName, String bucketName, List<OzoneAcl> removeAcls)
       throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
-      Preconditions.checkNotNull(bucketName);
+      HddsClientUtils.verifyResourceName(volumeName, bucketName);
       Preconditions.checkNotNull(removeAcls);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
 
@@ -454,8 +452,7 @@ public class RestClient implements ClientProtocol {
       String volumeName, String bucketName, Boolean versioning)
       throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
-      Preconditions.checkNotNull(bucketName);
+      HddsClientUtils.verifyResourceName(volumeName, bucketName);
       Preconditions.checkNotNull(versioning);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
 
@@ -477,8 +474,7 @@ public class RestClient implements ClientProtocol {
       String volumeName, String bucketName, StorageType storageType)
       throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
-      Preconditions.checkNotNull(bucketName);
+      HddsClientUtils.verifyResourceName(volumeName, bucketName);
       Preconditions.checkNotNull(storageType);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
 
@@ -498,8 +494,7 @@ public class RestClient implements ClientProtocol {
   public void deleteBucket(String volumeName, String bucketName)
       throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
-      Preconditions.checkNotNull(bucketName);
+      HddsClientUtils.verifyResourceName(volumeName, bucketName);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
       builder.setPath(PATH_SEPARATOR + volumeName +
           PATH_SEPARATOR + bucketName);
@@ -521,8 +516,7 @@ public class RestClient implements ClientProtocol {
   public OzoneBucket getBucketDetails(String volumeName, String bucketName)
       throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
-      Preconditions.checkNotNull(bucketName);
+      HddsClientUtils.verifyResourceName(volumeName, bucketName);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
       builder.setPath(PATH_SEPARATOR + volumeName +
           PATH_SEPARATOR + bucketName);
@@ -573,9 +567,8 @@ public class RestClient implements ClientProtocol {
     // TODO: Once ReplicationType and ReplicationFactor are supported in
     // OzoneHandler (in Datanode), set them in header.
     try {
-      Preconditions.checkNotNull(volumeName);
-      Preconditions.checkNotNull(bucketName);
-      Preconditions.checkNotNull(keyName);
+      HddsClientUtils.verifyResourceName(volumeName, bucketName);
+      HddsClientUtils.checkNotNull(keyName, type, factor);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
       builder.setPath(PATH_SEPARATOR + volumeName +
           PATH_SEPARATOR + bucketName +
@@ -617,8 +610,7 @@ public class RestClient implements ClientProtocol {
       String volumeName, String bucketName, String keyName)
       throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
-      Preconditions.checkNotNull(bucketName);
+      HddsClientUtils.verifyResourceName(volumeName, bucketName);
       Preconditions.checkNotNull(keyName);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
       builder.setPath(PATH_SEPARATOR + volumeName +
@@ -661,8 +653,7 @@ public class RestClient implements ClientProtocol {
   public void deleteKey(String volumeName, String bucketName, String keyName)
       throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
-      Preconditions.checkNotNull(bucketName);
+      HddsClientUtils.verifyResourceName(volumeName, bucketName);
       Preconditions.checkNotNull(keyName);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
       builder.setPath(PATH_SEPARATOR + volumeName +
@@ -679,10 +670,8 @@ public class RestClient implements ClientProtocol {
   public void renameKey(String volumeName, String bucketName,
       String fromKeyName, String toKeyName) throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
-      Preconditions.checkNotNull(bucketName);
-      Preconditions.checkNotNull(fromKeyName);
-      Preconditions.checkNotNull(toKeyName);
+      HddsClientUtils.verifyResourceName(volumeName, bucketName);
+      HddsClientUtils.checkNotNull(fromKeyName, toKeyName);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
       builder.setPath(PATH_SEPARATOR + volumeName + PATH_SEPARATOR + bucketName
           + PATH_SEPARATOR + fromKeyName);
@@ -708,8 +697,7 @@ public class RestClient implements ClientProtocol {
       String volumeName, String bucketName, String keyName)
       throws IOException {
     try {
-      Preconditions.checkNotNull(volumeName);
-      Preconditions.checkNotNull(bucketName);
+      HddsClientUtils.verifyResourceName(volumeName, bucketName);
       Preconditions.checkNotNull(keyName);
       URIBuilder builder = new URIBuilder(ozoneRestUri);
       builder.setPath(PATH_SEPARATOR + volumeName +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a9652e6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
index ffe93dd..43b94a1 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
@@ -22,6 +22,7 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -170,7 +171,7 @@ public class RpcClient implements ClientProtocol {
   @Override
   public void createVolume(String volumeName, VolumeArgs volArgs)
       throws IOException {
-    Preconditions.checkNotNull(volumeName);
+    HddsClientUtils.verifyResourceName(volumeName);
     Preconditions.checkNotNull(volArgs);
 
     String admin = volArgs.getAdmin() == null ?
@@ -214,7 +215,7 @@ public class RpcClient implements ClientProtocol {
   @Override
   public void setVolumeOwner(String volumeName, String owner)
       throws IOException {
-    Preconditions.checkNotNull(volumeName);
+    HddsClientUtils.verifyResourceName(volumeName);
     Preconditions.checkNotNull(owner);
     keySpaceManagerClient.setOwner(volumeName, owner);
   }
@@ -222,7 +223,7 @@ public class RpcClient implements ClientProtocol {
   @Override
   public void setVolumeQuota(String volumeName, OzoneQuota quota)
       throws IOException {
-    Preconditions.checkNotNull(volumeName);
+    HddsClientUtils.verifyResourceName(volumeName);
     Preconditions.checkNotNull(quota);
     long quotaInBytes = quota.sizeInBytes();
     keySpaceManagerClient.setQuota(volumeName, quotaInBytes);
@@ -231,7 +232,7 @@ public class RpcClient implements ClientProtocol {
   @Override
   public OzoneVolume getVolumeDetails(String volumeName)
       throws IOException {
-    Preconditions.checkNotNull(volumeName);
+    HddsClientUtils.verifyResourceName(volumeName);
     KsmVolumeArgs volume = keySpaceManagerClient.getVolumeInfo(volumeName);
     return new OzoneVolume(
         conf,
@@ -253,7 +254,7 @@ public class RpcClient implements ClientProtocol {
 
   @Override
   public void deleteVolume(String volumeName) throws IOException {
-    Preconditions.checkNotNull(volumeName);
+    HddsClientUtils.verifyResourceName(volumeName);
     keySpaceManagerClient.deleteVolume(volumeName);
   }
 
@@ -307,8 +308,7 @@ public class RpcClient implements ClientProtocol {
   public void createBucket(
       String volumeName, String bucketName, BucketArgs bucketArgs)
       throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
+    HddsClientUtils.verifyResourceName(volumeName, bucketName);
     Preconditions.checkNotNull(bucketArgs);
 
     Boolean isVersionEnabled = bucketArgs.getVersioning() == null ?
@@ -346,8 +346,7 @@ public class RpcClient implements ClientProtocol {
   public void addBucketAcls(
       String volumeName, String bucketName, List<OzoneAcl> addAcls)
       throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
+    HddsClientUtils.verifyResourceName(volumeName, bucketName);
     Preconditions.checkNotNull(addAcls);
     KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
     builder.setVolumeName(volumeName)
@@ -360,8 +359,7 @@ public class RpcClient implements ClientProtocol {
   public void removeBucketAcls(
       String volumeName, String bucketName, List<OzoneAcl> removeAcls)
       throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
+    HddsClientUtils.verifyResourceName(volumeName, bucketName);
     Preconditions.checkNotNull(removeAcls);
     KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
     builder.setVolumeName(volumeName)
@@ -374,8 +372,7 @@ public class RpcClient implements ClientProtocol {
   public void setBucketVersioning(
       String volumeName, String bucketName, Boolean versioning)
       throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
+    HddsClientUtils.verifyResourceName(volumeName, bucketName);
     Preconditions.checkNotNull(versioning);
     KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
     builder.setVolumeName(volumeName)
@@ -388,8 +385,7 @@ public class RpcClient implements ClientProtocol {
   public void setBucketStorageType(
       String volumeName, String bucketName, StorageType storageType)
       throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
+    HddsClientUtils.verifyResourceName(volumeName, bucketName);
     Preconditions.checkNotNull(storageType);
     KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
     builder.setVolumeName(volumeName)
@@ -401,8 +397,7 @@ public class RpcClient implements ClientProtocol {
   @Override
   public void deleteBucket(
       String volumeName, String bucketName) throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
+    HddsClientUtils.verifyResourceName(volumeName, bucketName);
     keySpaceManagerClient.deleteBucket(volumeName, bucketName);
   }
 
@@ -415,8 +410,7 @@ public class RpcClient implements ClientProtocol {
   @Override
   public OzoneBucket getBucketDetails(
       String volumeName, String bucketName) throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
+    HddsClientUtils.verifyResourceName(volumeName, bucketName);
     KsmBucketInfo bucketArgs =
         keySpaceManagerClient.getBucketInfo(volumeName, bucketName);
     return new OzoneBucket(
@@ -454,6 +448,8 @@ public class RpcClient implements ClientProtocol {
       String volumeName, String bucketName, String keyName, long size,
       ReplicationType type, ReplicationFactor factor)
       throws IOException {
+    HddsClientUtils.verifyResourceName(volumeName, bucketName);
+    HddsClientUtils.checkNotNull(keyName, type, factor);
     String requestId = UUID.randomUUID().toString();
     KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
         .setVolumeName(volumeName)
@@ -486,8 +482,7 @@ public class RpcClient implements ClientProtocol {
   public OzoneInputStream getKey(
       String volumeName, String bucketName, String keyName)
       throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
+    HddsClientUtils.verifyResourceName(volumeName, bucketName);
     Preconditions.checkNotNull(keyName);
     String requestId = UUID.randomUUID().toString();
     KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
@@ -508,8 +503,7 @@ public class RpcClient implements ClientProtocol {
   public void deleteKey(
       String volumeName, String bucketName, String keyName)
       throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
+    HddsClientUtils.verifyResourceName(volumeName, bucketName);
     Preconditions.checkNotNull(keyName);
     KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
         .setVolumeName(volumeName)
@@ -522,10 +516,8 @@ public class RpcClient implements ClientProtocol {
   @Override
   public void renameKey(String volumeName, String bucketName,
       String fromKeyName, String toKeyName) throws IOException {
-    Preconditions.checkNotNull(volumeName);
-    Preconditions.checkNotNull(bucketName);
-    Preconditions.checkNotNull(fromKeyName);
-    Preconditions.checkNotNull(toKeyName);
+    HddsClientUtils.verifyResourceName(volumeName, bucketName);
+    HddsClientUtils.checkNotNull(fromKeyName, toKeyName);
     KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
         .setVolumeName(volumeName)
         .setBucketName(bucketName)


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


[08/50] [abbrv] hadoop git commit: YARN-8191. Fair scheduler: queue deletion without RM restart. (Gergo Repas via Haibo Chen)

Posted by ha...@apache.org.
YARN-8191. Fair scheduler: queue deletion without RM restart. (Gergo Repas 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/86bc6425
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/86bc6425
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/86bc6425

Branch: refs/heads/HDDS-48
Commit: 86bc6425d425913899f1d951498bd040e453b3d0
Parents: d9852eb
Author: Haibo Chen <ha...@apache.org>
Authored: Thu May 24 17:07:21 2018 -0700
Committer: Haibo Chen <ha...@apache.org>
Committed: Thu May 24 17:12:34 2018 -0700

----------------------------------------------------------------------
 .../fair/AllocationFileLoaderService.java       |  16 +-
 .../scheduler/fair/FSLeafQueue.java             |  31 ++
 .../resourcemanager/scheduler/fair/FSQueue.java |   9 +
 .../scheduler/fair/FairScheduler.java           |  29 +-
 .../scheduler/fair/QueueManager.java            | 155 +++++++--
 .../fair/TestAllocationFileLoaderService.java   | 100 +++---
 .../scheduler/fair/TestQueueManager.java        | 337 +++++++++++++++++++
 7 files changed, 596 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/86bc6425/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.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/AllocationFileLoaderService.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/AllocationFileLoaderService.java
index d8d9051..7a40b6a 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/AllocationFileLoaderService.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/AllocationFileLoaderService.java
@@ -87,7 +87,7 @@ public class AllocationFileLoaderService extends AbstractService {
   private Path allocFile;
   private FileSystem fs;
 
-  private Listener reloadListener;
+  private final Listener reloadListener;
 
   @VisibleForTesting
   long reloadIntervalMs = ALLOC_RELOAD_INTERVAL_MS;
@@ -95,15 +95,16 @@ public class AllocationFileLoaderService extends AbstractService {
   private Thread reloadThread;
   private volatile boolean running = true;
 
-  public AllocationFileLoaderService() {
-    this(SystemClock.getInstance());
+  public AllocationFileLoaderService(Listener reloadListener) {
+    this(reloadListener, SystemClock.getInstance());
   }
 
   private List<Permission> defaultPermissions;
 
-  public AllocationFileLoaderService(Clock clock) {
+  public AllocationFileLoaderService(Listener reloadListener, Clock clock) {
     super(AllocationFileLoaderService.class.getName());
     this.clock = clock;
+    this.reloadListener = reloadListener;
   }
 
   @Override
@@ -114,6 +115,7 @@ public class AllocationFileLoaderService extends AbstractService {
       reloadThread = new Thread(() -> {
         while (running) {
           try {
+            reloadListener.onCheck();
             long time = clock.getTime();
             long lastModified =
                 fs.getFileStatus(allocFile).getModificationTime();
@@ -207,10 +209,6 @@ public class AllocationFileLoaderService extends AbstractService {
     return allocPath;
   }
 
-  public synchronized void setReloadListener(Listener reloadListener) {
-    this.reloadListener = reloadListener;
-  }
-
   /**
    * Updates the allocation list from the allocation config file. This file is
    * expected to be in the XML format specified in the design doc.
@@ -351,5 +349,7 @@ public class AllocationFileLoaderService extends AbstractService {
 
   public interface Listener {
     void onReload(AllocationConfiguration info) throws IOException;
+
+    void onCheck();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86bc6425/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.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/FSLeafQueue.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/FSLeafQueue.java
index 49d2166..e7da16f 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/FSLeafQueue.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/FSLeafQueue.java
@@ -21,7 +21,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -34,6 +36,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -56,6 +59,8 @@ public class FSLeafQueue extends FSQueue {
   // apps that are runnable
   private final List<FSAppAttempt> runnableApps = new ArrayList<>();
   private final List<FSAppAttempt> nonRunnableApps = new ArrayList<>();
+  // assignedApps keeps track of applications that have no appAttempts
+  private final Set<ApplicationId> assignedApps = new HashSet<>();
   // get a lock with fair distribution for app list updates
   private final ReadWriteLock rwl = new ReentrantReadWriteLock(true);
   private final Lock readLock = rwl.readLock();
@@ -89,6 +94,9 @@ public class FSLeafQueue extends FSQueue {
       } else {
         nonRunnableApps.add(app);
       }
+      // when an appAttempt is created for an application, we'd like to move
+      // it over from assignedApps to either runnableApps or nonRunnableApps
+      assignedApps.remove(app.getApplicationId());
       incUsedResource(app.getResourceUsage());
     } finally {
       writeLock.unlock();
@@ -440,6 +448,15 @@ public class FSLeafQueue extends FSQueue {
     return numPendingApps;
   }
 
+  public int getNumAssignedApps() {
+    readLock.lock();
+    try {
+      return assignedApps.size();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
   /**
    * TODO: Based on how frequently this is called, we might want to club
    * counting pending and active apps in the same method.
@@ -609,4 +626,18 @@ public class FSLeafQueue extends FSQueue {
         ", LastTimeAtMinShare: " + lastTimeAtMinShare +
         "}");
   }
+
+  /**
+   * This method is called when an application is assigned to this queue
+   * for book-keeping purposes (to be able to determine if the queue is empty).
+   * @param applicationId the application's id
+   */
+  public void addAssignedApp(ApplicationId applicationId) {
+    writeLock.lock();
+    try {
+      assignedApps.add(applicationId);
+    } finally {
+      writeLock.unlock();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86bc6425/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.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/FSQueue.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/FSQueue.java
index 4babfd5..6b88a32 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/FSQueue.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/FSQueue.java
@@ -83,6 +83,7 @@ public abstract class FSQueue implements Queue, Schedulable {
   private long minSharePreemptionTimeout = Long.MAX_VALUE;
   private float fairSharePreemptionThreshold = 0.5f;
   private boolean preemptable = true;
+  private boolean isDynamic = true;
 
   public FSQueue(String name, FairScheduler scheduler, FSParentQueue parent) {
     this.name = name;
@@ -585,4 +586,12 @@ public abstract class FSQueue implements Queue, Schedulable {
    * @param sb the {code StringBuilder} which holds queue states
    */
   protected abstract void dumpStateInternal(StringBuilder sb);
+
+  public boolean isDynamic() {
+    return isDynamic;
+  }
+
+  public void setDynamic(boolean dynamic) {
+    this.isDynamic = dynamic;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86bc6425/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 1c4bd51..4c84aa9 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
@@ -99,6 +99,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.EnumSet;
 import java.util.HashSet;
@@ -207,7 +208,8 @@ public class FairScheduler extends
   public FairScheduler() {
     super(FairScheduler.class.getName());
     context = new FSContext(this);
-    allocsLoader = new AllocationFileLoaderService();
+    allocsLoader =
+        new AllocationFileLoaderService(new AllocationReloadListener());
     queueMgr = new QueueManager(this);
     maxRunningEnforcer = new MaxRunningAppsEnforcer(this);
   }
@@ -516,6 +518,7 @@ public class FairScheduler extends
           new SchedulerApplication<FSAppAttempt>(queue, user);
       applications.put(applicationId, application);
       queue.getMetrics().submitApp(user);
+      queue.addAssignedApp(applicationId);
 
       LOG.info("Accepted application " + applicationId + " from user: " + user
           + ", in queue: " + queue.getName()
@@ -1435,7 +1438,6 @@ public class FairScheduler extends
     }
 
     allocsLoader.init(conf);
-    allocsLoader.setReloadListener(new AllocationReloadListener());
     // If we fail to load allocations file on initialize, we want to fail
     // immediately.  After a successful load, exceptions on future reloads
     // will just result in leaving things as they are.
@@ -1589,6 +1591,7 @@ public class FairScheduler extends
       // Commit the reload; also create any queue defined in the alloc file
       // if it does not already exist, so it can be displayed on the web UI.
 
+      Set<String> removedStaticQueues = getRemovedStaticQueues(queueInfo);
       writeLock.lock();
       try {
         if (queueInfo == null) {
@@ -1599,6 +1602,7 @@ public class FairScheduler extends
           setQueueAcls(allocConf.getQueueAcls());
           allocConf.getDefaultSchedulingPolicy().initialize(getContext());
           queueMgr.updateAllocationConfiguration(allocConf);
+          queueMgr.setQueuesToDynamic(removedStaticQueues);
           applyChildDefaults();
           maxRunningEnforcer.updateRunnabilityOnReload();
         }
@@ -1606,6 +1610,27 @@ public class FairScheduler extends
         writeLock.unlock();
       }
     }
+
+    private Set<String> getRemovedStaticQueues(
+        AllocationConfiguration queueInfo) {
+      if (queueInfo == null || allocConf == null) {
+        return Collections.emptySet();
+      }
+      Set<String> removedStaticQueues = new HashSet<>();
+      for (Set<String> queues : allocConf.getConfiguredQueues().values()) {
+        removedStaticQueues.addAll(queues);
+      }
+      for (Set<String> queues : queueInfo.getConfiguredQueues().values()) {
+        removedStaticQueues.removeAll(queues);
+      }
+      return removedStaticQueues;
+    }
+
+    @Override
+    public void onCheck() {
+      queueMgr.removeEmptyDynamicQueues();
+      queueMgr.removePendingIncompatibleQueues();
+    }
   }
 
   private void setQueueAcls(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86bc6425/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.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/QueueManager.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/QueueManager.java
index 8734877..632a842 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/QueueManager.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/QueueManager.java
@@ -22,13 +22,17 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CopyOnWriteArrayList;
 
 import javax.xml.parsers.ParserConfigurationException;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -52,6 +56,36 @@ public class QueueManager {
   public static final Log LOG = LogFactory.getLog(
     QueueManager.class.getName());
 
+  private final class IncompatibleQueueRemovalTask {
+
+    private final String queueToCreate;
+    private final FSQueueType queueType;
+
+    private IncompatibleQueueRemovalTask(String queueToCreate,
+        FSQueueType queueType) {
+      this.queueToCreate = queueToCreate;
+      this.queueType = queueType;
+    }
+
+    private void execute() {
+      Boolean removed =
+          removeEmptyIncompatibleQueues(queueToCreate, queueType).orElse(null);
+      if (Boolean.TRUE.equals(removed)) {
+        FSQueue queue = getQueue(queueToCreate, true, queueType, false);
+        if (queue != null &&
+            // if queueToCreate is present in the allocation config, set it
+            // to static
+            scheduler.allocConf.configuredQueues.values().stream()
+            .anyMatch(s -> s.contains(queueToCreate))) {
+          queue.setDynamic(false);
+        }
+      }
+      if (!Boolean.FALSE.equals(removed)) {
+        incompatibleQueuesPendingRemoval.remove(this);
+      }
+    }
+  }
+
   public static final String ROOT_QUEUE = "root";
   
   private final FairScheduler scheduler;
@@ -59,6 +93,8 @@ public class QueueManager {
   private final Collection<FSLeafQueue> leafQueues = 
       new CopyOnWriteArrayList<FSLeafQueue>();
   private final Map<String, FSQueue> queues = new HashMap<String, FSQueue>();
+  private Set<IncompatibleQueueRemovalTask> incompatibleQueuesPendingRemoval =
+      new HashSet<>();
   private FSParentQueue rootQueue;
 
   public QueueManager(FairScheduler scheduler) {
@@ -75,10 +111,13 @@ public class QueueManager {
     // SchedulingPolicy.DEFAULT_POLICY since the allocation file hasn't been
     // loaded yet.
     rootQueue = new FSParentQueue("root", scheduler, null);
+    rootQueue.setDynamic(false);
     queues.put(rootQueue.getName(), rootQueue);
 
     // Create the default queue
-    getLeafQueue(YarnConfiguration.DEFAULT_QUEUE_NAME, true);
+    FSLeafQueue defaultQueue =
+        getLeafQueue(YarnConfiguration.DEFAULT_QUEUE_NAME, true);
+    defaultQueue.setDynamic(false);
     // Recursively reinitialize to propagate queue properties
     rootQueue.reinit(true);
   }
@@ -121,7 +160,8 @@ public class QueueManager {
    */
   public boolean removeLeafQueue(String name) {
     name = ensureRootPrefix(name);
-    return removeEmptyIncompatibleQueues(name, FSQueueType.PARENT);
+    return !Boolean.FALSE.equals(
+        removeEmptyIncompatibleQueues(name, FSQueueType.PARENT).orElse(null));
   }
 
 
@@ -346,9 +386,13 @@ public class QueueManager {
    * 
    * We will never remove the root queue or the default queue in this way.
    *
-   * @return true if we can create queueToCreate or it already exists.
+   * @return Optional.of(Boolean.TRUE)  if there was an incompatible queue that
+   *                                    has been removed,
+   *         Optional.of(Boolean.FALSE) if there was an incompatible queue that
+   *                                    have not be removed,
+   *         Optional.empty()           if there is no incompatible queue.
    */
-  private boolean removeEmptyIncompatibleQueues(String queueToCreate,
+  private Optional<Boolean> removeEmptyIncompatibleQueues(String queueToCreate,
       FSQueueType queueType) {
     queueToCreate = ensureRootPrefix(queueToCreate);
 
@@ -357,7 +401,7 @@ public class QueueManager {
     if (queueToCreate.equals(ROOT_QUEUE) ||
         queueToCreate.startsWith(
             ROOT_QUEUE + "." + YarnConfiguration.DEFAULT_QUEUE_NAME + ".")) {
-      return false;
+      return Optional.empty();
     }
 
     FSQueue queue = queues.get(queueToCreate);
@@ -365,19 +409,18 @@ public class QueueManager {
     if (queue != null) {
       if (queue instanceof FSLeafQueue) {
         if (queueType == FSQueueType.LEAF) {
-          // if queue is already a leaf then return true
-          return true;
+          return Optional.empty();
         }
         // remove incompatibility since queue is a leaf currently
         // needs to change to a parent.
-        return removeQueueIfEmpty(queue);
+        return Optional.of(removeQueueIfEmpty(queue));
       } else {
         if (queueType == FSQueueType.PARENT) {
-          return true;
+          return Optional.empty();
         }
         // If it's an existing parent queue and needs to change to leaf, 
         // remove it if it's empty.
-        return removeQueueIfEmpty(queue);
+        return Optional.of(removeQueueIfEmpty(queue));
       }
     }
 
@@ -389,11 +432,51 @@ public class QueueManager {
       String prefixString = queueToCreate.substring(0, sepIndex);
       FSQueue prefixQueue = queues.get(prefixString);
       if (prefixQueue != null && prefixQueue instanceof FSLeafQueue) {
-        return removeQueueIfEmpty(prefixQueue);
+        return Optional.of(removeQueueIfEmpty(prefixQueue));
       }
       sepIndex = queueToCreate.lastIndexOf('.', sepIndex-1);
     }
-    return true;
+    return Optional.empty();
+  }
+
+  /**
+   * Removes all empty dynamic queues (including empty dynamic parent queues).
+   */
+  public void removeEmptyDynamicQueues() {
+    synchronized (queues) {
+      Set<FSParentQueue> parentQueuesToCheck = new HashSet<>();
+      for (FSQueue queue : getQueues()) {
+        if (queue.isDynamic() && queue.getChildQueues().isEmpty()) {
+          boolean removed = removeQueueIfEmpty(queue);
+          if (removed && queue.getParent().isDynamic()) {
+            parentQueuesToCheck.add(queue.getParent());
+          }
+        }
+      }
+      while (!parentQueuesToCheck.isEmpty()) {
+        FSParentQueue queue = parentQueuesToCheck.iterator().next();
+        if (queue.getChildQueues().isEmpty()) {
+          removeQueue(queue);
+          if (queue.getParent().isDynamic()) {
+            parentQueuesToCheck.add(queue.getParent());
+          }
+        }
+        parentQueuesToCheck.remove(queue);
+      }
+    }
+  }
+
+  /**
+   * Re-checking incompatible queues that could not be removed earlier due to
+   * not being empty, and removing those that became empty.
+   */
+  public void removePendingIncompatibleQueues() {
+    synchronized (queues) {
+      for (IncompatibleQueueRemovalTask removalTask :
+          ImmutableSet.copyOf(incompatibleQueuesPendingRemoval)) {
+        removalTask.execute();
+      }
+    }
   }
 
   /**
@@ -435,7 +518,8 @@ public class QueueManager {
     if (queue instanceof FSLeafQueue) {
       FSLeafQueue leafQueue = (FSLeafQueue)queue;
       return queue.getNumRunnableApps() == 0 &&
-          leafQueue.getNumNonRunnableApps() == 0;
+          leafQueue.getNumNonRunnableApps() == 0 &&
+          leafQueue.getNumAssignedApps() == 0;
     } else {
       for (FSQueue child : queue.getChildQueues()) {
         if (!isEmpty(child)) {
@@ -501,21 +585,13 @@ public class QueueManager {
         LOG.error("Setting scheduling policies for existing queues failed!");
       }
 
-      for (String name : queueConf.getConfiguredQueues().get(
-              FSQueueType.LEAF)) {
-        if (removeEmptyIncompatibleQueues(name, FSQueueType.LEAF)) {
-          getLeafQueue(name, true, false);
-        }
-      }
+      ensureQueueExistsAndIsCompatibleAndIsStatic(queueConf, FSQueueType.LEAF);
+
       // At this point all leaves and 'parents with
       // at least one child' would have been created.
       // Now create parents with no configured leaf.
-      for (String name : queueConf.getConfiguredQueues().get(
-          FSQueueType.PARENT)) {
-        if (removeEmptyIncompatibleQueues(name, FSQueueType.PARENT)) {
-          getParentQueue(name, true, false);
-        }
-      }
+      ensureQueueExistsAndIsCompatibleAndIsStatic(queueConf,
+          FSQueueType.PARENT);
     }
 
     // Initialize all queues recursively
@@ -524,6 +600,35 @@ public class QueueManager {
     rootQueue.recomputeSteadyShares();
   }
 
+  private void ensureQueueExistsAndIsCompatibleAndIsStatic(
+      AllocationConfiguration queueConf, FSQueueType queueType) {
+    for (String name : queueConf.getConfiguredQueues().get(queueType)) {
+      Boolean removed =
+          removeEmptyIncompatibleQueues(name, queueType).orElse(null);
+      if (Boolean.FALSE.equals(removed)) {
+        incompatibleQueuesPendingRemoval.add(
+            new IncompatibleQueueRemovalTask(name, queueType));
+      } else {
+        FSQueue queue = getQueue(name, true, queueType, false);
+        if (queue != null) {
+          queue.setDynamic(false);
+        }
+      }
+    }
+  }
+
+  /**
+   * Setting a set of queues to dynamic.
+   * @param queueNames The names of the queues to be set to dynamic
+   */
+  protected void setQueuesToDynamic(Set<String> queueNames) {
+    synchronized (queues) {
+      for (String queueName : queueNames) {
+        queues.get(queueName).setDynamic(true);
+      }
+    }
+  }
+
   /**
    * Check whether queue name is valid,
    * return true if it is valid, otherwise return false.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86bc6425/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.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/fair/TestAllocationFileLoaderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java
index 8591d67..30b8a91 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.AllocationFileLoaderService.Listener;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueuePlacementRule.NestedUserQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.AllocationFileWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
@@ -32,6 +33,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.Fai
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Test;
+import org.mockito.Mockito;
+
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.FileWriter;
@@ -79,7 +82,8 @@ public class TestAllocationFileLoaderService {
     fs.copyFromLocalFile(new Path(fschedURL.toURI()), new Path(fsAllocPath));
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, fsAllocPath);
 
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(Mockito.mock(Listener.class));
     Path allocationFile = allocLoader.getAllocationFile(conf);
     assertEquals(fsAllocPath, allocationFile.toString());
     assertTrue(fs.exists(allocationFile));
@@ -92,7 +96,8 @@ public class TestAllocationFileLoaderService {
       throws UnsupportedFileSystemException {
     Configuration conf = new YarnConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, "badfs:///badfile");
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(Mockito.mock(Listener.class));
 
     allocLoader.getAllocationFile(conf);
   }
@@ -105,7 +110,7 @@ public class TestAllocationFileLoaderService {
       conf.set(FairSchedulerConfiguration.ALLOCATION_FILE,
           TEST_FAIRSCHED_XML);
       AllocationFileLoaderService allocLoader =
-          new AllocationFileLoaderService();
+          new AllocationFileLoaderService(Mockito.mock(Listener.class));
       Path allocationFile = allocLoader.getAllocationFile(conf);
       assertEquals(TEST_FAIRSCHED_XML, allocationFile.getName());
       assertTrue(fs.exists(allocationFile));
@@ -134,12 +139,11 @@ public class TestAllocationFileLoaderService {
     Configuration conf = new Configuration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
 
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(
-        clock);
+    ReloadListener confHolder = new ReloadListener();
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(confHolder, clock);
     allocLoader.reloadIntervalMs = 5;
     allocLoader.init(conf);
-    ReloadListener confHolder = new ReloadListener();
-    allocLoader.setReloadListener(confHolder);
     allocLoader.reloadAllocations();
     AllocationConfiguration allocConf = confHolder.allocConf;
 
@@ -205,7 +209,9 @@ public class TestAllocationFileLoaderService {
   public void testAllocationFileParsing() throws Exception {
     Configuration conf = new Configuration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
+    ReloadListener confHolder = new ReloadListener();
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(confHolder);
 
     AllocationFileWriter
             .create()
@@ -278,8 +284,6 @@ public class TestAllocationFileLoaderService {
             .writeToFile(ALLOC_FILE);
 
     allocLoader.init(conf);
-    ReloadListener confHolder = new ReloadListener();
-    allocLoader.setReloadListener(confHolder);
     allocLoader.reloadAllocations();
     AllocationConfiguration queueConf = confHolder.allocConf;
 
@@ -427,7 +431,9 @@ public class TestAllocationFileLoaderService {
   public void testBackwardsCompatibleAllocationFileParsing() throws Exception {
     Configuration conf = new Configuration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
+    ReloadListener confHolder = new ReloadListener();
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(confHolder);
 
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
@@ -473,8 +479,6 @@ public class TestAllocationFileLoaderService {
     out.close();
 
     allocLoader.init(conf);
-    ReloadListener confHolder = new ReloadListener();
-    allocLoader.setReloadListener(confHolder);
     allocLoader.reloadAllocations();
     AllocationConfiguration queueConf = confHolder.allocConf;
 
@@ -550,10 +554,10 @@ public class TestAllocationFileLoaderService {
     out.println("</allocations>");
     out.close();
 
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
-    allocLoader.init(conf);
     ReloadListener confHolder = new ReloadListener();
-    allocLoader.setReloadListener(confHolder);
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(confHolder);
+    allocLoader.init(conf);
     allocLoader.reloadAllocations();
     AllocationConfiguration allocConf = confHolder.allocConf;
 
@@ -584,10 +588,10 @@ public class TestAllocationFileLoaderService {
     out.println("</allocations>");
     out.close();
 
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
-    allocLoader.init(conf);
     ReloadListener confHolder = new ReloadListener();
-    allocLoader.setReloadListener(confHolder);
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(confHolder);
+    allocLoader.init(conf);
     allocLoader.reloadAllocations();
   }
 
@@ -608,10 +612,10 @@ public class TestAllocationFileLoaderService {
     out.println("</allocations>");
     out.close();
 
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
-    allocLoader.init(conf);
     ReloadListener confHolder = new ReloadListener();
-    allocLoader.setReloadListener(confHolder);
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(confHolder);
+    allocLoader.init(conf);
     allocLoader.reloadAllocations();
   }
 
@@ -632,10 +636,10 @@ public class TestAllocationFileLoaderService {
     out.println("</allocations>");
     out.close();
 
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
-    allocLoader.init(conf);
     ReloadListener confHolder = new ReloadListener();
-    allocLoader.setReloadListener(confHolder);
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(confHolder);
+    allocLoader.init(conf);
     allocLoader.reloadAllocations();
   }
 
@@ -654,10 +658,10 @@ public class TestAllocationFileLoaderService {
     out.println("</allocations>");
     out.close();
 
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
-    allocLoader.init(conf);
     ReloadListener confHolder = new ReloadListener();
-    allocLoader.setReloadListener(confHolder);
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(confHolder);
+    allocLoader.init(conf);
     try {
       allocLoader.reloadAllocations();
     } catch (AllocationConfigurationException ex) {
@@ -685,10 +689,10 @@ public class TestAllocationFileLoaderService {
     out.println("</allocations>");
     out.close();
 
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
-    allocLoader.init(conf);
     ReloadListener confHolder = new ReloadListener();
-    allocLoader.setReloadListener(confHolder);
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(confHolder);
+    allocLoader.init(conf);
     try {
       allocLoader.reloadAllocations();
     } catch (AllocationConfigurationException ex) {
@@ -714,10 +718,10 @@ public class TestAllocationFileLoaderService {
     out.println("</allocations>");
     out.close();
 
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
-    allocLoader.init(conf);
     ReloadListener confHolder = new ReloadListener();
-    allocLoader.setReloadListener(confHolder);
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(confHolder);
+    allocLoader.init(conf);
     allocLoader.reloadAllocations();
     AllocationConfiguration queueConf = confHolder.allocConf;
     // Check whether queue 'parent' and 'child' are loaded successfully
@@ -745,10 +749,10 @@ public class TestAllocationFileLoaderService {
     out.println("</allocations>");
     out.close();
 
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
-    allocLoader.init(conf);
     ReloadListener confHolder = new ReloadListener();
-    allocLoader.setReloadListener(confHolder);
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(confHolder);
+    allocLoader.init(conf);
     allocLoader.reloadAllocations();
   }
 
@@ -767,10 +771,10 @@ public class TestAllocationFileLoaderService {
     out.println("</allocations>");
     out.close();
 
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
-    allocLoader.init(conf);
     ReloadListener confHolder = new ReloadListener();
-    allocLoader.setReloadListener(confHolder);
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(confHolder);
+    allocLoader.init(conf);
     allocLoader.reloadAllocations();
   }
 
@@ -793,10 +797,10 @@ public class TestAllocationFileLoaderService {
     out.println("</allocations>");
     out.close();
 
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
-    allocLoader.init(conf);
     ReloadListener confHolder = new ReloadListener();
-    allocLoader.setReloadListener(confHolder);
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(confHolder);
+    allocLoader.init(conf);
     allocLoader.reloadAllocations();
 
     AllocationConfiguration allocConf = confHolder.allocConf;
@@ -853,10 +857,10 @@ public class TestAllocationFileLoaderService {
     out.println("</allocations>");
     out.close();
 
-    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
-    allocLoader.init(conf);
     ReloadListener confHolder = new ReloadListener();
-    allocLoader.setReloadListener(confHolder);
+    AllocationFileLoaderService allocLoader =
+        new AllocationFileLoaderService(confHolder);
+    allocLoader.init(conf);
     allocLoader.reloadAllocations();
   }
 
@@ -867,5 +871,9 @@ public class TestAllocationFileLoaderService {
     public void onReload(AllocationConfiguration info) {
       allocConf = info;
     }
+
+    @Override
+    public void onCheck() {
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86bc6425/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManager.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/fair/TestQueueManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManager.java
index eb2d402..3674ffb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManager.java
@@ -20,15 +20,22 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
 
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
 
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
 
 public class TestQueueManager {
@@ -305,4 +312,334 @@ public class TestQueueManager {
     assertEquals("createQueue() returned wrong queue",
         "root.queue1.queue2", q2.getName());
   }
+
+  @Test
+  public void testRemovalOfDynamicLeafQueue() {
+    AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
+
+    queueManager.updateAllocationConfiguration(allocConf);
+
+    FSQueue q1 = queueManager.getLeafQueue("root.test.childB.dynamic1", true);
+
+    assertNotNull("Queue root.test.childB.dynamic1 was not created", q1);
+    assertEquals("createQueue() returned wrong queue",
+        "root.test.childB.dynamic1", q1.getName());
+    assertTrue("root.test.childB.dynamic1 is not a dynamic queue",
+        q1.isDynamic());
+
+    // an application is submitted to root.test.childB.dynamic1
+    notEmptyQueues.add(q1);
+
+    // root.test.childB.dynamic1 is not empty and should not be removed
+    queueManager.removePendingIncompatibleQueues();
+    queueManager.removeEmptyDynamicQueues();
+    q1 = queueManager.getLeafQueue("root.test.childB.dynamic1", false);
+    assertNotNull("Queue root.test.childB.dynamic1 was deleted", q1);
+
+    // the application finishes, the next removeEmptyDynamicQueues() should
+    // clean root.test.childB.dynamic1 up, but keep its static parent
+    notEmptyQueues.remove(q1);
+
+    queueManager.removePendingIncompatibleQueues();
+    queueManager.removeEmptyDynamicQueues();
+    q1 = queueManager.getLeafQueue("root.test.childB.dynamic1", false);
+    assertNull("Queue root.test.childB.dynamic1 was not deleted", q1);
+    assertNotNull("The static parent of root.test.childB.dynamic1 was deleted",
+        queueManager.getParentQueue("root.test.childB", false));
+  }
+
+  @Test
+  public void testRemovalOfDynamicParentQueue() {
+    AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
+
+    queueManager.updateAllocationConfiguration(allocConf);
+
+    FSQueue q1 = queueManager.getLeafQueue("root.parent1.dynamic1", true);
+
+    assertNotNull("Queue root.parent1.dynamic1 was not created", q1);
+    assertEquals("createQueue() returned wrong queue",
+        "root.parent1.dynamic1", q1.getName());
+    assertTrue("root.parent1.dynamic1 is not a dynamic queue", q1.isDynamic());
+
+    FSQueue p1 = queueManager.getParentQueue("root.parent1", false);
+    assertNotNull("Queue root.parent1 was not created", p1);
+    assertTrue("root.parent1 is not a dynamic queue", p1.isDynamic());
+
+    queueManager.removePendingIncompatibleQueues();
+    queueManager.removeEmptyDynamicQueues();
+    q1 = queueManager.getLeafQueue("root.parent1.dynamic1", false);
+    p1 = queueManager.getParentQueue("root.parent1", false);
+
+    assertNull("Queue root.parent1.dynamic1 was not deleted", q1);
+    assertNull("Queue root.parent1 was not deleted", p1);
+  }
+
+  @Test
+  public void testNonEmptyDynamicQueueBecomingStaticQueue() {
+    AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
+
+    queueManager.updateAllocationConfiguration(allocConf);
+
+    FSLeafQueue q1 = queueManager.getLeafQueue("root.leaf1", true);
+
+    assertNotNull("Queue root.leaf1 was not created", q1);
+    assertEquals("createQueue() returned wrong queue",
+        "root.leaf1", q1.getName());
+    assertTrue("root.leaf1 is not a dynamic queue", q1.isDynamic());
+
+    // pretend that we submitted an app to the queue
+    notEmptyQueues.add(q1);
+
+    // non-empty queues should not be deleted
+    queueManager.removePendingIncompatibleQueues();
+    queueManager.removeEmptyDynamicQueues();
+    q1 = queueManager.getLeafQueue("root.leaf1", false);
+    assertNotNull("Queue root.leaf1 was deleted", q1);
+
+    // next we add leaf1 under root in the allocation config
+    allocConf.configuredQueues.get(FSQueueType.LEAF).add("root.leaf1");
+    queueManager.updateAllocationConfiguration(allocConf);
+
+    // updateAllocationConfiguration() should make root.leaf1 a dynamic queue
+    assertFalse("root.leaf1 is not a static queue", q1.isDynamic());
+
+    // application finished now and the queue is empty, but since leaf1 is a
+    // static queue at this point, hence not affected by
+    // removeEmptyDynamicQueues()
+    notEmptyQueues.clear();
+    queueManager.removePendingIncompatibleQueues();
+    queueManager.removeEmptyDynamicQueues();
+    q1 = queueManager.getLeafQueue("root.leaf1", false);
+    assertNotNull("Queue root.leaf1 was deleted", q1);
+    assertFalse("root.leaf1 is not a static queue", q1.isDynamic());
+  }
+
+  @Test
+  public void testNonEmptyStaticQueueBecomingDynamicQueue() {
+    AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
+    queueManager.updateAllocationConfiguration(allocConf);
+
+    FSLeafQueue q1 = queueManager.getLeafQueue("root.test.childA", false);
+
+    assertNotNull("Queue root.test.childA does not exist", q1);
+    assertEquals("createQueue() returned wrong queue",
+        "root.test.childA", q1.getName());
+    assertFalse("root.test.childA is not a static queue", q1.isDynamic());
+
+    // we submitted an app to the queue
+    notEmptyQueues.add(q1);
+
+    // the next removeEmptyDynamicQueues() call should not modify
+    // root.test.childA
+    queueManager.removePendingIncompatibleQueues();
+    queueManager.removeEmptyDynamicQueues();
+    q1 = queueManager.getLeafQueue("root.test.childA", false);
+    assertNotNull("Queue root.test.childA was deleted", q1);
+    assertFalse("root.test.childA is not a dynamic queue", q1.isDynamic());
+
+    // next we remove all queues from the allocation config,
+    // this causes all queues to change to dynamic
+    for (Set<String> queueNames : allocConf.configuredQueues.values()) {
+      queueManager.setQueuesToDynamic(queueNames);
+      queueNames.clear();
+    }
+    queueManager.updateAllocationConfiguration(allocConf);
+
+    q1 = queueManager.getLeafQueue("root.test.childA", false);
+    assertNotNull("Queue root.test.childA was deleted", q1);
+    assertTrue("root.test.childA is not a dynamic queue", q1.isDynamic());
+
+    // application finished - the queue does not have runnable app
+    // the next removeEmptyDynamicQueues() call should remove the queues
+    notEmptyQueues.remove(q1);
+
+    queueManager.removePendingIncompatibleQueues();
+    queueManager.removeEmptyDynamicQueues();
+
+    q1 = queueManager.getLeafQueue("root.test.childA", false);
+    assertNull("Queue root.test.childA was not deleted", q1);
+
+    FSParentQueue p1 = queueManager.getParentQueue("root.test", false);
+    assertNull("Queue root.test was not deleted", p1);
+  }
+
+  @Test
+  public void testRemovalOfChildlessParentQueue() {
+    AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
+    queueManager.updateAllocationConfiguration(allocConf);
+
+    FSParentQueue q1 = queueManager.getParentQueue("root.test.childB", false);
+
+    assertNotNull("Queue root.test.childB was not created", q1);
+    assertEquals("createQueue() returned wrong queue",
+        "root.test.childB", q1.getName());
+    assertFalse("root.test.childB is a dynamic queue", q1.isDynamic());
+
+    // static queues should not be deleted
+    queueManager.removePendingIncompatibleQueues();
+    queueManager.removeEmptyDynamicQueues();
+    q1 = queueManager.getParentQueue("root.test.childB", false);
+    assertNotNull("Queue root.test.childB was deleted", q1);
+
+    // next we remove root.test.childB from the allocation config
+    allocConf.configuredQueues.get(FSQueueType.PARENT)
+        .remove("root.test.childB");
+    queueManager.updateAllocationConfiguration(allocConf);
+    queueManager.setQueuesToDynamic(Collections.singleton("root.test.childB"));
+
+    // the next removeEmptyDynamicQueues() call should clean
+    // root.test.childB up
+    queueManager.removePendingIncompatibleQueues();
+    queueManager.removeEmptyDynamicQueues();
+    q1 = queueManager.getParentQueue("root.leaf1", false);
+    assertNull("Queue root.leaf1 was not deleted", q1);
+  }
+
+  @Test
+  public void testQueueTypeChange() {
+    AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
+    queueManager.updateAllocationConfiguration(allocConf);
+
+    FSQueue q1 = queueManager.getLeafQueue("root.parent1.leaf1", true);
+    assertNotNull("Queue root.parent1.leaf1 was not created", q1);
+    assertEquals("createQueue() returned wrong queue",
+        "root.parent1.leaf1", q1.getName());
+    assertTrue("root.parent1.leaf1 is not a dynamic queue", q1.isDynamic());
+
+    FSQueue p1 = queueManager.getParentQueue("root.parent1", false);
+    assertNotNull("Queue root.parent1 was not created", p1);
+    assertTrue("root.parent1 is not a dynamic queue", p1.isDynamic());
+
+    // adding root.parent1.leaf1 and root.parent1 to the allocation config
+    allocConf.configuredQueues.get(FSQueueType.PARENT).add("root.parent1");
+    allocConf.configuredQueues.get(FSQueueType.LEAF)
+        .add("root.parent1.leaf1");
+
+    // updateAllocationConfiguration() should change both queues over to static
+    queueManager.updateAllocationConfiguration(allocConf);
+    q1 = queueManager.getLeafQueue("root.parent1.leaf1", false);
+    assertFalse("root.parent1.leaf1 is not a static queue", q1.isDynamic());
+    p1 = queueManager.getParentQueue("root.parent1", false);
+    assertFalse("root.parent1 is not a static queue", p1.isDynamic());
+
+    // removing root.parent1.leaf1 and root.parent1 from the allocation
+    // config
+    allocConf.configuredQueues.get(FSQueueType.PARENT).remove("root.parent1");
+    allocConf.configuredQueues.get(FSQueueType.LEAF)
+        .remove("root.parent1.leaf1");
+
+    // updateAllocationConfiguration() should change both queues
+    // to dynamic
+    queueManager.updateAllocationConfiguration(allocConf);
+    queueManager.setQueuesToDynamic(
+        ImmutableSet.of("root.parent1", "root.parent1.leaf1"));
+    q1 = queueManager.getLeafQueue("root.parent1.leaf1", false);
+    assertTrue("root.parent1.leaf1 is not a dynamic queue", q1.isDynamic());
+    p1 = queueManager.getParentQueue("root.parent1", false);
+    assertTrue("root.parent1 is not a dynamic queue", p1.isDynamic());
+  }
+
+  @Test
+  public void testApplicationAssignmentPreventsRemovalOfDynamicQueue()
+      throws Exception {
+    AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
+    queueManager = new QueueManager(scheduler);
+    queueManager.initialize(conf);
+    queueManager.updateAllocationConfiguration(allocConf);
+
+    FSLeafQueue q = queueManager.getLeafQueue("root.leaf1", true);
+    assertNotNull("root.leaf1 does not exist", q);
+    assertTrue("root.leaf1 is not empty", queueManager.isEmpty(q));
+
+    // assigning an application (without an appAttempt so far) to the queue
+    // removeEmptyDynamicQueues() should not remove the queue
+    ApplicationId applicationId = ApplicationId.newInstance(1L, 0);
+    q.addAssignedApp(applicationId);
+    q = queueManager.getLeafQueue("root.leaf1", false);
+    assertFalse("root.leaf1 is empty", queueManager.isEmpty(q));
+
+    queueManager.removePendingIncompatibleQueues();
+    queueManager.removeEmptyDynamicQueues();
+    q = queueManager.getLeafQueue("root.leaf1", false);
+    assertNotNull("root.leaf1 has been removed", q);
+    assertFalse("root.leaf1 is empty", queueManager.isEmpty(q));
+
+    ApplicationAttemptId applicationAttemptId =
+        ApplicationAttemptId.newInstance(applicationId, 0);
+    ActiveUsersManager activeUsersManager =
+        Mockito.mock(ActiveUsersManager.class);
+    RMContext rmContext = Mockito.mock(RMContext.class);
+
+    // the appAttempt is created
+    // removeEmptyDynamicQueues() should not remove the queue
+    FSAppAttempt appAttempt = new FSAppAttempt(scheduler, applicationAttemptId,
+        "a_user", q, activeUsersManager, rmContext);
+    q.addApp(appAttempt, true);
+    queueManager.removeEmptyDynamicQueues();
+    q = queueManager.getLeafQueue("root.leaf1", false);
+    assertNotNull("root.leaf1 has been removed", q);
+    assertFalse("root.leaf1 is empty", queueManager.isEmpty(q));
+
+    // the appAttempt finished, the queue should be empty
+    q.removeApp(appAttempt);
+    q = queueManager.getLeafQueue("root.leaf1", false);
+    assertTrue("root.leaf1 is not empty", queueManager.isEmpty(q));
+
+    // removeEmptyDynamicQueues() should remove the queue
+    queueManager.removePendingIncompatibleQueues();
+    queueManager.removeEmptyDynamicQueues();
+    q = queueManager.getLeafQueue("root.leaf1", false);
+    assertNull("root.leaf1 has not been removed", q);
+  }
+
+  @Test
+  public void testRemovalOfIncompatibleNonEmptyQueue()
+      throws Exception {
+    AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
+    allocConf.configuredQueues.get(FSQueueType.LEAF).add("root.a");
+    scheduler.allocConf = allocConf;
+    queueManager.updateAllocationConfiguration(allocConf);
+
+    FSLeafQueue q = queueManager.getLeafQueue("root.a", true);
+    assertNotNull("root.a does not exist", q);
+    assertTrue("root.a is not empty", queueManager.isEmpty(q));
+
+    // we start to run an application on root.a
+    notEmptyQueues.add(q);
+    q = queueManager.getLeafQueue("root.a", false);
+    assertNotNull("root.a does not exist", q);
+    assertFalse("root.a is empty", queueManager.isEmpty(q));
+
+    // root.a should not be removed by removeEmptyDynamicQueues or by
+    // removePendingIncompatibleQueues
+    queueManager.removePendingIncompatibleQueues();
+    queueManager.removeEmptyDynamicQueues();
+    q = queueManager.getLeafQueue("root.a", false);
+    assertNotNull("root.a does not exist", q);
+
+    // let's introduce queue incompatibility
+    allocConf.configuredQueues.get(FSQueueType.LEAF).remove("root.a");
+    allocConf.configuredQueues.get(FSQueueType.PARENT).add("root.a");
+    allocConf.configuredQueues.get(FSQueueType.LEAF).add("root.a.b");
+    queueManager.updateAllocationConfiguration(allocConf);
+
+    // since root.a has running applications, it should be still a leaf queue
+    q = queueManager.getLeafQueue("root.a", false);
+    assertNotNull("root.a has been removed", q);
+    assertFalse("root.a is empty", queueManager.isEmpty(q));
+
+    // removePendingIncompatibleQueues should still keep root.a as a leaf queue
+    queueManager.removePendingIncompatibleQueues();
+    q = queueManager.getLeafQueue("root.a", false);
+    assertNotNull("root.a has been removed", q);
+    assertFalse("root.a is empty", queueManager.isEmpty(q));
+
+    // when the application finishes, root.a should be a parent queue
+    notEmptyQueues.clear();
+    queueManager.removePendingIncompatibleQueues();
+    queueManager.removeEmptyDynamicQueues();
+    FSParentQueue p = queueManager.getParentQueue("root.a", false);
+    assertNotNull("root.a does not exist", p);
+  }
+
 }


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


[40/50] [abbrv] hadoop git commit: YARN-8362. Bugfix logic in container retries in node manager. Contributed by Chandni Singh

Posted by ha...@apache.org.
YARN-8362.  Bugfix logic in container retries in node manager.
            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/135941e0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/135941e0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/135941e0

Branch: refs/heads/HDDS-48
Commit: 135941e00d762a417c3b4cc524cdc59b0d1810b1
Parents: 2416906
Author: Eric Yang <ey...@apache.org>
Authored: Tue May 29 16:56:58 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Tue May 29 16:56:58 2018 -0400

----------------------------------------------------------------------
 .../container/ContainerImpl.java                |  4 +-
 .../container/SlidingWindowRetryPolicy.java     | 62 +++++++++++---------
 .../container/TestSlidingWindowRetryPolicy.java |  6 ++
 3 files changed, 44 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/135941e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.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/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index c09c7f1..5527ac4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -1602,8 +1602,10 @@ public class ContainerImpl implements Container {
         }
         container.addDiagnostics(exitEvent.getDiagnosticInfo() + "\n");
       }
-
       if (container.shouldRetry(container.exitCode)) {
+        // Updates to the retry context should  be protected from concurrent
+        // writes. It should only be called from this transition.
+        container.retryPolicy.updateRetryContext(container.windowRetryContext);
         container.storeRetryContext();
         doRelaunch(container,
             container.windowRetryContext.getRemainingRetries(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/135941e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/SlidingWindowRetryPolicy.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/container/SlidingWindowRetryPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/SlidingWindowRetryPolicy.java
index 0208879..36a8b91 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/SlidingWindowRetryPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/SlidingWindowRetryPolicy.java
@@ -42,49 +42,40 @@ public class SlidingWindowRetryPolicy {
 
   public boolean shouldRetry(RetryContext retryContext,
       int errorCode) {
-    ContainerRetryContext containerRC = retryContext
-        .containerRetryContext;
+    ContainerRetryContext containerRC = retryContext.containerRetryContext;
     Preconditions.checkNotNull(containerRC, "container retry context null");
     ContainerRetryPolicy retryPolicy = containerRC.getRetryPolicy();
     if (retryPolicy == ContainerRetryPolicy.RETRY_ON_ALL_ERRORS
         || (retryPolicy == ContainerRetryPolicy.RETRY_ON_SPECIFIC_ERROR_CODES
         && containerRC.getErrorCodes() != null
         && containerRC.getErrorCodes().contains(errorCode))) {
-      if (containerRC.getMaxRetries() == ContainerRetryContext.RETRY_FOREVER) {
-        return true;
-      }
-      int pendingRetries = calculatePendingRetries(retryContext);
-      updateRetryContext(retryContext, pendingRetries);
-      return pendingRetries > 0;
+      return containerRC.getMaxRetries() == ContainerRetryContext.RETRY_FOREVER
+          || calculateRemainingRetries(retryContext) > 0;
     }
     return false;
   }
 
   /**
-   * Calculates the pending number of retries.
-   * <p>
-   * When failuresValidityInterval is > 0, it also removes time entries from
-   * <code>restartTimes</code> which are outside the validity interval.
+   * Calculates the remaining number of retries.
    *
-   * @return the pending retries.
+   * @return the remaining retries.
    */
-  private int calculatePendingRetries(RetryContext retryContext) {
+  private int calculateRemainingRetries(RetryContext retryContext) {
     ContainerRetryContext containerRC =
         retryContext.containerRetryContext;
     if (containerRC.getFailuresValidityInterval() > 0) {
-      Iterator<Long> iterator = retryContext.getRestartTimes().iterator();
+      int validFailuresCount = 0;
       long currentTime = clock.getTime();
-      while (iterator.hasNext()) {
-        long restartTime = iterator.next();
+      for (int i = retryContext.restartTimes.size() - 1; i >= 0; i--) {
+        long restartTime = retryContext.restartTimes.get(i);
         if (currentTime - restartTime
-            > containerRC.getFailuresValidityInterval()) {
-          iterator.remove();
+            <= containerRC.getFailuresValidityInterval()) {
+          validFailuresCount++;
         } else {
           break;
         }
       }
-      return containerRC.getMaxRetries() -
-          retryContext.getRestartTimes().size();
+      return containerRC.getMaxRetries() - validFailuresCount;
     } else {
       return retryContext.getRemainingRetries();
     }
@@ -93,13 +84,30 @@ public class SlidingWindowRetryPolicy {
   /**
    * Updates remaining retries and the restart time when
    * required in the retryContext.
+   * <p>
+   * When failuresValidityInterval is > 0, it also removes time entries from
+   * <code>restartTimes</code> which are outside the validity interval.
    */
-  private void updateRetryContext(RetryContext retryContext,
-      int pendingRetries) {
-    retryContext.setRemainingRetries(pendingRetries - 1);
-    if (retryContext.containerRetryContext.getFailuresValidityInterval()
-        > 0) {
-      retryContext.getRestartTimes().add(clock.getTime());
+  protected void updateRetryContext(RetryContext retryContext) {
+    if (retryContext.containerRetryContext.getFailuresValidityInterval() > 0) {
+      ContainerRetryContext containerRC = retryContext.containerRetryContext;
+      Iterator<Long> iterator = retryContext.getRestartTimes().iterator();
+      long currentTime = clock.getTime();
+
+      while (iterator.hasNext()) {
+        long restartTime = iterator.next();
+        if (currentTime - restartTime
+            > containerRC.getFailuresValidityInterval()) {
+          iterator.remove();
+        } else {
+          break;
+        }
+      }
+      retryContext.setRemainingRetries(containerRC.getMaxRetries() -
+          retryContext.restartTimes.size());
+      retryContext.getRestartTimes().add(currentTime);
+    } else {
+      retryContext.remainingRetries--;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/135941e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestSlidingWindowRetryPolicy.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/containermanager/container/TestSlidingWindowRetryPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestSlidingWindowRetryPolicy.java
index 04889a9..bacf3bb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestSlidingWindowRetryPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestSlidingWindowRetryPolicy.java
@@ -64,12 +64,18 @@ public class TestSlidingWindowRetryPolicy {
         new SlidingWindowRetryPolicy.RetryContext(retryContext);
     Assert.assertTrue("retry 1",
         retryPolicy.shouldRetry(windowRetryContext, 12));
+    retryPolicy.updateRetryContext(windowRetryContext);
+
     clock.setTime(20);
     Assert.assertTrue("retry 2",
         retryPolicy.shouldRetry(windowRetryContext, 12));
+    retryPolicy.updateRetryContext(windowRetryContext);
+
     clock.setTime(40);
     Assert.assertTrue("retry 3",
         retryPolicy.shouldRetry(windowRetryContext, 12));
+    retryPolicy.updateRetryContext(windowRetryContext);
+
     clock.setTime(45);
     Assert.assertFalse("retry failed",
         retryPolicy.shouldRetry(windowRetryContext, 12));


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


[45/50] [abbrv] hadoop git commit: HDFS-13632. Randomize baseDir for MiniJournalCluster in MiniQJMHACluster for TestDFSAdminWithHA. Contributed by Anbang Hu.

Posted by ha...@apache.org.
HDFS-13632. Randomize baseDir for MiniJournalCluster in MiniQJMHACluster for TestDFSAdminWithHA. Contributed by Anbang Hu.


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

Branch: refs/heads/HDDS-48
Commit: 8197b9b56040113806bdf328bbee68e95dd0aadd
Parents: e44c084
Author: Inigo Goiri <in...@apache.org>
Authored: Wed May 30 10:02:19 2018 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Wed May 30 10:13:52 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java | 10 ++++++++--
 .../org/apache/hadoop/hdfs/tools/TestDFSAdminWithHA.java  |  4 +++-
 2 files changed, 11 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8197b9b5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
index 1005f7f..f1f74dc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
@@ -49,6 +49,7 @@ public class MiniQJMHACluster {
     private int numNNs = 2;
     private final MiniDFSCluster.Builder dfsBuilder;
     private boolean forceRemoteEditsOnly = false;
+    private String baseDir;
 
     public Builder(Configuration conf) {
       this.conf = conf;
@@ -69,6 +70,11 @@ public class MiniQJMHACluster {
       this.startOpt = startOpt;
     }
 
+    public Builder baseDir(String d) {
+      this.baseDir = d;
+      return this;
+    }
+
     public Builder setNumNameNodes(int nns) {
       this.numNNs = nns;
       return this;
@@ -104,8 +110,8 @@ public class MiniQJMHACluster {
         basePort = 10000 + RANDOM.nextInt(1000) * 4;
         LOG.info("Set MiniQJMHACluster basePort to " + basePort);
         // start 3 journal nodes
-        journalCluster = new MiniJournalCluster.Builder(conf).format(true)
-            .build();
+        journalCluster = new MiniJournalCluster.Builder(conf)
+            .baseDir(builder.baseDir).format(true).build();
         journalCluster.waitActive();
         journalCluster.setNamenodeSharedEditsConf(NAMESERVICE);
         URI journalURI = journalCluster.getQuorumJournalURI(NAMESERVICE);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8197b9b5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdminWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdminWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdminWithHA.java
index b21084e..aa4d481 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdminWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdminWithHA.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.ha.BootstrapStandby;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Test;
 
@@ -85,7 +86,8 @@ public class TestDFSAdminWithHA {
     conf = new Configuration();
     conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
         security);
-    cluster = new MiniQJMHACluster.Builder(conf).build();
+    String baseDir = GenericTestUtils.getRandomizedTempPath();
+    cluster = new MiniQJMHACluster.Builder(conf).baseDir(baseDir).build();
     setHAConf(conf, cluster.getDfsCluster().getNameNode(0).getHostAndPort(),
         cluster.getDfsCluster().getNameNode(1).getHostAndPort());
     cluster.getDfsCluster().getNameNode(0).getHostAndPort();


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


[16/50] [abbrv] hadoop git commit: HDFS-13620. Randomize the test directory path for TestHDFSFileSystemContract. Contributed by Anbang Hu.

Posted by ha...@apache.org.
HDFS-13620. Randomize the test directory path for TestHDFSFileSystemContract. Contributed by Anbang Hu.


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

Branch: refs/heads/HDDS-48
Commit: 8605a38514b4f7a2a549c7ecf8e1421e61bb4d67
Parents: 2a9652e
Author: Inigo Goiri <in...@apache.org>
Authored: Fri May 25 19:43:33 2018 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Fri May 25 19:43:33 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/TestHDFSFileSystemContract.java     | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8605a385/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSFileSystemContract.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSFileSystemContract.java
index 50d1e75..6da46de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSFileSystemContract.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSFileSystemContract.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdfs;
 
+import java.io.File;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
@@ -25,6 +26,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystemContractBaseTest;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -39,7 +41,9 @@ public class TestHDFSFileSystemContract extends FileSystemContractBaseTest {
     Configuration conf = new HdfsConfiguration();
     conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY,
         FileSystemContractBaseTest.TEST_UMASK);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+    File basedir = GenericTestUtils.getRandomizedTestDir();
+    cluster = new MiniDFSCluster.Builder(conf, basedir).numDataNodes(2)
+        .build();
     fs = cluster.getFileSystem();
     defaultWorkingDirectory = "/user/" + 
            UserGroupInformation.getCurrentUser().getShortUserName();


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


[18/50] [abbrv] hadoop git commit: HDDS-78. Add per volume level storage stats in SCM. Contributed by Shashikant Banerjee.

Posted by ha...@apache.org.
HDDS-78. Add per volume level storage stats in SCM.
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/0cf6e87f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0cf6e87f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0cf6e87f

Branch: refs/heads/HDDS-48
Commit: 0cf6e87f9212af10eae39cdcb1fe60e6d8191772
Parents: f24c842
Author: Anu Engineer <ae...@apache.org>
Authored: Sat May 26 11:06:22 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Sat May 26 11:11:14 2018 -0700

----------------------------------------------------------------------
 .../placement/metrics/SCMNodeStat.java          |  21 --
 .../hdds/scm/node/SCMNodeStorageStatMXBean.java |   8 +
 .../hdds/scm/node/SCMNodeStorageStatMap.java    | 230 +++++++++++++------
 .../hdds/scm/node/StorageReportResult.java      |  87 +++++++
 .../scm/node/TestSCMNodeStorageStatMap.java     | 141 +++++++++---
 5 files changed, 356 insertions(+), 131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0cf6e87f/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/metrics/SCMNodeStat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/metrics/SCMNodeStat.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/metrics/SCMNodeStat.java
index 4fe72fc..3c871d3 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/metrics/SCMNodeStat.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/metrics/SCMNodeStat.java
@@ -136,25 +136,4 @@ public class SCMNodeStat implements NodeStat {
   public int hashCode() {
     return Long.hashCode(capacity.get() ^ scmUsed.get() ^ remaining.get());
   }
-
-
-  /**
-   * Truncate to 4 digits since uncontrolled precision is some times
-   * counter intuitive to what users expect.
-   * @param value - double.
-   * @return double.
-   */
-  private double truncateDecimals(double value) {
-    final int multiplier = 10000;
-    return (double) ((long) (value * multiplier)) / multiplier;
-  }
-
-  /**
-   * get the scmUsed ratio
-   */
-  public  double getScmUsedratio() {
-    double scmUsedRatio =
-        truncateDecimals(getScmUsed().get() / (double) getCapacity().get());
-    return scmUsedRatio;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0cf6e87f/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMXBean.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMXBean.java
index f17a970..d81ff0f 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMXBean.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMXBean.java
@@ -19,7 +19,9 @@
 package org.apache.hadoop.hdds.scm.node;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ozone.container.common.impl.StorageLocationReport;
 
+import java.util.Set;
 import java.util.UUID;
 
 /**
@@ -66,4 +68,10 @@ public interface SCMNodeStorageStatMXBean {
    * @return long
    */
   long getTotalFreeSpace();
+
+  /**
+   * Returns the set of disks for a given Datanode.
+   * @return set of storage volumes
+   */
+  Set<StorageLocationReport> getStorageVolumes(UUID datanodeId);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0cf6e87f/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMap.java
index 25cb357..f8ad2af 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeStorageStatMap.java
@@ -22,18 +22,18 @@ package org.apache.hadoop.hdds.scm.node;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
-import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
+import org.apache.hadoop.hdds.protocol.proto.
+    StorageContainerDatanodeProtocolProtos.SCMStorageReport;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.container.common.impl.StorageLocationReport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.management.ObjectName;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import java.io.IOException;
+import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.stream.Collectors;
 
@@ -52,16 +52,15 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
   private final double warningUtilizationThreshold;
   private final double criticalUtilizationThreshold;
 
-  private final Map<UUID, SCMNodeStat> scmNodeStorageStatMap;
+  private final Map<UUID, Set<StorageLocationReport>> scmNodeStorageReportMap;
   // NodeStorageInfo MXBean
   private ObjectName scmNodeStorageInfoBean;
-  // Aggregated node stats
-  private SCMNodeStat clusterStat;
   /**
-   * constructs the scmNodeStorageStatMap object
+   * constructs the scmNodeStorageReportMap object
    */
   public SCMNodeStorageStatMap(OzoneConfiguration conf) {
-    scmNodeStorageStatMap = new ConcurrentHashMap<>();
+    // scmNodeStorageReportMap = new ConcurrentHashMap<>();
+    scmNodeStorageReportMap = new ConcurrentHashMap<>();
     warningUtilizationThreshold = conf.getDouble(
         OzoneConfigKeys.
             HDDS_DATANODE_STORAGE_UTILIZATION_WARNING_THRESHOLD,
@@ -72,7 +71,6 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
             HDDS_DATANODE_STORAGE_UTILIZATION_CRITICAL_THRESHOLD,
         OzoneConfigKeys.
             HDDS_DATANODE_STORAGE_UTILIZATION_CRITICAL_THRESHOLD_DEFAULT);
-    clusterStat = new SCMNodeStat();
   }
 
   public enum UtilizationThreshold {
@@ -81,20 +79,22 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
 
   /**
    * Returns true if this a datanode that is already tracked by
-   * scmNodeStorageStatMap.
+   * scmNodeStorageReportMap.
    *
    * @param datanodeID - UUID of the Datanode.
    * @return True if this is tracked, false if this map does not know about it.
    */
   public boolean isKnownDatanode(UUID datanodeID) {
     Preconditions.checkNotNull(datanodeID);
-    return scmNodeStorageStatMap.containsKey(datanodeID);
+    return scmNodeStorageReportMap.containsKey(datanodeID);
   }
 
   public List<UUID> getDatanodeList(
       UtilizationThreshold threshold) {
-    return scmNodeStorageStatMap.entrySet().stream()
-        .filter(entry -> (isThresholdReached(threshold, entry.getValue())))
+    return scmNodeStorageReportMap.entrySet().stream().filter(
+        entry -> (isThresholdReached(threshold,
+            getScmUsedratio(getUsedSpace(entry.getKey()),
+                getCapacity(entry.getKey())))))
         .map(Map.Entry::getKey)
         .collect(Collectors.toList());
   }
@@ -105,19 +105,19 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
    * Insert a new datanode into Node2Container Map.
    *
    * @param datanodeID -- Datanode UUID
-   * @param stat - scmNode stat for the Datanode.
+   * @param report - set if StorageReports.
    */
-  public void insertNewDatanode(UUID datanodeID, SCMNodeStat stat)
+  public void insertNewDatanode(UUID datanodeID, Set<StorageLocationReport> report)
       throws SCMException {
-    Preconditions.checkNotNull(stat);
+    Preconditions.checkNotNull(report);
+    Preconditions.checkState(report.size() != 0);
     Preconditions.checkNotNull(datanodeID);
-    synchronized (scmNodeStorageStatMap) {
+    synchronized (scmNodeStorageReportMap) {
       if (isKnownDatanode(datanodeID)) {
         throw new SCMException("Node already exists in the map",
             DUPLICATE_DATANODE);
       }
-      scmNodeStorageStatMap.put(datanodeID, stat);
-      clusterStat.add(stat);
+      scmNodeStorageReportMap.putIfAbsent(datanodeID, report);
     }
   }
 
@@ -138,72 +138,103 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
    * Updates the Container list of an existing DN.
    *
    * @param datanodeID - UUID of DN.
-   * @param stat - scmNode stat for the Datanode.
+   * @param report - set of Storage Reports for the Datanode.
    * @throws SCMException - if we don't know about this datanode, for new DN
    *                      use insertNewDatanode.
    */
-  public void updateDatanodeMap(UUID datanodeID, SCMNodeStat stat)
+  public void updateDatanodeMap(UUID datanodeID, Set<StorageLocationReport> report)
       throws SCMException {
     Preconditions.checkNotNull(datanodeID);
-    Preconditions.checkNotNull(stat);
-    synchronized (scmNodeStorageStatMap) {
-      if (!scmNodeStorageStatMap.containsKey(datanodeID)) {
+    Preconditions.checkNotNull(report);
+    Preconditions.checkState(report.size() != 0);
+    synchronized (scmNodeStorageReportMap) {
+      if (!scmNodeStorageReportMap.containsKey(datanodeID)) {
         throw new SCMException("No such datanode", NO_SUCH_DATANODE);
       }
-      SCMNodeStat removed = scmNodeStorageStatMap.get(datanodeID);
-      clusterStat.subtract(removed);
-      scmNodeStorageStatMap.put(datanodeID, stat);
-      clusterStat.add(stat);
+      scmNodeStorageReportMap.put(datanodeID, report);
     }
   }
 
-  public NodeReportStatus processNodeReport(UUID datanodeID,
+  public StorageReportResult processNodeReport(UUID datanodeID,
       StorageContainerDatanodeProtocolProtos.SCMNodeReport nodeReport)
-      throws SCMException {
+      throws IOException {
     Preconditions.checkNotNull(datanodeID);
     Preconditions.checkNotNull(nodeReport);
+
     long totalCapacity = 0;
     long totalRemaining = 0;
     long totalScmUsed = 0;
-    List<StorageContainerDatanodeProtocolProtos.SCMStorageReport>
+    Set<StorageLocationReport> storagReportSet = new HashSet<>();
+    Set<StorageLocationReport> fullVolumeSet = new HashSet<>();
+    Set<StorageLocationReport> failedVolumeSet = new HashSet<>();
+    List<SCMStorageReport>
         storageReports = nodeReport.getStorageReportList();
-    for (StorageContainerDatanodeProtocolProtos.SCMStorageReport report : storageReports) {
+    for (SCMStorageReport report : storageReports) {
+      StorageLocationReport storageReport =
+          StorageLocationReport.getFromProtobuf(report);
+      storagReportSet.add(storageReport);
+      if (report.hasFailed() && report.getFailed()) {
+        failedVolumeSet.add(storageReport);
+      } else if (isThresholdReached(UtilizationThreshold.CRITICAL,
+          getScmUsedratio(report.getScmUsed(), report.getCapacity()))) {
+        fullVolumeSet.add(storageReport);
+      }
       totalCapacity += report.getCapacity();
       totalRemaining += report.getRemaining();
       totalScmUsed += report.getScmUsed();
     }
-    SCMNodeStat stat = scmNodeStorageStatMap.get(datanodeID);
-    if (stat == null) {
-      stat = new SCMNodeStat();
-      stat.set(totalCapacity, totalScmUsed, totalRemaining);
-      insertNewDatanode(datanodeID, stat);
+
+    if (!isKnownDatanode(datanodeID)) {
+      insertNewDatanode(datanodeID, storagReportSet);
     } else {
-      stat.set(totalCapacity, totalScmUsed, totalRemaining);
-      updateDatanodeMap(datanodeID, stat);
+      updateDatanodeMap(datanodeID, storagReportSet);
     }
-    if (isThresholdReached(UtilizationThreshold.CRITICAL, stat)) {
+    if (isThresholdReached(UtilizationThreshold.CRITICAL,
+        getScmUsedratio(totalScmUsed, totalCapacity))) {
       LOG.warn("Datanode {} is out of storage space. Capacity: {}, Used: {}",
-          datanodeID, stat.getCapacity().get(), stat.getScmUsed().get());
-      return NodeReportStatus.DATANODE_OUT_OF_SPACE;
-    } else {
-      if (isThresholdReached(UtilizationThreshold.WARN, stat)) {
-       LOG.warn("Datanode {} is low on storage space. Capacity: {}, Used: {}",
-           datanodeID, stat.getCapacity().get(), stat.getScmUsed().get());
-      }
-      return NodeReportStatus.ALL_IS_WELL;
+          datanodeID, totalCapacity, totalScmUsed);
+      return StorageReportResult.ReportResultBuilder.newBuilder()
+          .setStatus(ReportStatus.DATANODE_OUT_OF_SPACE)
+          .setFullVolumeSet(fullVolumeSet).setFailedVolumeSet(failedVolumeSet)
+          .build();
+    }
+    if (isThresholdReached(UtilizationThreshold.WARN,
+        getScmUsedratio(totalScmUsed, totalCapacity))) {
+      LOG.warn("Datanode {} is low on storage space. Capacity: {}, Used: {}",
+          datanodeID, totalCapacity, totalScmUsed);
     }
+
+    if (failedVolumeSet.isEmpty() && !fullVolumeSet.isEmpty()) {
+      return StorageReportResult.ReportResultBuilder.newBuilder()
+          .setStatus(ReportStatus.STORAGE_OUT_OF_SPACE)
+          .setFullVolumeSet(fullVolumeSet).build();
+    }
+
+    if (!failedVolumeSet.isEmpty() && fullVolumeSet.isEmpty()) {
+      return StorageReportResult.ReportResultBuilder.newBuilder()
+          .setStatus(ReportStatus.FAILED_STORAGE)
+          .setFailedVolumeSet(failedVolumeSet).build();
+    }
+    if (!failedVolumeSet.isEmpty() && !fullVolumeSet.isEmpty()) {
+      return StorageReportResult.ReportResultBuilder.newBuilder()
+          .setStatus(ReportStatus.FAILED_AND_OUT_OF_SPACE_STORAGE)
+          .setFailedVolumeSet(failedVolumeSet).setFullVolumeSet(fullVolumeSet)
+          .build();
+    }
+    return StorageReportResult.ReportResultBuilder.newBuilder()
+        .setStatus(ReportStatus.ALL_IS_WELL).build();
   }
 
   private boolean isThresholdReached(UtilizationThreshold threshold,
-      SCMNodeStat stat) {
+      double scmUsedratio) {
     switch (threshold) {
     case NORMAL:
-      return stat.getScmUsedratio() < warningUtilizationThreshold;
+      return scmUsedratio < warningUtilizationThreshold;
     case WARN:
-      return stat.getScmUsedratio() >= warningUtilizationThreshold &&
-          stat.getScmUsedratio() < criticalUtilizationThreshold;
+      return scmUsedratio >= warningUtilizationThreshold
+          && scmUsedratio < criticalUtilizationThreshold;
     case CRITICAL:
-      return stat.getScmUsedratio() >= criticalUtilizationThreshold;
+      return scmUsedratio >= criticalUtilizationThreshold;
     default:
       throw new RuntimeException("Unknown UtilizationThreshold value");
     }
@@ -211,67 +242,120 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
 
   @Override
   public long getCapacity(UUID dnId) {
-    return scmNodeStorageStatMap.get(dnId).getCapacity().get();
+    long capacity = 0;
+    Set<StorageLocationReport> reportSet = scmNodeStorageReportMap.get(dnId);
+    for (StorageLocationReport report : reportSet) {
+      capacity += report.getCapacity();
+    }
+    return capacity;
   }
 
   @Override
   public long getRemainingSpace(UUID dnId) {
-    return scmNodeStorageStatMap.get(dnId).getRemaining().get();
+    long remaining = 0;
+    Set<StorageLocationReport> reportSet = scmNodeStorageReportMap.get(dnId);
+    for (StorageLocationReport report : reportSet) {
+      remaining += report.getRemaining();
+    }
+    return remaining;
   }
 
   @Override
   public long getUsedSpace(UUID dnId) {
-    return scmNodeStorageStatMap.get(dnId).getScmUsed().get();
+    long scmUsed = 0;
+    Set<StorageLocationReport> reportSet = scmNodeStorageReportMap.get(dnId);
+    for (StorageLocationReport report : reportSet) {
+      scmUsed += report.getScmUsed();
+    }
+    return scmUsed;
   }
 
   @Override
   public long getTotalCapacity() {
-    return clusterStat.getCapacity().get();
+    long capacity = 0;
+    Set<UUID> dnIdSet = scmNodeStorageReportMap.keySet();
+    for (UUID id : dnIdSet) {
+      capacity += getCapacity(id);
+    }
+    return capacity;
   }
 
   @Override
   public long getTotalSpaceUsed() {
-    return clusterStat.getScmUsed().get();
+    long scmUsed = 0;
+    Set<UUID> dnIdSet = scmNodeStorageReportMap.keySet();
+    for (UUID id : dnIdSet) {
+      scmUsed += getUsedSpace(id);
+    }
+    return scmUsed;
   }
 
   @Override
   public long getTotalFreeSpace() {
-    return clusterStat.getRemaining().get();
+    long remaining = 0;
+    Set<UUID> dnIdSet = scmNodeStorageReportMap.keySet();
+    for (UUID id : dnIdSet) {
+      remaining += getRemainingSpace(id);
+    }
+    return remaining;
   }
 
   /**
-   * removes the dataNode from scmNodeStorageStatMap
+   * removes the dataNode from scmNodeStorageReportMap
    * @param datanodeID
    * @throws SCMException in case the dataNode is not found in the map.
    */
   public void removeDatanode(UUID datanodeID) throws SCMException {
     Preconditions.checkNotNull(datanodeID);
-    synchronized (scmNodeStorageStatMap) {
-      if (!scmNodeStorageStatMap.containsKey(datanodeID)) {
+    synchronized (scmNodeStorageReportMap) {
+      if (!scmNodeStorageReportMap.containsKey(datanodeID)) {
         throw new SCMException("No such datanode", NO_SUCH_DATANODE);
       }
-      SCMNodeStat stat = scmNodeStorageStatMap.remove(datanodeID);
-      clusterStat.subtract(stat);
+      scmNodeStorageReportMap.remove(datanodeID);
     }
   }
 
   /**
-   * Gets the SCMNodeStat for the datanode
+   * Returns the set of storage volumes for a Datanode.
    * @param  datanodeID
-   * @return SCMNodeStat
+   * @return set of storage volumes.
    */
 
-  SCMNodeStat getNodeStat(UUID datanodeID) {
-    return scmNodeStorageStatMap.get(datanodeID);
+  @Override
+  public Set<StorageLocationReport> getStorageVolumes(UUID datanodeID) {
+    return scmNodeStorageReportMap.get(datanodeID);
   }
 
+
+  /**
+   * Truncate to 4 digits since uncontrolled precision is some times
+   * counter intuitive to what users expect.
+   * @param value - double.
+   * @return double.
+   */
+  private double truncateDecimals(double value) {
+    final int multiplier = 10000;
+    return (double) ((long) (value * multiplier)) / multiplier;
+  }
+
+  /**
+   * get the scmUsed ratio
+   */
+  public  double getScmUsedratio(long scmUsed, long capacity) {
+    double scmUsedRatio =
+        truncateDecimals (scmUsed / (double) capacity);
+    return scmUsedRatio;
+  }
   /**
    * Results possible from processing a Node report by
    * Node2ContainerMapper.
    */
-  public enum NodeReportStatus {
+  public enum ReportStatus {
     ALL_IS_WELL,
-    DATANODE_OUT_OF_SPACE
+    DATANODE_OUT_OF_SPACE,
+    STORAGE_OUT_OF_SPACE,
+    FAILED_STORAGE,
+    FAILED_AND_OUT_OF_SPACE_STORAGE
   }
 
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0cf6e87f/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StorageReportResult.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StorageReportResult.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StorageReportResult.java
new file mode 100644
index 0000000..3436e77
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StorageReportResult.java
@@ -0,0 +1,87 @@
+
+/*
+ * 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.node;
+
+import org.apache.hadoop.ozone.container.common.impl.StorageLocationReport;
+
+import java.util.Set;
+
+/**
+ * A Container Report gets processsed by the Node2Container and returns the
+ * Report Result class.
+ */
+public class StorageReportResult {
+  private SCMNodeStorageStatMap.ReportStatus status;
+  private Set<StorageLocationReport> fullVolumes;
+  private Set<StorageLocationReport> failedVolumes;
+
+  StorageReportResult(SCMNodeStorageStatMap.ReportStatus status,
+      Set<StorageLocationReport> fullVolumes,
+      Set<StorageLocationReport> failedVolumes) {
+    this.status = status;
+    this.fullVolumes = fullVolumes;
+    this.failedVolumes = failedVolumes;
+  }
+
+  public SCMNodeStorageStatMap.ReportStatus getStatus() {
+    return status;
+  }
+
+  public Set<StorageLocationReport> getFullVolumes() {
+    return fullVolumes;
+  }
+
+  public Set<StorageLocationReport> getFailedVolumes() {
+    return failedVolumes;
+  }
+
+  static class ReportResultBuilder {
+    private SCMNodeStorageStatMap.ReportStatus status;
+    private Set<StorageLocationReport> fullVolumes;
+    private Set<StorageLocationReport> failedVolumes;
+
+    static ReportResultBuilder newBuilder() {
+      return new ReportResultBuilder();
+    }
+
+    public ReportResultBuilder setStatus(
+        SCMNodeStorageStatMap.ReportStatus newstatus) {
+      this.status = newstatus;
+      return this;
+    }
+
+    public ReportResultBuilder setFullVolumeSet(
+        Set<StorageLocationReport> fullVolumes) {
+      this.fullVolumes = fullVolumes;
+      return this;
+    }
+
+    public ReportResultBuilder setFailedVolumeSet(
+        Set<StorageLocationReport> failedVolumes) {
+      this.failedVolumes = failedVolumes;
+      return this;
+    }
+
+    StorageReportResult build() {
+      return new StorageReportResult(status, fullVolumes, failedVolumes);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0cf6e87f/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeStorageStatMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeStorageStatMap.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeStorageStatMap.java
index 2fa786b..571de77 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeStorageStatMap.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeStorageStatMap.java
@@ -17,38 +17,56 @@
  */
 package org.apache.hadoop.hdds.scm.node;
 
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.protocol.proto.
+    StorageContainerDatanodeProtocolProtos.SCMNodeReport;
+import org.apache.hadoop.hdds.protocol.proto.
+    StorageContainerDatanodeProtocolProtos.SCMStorageReport;
 import org.apache.hadoop.hdds.scm.TestUtils;
-import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMStorageReport;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.impl.StorageLocationReport;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.*;
+import org.junit.Rule;
 import org.junit.rules.ExpectedException;
 
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.io.IOException;
 import java.util.concurrent.ConcurrentHashMap;
 
 public class TestSCMNodeStorageStatMap {
-  private final static int DATANODE_COUNT = 300;
+  private final static int DATANODE_COUNT = 100;
   final long capacity = 10L * OzoneConsts.GB;
   final long used = 2L * OzoneConsts.GB;
   final long remaining = capacity - used;
   private static OzoneConfiguration conf = new OzoneConfiguration();
-  private final Map<UUID, SCMNodeStat> testData = new ConcurrentHashMap<>();
+  private final Map<UUID, Set<StorageLocationReport>> testData =
+      new ConcurrentHashMap<>();
 
   @Rule
   public ExpectedException thrown = ExpectedException.none();
 
   private void generateData() {
-    SCMNodeStat stat = new SCMNodeStat();
-    stat.set(capacity, used, remaining);
     for (int dnIndex = 1; dnIndex <= DATANODE_COUNT; dnIndex++) {
-      testData.put(UUID.randomUUID(), stat);
+      UUID dnId = UUID.randomUUID();
+      Set<StorageLocationReport> reportSet = new HashSet<>();
+      String path = GenericTestUtils.getTempPath(
+          TestSCMNodeStorageStatMap.class.getSimpleName() + "-" + Integer
+              .toString(dnIndex));
+      StorageLocationReport.Builder builder = StorageLocationReport.newBuilder();
+      builder.setStorageType(StorageType.DISK).setId(dnId.toString())
+          .setStorageLocation(path).setScmUsed(used).setRemaining(remaining)
+          .setCapacity(capacity).setFailed(false);
+      reportSet.add(builder.build());
+      testData.put(UUID.randomUUID(), reportSet);
     }
   }
 
@@ -70,8 +88,8 @@ public class TestSCMNodeStorageStatMap {
     SCMNodeStorageStatMap map = new SCMNodeStorageStatMap(conf);
     UUID knownNode = getFirstKey();
     UUID unknownNode = UUID.randomUUID();
-    SCMNodeStat stat = testData.get(knownNode);
-    map.insertNewDatanode(knownNode, stat);
+    Set<StorageLocationReport> report = testData.get(knownNode);
+    map.insertNewDatanode(knownNode, report);
     Assert.assertTrue("Not able to detect a known node",
         map.isKnownDatanode(knownNode));
     Assert.assertFalse("Unknown node detected",
@@ -82,54 +100,89 @@ public class TestSCMNodeStorageStatMap {
   public void testInsertNewDatanode() throws SCMException {
     SCMNodeStorageStatMap map = new SCMNodeStorageStatMap(conf);
     UUID knownNode = getFirstKey();
-    SCMNodeStat stat = testData.get(knownNode);
-    map.insertNewDatanode(knownNode, stat);
-    Assert.assertEquals(map.getNodeStat(knownNode).getScmUsed(),
-        testData.get(knownNode).getScmUsed());
+    Set<StorageLocationReport> report = testData.get(knownNode);
+    map.insertNewDatanode(knownNode, report);
+    Assert.assertEquals(map.getStorageVolumes(knownNode),
+        testData.get(knownNode));
     thrown.expect(SCMException.class);
     thrown.expectMessage("already exists");
-    map.insertNewDatanode(knownNode, stat);
+    map.insertNewDatanode(knownNode, report);
   }
 
   @Test
   public void testUpdateUnknownDatanode() throws SCMException {
     SCMNodeStorageStatMap map = new SCMNodeStorageStatMap(conf);
     UUID unknownNode = UUID.randomUUID();
-    SCMNodeStat stat = new SCMNodeStat();
-
+    String path = GenericTestUtils.getTempPath(
+        TestSCMNodeStorageStatMap.class.getSimpleName() + "-" + unknownNode
+            .toString());
+    Set<StorageLocationReport> reportSet = new HashSet<>();
+    StorageLocationReport.Builder builder = StorageLocationReport.newBuilder();
+    builder.setStorageType(StorageType.DISK).setId(unknownNode.toString())
+        .setStorageLocation(path).setScmUsed(used).setRemaining(remaining)
+        .setCapacity(capacity).setFailed(false);
+    reportSet.add(builder.build());
     thrown.expect(SCMException.class);
     thrown.expectMessage("No such datanode");
-    map.updateDatanodeMap(unknownNode, stat);
+    map.updateDatanodeMap(unknownNode, reportSet);
   }
 
   @Test
-  public void testProcessNodeReportCheckOneNode() throws SCMException {
+  public void testProcessNodeReportCheckOneNode() throws IOException {
     UUID key = getFirstKey();
-    SCMNodeStat value = testData.get(key);
+    List<SCMStorageReport> reportList = new ArrayList<>();
+    Set<StorageLocationReport> reportSet = testData.get(key);
     SCMNodeStorageStatMap map = new SCMNodeStorageStatMap(conf);
-    map.insertNewDatanode(key, value);
+    map.insertNewDatanode(key, reportSet);
     Assert.assertTrue(map.isKnownDatanode(key));
     String storageId = UUID.randomUUID().toString();
     String path =
         GenericTestUtils.getRandomizedTempPath().concat("/" + storageId);
-    long capacity = value.getCapacity().get();
-    long used = value.getScmUsed().get();
-    long remaining = value.getRemaining().get();
+    StorageLocationReport report = reportSet.iterator().next();
+    long capacity = report.getCapacity();
+    long used = report.getScmUsed();
+    long remaining = report.getRemaining();
     List<SCMStorageReport> reports = TestUtils
         .createStorageReport(capacity, used, remaining, path, null, storageId,
             1);
-    SCMNodeStorageStatMap.NodeReportStatus status =
+    StorageReportResult result =
         map.processNodeReport(key, TestUtils.createNodeReport(reports));
-    Assert.assertEquals(status,
-        SCMNodeStorageStatMap.NodeReportStatus.ALL_IS_WELL);
+    Assert.assertEquals(result.getStatus(),
+        SCMNodeStorageStatMap.ReportStatus.ALL_IS_WELL);
+    StorageContainerDatanodeProtocolProtos.SCMNodeReport.Builder nrb =
+        SCMNodeReport.newBuilder();
+    SCMStorageReport srb = reportSet.iterator().next().getProtoBufMessage();
+    reportList.add(srb);
+    result = map.processNodeReport(key, TestUtils.createNodeReport(reportList));
+    Assert.assertEquals(result.getStatus(),
+        SCMNodeStorageStatMap.ReportStatus.ALL_IS_WELL);
+
+    reportList.add(TestUtils
+        .createStorageReport(capacity, capacity, 0, path, null,
+            UUID.randomUUID().toString(), 1).get(0));
+    result = map.processNodeReport(key, TestUtils.createNodeReport(reportList));
+    Assert.assertEquals(result.getStatus(),
+        SCMNodeStorageStatMap.ReportStatus.STORAGE_OUT_OF_SPACE);
+    // Mark a disk failed 
+    SCMStorageReport srb2 = SCMStorageReport.newBuilder()
+        .setStorageUuid(UUID.randomUUID().toString())
+        .setStorageLocation(srb.getStorageLocation()).setScmUsed(capacity)
+        .setCapacity(capacity).setRemaining(0).setFailed(true).build();
+    reportList.add(srb2);
+    nrb.addAllStorageReport(reportList);
+    result = map.processNodeReport(key, nrb.addStorageReport(srb).build());
+    Assert.assertEquals(result.getStatus(),
+        SCMNodeStorageStatMap.ReportStatus.FAILED_AND_OUT_OF_SPACE_STORAGE);
+
   }
 
   @Test
-  public void testProcessNodeReportAndSCMStats() throws SCMException {
+  public void testProcessMultipleNodeReports() throws SCMException {
     SCMNodeStorageStatMap map = new SCMNodeStorageStatMap(conf);
     int counter = 1;
     // Insert all testData into the SCMNodeStorageStatMap Map.
-    for (Map.Entry<UUID, SCMNodeStat> keyEntry : testData.entrySet()) {
+    for (Map.Entry<UUID, Set<StorageLocationReport>> keyEntry : testData
+        .entrySet()) {
       map.insertNewDatanode(keyEntry.getKey(), keyEntry.getValue());
     }
     Assert.assertEquals(DATANODE_COUNT * capacity, map.getTotalCapacity());
@@ -137,9 +190,21 @@ public class TestSCMNodeStorageStatMap {
     Assert.assertEquals(DATANODE_COUNT * used, map.getTotalSpaceUsed());
 
     // upadate 1/4th of the datanode to be full
-    for (Map.Entry<UUID, SCMNodeStat> keyEntry : testData.entrySet()) {
-      SCMNodeStat stat = new SCMNodeStat(capacity, capacity, 0);
-      map.updateDatanodeMap(keyEntry.getKey(), stat);
+    for (Map.Entry<UUID, Set<StorageLocationReport>> keyEntry : testData
+        .entrySet()) {
+      Set<StorageLocationReport> reportSet = new HashSet<>();
+      String path = GenericTestUtils.getTempPath(
+          TestSCMNodeStorageStatMap.class.getSimpleName() + "-" + keyEntry
+              .getKey().toString());
+      StorageLocationReport.Builder builder =
+          StorageLocationReport.newBuilder();
+      builder.setStorageType(StorageType.DISK)
+          .setId(keyEntry.getKey().toString()).setStorageLocation(path)
+          .setScmUsed(capacity).setRemaining(0).setCapacity(capacity)
+          .setFailed(false);
+      reportSet.add(builder.build());
+
+      map.updateDatanodeMap(keyEntry.getKey(), reportSet);
       counter++;
       if (counter > DATANODE_COUNT / 4) {
         break;
@@ -163,7 +228,8 @@ public class TestSCMNodeStorageStatMap {
         map.getTotalSpaceUsed(), 0);
     counter = 1;
     // Remove 1/4 of the DataNodes from the Map
-    for (Map.Entry<UUID, SCMNodeStat> keyEntry : testData.entrySet()) {
+    for (Map.Entry<UUID, Set<StorageLocationReport>> keyEntry : testData
+        .entrySet()) {
       map.removeDatanode(keyEntry.getKey());
       counter++;
       if (counter > DATANODE_COUNT / 4) {
@@ -181,12 +247,13 @@ public class TestSCMNodeStorageStatMap {
         map.getDatanodeList(SCMNodeStorageStatMap.UtilizationThreshold.NORMAL)
             .size(), 0);
 
-    Assert.assertEquals(0.75 * DATANODE_COUNT * capacity, map.getTotalCapacity(), 0);
+    Assert
+        .assertEquals(0.75 * DATANODE_COUNT * capacity, map.getTotalCapacity(),
+            0);
     Assert.assertEquals(0.75 * DATANODE_COUNT * remaining,
         map.getTotalFreeSpace(), 0);
-    Assert.assertEquals(
-        0.75 * DATANODE_COUNT * used ,
-        map.getTotalSpaceUsed(), 0);
+    Assert
+        .assertEquals(0.75 * DATANODE_COUNT * used, map.getTotalSpaceUsed(), 0);
 
   }
 }


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


[14/50] [abbrv] hadoop git commit: HDFS-13619. TestAuditLoggerWithCommands fails on Windows. Contributed by Anbang Hu.

Posted by ha...@apache.org.
HDFS-13619. TestAuditLoggerWithCommands fails on Windows. Contributed by Anbang Hu.


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

Branch: refs/heads/HDDS-48
Commit: 13d25289076b39daf481fb1ee15939dbfe4a6b23
Parents: 8733012
Author: Inigo Goiri <in...@apache.org>
Authored: Fri May 25 13:32:34 2018 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Fri May 25 13:32:34 2018 -0700

----------------------------------------------------------------------
 .../hdfs/server/namenode/TestAuditLoggerWithCommands.java       | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/13d25289/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
index 41ee03f..222a1de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
@@ -1264,8 +1264,9 @@ public class TestAuditLoggerWithCommands {
   }
 
   private int verifyAuditLogs(String pattern) {
-    int length = auditlog.getOutput().split("\n").length;
-    String lastAudit = auditlog.getOutput().split("\n")[length - 1];
+    int length = auditlog.getOutput().split(System.lineSeparator()).length;
+    String lastAudit = auditlog.getOutput()
+        .split(System.lineSeparator())[length - 1];
     assertTrue("Unexpected log!", lastAudit.matches(pattern));
     return length;
   }


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


[13/50] [abbrv] hadoop git commit: HDDS-96. Add an option in ozone script to generate a site file with minimally required ozone configs. Contributed by Dinesh Chitlangia.

Posted by ha...@apache.org.
HDDS-96. Add an option in ozone script to generate a site file with minimally required ozone configs.
Contributed by Dinesh Chitlangia.


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

Branch: refs/heads/HDDS-48
Commit: 8733012ae35f2762d704f94975a762885d116795
Parents: 1e0d4b1
Author: Anu Engineer <ae...@apache.org>
Authored: Fri May 25 13:06:14 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri May 25 13:06:14 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hdds/conf/OzoneConfiguration.java    |   6 +-
 hadoop-ozone/common/src/main/bin/ozone          |   4 +
 ...TestGenerateOzoneRequiredConfigurations.java | 100 +++++++++++
 .../GenerateOzoneRequiredConfigurations.java    | 174 +++++++++++++++++++
 .../hadoop/ozone/genconf/package-info.java      |  24 +++
 5 files changed, 305 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8733012a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/OzoneConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/OzoneConfiguration.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/OzoneConfiguration.java
index f07718c..36d953c 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/OzoneConfiguration.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/OzoneConfiguration.java
@@ -137,7 +137,7 @@ public class OzoneConfiguration extends Configuration {
 
     @Override
     public String toString() {
-      return this.getName() + " " + this.getValue() + this.getTag();
+      return this.getName() + " " + this.getValue() + " " + this.getTag();
     }
 
     @Override
@@ -152,11 +152,11 @@ public class OzoneConfiguration extends Configuration {
     }
   }
 
-  public static void activate(){
+  public static void activate() {
     // adds the default resources
     Configuration.addDefaultResource("hdfs-default.xml");
     Configuration.addDefaultResource("hdfs-site.xml");
     Configuration.addDefaultResource("ozone-default.xml");
     Configuration.addDefaultResource("ozone-site.xml");
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8733012a/hadoop-ozone/common/src/main/bin/ozone
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/bin/ozone b/hadoop-ozone/common/src/main/bin/ozone
index 00261c7..6843bdd 100755
--- a/hadoop-ozone/common/src/main/bin/ozone
+++ b/hadoop-ozone/common/src/main/bin/ozone
@@ -47,6 +47,7 @@ function hadoop_usage
   hadoop_add_subcommand "scm" daemon "run the Storage Container Manager service"
   hadoop_add_subcommand "scmcli" client "run the CLI of the Storage Container Manager "
   hadoop_add_subcommand "version" client "print the version"
+  hadoop_add_subcommand "genconf" client "generate minimally required ozone configs and output to ozone-site.xml in specified path"
 
   hadoop_generate_usage "${HADOOP_SHELL_EXECNAME}" false
 }
@@ -118,6 +119,9 @@ function ozonecmd_case
     version)
       HADOOP_CLASSNAME=org.apache.hadoop.util.VersionInfo
     ;;
+    genconf)
+      HADOOP_CLASSNAME=org.apache.hadoop.ozone.genconf.GenerateOzoneRequiredConfigurations
+    ;;
     *)
       HADOOP_CLASSNAME="${subcmd}"
       if ! hadoop_validate_classname "${HADOOP_CLASSNAME}"; then

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8733012a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/genconf/TestGenerateOzoneRequiredConfigurations.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/genconf/TestGenerateOzoneRequiredConfigurations.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/genconf/TestGenerateOzoneRequiredConfigurations.java
new file mode 100644
index 0000000..82582a6
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/genconf/TestGenerateOzoneRequiredConfigurations.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
+ * <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.genconf;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+
+
+/**
+ * Tests GenerateOzoneRequiredConfigurations.
+ */
+public class TestGenerateOzoneRequiredConfigurations {
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration conf;
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   * <p>
+   * Ozone is made active by setting OZONE_ENABLED = true and
+   * OZONE_HANDLER_TYPE_KEY = "distributed"
+   *
+   * @throws IOException
+   */
+  @BeforeClass
+  public static void init() throws Exception {
+    conf = new OzoneConfiguration();
+    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
+    cluster.waitForClusterToBeReady();
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterClass
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Tests a valid path and generates ozone-site.xml.
+   * @throws Exception
+   */
+  @Test
+  public void generateConfigurationsSuccess() throws Exception {
+    String[] args = new String[]{"-output", "."};
+    GenerateOzoneRequiredConfigurations.main(args);
+
+    Assert.assertEquals("Path is valid",
+        true, GenerateOzoneRequiredConfigurations.isValidPath(args[1]));
+
+    Assert.assertEquals("Permission is valid",
+        true, GenerateOzoneRequiredConfigurations.canWrite(args[1]));
+
+    Assert.assertEquals("Config file generated",
+        0, GenerateOzoneRequiredConfigurations.generateConfigurations(args[1]));
+  }
+
+  /**
+   * Test to avoid generating ozone-site.xml when invalid permission.
+   * @throws Exception
+   */
+  @Test
+  public void generateConfigurationsFailure() throws Exception {
+    String[] args = new String[]{"-output", "/"};
+    GenerateOzoneRequiredConfigurations.main(args);
+
+    Assert.assertEquals("Path is valid",
+        true, GenerateOzoneRequiredConfigurations.isValidPath(args[1]));
+
+    Assert.assertEquals("Invalid permission",
+        false, GenerateOzoneRequiredConfigurations.canWrite(args[1]));
+
+    Assert.assertEquals("Config file not generated",
+        1, GenerateOzoneRequiredConfigurations.generateConfigurations(args[1]));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8733012a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genconf/GenerateOzoneRequiredConfigurations.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genconf/GenerateOzoneRequiredConfigurations.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genconf/GenerateOzoneRequiredConfigurations.java
new file mode 100644
index 0000000..6296c9d
--- /dev/null
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genconf/GenerateOzoneRequiredConfigurations.java
@@ -0,0 +1,174 @@
+/**
+ * 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.genconf;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import java.io.File;
+import java.net.URL;
+import java.nio.file.InvalidPathException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * GenerateOzoneRequiredConfigurations - A tool to generate ozone-site.xml<br>
+ * This tool generates an ozone-site.xml with minimally required configs.
+ * This tool can be invoked as follows:<br>
+ * <ul>
+ * <li>ozone genconf -output <Path to output file></li>
+ * <li>ozone genconf -help</li>
+ * </ul>
+ */
+public final class GenerateOzoneRequiredConfigurations {
+
+  private static final String OUTPUT = "-output";
+  private static final String HELP = "-help";
+  private static final String USAGE = "Usage: \nozone genconf "
+      + OUTPUT + " <Path to output file> \n"
+      + "ozone genconf "
+      + HELP;
+  private static final int SUCCESS = 0;
+  private static final int FAILURE = 1;
+
+  private GenerateOzoneRequiredConfigurations() {
+
+  }
+  /**
+   * Entry point for using genconf tool.
+   *
+   * @param args
+   * @throws JAXBException
+   */
+  public static void main(String[] args) {
+
+    try {
+      if (args.length == 0) {
+        System.out.println(USAGE);
+        System.exit(1);
+      }
+
+      switch (args[0]) {
+      case OUTPUT:
+        if (args.length > 1) {
+          int result = generateConfigurations(args[1]);
+        } else {
+          System.out.println("Path to output file is mandatory");
+          System.out.println(USAGE);
+          System.exit(1);
+        }
+        break;
+
+      case HELP:
+        System.out.println(USAGE);
+        System.exit(0);
+        break;
+
+      default:
+        System.out.println(USAGE);
+        System.exit(1);
+      }
+
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+  /**
+   * Check if the path is valid.
+   *
+   * @param path
+   * @return true, if path is valid, else return false
+   */
+  public static boolean isValidPath(String path) {
+    try {
+      Paths.get(path);
+    } catch (InvalidPathException | NullPointerException ex) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Check if user has permission to write in the specified path.
+   *
+   * @param path
+   * @return true, if the user has permission to write, else returns false
+   */
+  public static boolean canWrite(String path) {
+    File file = new File(path);
+    return file.canWrite();
+  }
+
+  /**
+   * Generate ozone-site.xml at specified path.
+   *
+   * @param path
+   * @return SUCCESS(0) if file can be generated, else returns FAILURE(1)
+   * @throws JAXBException
+   */
+  public static int generateConfigurations(String path) throws JAXBException {
+
+    if (!isValidPath(path)) {
+      System.out.println("Invalid path or insufficient permission");
+      return FAILURE;
+    }
+
+    if (!canWrite(path)) {
+      System.out.println("Invalid path or insufficient permission");
+      return FAILURE;
+    }
+
+    OzoneConfiguration oc = new OzoneConfiguration();
+
+    ClassLoader cL = Thread.currentThread().getContextClassLoader();
+    if (cL == null) {
+      cL = OzoneConfiguration.class.getClassLoader();
+    }
+    URL url = cL.getResource("ozone-default.xml");
+
+    List<OzoneConfiguration.Property> allProperties =
+        oc.readPropertyFromXml(url);
+
+    List<OzoneConfiguration.Property> requiredProperties = new ArrayList<>();
+
+    for (OzoneConfiguration.Property p : allProperties) {
+      if (p.getTag() != null && p.getTag().contains("REQUIRED")) {
+        requiredProperties.add(p);
+      }
+    }
+
+    OzoneConfiguration.XMLConfiguration requiredConfig =
+        new OzoneConfiguration.XMLConfiguration();
+    requiredConfig.setProperties(requiredProperties);
+
+    JAXBContext context =
+        JAXBContext.newInstance(OzoneConfiguration.XMLConfiguration.class);
+    Marshaller m = context.createMarshaller();
+    m.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, Boolean.TRUE);
+    m.marshal(requiredConfig, new File(path, "ozone-site.xml"));
+
+    System.out.println("ozone-site.xml has been generated at " + path);
+
+    return SUCCESS;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8733012a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genconf/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genconf/package-info.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genconf/package-info.java
new file mode 100644
index 0000000..4817d39
--- /dev/null
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genconf/package-info.java
@@ -0,0 +1,24 @@
+/**
+ * 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.genconf;
+
+/**
+ * Command line tool to generate required Ozone configs to an ozone-site.xml.
+ */
+


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


[48/50] [abbrv] hadoop git commit: YARN-8350. NPE in service AM related to placement policy. Contributed by Gour Saha

Posted by ha...@apache.org.
YARN-8350. NPE in service AM related to placement policy. 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/778a4a24
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/778a4a24
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/778a4a24

Branch: refs/heads/HDDS-48
Commit: 778a4a24be176382a5704f709c00bdfcfe6ddc8c
Parents: 96eefcc
Author: Billie Rinaldi <bi...@apache.org>
Authored: Wed May 30 13:19:13 2018 -0700
Committer: Billie Rinaldi <bi...@apache.org>
Committed: Wed May 30 13:19:13 2018 -0700

----------------------------------------------------------------------
 .../yarn/service/component/Component.java       | 114 ++++++++++---------
 .../exceptions/RestApiErrorMessages.java        |   8 ++
 .../yarn/service/utils/ServiceApiUtil.java      |  24 +++-
 .../hadoop/yarn/service/TestServiceApiUtil.java |  44 ++++++-
 4 files changed, 130 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/778a4a24/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 931877e..a1ee796 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
@@ -694,62 +694,66 @@ public class Component implements EventHandler<ComponentEvent> {
       // composite constraints then this AND-ed composite constraint is not
       // used.
       PlacementConstraint finalConstraint = null;
-      for (org.apache.hadoop.yarn.service.api.records.PlacementConstraint
-          yarnServiceConstraint : placementPolicy.getConstraints()) {
-        List<TargetExpression> targetExpressions = new ArrayList<>();
-        // Currently only intra-application allocation tags are supported.
-        if (!yarnServiceConstraint.getTargetTags().isEmpty()) {
-          targetExpressions.add(PlacementTargets.allocationTag(
-              yarnServiceConstraint.getTargetTags().toArray(new String[0])));
-        }
-        // Add all node attributes
-        for (Map.Entry<String, List<String>> attribute : yarnServiceConstraint
-            .getNodeAttributes().entrySet()) {
-          targetExpressions.add(PlacementTargets.nodeAttribute(
-              attribute.getKey(), attribute.getValue().toArray(new String[0])));
-        }
-        // Add all node partitions
-        if (!yarnServiceConstraint.getNodePartitions().isEmpty()) {
-          targetExpressions
-              .add(PlacementTargets.nodePartition(yarnServiceConstraint
-                  .getNodePartitions().toArray(new String[0])));
-        }
-        PlacementConstraint constraint = null;
-        switch (yarnServiceConstraint.getType()) {
-        case AFFINITY:
-          constraint = PlacementConstraints
-              .targetIn(yarnServiceConstraint.getScope().getValue(),
-                  targetExpressions.toArray(new TargetExpression[0]))
-              .build();
-          break;
-        case ANTI_AFFINITY:
-          constraint = PlacementConstraints
-              .targetNotIn(yarnServiceConstraint.getScope().getValue(),
-                  targetExpressions.toArray(new TargetExpression[0]))
-              .build();
-          break;
-        case AFFINITY_WITH_CARDINALITY:
-          constraint = PlacementConstraints.targetCardinality(
-              yarnServiceConstraint.getScope().name().toLowerCase(),
-              yarnServiceConstraint.getMinCardinality() == null ? 0
-                  : yarnServiceConstraint.getMinCardinality().intValue(),
-              yarnServiceConstraint.getMaxCardinality() == null
-                  ? Integer.MAX_VALUE
-                  : yarnServiceConstraint.getMaxCardinality().intValue(),
-              targetExpressions.toArray(new TargetExpression[0])).build();
-          break;
-        }
-        // The default AND-ed final composite constraint
-        if (finalConstraint != null) {
-          finalConstraint = PlacementConstraints
-              .and(constraint.getConstraintExpr(),
-                  finalConstraint.getConstraintExpr())
-              .build();
-        } else {
-          finalConstraint = constraint;
+      if (placementPolicy != null) {
+        for (org.apache.hadoop.yarn.service.api.records.PlacementConstraint
+            yarnServiceConstraint : placementPolicy.getConstraints()) {
+          List<TargetExpression> targetExpressions = new ArrayList<>();
+          // Currently only intra-application allocation tags are supported.
+          if (!yarnServiceConstraint.getTargetTags().isEmpty()) {
+            targetExpressions.add(PlacementTargets.allocationTag(
+                yarnServiceConstraint.getTargetTags().toArray(new String[0])));
+          }
+          // Add all node attributes
+          for (Map.Entry<String, List<String>> attribute : yarnServiceConstraint
+              .getNodeAttributes().entrySet()) {
+            targetExpressions
+                .add(PlacementTargets.nodeAttribute(attribute.getKey(),
+                    attribute.getValue().toArray(new String[0])));
+          }
+          // Add all node partitions
+          if (!yarnServiceConstraint.getNodePartitions().isEmpty()) {
+            targetExpressions
+                .add(PlacementTargets.nodePartition(yarnServiceConstraint
+                    .getNodePartitions().toArray(new String[0])));
+          }
+          PlacementConstraint constraint = null;
+          switch (yarnServiceConstraint.getType()) {
+          case AFFINITY:
+            constraint = PlacementConstraints
+                .targetIn(yarnServiceConstraint.getScope().getValue(),
+                    targetExpressions.toArray(new TargetExpression[0]))
+                .build();
+            break;
+          case ANTI_AFFINITY:
+            constraint = PlacementConstraints
+                .targetNotIn(yarnServiceConstraint.getScope().getValue(),
+                    targetExpressions.toArray(new TargetExpression[0]))
+                .build();
+            break;
+          case AFFINITY_WITH_CARDINALITY:
+            constraint = PlacementConstraints.targetCardinality(
+                yarnServiceConstraint.getScope().name().toLowerCase(),
+                yarnServiceConstraint.getMinCardinality() == null ? 0
+                    : yarnServiceConstraint.getMinCardinality().intValue(),
+                yarnServiceConstraint.getMaxCardinality() == null
+                    ? Integer.MAX_VALUE
+                    : yarnServiceConstraint.getMaxCardinality().intValue(),
+                targetExpressions.toArray(new TargetExpression[0])).build();
+            break;
+          }
+          // The default AND-ed final composite constraint
+          if (finalConstraint != null) {
+            finalConstraint = PlacementConstraints
+                .and(constraint.getConstraintExpr(),
+                    finalConstraint.getConstraintExpr())
+                .build();
+          } else {
+            finalConstraint = constraint;
+          }
+          LOG.debug("[COMPONENT {}] Placement constraint: {}",
+              componentSpec.getName(),
+              constraint.getConstraintExpr().toString());
         }
-        LOG.debug("[COMPONENT {}] Placement constraint: {}",
-            componentSpec.getName(), constraint.getConstraintExpr().toString());
       }
       ResourceSizing resourceSizing = ResourceSizing.newInstance((int) count,
           resource);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/778a4a24/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 5b6eac3..1d2d719 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
@@ -91,6 +91,14 @@ public interface RestApiErrorMessages {
 
   String ERROR_QUICKLINKS_FOR_COMP_INVALID = "Quicklinks specified at"
       + " component level, needs corresponding values set at service level";
+  // Note: %sin is not a typo. Constraint name is optional so the error messages
+  // below handle that scenario by adding a space if name is specified.
+  String ERROR_PLACEMENT_POLICY_CONSTRAINT_TYPE_NULL = "Type not specified "
+      + "for constraint %sin placement policy of component %s.";
+  String ERROR_PLACEMENT_POLICY_CONSTRAINT_SCOPE_NULL = "Scope not specified "
+      + "for constraint %sin placement policy of component %s.";
+  String ERROR_PLACEMENT_POLICY_CONSTRAINT_TAGS_NULL = "Tag(s) not specified "
+      + "for constraint %sin placement policy of component %s.";
   String ERROR_PLACEMENT_POLICY_TAG_NAME_NOT_SAME = "Invalid target tag %s "
       + "specified in placement policy of component %s. For now, target tags "
       + "support self reference only. Specifying anything other than its "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/778a4a24/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 2f826fa..6101bf0 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
@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.service.api.records.Component;
 import org.apache.hadoop.yarn.service.api.records.Configuration;
 import org.apache.hadoop.yarn.service.api.records.KerberosPrincipal;
 import org.apache.hadoop.yarn.service.api.records.PlacementConstraint;
+import org.apache.hadoop.yarn.service.api.records.PlacementPolicy;
 import org.apache.hadoop.yarn.service.api.records.Resource;
 import org.apache.hadoop.yarn.service.exceptions.SliderException;
 import org.apache.hadoop.yarn.service.conf.RestApiConstants;
@@ -314,9 +315,28 @@ public class ServiceApiUtil {
   private static void validatePlacementPolicy(List<Component> components,
       Set<String> componentNames) {
     for (Component comp : components) {
-      if (comp.getPlacementPolicy() != null) {
-        for (PlacementConstraint constraint : comp.getPlacementPolicy()
+      PlacementPolicy placementPolicy = comp.getPlacementPolicy();
+      if (placementPolicy != null) {
+        for (PlacementConstraint constraint : placementPolicy
             .getConstraints()) {
+          if (constraint.getType() == null) {
+            throw new IllegalArgumentException(String.format(
+              RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_TYPE_NULL,
+              constraint.getName() == null ? "" : constraint.getName() + " ",
+              comp.getName()));
+          }
+          if (constraint.getScope() == null) {
+            throw new IllegalArgumentException(String.format(
+              RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_SCOPE_NULL,
+              constraint.getName() == null ? "" : constraint.getName() + " ",
+              comp.getName()));
+          }
+          if (constraint.getTargetTags().isEmpty()) {
+            throw new IllegalArgumentException(String.format(
+              RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_TAGS_NULL,
+              constraint.getName() == null ? "" : constraint.getName() + " ",
+              comp.getName()));
+          }
           for (String targetTag : constraint.getTargetTags()) {
             if (!comp.getName().equals(targetTag)) {
               throw new IllegalArgumentException(String.format(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/778a4a24/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 b209bbb..243c6b3 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
@@ -25,6 +25,8 @@ import org.apache.hadoop.yarn.service.api.records.Component;
 import org.apache.hadoop.yarn.service.api.records.KerberosPrincipal;
 import org.apache.hadoop.yarn.service.api.records.PlacementConstraint;
 import org.apache.hadoop.yarn.service.api.records.PlacementPolicy;
+import org.apache.hadoop.yarn.service.api.records.PlacementScope;
+import org.apache.hadoop.yarn.service.api.records.PlacementType;
 import org.apache.hadoop.yarn.service.api.records.Resource;
 import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages;
@@ -503,13 +505,48 @@ public class TestServiceApiUtil {
     PlacementPolicy pp = new PlacementPolicy();
     PlacementConstraint pc = new PlacementConstraint();
     pc.setName("CA1");
-    pc.setTargetTags(Collections.singletonList("comp-invalid"));
     pp.setConstraints(Collections.singletonList(pc));
     comp.setPlacementPolicy(pp);
 
     try {
       ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with empty placement");
+      Assert.fail(EXCEPTION_PREFIX + "constraint with no type");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_TYPE_NULL,
+          "CA1 ", "comp-a"), e.getMessage());
+    }
+
+    // Set the type
+    pc.setType(PlacementType.ANTI_AFFINITY);
+
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "constraint with no scope");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_SCOPE_NULL,
+          "CA1 ", "comp-a"), e.getMessage());
+    }
+
+    // Set the scope
+    pc.setScope(PlacementScope.NODE);
+
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "constraint with no tag(s)");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_TAGS_NULL,
+          "CA1 ", "comp-a"), e.getMessage());
+    }
+
+    // Set a target tag - but an invalid one
+    pc.setTargetTags(Collections.singletonList("comp-invalid"));
+
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "constraint with invalid tag name");
     } catch (IllegalArgumentException e) {
       assertEquals(
           String.format(
@@ -518,9 +555,10 @@ public class TestServiceApiUtil {
           e.getMessage());
     }
 
+    // Set valid target tags now
     pc.setTargetTags(Collections.singletonList("comp-a"));
 
-    // now it should succeed
+    // Finally it should succeed
     try {
       ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
     } catch (IllegalArgumentException e) {


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


[44/50] [abbrv] hadoop git commit: YARN-8377: Javadoc build failed in hadoop-yarn-server-nodemanager. Contributed by Takanobu Asanuma

Posted by ha...@apache.org.
YARN-8377: Javadoc build failed in hadoop-yarn-server-nodemanager. Contributed by Takanobu Asanuma


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

Branch: refs/heads/HDDS-48
Commit: e44c0849d7982c8f1ed43af25d2092090881d19f
Parents: 3b34148
Author: Eric E Payne <er...@oath.com>
Authored: Wed May 30 16:50:19 2018 +0000
Committer: Eric E Payne <er...@oath.com>
Committed: Wed May 30 16:50:19 2018 +0000

----------------------------------------------------------------------
 .../containermanager/container/SlidingWindowRetryPolicy.java    | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e44c0849/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/SlidingWindowRetryPolicy.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/container/SlidingWindowRetryPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/SlidingWindowRetryPolicy.java
index 36a8b91..9360669 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/SlidingWindowRetryPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/SlidingWindowRetryPolicy.java
@@ -85,8 +85,9 @@ public class SlidingWindowRetryPolicy {
    * Updates remaining retries and the restart time when
    * required in the retryContext.
    * <p>
-   * When failuresValidityInterval is > 0, it also removes time entries from
-   * <code>restartTimes</code> which are outside the validity interval.
+   * When failuresValidityInterval is {@literal >} 0, it also removes time
+   * entries from <code>restartTimes</code> which are outside the validity
+   * interval.
    */
   protected void updateRetryContext(RetryContext retryContext) {
     if (retryContext.containerRetryContext.getFailuresValidityInterval() > 0) {


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


[49/50] [abbrv] hadoop git commit: HDDS-92. Use DBType during parsing datanode .container files. Contributed by Bharat Viswanadham.

Posted by ha...@apache.org.
HDDS-92. Use DBType during parsing datanode .container files. 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/ee1e0e20
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ee1e0e20
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ee1e0e20

Branch: refs/heads/HDDS-48
Commit: ee1e0e20368e825a6cef347ed9637c9cf67be45f
Parents: 778a4a2
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Thu May 24 18:25:35 2018 -0700
Committer: Hanisha Koneru <ha...@apache.org>
Committed: Wed May 30 14:04:19 2018 -0700

----------------------------------------------------------------------
 .../hadoop/utils/MetadataStoreBuilder.java      |  40 +-
 .../apache/hadoop/ozone/TestMetadataStore.java  | 414 -----------------
 .../apache/hadoop/utils/TestMetadataStore.java  | 463 +++++++++++++++++++
 .../container/common/helpers/KeyUtils.java      |   3 +-
 .../container/common/utils/ContainerCache.java  |   6 +-
 5 files changed, 503 insertions(+), 423 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee1e0e20/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStoreBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStoreBuilder.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStoreBuilder.java
index 9e9c32a..85cebed 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStoreBuilder.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStoreBuilder.java
@@ -18,12 +18,15 @@
 
 package org.apache.hadoop.utils;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.iq80.leveldb.Options;
 import org.rocksdb.BlockBasedTableConfig;
 import org.rocksdb.Statistics;
 import org.rocksdb.StatsLevel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
@@ -44,10 +47,14 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys
  */
 public class MetadataStoreBuilder {
 
+  @VisibleForTesting
+  static final Logger LOG =
+      LoggerFactory.getLogger(MetadataStoreBuilder.class);
   private File dbFile;
   private long cacheSize;
   private boolean createIfMissing = true;
   private Configuration conf;
+  private String dbType;
 
   public static MetadataStoreBuilder newBuilder() {
     return new MetadataStoreBuilder();
@@ -73,6 +80,17 @@ public class MetadataStoreBuilder {
     return this;
   }
 
+  /**
+   * Set the container DB Type.
+   * @param type
+   * @return MetadataStoreBuilder
+   */
+  public MetadataStoreBuilder setDBType(String type) {
+    this.dbType = type;
+    return this;
+  }
+
+
   public MetadataStore build() throws IOException {
     if (dbFile == null) {
       throw new IllegalArgumentException("Failed to build metadata store, "
@@ -81,18 +99,26 @@ public class MetadataStoreBuilder {
 
     // Build db store based on configuration
     MetadataStore store = null;
-    String impl = conf == null ?
-        OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_DEFAULT :
-        conf.getTrimmed(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL,
-            OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_DEFAULT);
-    if (OZONE_METADATA_STORE_IMPL_LEVELDB.equals(impl)) {
+
+    if(dbType == null) {
+      LOG.debug("dbType is null, using ");
+      dbType = conf == null ?
+          OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_DEFAULT :
+          conf.getTrimmed(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL,
+              OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_DEFAULT);
+      LOG.debug("dbType is null, using dbType {} from ozone configuration",
+          dbType);
+    } else {
+      LOG.debug("Using dbType {} for metastore", dbType);
+    }
+    if (OZONE_METADATA_STORE_IMPL_LEVELDB.equals(dbType)) {
       Options options = new Options();
       options.createIfMissing(createIfMissing);
       if (cacheSize > 0) {
         options.cacheSize(cacheSize);
       }
       store = new LevelDBStore(dbFile, options);
-    } else if (OZONE_METADATA_STORE_IMPL_ROCKSDB.equals(impl)) {
+    } else if (OZONE_METADATA_STORE_IMPL_ROCKSDB.equals(dbType)) {
       org.rocksdb.Options opts = new org.rocksdb.Options();
       opts.setCreateIfMissing(createIfMissing);
 
@@ -119,7 +145,7 @@ public class MetadataStoreBuilder {
           + OzoneConfigKeys.OZONE_METADATA_STORE_IMPL
           + ". Expecting " + OZONE_METADATA_STORE_IMPL_LEVELDB
           + " or " + OZONE_METADATA_STORE_IMPL_ROCKSDB
-          + ", but met " + impl);
+          + ", but met " + dbType);
     }
     return store;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee1e0e20/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/TestMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/TestMetadataStore.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/TestMetadataStore.java
deleted file mode 100644
index 6b26b60..0000000
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/TestMetadataStore.java
+++ /dev/null
@@ -1,414 +0,0 @@
-/**
- * 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;
-
-import com.google.common.collect.Lists;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.tuple.ImmutablePair;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.DFSUtilClient;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.utils.BatchOperation;
-import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
-import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
-import org.apache.hadoop.utils.MetadataStore;
-import org.apache.hadoop.utils.MetadataStoreBuilder;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.junit.runners.Parameterized.Parameters;
-
-/**
- * Test class for ozone metadata store.
- */
-@RunWith(Parameterized.class)
-public class TestMetadataStore {
-
-  private final String storeImpl;
-
-  public TestMetadataStore(String metadataImpl) {
-    this.storeImpl = metadataImpl;
-  }
-
-  @Parameters
-  public static Collection<Object[]> data() {
-    return Arrays.asList(new Object[][] {
-        {OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB},
-        {OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_ROCKSDB}
-    });
-  }
-
-  private MetadataStore store;
-  private File testDir;
-  private final static int MAX_GETRANGE_LENGTH = 100;
-
-  @Rule
-  public ExpectedException expectedException = ExpectedException.none();
-
-  @Before
-  public void init() throws IOException {
-    testDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
-        + "-" + storeImpl.toLowerCase());
-
-    Configuration conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
-
-    store = MetadataStoreBuilder.newBuilder()
-        .setConf(conf)
-        .setCreateIfMissing(true)
-        .setDbFile(testDir)
-        .build();
-
-    // Add 20 entries.
-    // {a0 : a-value0} to {a9 : a-value9}
-    // {b0 : b-value0} to {b9 : b-value9}
-    for (int i=0; i<10; i++) {
-      store.put(getBytes("a" + i), getBytes("a-value" + i));
-      store.put(getBytes("b" + i), getBytes("b-value" + i));
-    }
-  }
-
-  @After
-  public void cleanup() throws IOException {
-    store.close();
-    store.destroy();
-    FileUtils.deleteDirectory(testDir);
-  }
-
-  private byte[] getBytes(String str) {
-    return str == null ? null :
-        DFSUtilClient.string2Bytes(str);
-  }
-
-  private String getString(byte[] bytes) {
-    return bytes == null ? null :
-        DFSUtilClient.bytes2String(bytes);
-  }
-
-  @Test
-  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));
-
-      byte[] vb = store.get(getBytes("b" + i));
-      Assert.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))));
-    store.delete(getBytes(keyToDel));
-    Assert.assertEquals(null, store.get(getBytes(keyToDel)));
-  }
-
-  @Test
-  public void testPeekFrom() throws IOException {
-    // Test peek from an element that has prev as well as next
-    testPeek("a3", "a2", "a4");
-
-    // Test peek from an element that only has prev
-    testPeek("b9", "b8", null);
-
-    // Test peek from an element that only has next
-    testPeek("a0", null, "a1");
-  }
-
-  private String getExpectedValue(String key) {
-    if (key == null) {
-      return null;
-    }
-    char[] arr = key.toCharArray();
-    return new StringBuffer().append(arr[0]).append("-value")
-        .append(arr[arr.length - 1]).toString();
-  }
-
-  private void testPeek(String peekKey, String prevKey, String nextKey)
-      throws IOException {
-    // Look for current
-    String k = null;
-    String v = null;
-    ImmutablePair<byte[], byte[]> current =
-        store.peekAround(0, getBytes(peekKey));
-    if (current != null) {
-      k = getString(current.getKey());
-      v = getString(current.getValue());
-    }
-    Assert.assertEquals(peekKey, k);
-    Assert.assertEquals(v, getExpectedValue(peekKey));
-
-    // Look for prev
-    k = null;
-    v = null;
-    ImmutablePair<byte[], byte[]> prev =
-        store.peekAround(-1, getBytes(peekKey));
-    if (prev != null) {
-      k = getString(prev.getKey());
-      v = getString(prev.getValue());
-    }
-    Assert.assertEquals(prevKey, k);
-    Assert.assertEquals(v, getExpectedValue(prevKey));
-
-    // Look for next
-    k = null;
-    v = null;
-    ImmutablePair<byte[], byte[]> next =
-        store.peekAround(1, getBytes(peekKey));
-    if (next != null) {
-      k = getString(next.getKey());
-      v = getString(next.getValue());
-    }
-    Assert.assertEquals(nextKey, k);
-    Assert.assertEquals(v, getExpectedValue(nextKey));
-  }
-
-  @Test
-  public void testIterateKeys() throws IOException {
-    // iterate keys from b0
-    ArrayList<String> result = Lists.newArrayList();
-    store.iterate(getBytes("b0"), (k, v) -> {
-      // b-value{i}
-      String value = getString(v);
-      char num = value.charAt(value.length() - 1);
-      // each value adds 1
-      int i = Character.getNumericValue(num) + 1;
-      value =  value.substring(0, value.length() - 1) + i;
-      result.add(value);
-      return true;
-    });
-
-    Assert.assertFalse(result.isEmpty());
-    for (int i=0; i<result.size(); i++) {
-      Assert.assertEquals("b-value" + (i+1), result.get(i));
-    }
-
-    // iterate from a non exist key
-    result.clear();
-    store.iterate(getBytes("xyz"), (k, v) -> {
-      result.add(getString(v));
-      return true;
-    });
-    Assert.assertTrue(result.isEmpty());
-
-    // iterate from the beginning
-    result.clear();
-    store.iterate(null, (k, v) -> {
-      result.add(getString(v));
-      return true;
-    });
-    Assert.assertEquals(20, result.size());
-  }
-
-  @Test
-  public void testGetRangeKVs() throws IOException {
-    List<Map.Entry<byte[], byte[]>> result = null;
-
-    // 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()));
-
-    // Empty list if startKey doesn't exist.
-    result = store.getRangeKVs(getBytes("a12"), 5);
-    Assert.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()));
-    result = store.getRangeKVs(getBytes("b0"), 5);
-    Assert.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()));
-
-    // Filter keys by prefix.
-    // It should returns all "b*" entries.
-    MetadataKeyFilter filter1 = new KeyPrefixFilter("b");
-    result = store.getRangeKVs(null, 100, filter1);
-    Assert.assertEquals(10, result.size());
-    Assert.assertTrue(result.stream().allMatch(entry ->
-        new String(entry.getKey()).startsWith("b")
-    ));
-    Assert.assertEquals(20, filter1.getKeysScannedNum());
-    Assert.assertEquals(10, filter1.getKeysHintedNum());
-    result = store.getRangeKVs(null, 3, filter1);
-    Assert.assertEquals(3, result.size());
-    result = store.getRangeKVs(getBytes("b3"), 1, filter1);
-    Assert.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()));
-    result = store.getRangeKVs(null, 1, filter2);
-    Assert.assertEquals(1, result.size());
-    Assert.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()));
-
-    // 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()));
-  }
-
-  @Test
-  public void testGetSequentialRangeKVs() throws IOException {
-    MetadataKeyFilter suffixFilter = (preKey, currentKey, nextKey)
-        -> DFSUtil.bytes2String(currentKey).endsWith("2");
-    // 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()));
-
-    // 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()));
-  }
-
-  @Test
-  public void testGetRangeLength() throws IOException {
-    List<Map.Entry<byte[], byte[]>> result = null;
-
-    result = store.getRangeKVs(null, 0);
-    Assert.assertEquals(0, result.size());
-
-    result = store.getRangeKVs(null, 1);
-    Assert.assertEquals(1, result.size());
-
-    // Count less than zero is invalid.
-    expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage("Invalid count given");
-    store.getRangeKVs(null, -1);
-  }
-
-  @Test
-  public void testInvalidStartKey() throws IOException {
-    // 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);
-  }
-
-  @Test
-  public void testDestroyDB() throws IOException {
-    // create a new DB to test db destroy
-    Configuration conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
-
-    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
-        + "-" + storeImpl.toLowerCase() + "-toDestroy");
-    MetadataStore dbStore = MetadataStoreBuilder.newBuilder()
-        .setConf(conf)
-        .setCreateIfMissing(true)
-        .setDbFile(dbDir)
-        .build();
-
-    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);
-
-    dbStore.destroy();
-
-    Assert.assertFalse(dbDir.exists());
-  }
-
-  @Test
-  public void testBatchWrite() throws IOException {
-    Configuration conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
-
-    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
-        + "-" + storeImpl.toLowerCase() + "-batchWrite");
-    MetadataStore dbStore = MetadataStoreBuilder.newBuilder()
-        .setConf(conf)
-        .setCreateIfMissing(true)
-        .setDbFile(dbDir)
-        .build();
-
-    List<String> expectedResult = Lists.newArrayList();
-    for (int i = 0; i<10; i++) {
-      dbStore.put(getBytes("batch-" + i), getBytes("batch-value-" + i));
-      expectedResult.add("batch-" + i);
-    }
-
-    BatchOperation batch = new BatchOperation();
-    batch.delete(getBytes("batch-2"));
-    batch.delete(getBytes("batch-3"));
-    batch.delete(getBytes("batch-4"));
-    batch.put(getBytes("batch-new-2"), getBytes("batch-new-value-2"));
-
-    expectedResult.remove("batch-2");
-    expectedResult.remove("batch-3");
-    expectedResult.remove("batch-4");
-    expectedResult.add("batch-new-2");
-
-    dbStore.writeBatch(batch);
-
-    Iterator<String> it = expectedResult.iterator();
-    AtomicInteger count = new AtomicInteger(0);
-    dbStore.iterate(null, (key, value) -> {
-      count.incrementAndGet();
-      return it.hasNext() && it.next().equals(getString(key));
-    });
-
-    Assert.assertEquals(8, count.get());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee1e0e20/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
new file mode 100644
index 0000000..06db1e1
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
@@ -0,0 +1,463 @@
+/**
+ * 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.utils;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+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.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;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.event.Level;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Test class for ozone metadata store.
+ */
+@RunWith(Parameterized.class)
+public class TestMetadataStore {
+
+  private final String storeImpl;
+
+  public TestMetadataStore(String metadataImpl) {
+    this.storeImpl = metadataImpl;
+  }
+
+  @Parameters
+  public static Collection<Object[]> data() {
+    return Arrays.asList(new Object[][] {
+        {OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB},
+        {OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_ROCKSDB}
+    });
+  }
+
+  private MetadataStore store;
+  private File testDir;
+  private final static int MAX_GETRANGE_LENGTH = 100;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  @Before
+  public void init() throws IOException {
+    testDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
+        + "-" + storeImpl.toLowerCase());
+
+    Configuration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
+
+    store = MetadataStoreBuilder.newBuilder()
+        .setConf(conf)
+        .setCreateIfMissing(true)
+        .setDbFile(testDir)
+        .build();
+
+    // Add 20 entries.
+    // {a0 : a-value0} to {a9 : a-value9}
+    // {b0 : b-value0} to {b9 : b-value9}
+    for (int i=0; i<10; i++) {
+      store.put(getBytes("a" + i), getBytes("a-value" + i));
+      store.put(getBytes("b" + i), getBytes("b-value" + i));
+    }
+  }
+
+  @Test
+  public void testMetaStoreConfigDifferentFromType() throws IOException {
+
+    Configuration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
+    String dbType;
+    GenericTestUtils.setLogLevel(MetadataStoreBuilder.LOG, Level.DEBUG);
+    GenericTestUtils.LogCapturer logCapturer =
+        GenericTestUtils.LogCapturer.captureLogs(MetadataStoreBuilder.LOG);
+    if(storeImpl.equals(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB)) {
+      dbType = "RocksDB";
+    } else {
+      dbType = "LevelDB";
+    }
+
+    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
+        + "-" + dbType.toLowerCase() + "-test");
+    MetadataStore dbStore = MetadataStoreBuilder.newBuilder().setConf(conf)
+        .setCreateIfMissing(true).setDbFile(dbDir).setDBType(dbType).build();
+    assertTrue(logCapturer.getOutput().contains("Using dbType " + dbType + "" +
+        " for metastore"));
+    dbStore.close();
+    dbStore.destroy();
+    FileUtils.deleteDirectory(dbDir);
+
+  }
+
+  @Test
+  public void testdbTypeNotSet() throws IOException {
+
+    Configuration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
+    GenericTestUtils.setLogLevel(MetadataStoreBuilder.LOG, Level.DEBUG);
+    GenericTestUtils.LogCapturer logCapturer =
+        GenericTestUtils.LogCapturer.captureLogs(MetadataStoreBuilder.LOG);
+
+
+    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
+        + "-" + storeImpl.toLowerCase() + "-test");
+    MetadataStore dbStore = MetadataStoreBuilder.newBuilder().setConf(conf)
+        .setCreateIfMissing(true).setDbFile(dbDir).build();
+    assertTrue(logCapturer.getOutput().contains("dbType is null, using dbType" +
+        " " + storeImpl));
+    dbStore.close();
+    dbStore.destroy();
+    FileUtils.deleteDirectory(dbDir);
+
+  }
+
+  @After
+  public void cleanup() throws IOException {
+    store.close();
+    store.destroy();
+    FileUtils.deleteDirectory(testDir);
+  }
+
+  private byte[] getBytes(String str) {
+    return str == null ? null :
+        DFSUtilClient.string2Bytes(str);
+  }
+
+  private String getString(byte[] bytes) {
+    return bytes == null ? null :
+        DFSUtilClient.bytes2String(bytes);
+  }
+
+  @Test
+  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));
+
+      byte[] vb = store.get(getBytes("b" + i));
+      Assert.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))));
+    store.delete(getBytes(keyToDel));
+    Assert.assertEquals(null, store.get(getBytes(keyToDel)));
+  }
+
+  @Test
+  public void testPeekFrom() throws IOException {
+    // Test peek from an element that has prev as well as next
+    testPeek("a3", "a2", "a4");
+
+    // Test peek from an element that only has prev
+    testPeek("b9", "b8", null);
+
+    // Test peek from an element that only has next
+    testPeek("a0", null, "a1");
+  }
+
+  private String getExpectedValue(String key) {
+    if (key == null) {
+      return null;
+    }
+    char[] arr = key.toCharArray();
+    return new StringBuffer().append(arr[0]).append("-value")
+        .append(arr[arr.length - 1]).toString();
+  }
+
+  private void testPeek(String peekKey, String prevKey, String nextKey)
+      throws IOException {
+    // Look for current
+    String k = null;
+    String v = null;
+    ImmutablePair<byte[], byte[]> current =
+        store.peekAround(0, getBytes(peekKey));
+    if (current != null) {
+      k = getString(current.getKey());
+      v = getString(current.getValue());
+    }
+    Assert.assertEquals(peekKey, k);
+    Assert.assertEquals(v, getExpectedValue(peekKey));
+
+    // Look for prev
+    k = null;
+    v = null;
+    ImmutablePair<byte[], byte[]> prev =
+        store.peekAround(-1, getBytes(peekKey));
+    if (prev != null) {
+      k = getString(prev.getKey());
+      v = getString(prev.getValue());
+    }
+    Assert.assertEquals(prevKey, k);
+    Assert.assertEquals(v, getExpectedValue(prevKey));
+
+    // Look for next
+    k = null;
+    v = null;
+    ImmutablePair<byte[], byte[]> next =
+        store.peekAround(1, getBytes(peekKey));
+    if (next != null) {
+      k = getString(next.getKey());
+      v = getString(next.getValue());
+    }
+    Assert.assertEquals(nextKey, k);
+    Assert.assertEquals(v, getExpectedValue(nextKey));
+  }
+
+  @Test
+  public void testIterateKeys() throws IOException {
+    // iterate keys from b0
+    ArrayList<String> result = Lists.newArrayList();
+    store.iterate(getBytes("b0"), (k, v) -> {
+      // b-value{i}
+      String value = getString(v);
+      char num = value.charAt(value.length() - 1);
+      // each value adds 1
+      int i = Character.getNumericValue(num) + 1;
+      value =  value.substring(0, value.length() - 1) + i;
+      result.add(value);
+      return true;
+    });
+
+    Assert.assertFalse(result.isEmpty());
+    for (int i=0; i<result.size(); i++) {
+      Assert.assertEquals("b-value" + (i+1), result.get(i));
+    }
+
+    // iterate from a non exist key
+    result.clear();
+    store.iterate(getBytes("xyz"), (k, v) -> {
+      result.add(getString(v));
+      return true;
+    });
+    Assert.assertTrue(result.isEmpty());
+
+    // iterate from the beginning
+    result.clear();
+    store.iterate(null, (k, v) -> {
+      result.add(getString(v));
+      return true;
+    });
+    Assert.assertEquals(20, result.size());
+  }
+
+  @Test
+  public void testGetRangeKVs() throws IOException {
+    List<Map.Entry<byte[], byte[]>> result = null;
+
+    // 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()));
+
+    // Empty list if startKey doesn't exist.
+    result = store.getRangeKVs(getBytes("a12"), 5);
+    Assert.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()));
+    result = store.getRangeKVs(getBytes("b0"), 5);
+    Assert.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()));
+
+    // Filter keys by prefix.
+    // It should returns all "b*" entries.
+    MetadataKeyFilter filter1 = new KeyPrefixFilter("b");
+    result = store.getRangeKVs(null, 100, filter1);
+    Assert.assertEquals(10, result.size());
+    Assert.assertTrue(result.stream().allMatch(entry ->
+        new String(entry.getKey()).startsWith("b")
+    ));
+    Assert.assertEquals(20, filter1.getKeysScannedNum());
+    Assert.assertEquals(10, filter1.getKeysHintedNum());
+    result = store.getRangeKVs(null, 3, filter1);
+    Assert.assertEquals(3, result.size());
+    result = store.getRangeKVs(getBytes("b3"), 1, filter1);
+    Assert.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()));
+    result = store.getRangeKVs(null, 1, filter2);
+    Assert.assertEquals(1, result.size());
+    Assert.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()));
+
+    // 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()));
+  }
+
+  @Test
+  public void testGetSequentialRangeKVs() throws IOException {
+    MetadataKeyFilter suffixFilter = (preKey, currentKey, nextKey)
+        -> DFSUtil.bytes2String(currentKey).endsWith("2");
+    // 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()));
+
+    // 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()));
+  }
+
+  @Test
+  public void testGetRangeLength() throws IOException {
+    List<Map.Entry<byte[], byte[]>> result = null;
+
+    result = store.getRangeKVs(null, 0);
+    Assert.assertEquals(0, result.size());
+
+    result = store.getRangeKVs(null, 1);
+    Assert.assertEquals(1, result.size());
+
+    // Count less than zero is invalid.
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage("Invalid count given");
+    store.getRangeKVs(null, -1);
+  }
+
+  @Test
+  public void testInvalidStartKey() throws IOException {
+    // 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);
+  }
+
+  @Test
+  public void testDestroyDB() throws IOException {
+    // create a new DB to test db destroy
+    Configuration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
+
+    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
+        + "-" + storeImpl.toLowerCase() + "-toDestroy");
+    MetadataStore dbStore = MetadataStoreBuilder.newBuilder()
+        .setConf(conf)
+        .setCreateIfMissing(true)
+        .setDbFile(dbDir)
+        .build();
+
+    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);
+
+    dbStore.destroy();
+
+    Assert.assertFalse(dbDir.exists());
+  }
+
+  @Test
+  public void testBatchWrite() throws IOException {
+    Configuration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
+
+    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
+        + "-" + storeImpl.toLowerCase() + "-batchWrite");
+    MetadataStore dbStore = MetadataStoreBuilder.newBuilder()
+        .setConf(conf)
+        .setCreateIfMissing(true)
+        .setDbFile(dbDir)
+        .build();
+
+    List<String> expectedResult = Lists.newArrayList();
+    for (int i = 0; i<10; i++) {
+      dbStore.put(getBytes("batch-" + i), getBytes("batch-value-" + i));
+      expectedResult.add("batch-" + i);
+    }
+
+    BatchOperation batch = new BatchOperation();
+    batch.delete(getBytes("batch-2"));
+    batch.delete(getBytes("batch-3"));
+    batch.delete(getBytes("batch-4"));
+    batch.put(getBytes("batch-new-2"), getBytes("batch-new-value-2"));
+
+    expectedResult.remove("batch-2");
+    expectedResult.remove("batch-3");
+    expectedResult.remove("batch-4");
+    expectedResult.add("batch-new-2");
+
+    dbStore.writeBatch(batch);
+
+    Iterator<String> it = expectedResult.iterator();
+    AtomicInteger count = new AtomicInteger(0);
+    dbStore.iterate(null, (key, value) -> {
+      count.incrementAndGet();
+      return it.hasNext() && it.next().equals(getString(key));
+    });
+
+    Assert.assertEquals(8, count.get());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee1e0e20/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java
index f831d45..d52bc18 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java
@@ -63,7 +63,8 @@ public final class KeyUtils {
     ContainerCache cache = ContainerCache.getInstance(conf);
     Preconditions.checkNotNull(cache);
     try {
-      return cache.getDB(container.getContainerID(), container.getDBPath());
+      return cache.getDB(container.getContainerID(), container
+          .getContainerDBType(), container.getDBPath());
     } catch (IOException ex) {
       String message =
           String.format("Unable to open DB. DB Name: %s, Path: %s. ex: %s",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee1e0e20/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
index 4d9c690..d9dd360 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
@@ -121,9 +121,12 @@ public final class ContainerCache extends LRUMap {
    * Returns a DB handle if available, create the handler otherwise.
    *
    * @param containerID - ID of the container.
+   * @param containerDBType - DB type of the container.
+   * @param containerDBPath - DB path of the container.
    * @return MetadataStore.
    */
-  public MetadataStore getDB(long containerID, String containerDBPath)
+  public MetadataStore getDB(long containerID, String containerDBType, String
+                             containerDBPath)
       throws IOException {
     Preconditions.checkState(containerID >= 0, "Container ID cannot be negative.");
     lock.lock();
@@ -134,6 +137,7 @@ public final class ContainerCache extends LRUMap {
         db = MetadataStoreBuilder.newBuilder()
             .setDbFile(new File(containerDBPath))
             .setCreateIfMissing(false)
+            .setDBType(containerDBType)
             .build();
         this.put(containerID, db);
       }


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


[22/50] [abbrv] hadoop git commit: HADOOP-15449. Increase default timeout of ZK session to avoid frequent NameNode failover

Posted by ha...@apache.org.
HADOOP-15449. Increase default timeout of ZK session to avoid frequent NameNode failover

Signed-off-by: Akira Ajisaka <aa...@apache.org>


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

Branch: refs/heads/HDDS-48
Commit: 61df174e8b3d582183306cabfa2347c8b96322ff
Parents: 04757e5
Author: Karthik Palanisamy <ka...@gmail.com>
Authored: Mon May 28 19:41:07 2018 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon May 28 19:41:07 2018 +0900

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/ha/ZKFailoverController.java   | 2 +-
 .../hadoop-common/src/main/resources/core-default.xml              | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/61df174e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
index a8c19ab..9295288 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
@@ -63,7 +63,7 @@ public abstract class ZKFailoverController {
   
   public static final String ZK_QUORUM_KEY = "ha.zookeeper.quorum";
   private static final String ZK_SESSION_TIMEOUT_KEY = "ha.zookeeper.session-timeout.ms";
-  private static final int ZK_SESSION_TIMEOUT_DEFAULT = 5*1000;
+  private static final int ZK_SESSION_TIMEOUT_DEFAULT = 10*1000;
   private static final String ZK_PARENT_ZNODE_KEY = "ha.zookeeper.parent-znode";
   public static final String ZK_ACL_KEY = "ha.zookeeper.acl";
   private static final String ZK_ACL_DEFAULT = "world:anyone:rwcda";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/61df174e/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 9564587..75acf48 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -2168,7 +2168,7 @@
 
 <property>
   <name>ha.zookeeper.session-timeout.ms</name>
-  <value>5000</value>
+  <value>10000</value>
   <description>
     The session timeout to use when the ZKFC connects to ZooKeeper.
     Setting this value to a lower value implies that server crashes


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


[29/50] [abbrv] hadoop git commit: HADOOP-15497. TestTrash should use proper test path to avoid failing on Windows. Contributed by Anbang Hu.

Posted by ha...@apache.org.
HADOOP-15497. TestTrash should use proper test path to avoid failing on Windows. Contributed by Anbang Hu.


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

Branch: refs/heads/HDDS-48
Commit: 3c75f8e4933221fa60a87e86a3db5e4727530b6f
Parents: 31ab960
Author: Inigo Goiri <in...@apache.org>
Authored: Tue May 29 09:11:08 2018 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Tue May 29 09:11:08 2018 -0700

----------------------------------------------------------------------
 .../src/test/java/org/apache/hadoop/fs/TestTrash.java     | 10 ++++++----
 1 file changed, 6 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c75f8e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
index 12aed29..fa2d21f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
@@ -49,9 +49,11 @@ import org.apache.hadoop.util.Time;
  */
 public class TestTrash {
 
-  private final static Path TEST_DIR = new Path(GenericTestUtils.getTempPath(
+  private final static File BASE_PATH = new File(GenericTestUtils.getTempPath(
       "testTrash"));
 
+  private final static Path TEST_DIR = new Path(BASE_PATH.getAbsolutePath());
+
   @Before
   public void setUp() throws IOException {
     // ensure each test initiates a FileSystem instance,
@@ -682,7 +684,7 @@ public class TestTrash {
   static class TestLFS extends LocalFileSystem {
     Path home;
     TestLFS() {
-      this(new Path(TEST_DIR, "user/test"));
+      this(TEST_DIR);
     }
     TestLFS(final Path home) {
       super(new RawLocalFileSystem() {
@@ -809,8 +811,8 @@ public class TestTrash {
    */
   public static void verifyTrashPermission(FileSystem fs, Configuration conf)
       throws IOException {
-    Path caseRoot = new Path(
-        GenericTestUtils.getTempPath("testTrashPermission"));
+    Path caseRoot = new Path(BASE_PATH.getPath(),
+        "testTrashPermission");
     try (FileSystem fileSystem = fs){
       Trash trash = new Trash(fileSystem, conf);
       FileSystemTestWrapper wrapper =


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