Jelajahi Sumber

HDDS-1. Remove SCM Block DB. Contributed by Xiaoyu Yao.

Anu Engineer 7 tahun lalu
induk
melakukan
3a43ac2851
100 mengubah file dengan 1396 tambahan dan 1718 penghapusan
  1. 12 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java
  2. 8 9
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java
  3. 61 60
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java
  4. 7 6
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java
  5. 9 9
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java
  6. 59 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
  7. 22 24
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
  8. 11 9
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/AllocatedBlock.java
  9. 3 19
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
  10. 9 7
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/DeleteBlockResult.java
  11. 2 16
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/Pipeline.java
  12. 2 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/PipelineChannel.java
  13. 0 12
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocol.java
  14. 13 13
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
  15. 2 44
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java
  16. 25 35
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
  17. 30 36
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
  18. 18 8
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/BlockGroup.java
  19. 7 6
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/DeleteBlockGroupResult.java
  20. 23 18
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
  21. 1 37
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/ScmBlockLocationProtocolServerSideTranslatorPB.java
  22. 11 19
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java
  23. 3 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java
  24. 17 3
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataKeyFilters.java
  25. 1 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java
  26. 34 48
      hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
  27. 3 33
      hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto
  28. 7 9
      hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto
  29. 9 4
      hadoop-hdds/common/src/main/proto/hdds.proto
  30. 9 11
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java
  31. 12 21
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerData.java
  32. 4 15
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
  33. 10 4
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
  34. 10 9
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DeletedContainerBlocksSummary.java
  35. 2 1
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/FileUtils.java
  36. 3 3
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java
  37. 37 39
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkManagerImpl.java
  38. 106 104
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java
  39. 50 71
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
  40. 32 34
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyManagerImpl.java
  41. 2 2
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
  42. 2 2
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
  43. 8 11
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java
  44. 1 1
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java
  45. 35 41
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java
  46. 7 10
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/KeyManager.java
  47. 4 4
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java
  48. 5 4
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerHandler.java
  49. 5 4
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
  50. 1 1
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
  51. 10 11
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
  52. 17 19
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
  53. 7 7
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CloseContainerCommand.java
  54. 5 4
      hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
  55. 1 1
      hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
  56. 2 10
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java
  57. 32 139
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
  58. 3 3
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DatanodeDeletedBlockTransactions.java
  59. 3 3
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java
  60. 15 9
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java
  61. 32 48
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
  62. 4 8
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
  63. 14 19
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
  64. 8 8
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/closer/ContainerCloser.java
  65. 5 5
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java
  66. 1 1
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodePoolManager.java
  67. 8 7
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java
  68. 4 4
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
  69. 1 1
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java
  70. 2 18
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java
  71. 26 21
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
  72. 4 5
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
  73. 3 31
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
  74. 30 26
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java
  75. 27 52
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
  76. 10 15
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
  77. 4 5
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
  78. 4 3
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
  79. 17 15
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerSupervisor.java
  80. 6 5
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationDatanodeStateManager.java
  81. 14 12
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java
  82. 0 1
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommandHandler.java
  83. 2 20
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateContainerHandler.java
  84. 14 12
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java
  85. 17 14
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java
  86. 4 8
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListContainerHandler.java
  87. 13 12
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java
  88. 20 15
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
  89. 5 6
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneContainerTranslation.java
  90. 17 24
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfo.java
  91. 1 1
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfoGroup.java
  92. 6 2
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/handlers/UserArgs.java
  93. 5 6
      hadoop-ozone/common/src/main/proto/KeySpaceManagerProtocol.proto
  94. 64 60
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManager.java
  95. 6 4
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java
  96. 1 2
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java
  97. 33 31
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
  98. 17 16
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
  99. 5 5
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
  100. 93 87
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java

+ 12 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java

@@ -18,6 +18,8 @@
 package org.apache.hadoop.util;
 
 import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.TimeZone;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -34,6 +36,8 @@ public final class Time {
    */
   private static final long NANOSECONDS_PER_MILLISECOND = 1000000;
 
+  private static final TimeZone UTC_ZONE = TimeZone.getTimeZone("UTC");
+
   private static final ThreadLocal<SimpleDateFormat> DATE_FORMAT =
       new ThreadLocal<SimpleDateFormat>() {
     @Override
@@ -82,4 +86,12 @@ public final class Time {
   public static String formatTime(long millis) {
     return DATE_FORMAT.get().format(millis);
   }
+
+  /**
+   * Get the current UTC time in milliseconds.
+   * @return the current UTC time in milliseconds.
+   */
+  public static long getUtcTime() {
+    return Calendar.getInstance(UTC_ZONE).getTimeInMillis();
+  }
 }

+ 8 - 9
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java

@@ -60,7 +60,7 @@ public class XceiverClientManager implements Closeable {
 
   //TODO : change this to SCM configuration class
   private final Configuration conf;
-  private final Cache<String, XceiverClientSpi> clientCache;
+  private final Cache<Long, XceiverClientSpi> clientCache;
   private final boolean useRatis;
 
   private static XceiverClientMetrics metrics;
@@ -84,10 +84,10 @@ public class XceiverClientManager implements Closeable {
         .expireAfterAccess(staleThresholdMs, TimeUnit.MILLISECONDS)
         .maximumSize(maxSize)
         .removalListener(
-            new RemovalListener<String, XceiverClientSpi>() {
+            new RemovalListener<Long, XceiverClientSpi>() {
             @Override
             public void onRemoval(
-                RemovalNotification<String, XceiverClientSpi>
+                RemovalNotification<Long, XceiverClientSpi>
                   removalNotification) {
               synchronized (clientCache) {
                 // Mark the entry as evicted
@@ -99,7 +99,7 @@ public class XceiverClientManager implements Closeable {
   }
 
   @VisibleForTesting
-  public Cache<String, XceiverClientSpi> getClientCache() {
+  public Cache<Long, XceiverClientSpi> getClientCache() {
     return clientCache;
   }
 
@@ -114,14 +114,14 @@ public class XceiverClientManager implements Closeable {
    * @return XceiverClientSpi connected to a container
    * @throws IOException if a XceiverClientSpi cannot be acquired
    */
-  public XceiverClientSpi acquireClient(Pipeline pipeline)
+  public XceiverClientSpi acquireClient(Pipeline pipeline, long containerID)
       throws IOException {
     Preconditions.checkNotNull(pipeline);
     Preconditions.checkArgument(pipeline.getMachines() != null);
     Preconditions.checkArgument(!pipeline.getMachines().isEmpty());
 
     synchronized (clientCache) {
-      XceiverClientSpi info = getClient(pipeline);
+      XceiverClientSpi info = getClient(pipeline, containerID);
       info.incrementReference();
       return info;
     }
@@ -139,11 +139,10 @@ public class XceiverClientManager implements Closeable {
     }
   }
 
-  private XceiverClientSpi getClient(Pipeline pipeline)
+  private XceiverClientSpi getClient(Pipeline pipeline, long containerID)
       throws IOException {
-    String containerName = pipeline.getContainerName();
     try {
-      return clientCache.get(containerName,
+      return clientCache.get(containerID,
           new Callable<XceiverClientSpi>() {
           @Override
           public XceiverClientSpi call() throws Exception {

+ 61 - 60
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java

@@ -86,15 +86,16 @@ public class ContainerOperationClient implements ScmClient {
    * @inheritDoc
    */
   @Override
-  public Pipeline createContainer(String containerId, String owner)
+  public ContainerInfo createContainer(String owner)
       throws IOException {
     XceiverClientSpi client = null;
     try {
-      Pipeline pipeline =
+      ContainerInfo container =
           storageContainerLocationClient.allocateContainer(
               xceiverClientManager.getType(),
-              xceiverClientManager.getFactor(), containerId, owner);
-      client = xceiverClientManager.acquireClient(pipeline);
+              xceiverClientManager.getFactor(), owner);
+      Pipeline pipeline = container.getPipeline();
+      client = xceiverClientManager.acquireClient(pipeline, container.getContainerID());
 
       // Allocated State means that SCM has allocated this pipeline in its
       // namespace. The client needs to create the pipeline on the machines
@@ -104,10 +105,8 @@ public class ContainerOperationClient implements ScmClient {
       if (pipeline.getLifeCycleState() == ALLOCATED) {
         createPipeline(client, pipeline);
       }
-      // TODO : Container Client State needs to be updated.
-      // TODO : Return ContainerInfo instead of Pipeline
-      createContainer(containerId, client, pipeline);
-      return pipeline;
+      createContainer(client, container.getContainerID());
+      return container;
     } finally {
       if (client != null) {
         xceiverClientManager.releaseClient(client);
@@ -118,20 +117,19 @@ public class ContainerOperationClient implements ScmClient {
   /**
    * Create a container over pipeline specified by the SCM.
    *
-   * @param containerId - Container ID
-   * @param client - Client to communicate with Datanodes
-   * @param pipeline - A pipeline that is already created.
+   * @param client - Client to communicate with Datanodes.
+   * @param containerId - Container ID.
    * @throws IOException
    */
-  public void createContainer(String containerId, XceiverClientSpi client,
-      Pipeline pipeline) throws IOException {
+  public void createContainer(XceiverClientSpi client,
+      long containerId) throws IOException {
     String traceID = UUID.randomUUID().toString();
     storageContainerLocationClient.notifyObjectStageChange(
         ObjectStageChangeRequestProto.Type.container,
         containerId,
         ObjectStageChangeRequestProto.Op.create,
         ObjectStageChangeRequestProto.Stage.begin);
-    ContainerProtocolCalls.createContainer(client, traceID);
+    ContainerProtocolCalls.createContainer(client, containerId, traceID);
     storageContainerLocationClient.notifyObjectStageChange(
         ObjectStageChangeRequestProto.Type.container,
         containerId,
@@ -142,8 +140,8 @@ public class ContainerOperationClient implements ScmClient {
     // creation state.
     if (LOG.isDebugEnabled()) {
       LOG.debug("Created container " + containerId
-          + " leader:" + pipeline.getLeader()
-          + " machines:" + pipeline.getMachines());
+          + " leader:" + client.getPipeline().getLeader()
+          + " machines:" + client.getPipeline().getMachines());
     }
   }
 
@@ -168,20 +166,25 @@ public class ContainerOperationClient implements ScmClient {
     // 2. Talk to Datanodes to create the pipeline.
     //
     // 3. update SCM that pipeline creation was successful.
-    storageContainerLocationClient.notifyObjectStageChange(
-        ObjectStageChangeRequestProto.Type.pipeline,
-        pipeline.getPipelineName(),
-        ObjectStageChangeRequestProto.Op.create,
-        ObjectStageChangeRequestProto.Stage.begin);
+
+    // TODO: this has not been fully implemented on server side
+    // SCMClientProtocolServer#notifyObjectStageChange
+    // TODO: when implement the pipeline state machine, change
+    // the pipeline name (string) to pipeline id (long)
+    //storageContainerLocationClient.notifyObjectStageChange(
+    //    ObjectStageChangeRequestProto.Type.pipeline,
+    //    pipeline.getPipelineName(),
+    //    ObjectStageChangeRequestProto.Op.create,
+    //    ObjectStageChangeRequestProto.Stage.begin);
 
     client.createPipeline(pipeline.getPipelineName(),
         pipeline.getMachines());
 
-    storageContainerLocationClient.notifyObjectStageChange(
-        ObjectStageChangeRequestProto.Type.pipeline,
-        pipeline.getPipelineName(),
-        ObjectStageChangeRequestProto.Op.create,
-        ObjectStageChangeRequestProto.Stage.complete);
+    //storageContainerLocationClient.notifyObjectStageChange(
+    //    ObjectStageChangeRequestProto.Type.pipeline,
+    //    pipeline.getPipelineName(),
+    //    ObjectStageChangeRequestProto.Op.create,
+    //    ObjectStageChangeRequestProto.Stage.complete);
 
     // TODO : Should we change the state on the client side ??
     // That makes sense, but it is not needed for the client to work.
@@ -193,16 +196,17 @@ public class ContainerOperationClient implements ScmClient {
    * @inheritDoc
    */
   @Override
-  public Pipeline createContainer(HddsProtos.ReplicationType type,
-      HddsProtos.ReplicationFactor factor,
-      String containerId, String owner) throws IOException {
+  public ContainerInfo createContainer(HddsProtos.ReplicationType type,
+      HddsProtos.ReplicationFactor factor, String owner) throws IOException {
     XceiverClientSpi client = null;
     try {
       // allocate container on SCM.
-      Pipeline pipeline =
+      ContainerInfo container =
           storageContainerLocationClient.allocateContainer(type, factor,
-              containerId, owner);
-      client = xceiverClientManager.acquireClient(pipeline);
+              owner);
+      Pipeline pipeline = container.getPipeline();
+      client = xceiverClientManager.acquireClient(pipeline,
+          container.getContainerID());
 
       // Allocated State means that SCM has allocated this pipeline in its
       // namespace. The client needs to create the pipeline on the machines
@@ -210,12 +214,11 @@ public class ContainerOperationClient implements ScmClient {
       if (pipeline.getLifeCycleState() == ALLOCATED) {
         createPipeline(client, pipeline);
       }
-
-      // TODO : Return ContainerInfo instead of Pipeline
       // connect to pipeline leader and allocate container on leader datanode.
-      client = xceiverClientManager.acquireClient(pipeline);
-      createContainer(containerId, client, pipeline);
-      return pipeline;
+      client = xceiverClientManager.acquireClient(pipeline,
+          container.getContainerID());
+      createContainer(client, container.getContainerID());
+      return container;
     } finally {
       if (client != null) {
         xceiverClientManager.releaseClient(client);
@@ -258,18 +261,18 @@ public class ContainerOperationClient implements ScmClient {
    * @throws IOException
    */
   @Override
-  public void deleteContainer(Pipeline pipeline, boolean force)
+  public void deleteContainer(long containerID, Pipeline pipeline, boolean force)
       throws IOException {
     XceiverClientSpi client = null;
     try {
-      client = xceiverClientManager.acquireClient(pipeline);
+      client = xceiverClientManager.acquireClient(pipeline, containerID);
       String traceID = UUID.randomUUID().toString();
-      ContainerProtocolCalls.deleteContainer(client, force, traceID);
+      ContainerProtocolCalls.deleteContainer(client, containerID, force, traceID);
       storageContainerLocationClient
-          .deleteContainer(pipeline.getContainerName());
+          .deleteContainer(containerID);
       if (LOG.isDebugEnabled()) {
         LOG.debug("Deleted container {}, leader: {}, machines: {} ",
-            pipeline.getContainerName(),
+            containerID,
             pipeline.getLeader(),
             pipeline.getMachines());
       }
@@ -284,11 +287,10 @@ public class ContainerOperationClient implements ScmClient {
    * {@inheritDoc}
    */
   @Override
-  public List<ContainerInfo> listContainer(String startName,
-      String prefixName, int count)
-      throws IOException {
+  public List<ContainerInfo> listContainer(long startContainerID,
+      int count) throws IOException {
     return storageContainerLocationClient.listContainer(
-        startName, prefixName, count);
+        startContainerID, count);
   }
 
   /**
@@ -300,17 +302,17 @@ public class ContainerOperationClient implements ScmClient {
    * @throws IOException
    */
   @Override
-  public ContainerData readContainer(Pipeline pipeline) throws IOException {
+  public ContainerData readContainer(long containerID,
+      Pipeline pipeline) throws IOException {
     XceiverClientSpi client = null;
     try {
-      client = xceiverClientManager.acquireClient(pipeline);
+      client = xceiverClientManager.acquireClient(pipeline, containerID);
       String traceID = UUID.randomUUID().toString();
       ReadContainerResponseProto response =
-          ContainerProtocolCalls.readContainer(client,
-              pipeline.getContainerName(), traceID);
+          ContainerProtocolCalls.readContainer(client, containerID, traceID);
       if (LOG.isDebugEnabled()) {
         LOG.debug("Read container {}, leader: {}, machines: {} ",
-            pipeline.getContainerName(),
+            containerID,
             pipeline.getLeader(),
             pipeline.getMachines());
       }
@@ -329,7 +331,7 @@ public class ContainerOperationClient implements ScmClient {
    * @throws IOException
    */
   @Override
-  public Pipeline getContainer(String containerId) throws
+  public ContainerInfo getContainer(long containerId) throws
       IOException {
     return storageContainerLocationClient.getContainer(containerId);
   }
@@ -341,7 +343,8 @@ public class ContainerOperationClient implements ScmClient {
    * @throws IOException
    */
   @Override
-  public void closeContainer(Pipeline pipeline) throws IOException {
+  public void closeContainer(long containerId, Pipeline pipeline)
+      throws IOException {
     XceiverClientSpi client = null;
     try {
       LOG.debug("Close container {}", pipeline);
@@ -364,18 +367,16 @@ public class ContainerOperationClient implements ScmClient {
       For now, take the #2 way.
        */
       // Actually close the container on Datanode
-      client = xceiverClientManager.acquireClient(pipeline);
+      client = xceiverClientManager.acquireClient(pipeline, containerId);
       String traceID = UUID.randomUUID().toString();
 
-      String containerId = pipeline.getContainerName();
-
       storageContainerLocationClient.notifyObjectStageChange(
           ObjectStageChangeRequestProto.Type.container,
           containerId,
           ObjectStageChangeRequestProto.Op.close,
           ObjectStageChangeRequestProto.Stage.begin);
 
-      ContainerProtocolCalls.closeContainer(client, traceID);
+      ContainerProtocolCalls.closeContainer(client, containerId, traceID);
       // Notify SCM to close the container
       storageContainerLocationClient.notifyObjectStageChange(
           ObjectStageChangeRequestProto.Type.container,
@@ -391,13 +392,13 @@ public class ContainerOperationClient implements ScmClient {
 
   /**
    * Get the the current usage information.
-   * @param pipeline - Pipeline
+   * @param containerID - ID of the container.
    * @return the size of the given container.
    * @throws IOException
    */
   @Override
-  public long getContainerSize(Pipeline pipeline) throws IOException {
-    // TODO : Pipeline can be null, handle it correctly.
+  public long getContainerSize(long containerID) throws IOException {
+    // TODO : Fix this, it currently returns the capacity but not the current usage.
     long size = getContainerSizeB();
     if (size == -1) {
       throw new IOException("Container size unknown!");

+ 7 - 6
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.scm.XceiverClientSpi;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.ChunkInfo;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos
     .ReadChunkResponseProto;
+import org.apache.hadoop.hdds.client.BlockID;
 
 import java.io.EOFException;
 import java.io.IOException;
@@ -45,7 +46,7 @@ public class ChunkInputStream extends InputStream implements Seekable {
 
   private static final int EOF = -1;
 
-  private final String key;
+  private final BlockID blockID;
   private final String traceID;
   private XceiverClientManager xceiverClientManager;
   private XceiverClientSpi xceiverClient;
@@ -58,15 +59,15 @@ public class ChunkInputStream extends InputStream implements Seekable {
   /**
    * Creates a new ChunkInputStream.
    *
-   * @param key chunk key
+   * @param blockID block ID of the chunk
    * @param xceiverClientManager client manager that controls client
    * @param xceiverClient client to perform container calls
    * @param chunks list of chunks to read
    * @param traceID container protocol call traceID
    */
-  public ChunkInputStream(String key, XceiverClientManager xceiverClientManager,
+  public ChunkInputStream(BlockID blockID, XceiverClientManager xceiverClientManager,
       XceiverClientSpi xceiverClient, List<ChunkInfo> chunks, String traceID) {
-    this.key = key;
+    this.blockID = blockID;
     this.traceID = traceID;
     this.xceiverClientManager = xceiverClientManager;
     this.xceiverClient = xceiverClient;
@@ -196,7 +197,7 @@ public class ChunkInputStream extends InputStream implements Seekable {
     final ReadChunkResponseProto readChunkResponse;
     try {
       readChunkResponse = ContainerProtocolCalls.readChunk(xceiverClient,
-          chunks.get(chunkIndex), key, traceID);
+          chunks.get(chunkIndex), blockID, traceID);
     } catch (IOException e) {
       throw new IOException("Unexpected OzoneException: " + e.toString(), e);
     }
@@ -211,7 +212,7 @@ public class ChunkInputStream extends InputStream implements Seekable {
         || pos >= chunkOffset[chunks.size() - 1] + chunks.get(chunks.size() - 1)
         .getLen()) {
       throw new EOFException(
-          "EOF encountered pos: " + pos + " container key: " + key);
+          "EOF encountered pos: " + pos + " container key: " + blockID.getLocalID());
     }
     if (chunkIndex == -1) {
       chunkIndex = Arrays.binarySearch(chunkOffset, pos);

+ 9 - 9
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.scm.XceiverClientSpi;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.ChunkInfo;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.KeyData;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.KeyValue;
+import org.apache.hadoop.hdds.client.BlockID;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -53,7 +54,7 @@ import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls
  */
 public class ChunkOutputStream extends OutputStream {
 
-  private final String containerKey;
+  private final BlockID blockID;
   private final String key;
   private final String traceID;
   private final KeyData.Builder containerKeyData;
@@ -67,25 +68,24 @@ public class ChunkOutputStream extends OutputStream {
   /**
    * Creates a new ChunkOutputStream.
    *
-   * @param containerKey container key
+   * @param blockID block ID
    * @param key chunk key
    * @param xceiverClientManager client manager that controls client
    * @param xceiverClient client to perform container calls
    * @param traceID container protocol call args
    * @param chunkSize chunk size
    */
-  public ChunkOutputStream(String containerKey, String key,
-      XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient,
-      String traceID, int chunkSize) {
-    this.containerKey = containerKey;
+  public ChunkOutputStream(BlockID blockID, String key,
+       XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient,
+       String traceID, int chunkSize) {
+    this.blockID = blockID;
     this.key = key;
     this.traceID = traceID;
     this.chunkSize = chunkSize;
     KeyValue keyValue = KeyValue.newBuilder()
         .setKey("TYPE").setValue("KEY").build();
     this.containerKeyData = KeyData.newBuilder()
-        .setContainerName(xceiverClient.getPipeline().getContainerName())
-        .setName(containerKey)
+        .setBlockID(blockID.getProtobuf())
         .addMetadata(keyValue);
     this.xceiverClientManager = xceiverClientManager;
     this.xceiverClient = xceiverClient;
@@ -217,7 +217,7 @@ public class ChunkOutputStream extends OutputStream {
         .setLen(data.size())
         .build();
     try {
-      writeChunk(xceiverClient, chunk, key, data, traceID);
+      writeChunk(xceiverClient, chunk, blockID, data, traceID);
     } catch (IOException e) {
       throw new IOException(
           "Unexpected Storage Container Exception: " + e.toString(), e);

+ 59 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java

@@ -0,0 +1,59 @@
+/**
+ * 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.hdds.client;
+
+import org.apache.commons.lang.builder.ToStringBuilder;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+
+/**
+ * BlockID of ozone (containerID + localID)
+ */
+public class BlockID {
+  private long containerID;
+  private long localID;
+
+  public BlockID(long containerID, long localID) {
+    this.containerID = containerID;
+    this.localID = localID;
+  }
+
+  public long getContainerID() {
+    return containerID;
+  }
+
+  public long getLocalID() {
+    return localID;
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this).
+        append("containerID", containerID).
+        append("localID", localID).
+        toString();
+  }
+
+  public HddsProtos.BlockID getProtobuf() {
+    return HddsProtos.BlockID.newBuilder().
+        setContainerID(containerID).setLocalID(localID).build();
+  }
+
+  public static BlockID getFromProtobuf(HddsProtos.BlockID blockID) {
+    return new BlockID(blockID.getContainerID(),
+        blockID.getLocalID());
+  }
+}

+ 22 - 24
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java

@@ -41,78 +41,76 @@ import java.util.List;
 public interface ScmClient {
   /**
    * Creates a Container on SCM and returns the pipeline.
-   * @param containerId - String container ID
-   * @return Pipeline
+   * @return ContainerInfo
    * @throws IOException
    */
-  Pipeline createContainer(String containerId, String owner) throws IOException;
+  ContainerInfo createContainer(String owner) throws IOException;
 
   /**
    * Gets a container by Name -- Throws if the container does not exist.
-   * @param containerId - String Container ID
+   * @param containerId - Container ID
    * @return Pipeline
    * @throws IOException
    */
-  Pipeline getContainer(String containerId) throws IOException;
+  ContainerInfo getContainer(long containerId) throws IOException;
 
   /**
-   * Close a container by name.
+   * Close a container.
    *
-   * @param pipeline the container to be closed.
+   * @param containerId - ID of the container.
+   * @param pipeline - Pipeline where the container is located.
    * @throws IOException
    */
-  void closeContainer(Pipeline pipeline) throws IOException;
+  void closeContainer(long containerId, Pipeline pipeline) throws IOException;
 
   /**
    * Deletes an existing container.
+   * @param containerId - ID of the container.
    * @param pipeline - Pipeline that represents the container.
    * @param force - true to forcibly delete the container.
    * @throws IOException
    */
-  void deleteContainer(Pipeline pipeline, boolean force) throws IOException;
+  void deleteContainer(long containerId, Pipeline pipeline, boolean force) throws IOException;
 
   /**
    * Lists a range of containers and get their info.
    *
-   * @param startName start name, if null, start searching at the head.
-   * @param prefixName prefix name, if null, then filter is disabled.
-   * @param count count, if count < 0, the max size is unlimited.(
-   *              Usually the count will be replace with a very big
-   *              value instead of being unlimited in case the db is very big)
+   * @param startContainerID start containerID.
+   * @param count count must be > 0.
    *
    * @return a list of pipeline.
    * @throws IOException
    */
-  List<ContainerInfo> listContainer(String startName, String prefixName,
+  List<ContainerInfo> listContainer(long startContainerID,
       int count) throws IOException;
 
   /**
    * Read meta data from an existing container.
-   * @param pipeline - Pipeline that represents the container.
+   * @param containerID - ID of the container.
+   * @param pipeline - Pipeline where the container is located.
    * @return ContainerInfo
    * @throws IOException
    */
-  ContainerData readContainer(Pipeline pipeline) throws IOException;
-
+  ContainerData readContainer(long containerID, Pipeline pipeline)
+      throws IOException;
 
   /**
    * Gets the container size -- Computed by SCM from Container Reports.
-   * @param pipeline - Pipeline
+   * @param containerID - ID of the container.
    * @return number of bytes used by this container.
    * @throws IOException
    */
-  long getContainerSize(Pipeline pipeline) throws IOException;
+  long getContainerSize(long containerID) throws IOException;
 
   /**
    * Creates a Container on SCM and returns the pipeline.
    * @param type - Replication Type.
    * @param replicationFactor - Replication Factor
-   * @param containerId - Container ID
-   * @return Pipeline
+   * @return ContainerInfo
    * @throws IOException - in case of error.
    */
-  Pipeline createContainer(HddsProtos.ReplicationType type,
-      HddsProtos.ReplicationFactor replicationFactor, String containerId,
+  ContainerInfo createContainer(HddsProtos.ReplicationType type,
+      HddsProtos.ReplicationFactor replicationFactor,
       String owner) throws IOException;
 
   /**

+ 11 - 9
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/AllocatedBlock.java

@@ -18,13 +18,15 @@
 
 package org.apache.hadoop.hdds.scm.container.common.helpers;
 
+import org.apache.hadoop.hdds.client.BlockID;
+
 /**
  * Allocated block wraps the result returned from SCM#allocateBlock which
  * contains a Pipeline and the key.
  */
 public final class AllocatedBlock {
   private Pipeline pipeline;
-  private String key;
+  private BlockID blockID;
   // Indicates whether the client should create container before writing block.
   private boolean shouldCreateContainer;
 
@@ -33,7 +35,7 @@ public final class AllocatedBlock {
    */
   public static class Builder {
     private Pipeline pipeline;
-    private String key;
+    private BlockID blockID;
     private boolean shouldCreateContainer;
 
     public Builder setPipeline(Pipeline p) {
@@ -41,8 +43,8 @@ public final class AllocatedBlock {
       return this;
     }
 
-    public Builder setKey(String k) {
-      this.key = k;
+    public Builder setBlockID(BlockID blockID) {
+      this.blockID = blockID;
       return this;
     }
 
@@ -52,14 +54,14 @@ public final class AllocatedBlock {
     }
 
     public AllocatedBlock build() {
-      return new AllocatedBlock(pipeline, key, shouldCreateContainer);
+      return new AllocatedBlock(pipeline, blockID, shouldCreateContainer);
     }
   }
 
-  private AllocatedBlock(Pipeline pipeline, String key,
+  private AllocatedBlock(Pipeline pipeline, BlockID blockID,
       boolean shouldCreateContainer) {
     this.pipeline = pipeline;
-    this.key = key;
+    this.blockID = blockID;
     this.shouldCreateContainer = shouldCreateContainer;
   }
 
@@ -67,8 +69,8 @@ public final class AllocatedBlock {
     return pipeline;
   }
 
-  public String getKey() {
-    return key;
+  public BlockID getBlockID() {
+    return blockID;
   }
 
   public boolean getCreateContainer() {

+ 3 - 19
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java

@@ -43,11 +43,9 @@ public class ContainerInfo
   // The wall-clock ms since the epoch at which the current state enters.
   private long stateEnterTime;
   private String owner;
-  private String containerName;
   private long containerID;
   ContainerInfo(
       long containerID,
-      final String containerName,
       HddsProtos.LifeCycleState state,
       Pipeline pipeline,
       long allocatedBytes,
@@ -56,7 +54,6 @@ public class ContainerInfo
       long stateEnterTime,
       String owner) {
     this.containerID = containerID;
-    this.containerName = containerName;
     this.pipeline = pipeline;
     this.allocatedBytes = allocatedBytes;
     this.usedBytes = usedBytes;
@@ -82,7 +79,6 @@ public class ContainerInfo
     builder.setState(info.getState());
     builder.setStateEnterTime(info.getStateEnterTime());
     builder.setOwner(info.getOwner());
-    builder.setContainerName(info.getContainerName());
     builder.setContainerID(info.getContainerID());
     return builder.build();
   }
@@ -91,10 +87,6 @@ public class ContainerInfo
     return containerID;
   }
 
-  public String getContainerName() {
-    return containerName;
-  }
-
   public HddsProtos.LifeCycleState getState() {
     return state;
   }
@@ -170,7 +162,6 @@ public class ContainerInfo
     if (getOwner() != null) {
       builder.setOwner(getOwner());
     }
-    builder.setContainerName(getContainerName());
     return builder.build();
   }
 
@@ -189,7 +180,6 @@ public class ContainerInfo
         + ", pipeline=" + pipeline
         + ", stateEnterTime=" + stateEnterTime
         + ", owner=" + owner
-        + ", containerName='" + containerName
         + '}';
   }
 
@@ -206,7 +196,7 @@ public class ContainerInfo
     ContainerInfo that = (ContainerInfo) o;
 
     return new EqualsBuilder()
-        .append(pipeline.getContainerName(), that.pipeline.getContainerName())
+        .append(getContainerID(), that.getContainerID())
 
         // TODO : Fix this later. If we add these factors some tests fail.
         // So Commenting this to continue and will enforce this with
@@ -221,7 +211,7 @@ public class ContainerInfo
   @Override
   public int hashCode() {
     return new HashCodeBuilder(11, 811)
-        .append(pipeline.getContainerName())
+        .append(getContainerID())
         .append(pipeline.getFactor())
         .append(pipeline.getType())
         .append(owner)
@@ -275,7 +265,6 @@ public class ContainerInfo
     private long keys;
     private long stateEnterTime;
     private String owner;
-    private String containerName;
     private long containerID;
 
     public Builder setContainerID(long id) {
@@ -319,14 +308,9 @@ public class ContainerInfo
       return this;
     }
 
-    public Builder setContainerName(String container) {
-      this.containerName = container;
-      return this;
-    }
-
     public ContainerInfo build() {
       return new
-          ContainerInfo(containerID, containerName, state, pipeline,
+          ContainerInfo(containerID, state, pipeline,
           allocated, used, keys, stateEnterTime, owner);
     }
   }

+ 9 - 7
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/DeleteBlockResult.java

@@ -17,6 +17,8 @@
 
 package org.apache.hadoop.hdds.scm.container.common.helpers;
 
+import org.apache.hadoop.hdds.client.BlockID;
+
 import static org.apache.hadoop.hdds.protocol.proto
     .ScmBlockLocationProtocolProtos.DeleteScmBlockResult;
 
@@ -24,21 +26,21 @@ import static org.apache.hadoop.hdds.protocol.proto
  * Class wraps storage container manager block deletion results.
  */
 public class DeleteBlockResult {
-  private String key;
+  private BlockID blockID;
   private DeleteScmBlockResult.Result result;
 
-  public DeleteBlockResult(final String key,
+  public DeleteBlockResult(final BlockID blockID,
       final DeleteScmBlockResult.Result result) {
-    this.key = key;
+    this.blockID = blockID;
     this.result = result;
   }
 
   /**
-   * Get key deleted.
-   * @return key name.
+   * Get block id deleted.
+   * @return block id.
    */
-  public String getKey() {
-    return key;
+  public BlockID getBlockID() {
+    return blockID;
   }
 
   /**

+ 2 - 16
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/Pipeline.java

@@ -57,7 +57,6 @@ public class Pipeline {
     WRITER = mapper.writer(filters);
   }
 
-  private String containerName;
   private PipelineChannel pipelineChannel;
   /**
    * Allows you to maintain private data on pipelines. This is not serialized
@@ -68,11 +67,9 @@ public class Pipeline {
   /**
    * Constructs a new pipeline data structure.
    *
-   * @param containerName - Container
    * @param pipelineChannel - transport information for this container
    */
-  public Pipeline(String containerName, PipelineChannel pipelineChannel) {
-    this.containerName = containerName;
+  public Pipeline(PipelineChannel pipelineChannel) {
     this.pipelineChannel = pipelineChannel;
     data = null;
   }
@@ -87,7 +84,7 @@ public class Pipeline {
     Preconditions.checkNotNull(pipeline);
     PipelineChannel pipelineChannel =
         PipelineChannel.getFromProtoBuf(pipeline.getPipelineChannel());
-    return new Pipeline(pipeline.getContainerName(), pipelineChannel);
+    return new Pipeline(pipelineChannel);
   }
 
   public HddsProtos.ReplicationFactor getFactor() {
@@ -146,20 +143,10 @@ public class Pipeline {
   public HddsProtos.Pipeline getProtobufMessage() {
     HddsProtos.Pipeline.Builder builder =
         HddsProtos.Pipeline.newBuilder();
-    builder.setContainerName(this.containerName);
     builder.setPipelineChannel(this.pipelineChannel.getProtobufMessage());
     return builder.build();
   }
 
-  /**
-   * Returns containerName if available.
-   *
-   * @return String.
-   */
-  public String getContainerName() {
-    return containerName;
-  }
-
   /**
    * Returns private data that is set on this pipeline.
    *
@@ -223,7 +210,6 @@ public class Pipeline {
     pipelineChannel.getDatanodes().keySet().stream()
         .forEach(id -> b.
             append(id.endsWith(pipelineChannel.getLeaderID()) ? "*" + id : id));
-    b.append("] container:").append(containerName);
     b.append(" name:").append(getPipelineName());
     if (getType() != null) {
       b.append(" type:").append(getType().toString());

+ 2 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/PipelineChannel.java

@@ -40,6 +40,8 @@ public class PipelineChannel {
   private ReplicationType type;
   private ReplicationFactor factor;
   private String name;
+  // TODO: change to long based id
+  //private long id;
 
   public PipelineChannel(String leaderID, LifeCycleState lifeCycleState,
       ReplicationType replicationType, ReplicationFactor replicationFactor,

+ 0 - 12
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocol.java

@@ -26,7 +26,6 @@ import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Set;
 
 /**
  * ScmBlockLocationProtocol is used by an HDFS node to find the set of nodes
@@ -34,17 +33,6 @@ import java.util.Set;
  */
 public interface ScmBlockLocationProtocol {
 
-  /**
-   * Find the set of nodes to read/write a block, as
-   * identified by the block key.  This method supports batch lookup by
-   * passing multiple keys.
-   *
-   * @param keys batch of block keys to find
-   * @return allocated blocks for each block key
-   * @throws IOException if there is any failure
-   */
-  Set<AllocatedBlock> getBlockLocations(Set<String> keys) throws IOException;
-
   /**
    * Asks SCM where a block should be allocated. SCM responds with the
    * set of datanodes that should be used creating this block.

+ 13 - 13
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java

@@ -38,19 +38,20 @@ public interface StorageContainerLocationProtocol {
    * set of datanodes that should be used creating this container.
    *
    */
-  Pipeline allocateContainer(HddsProtos.ReplicationType replicationType,
-      HddsProtos.ReplicationFactor factor, String containerName, String owner)
+  ContainerInfo allocateContainer(HddsProtos.ReplicationType replicationType,
+      HddsProtos.ReplicationFactor factor, String owner)
       throws IOException;
 
   /**
    * Ask SCM the location of the container. SCM responds with a group of
    * nodes where this container and its replicas are located.
    *
-   * @param containerName - Name of the container.
-   * @return Pipeline - the pipeline where container locates.
+   * @param containerID - ID of the container.
+   * @return ContainerInfo - the container info such as where the pipeline
+   *                         is located.
    * @throws IOException
    */
-  Pipeline getContainer(String containerName) throws IOException;
+  ContainerInfo getContainer(long containerID) throws IOException;
 
   /**
    * Ask SCM a list of containers with a range of container names
@@ -59,8 +60,7 @@ public interface StorageContainerLocationProtocol {
    * use prefix name to filter the result. the max size of the
    * searching range cannot exceed the value of count.
    *
-   * @param startName start name, if null, start searching at the head.
-   * @param prefixName prefix name, if null, then filter is disabled.
+   * @param startContainerID start container ID.
    * @param count count, if count < 0, the max size is unlimited.(
    *              Usually the count will be replace with a very big
    *              value instead of being unlimited in case the db is very big)
@@ -68,18 +68,18 @@ public interface StorageContainerLocationProtocol {
    * @return a list of container.
    * @throws IOException
    */
-  List<ContainerInfo> listContainer(String startName, String prefixName,
-      int count) throws IOException;
+  List<ContainerInfo> listContainer(long startContainerID, int count)
+      throws IOException;
 
   /**
    * Deletes a container in SCM.
    *
-   * @param containerName
+   * @param containerID
    * @throws IOException
    *   if failed to delete the container mapping from db store
    *   or container doesn't exist.
    */
-  void deleteContainer(String containerName) throws IOException;
+  void deleteContainer(long containerID) throws IOException;
 
   /**
    *  Queries a list of Node Statuses.
@@ -94,12 +94,12 @@ public interface StorageContainerLocationProtocol {
    * or containers on datanodes.
    * Container will be in Operational state after that.
    * @param type object type
-   * @param name object name
+   * @param id object id
    * @param op operation type (e.g., create, close, delete)
    * @param stage creation stage
    */
   void notifyObjectStageChange(
-      ObjectStageChangeRequestProto.Type type, String name,
+      ObjectStageChangeRequestProto.Type type, long id,
       ObjectStageChangeRequestProto.Op op,
       ObjectStageChangeRequestProto.Stage stage) throws IOException;
 

+ 2 - 44
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java

@@ -17,10 +17,10 @@
 package org.apache.hadoop.hdds.scm.protocolPB;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Sets;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.scm.ScmInfo;
 import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
@@ -34,14 +34,8 @@ import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
     .DeleteScmKeyBlocksRequestProto;
 import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
     .DeleteScmKeyBlocksResponseProto;
-import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
-    .GetScmBlockLocationsRequestProto;
-import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
-    .GetScmBlockLocationsResponseProto;
 import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
     .KeyBlocks;
-import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
-    .ScmLocatedBlockProto;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtocolTranslator;
 import org.apache.hadoop.ipc.RPC;
@@ -52,7 +46,6 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 import java.util.stream.Collectors;
 
 /**
@@ -81,41 +74,6 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
     this.rpcProxy = rpcProxy;
   }
 
-  /**
-   * Find the set of nodes to read/write a block, as
-   * identified by the block key.  This method supports batch lookup by
-   * passing multiple keys.
-   *
-   * @param keys batch of block keys to find
-   * @return allocated blocks for each block key
-   * @throws IOException if there is any failure
-   */
-  @Override
-  public Set<AllocatedBlock> getBlockLocations(Set<String> keys)
-      throws IOException {
-    GetScmBlockLocationsRequestProto.Builder req =
-        GetScmBlockLocationsRequestProto.newBuilder();
-    for (String key : keys) {
-      req.addKeys(key);
-    }
-    final GetScmBlockLocationsResponseProto resp;
-    try {
-      resp = rpcProxy.getScmBlockLocations(NULL_RPC_CONTROLLER,
-          req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    Set<AllocatedBlock> locatedBlocks =
-        Sets.newLinkedHashSetWithExpectedSize(resp.getLocatedBlocksCount());
-    for (ScmLocatedBlockProto locatedBlock : resp.getLocatedBlocksList()) {
-      locatedBlocks.add(new AllocatedBlock.Builder()
-          .setKey(locatedBlock.getKey())
-          .setPipeline(Pipeline.getFromProtoBuf(locatedBlock.getPipeline()))
-          .build());
-    }
-    return locatedBlocks;
-  }
-
   /**
    * Asks SCM where a block should be allocated. SCM responds with the
    * set of datanodes that should be used creating this block.
@@ -144,7 +102,7 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
           response.getErrorMessage() : "Allocate block failed.");
     }
     AllocatedBlock.Builder builder = new AllocatedBlock.Builder()
-        .setKey(response.getKey())
+        .setBlockID(BlockID.getFromProtobuf(response.getBlockID()))
         .setPipeline(Pipeline.getFromProtoBuf(response.getPipeline()))
         .setShouldCreateContainer(response.getCreateContainer());
     return builder.build();

+ 25 - 35
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java

@@ -17,7 +17,6 @@
 package org.apache.hadoop.hdds.scm.protocolPB;
 
 import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -92,20 +91,14 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB
    * supports replication factor of either 1 or 3.
    * @param type - Replication Type
    * @param factor - Replication Count
-   * @param containerName - Name
    * @return
    * @throws IOException
    */
   @Override
-  public Pipeline allocateContainer(HddsProtos.ReplicationType type,
-      HddsProtos.ReplicationFactor factor, String
-      containerName, String owner) throws IOException {
+  public ContainerInfo allocateContainer(HddsProtos.ReplicationType type,
+      HddsProtos.ReplicationFactor factor, String owner) throws IOException {
 
-    Preconditions.checkNotNull(containerName, "Container Name cannot be Null");
-    Preconditions.checkState(!containerName.isEmpty(), "Container name cannot" +
-        " be empty");
     ContainerRequestProto request = ContainerRequestProto.newBuilder()
-        .setContainerName(containerName)
         .setReplicationFactor(factor)
         .setReplicationType(type)
         .setOwner(owner)
@@ -121,22 +114,20 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB
       throw new IOException(response.hasErrorMessage() ?
           response.getErrorMessage() : "Allocate container failed.");
     }
-    return Pipeline.getFromProtoBuf(response.getPipeline());
+    return ContainerInfo.fromProtobuf(response.getContainerInfo());
   }
 
-  public Pipeline getContainer(String containerName) throws IOException {
-    Preconditions.checkNotNull(containerName,
-        "Container Name cannot be Null");
-    Preconditions.checkState(!containerName.isEmpty(),
-        "Container name cannot be empty");
+  public ContainerInfo getContainer(long containerID) throws IOException {
+    Preconditions.checkState(containerID >= 0,
+        "Container ID cannot be negative");
     GetContainerRequestProto request = GetContainerRequestProto
         .newBuilder()
-        .setContainerName(containerName)
+        .setContainerID(containerID)
         .build();
     try {
       GetContainerResponseProto response =
           rpcProxy.getContainer(NULL_RPC_CONTROLLER, request);
-      return Pipeline.getFromProtoBuf(response.getPipeline());
+      return ContainerInfo.fromProtobuf(response.getContainerInfo());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -146,16 +137,15 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB
    * {@inheritDoc}
    */
   @Override
-  public List<ContainerInfo> listContainer(String startName, String prefixName,
-      int count) throws IOException {
+  public List<ContainerInfo> listContainer(long startContainerID, int count)
+      throws IOException {
+    Preconditions.checkState(startContainerID >= 0,
+        "Container ID cannot be negative.");
+    Preconditions.checkState(count > 0,
+        "Container count must be greater than 0.");
     SCMListContainerRequestProto.Builder builder = SCMListContainerRequestProto
         .newBuilder();
-    if (prefixName != null) {
-      builder.setPrefixName(prefixName);
-    }
-    if (startName != null) {
-      builder.setStartName(startName);
-    }
+    builder.setStartContainerID(startContainerID);
     builder.setCount(count);
     SCMListContainerRequestProto request = builder.build();
 
@@ -177,17 +167,17 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB
    * Ask SCM to delete a container by name. SCM will remove
    * the container mapping in its database.
    *
-   * @param containerName
+   * @param containerID
    * @throws IOException
    */
   @Override
-  public void deleteContainer(String containerName)
+  public void deleteContainer(long containerID)
       throws IOException {
-    Preconditions.checkState(!Strings.isNullOrEmpty(containerName),
-        "Container name cannot be null or empty");
+    Preconditions.checkState(containerID >= 0,
+        "Container ID cannot be negative");
     SCMDeleteContainerRequestProto request = SCMDeleteContainerRequestProto
         .newBuilder()
-        .setContainerName(containerName)
+        .setContainerID(containerID)
         .build();
     try {
       rpcProxy.deleteContainer(NULL_RPC_CONTROLLER, request);
@@ -226,21 +216,21 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB
   /**
    * Notify from client that creates object on datanodes.
    * @param type object type
-   * @param name object name
+   * @param id object id
    * @param op operation type (e.g., create, close, delete)
    * @param stage object creation stage : begin/complete
    */
   @Override
   public void notifyObjectStageChange(
-      ObjectStageChangeRequestProto.Type type, String name,
+      ObjectStageChangeRequestProto.Type type, long id,
       ObjectStageChangeRequestProto.Op op,
       ObjectStageChangeRequestProto.Stage stage) throws IOException {
-    Preconditions.checkState(!Strings.isNullOrEmpty(name),
-        "Object name cannot be null or empty");
+    Preconditions.checkState(id >= 0,
+        "Object id cannot be negative.");
     ObjectStageChangeRequestProto request =
         ObjectStageChangeRequestProto.newBuilder()
             .setType(type)
-            .setName(name)
+            .setId(id)
             .setOp(op)
             .setStage(stage)
             .build();

+ 30 - 36
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java

@@ -51,6 +51,7 @@ import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Type;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos
     .WriteChunkRequestProto;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.KeyValue;
+import org.apache.hadoop.hdds.client.BlockID;
 
 import java.io.IOException;
 
@@ -79,7 +80,6 @@ public final class ContainerProtocolCalls  {
       KeyData containerKeyData, String traceID) throws IOException {
     GetKeyRequestProto.Builder readKeyRequest = GetKeyRequestProto
         .newBuilder()
-        .setPipeline(xceiverClient.getPipeline().getProtobufMessage())
         .setKeyData(containerKeyData);
     String id = xceiverClient.getPipeline().getLeader().getUuidString();
     ContainerCommandRequestProto request = ContainerCommandRequestProto
@@ -106,7 +106,6 @@ public final class ContainerProtocolCalls  {
       KeyData containerKeyData, String traceID) throws IOException {
     PutKeyRequestProto.Builder createKeyRequest = PutKeyRequestProto
         .newBuilder()
-        .setPipeline(xceiverClient.getPipeline().getProtobufMessage())
         .setKeyData(containerKeyData);
     String id = xceiverClient.getPipeline().getLeader().getUuidString();
     ContainerCommandRequestProto request = ContainerCommandRequestProto
@@ -125,18 +124,16 @@ public final class ContainerProtocolCalls  {
    *
    * @param xceiverClient client to perform call
    * @param chunk information about chunk to read
-   * @param key the key name
+   * @param blockID ID of the block
    * @param traceID container protocol call args
    * @return container protocol read chunk response
    * @throws IOException if there is an I/O error while performing the call
    */
   public static ReadChunkResponseProto readChunk(XceiverClientSpi xceiverClient,
-      ChunkInfo chunk, String key, String traceID)
-      throws IOException {
+        ChunkInfo chunk, BlockID blockID, String traceID) throws IOException {
     ReadChunkRequestProto.Builder readChunkRequest = ReadChunkRequestProto
         .newBuilder()
-        .setPipeline(xceiverClient.getPipeline().getProtobufMessage())
-        .setKeyName(key)
+        .setBlockID(blockID.getProtobuf())
         .setChunkData(chunk);
     String id = xceiverClient.getPipeline().getLeader().getUuidString();
     ContainerCommandRequestProto request = ContainerCommandRequestProto
@@ -156,18 +153,17 @@ public final class ContainerProtocolCalls  {
    *
    * @param xceiverClient client to perform call
    * @param chunk information about chunk to write
-   * @param key the key name
+   * @param blockID ID of the block
    * @param data the data of the chunk to write
    * @param traceID container protocol call args
    * @throws IOException if there is an I/O error while performing the call
    */
   public static void writeChunk(XceiverClientSpi xceiverClient, ChunkInfo chunk,
-      String key, ByteString data, String traceID)
+      BlockID blockID, ByteString data, String traceID)
       throws IOException {
     WriteChunkRequestProto.Builder writeChunkRequest = WriteChunkRequestProto
         .newBuilder()
-        .setPipeline(xceiverClient.getPipeline().getProtobufMessage())
-        .setKeyName(key)
+        .setBlockID(blockID.getProtobuf())
         .setChunkData(chunk)
         .setData(data);
     String id = xceiverClient.getPipeline().getLeader().getUuidString();
@@ -189,30 +185,29 @@ public final class ContainerProtocolCalls  {
    * than 1 MB.
    *
    * @param client - client that communicates with the container.
-   * @param containerName - Name of the container
-   * @param key - Name of the Key
+   * @param blockID - ID of the block
    * @param data - Data to be written into the container.
    * @param traceID - Trace ID for logging purpose.
    * @throws IOException
    */
   public static void writeSmallFile(XceiverClientSpi client,
-      String containerName, String key, byte[] data, String traceID)
+      BlockID blockID, byte[] data, String traceID)
       throws IOException {
 
     KeyData containerKeyData =
-        KeyData.newBuilder().setContainerName(containerName).setName(key)
+        KeyData.newBuilder().setBlockID(blockID.getProtobuf())
             .build();
     PutKeyRequestProto.Builder createKeyRequest =
         PutKeyRequestProto.newBuilder()
-            .setPipeline(client.getPipeline().getProtobufMessage())
             .setKeyData(containerKeyData);
 
     KeyValue keyValue =
         KeyValue.newBuilder().setKey("OverWriteRequested").setValue("true")
             .build();
     ChunkInfo chunk =
-        ChunkInfo.newBuilder().setChunkName(key + "_chunk").setOffset(0)
-            .setLen(data.length).addMetadata(keyValue).build();
+        ChunkInfo.newBuilder().setChunkName(blockID.getLocalID()
+            + "_chunk").setOffset(0).setLen(data.length).
+            addMetadata(keyValue).build();
 
     PutSmallFileRequestProto putSmallFileRequest =
         PutSmallFileRequestProto.newBuilder().setChunkInfo(chunk)
@@ -234,17 +229,18 @@ public final class ContainerProtocolCalls  {
   /**
    * createContainer call that creates a container on the datanode.
    * @param client  - client
+   * @param containerID - ID of container
    * @param traceID - traceID
    * @throws IOException
    */
-  public static void createContainer(XceiverClientSpi client, String traceID)
-      throws IOException {
+  public static void createContainer(XceiverClientSpi client, long containerID,
+      String traceID) throws IOException {
     ContainerProtos.CreateContainerRequestProto.Builder createRequest =
         ContainerProtos.CreateContainerRequestProto
             .newBuilder();
     ContainerProtos.ContainerData.Builder containerData = ContainerProtos
         .ContainerData.newBuilder();
-    containerData.setName(client.getPipeline().getContainerName());
+    containerData.setContainerID(containerID);
     createRequest.setPipeline(client.getPipeline().getProtobufMessage());
     createRequest.setContainerData(containerData.build());
 
@@ -268,12 +264,11 @@ public final class ContainerProtocolCalls  {
    * @param traceID
    * @throws IOException
    */
-  public static void deleteContainer(XceiverClientSpi client,
+  public static void deleteContainer(XceiverClientSpi client, long containerID,
       boolean force, String traceID) throws IOException {
     ContainerProtos.DeleteContainerRequestProto.Builder deleteRequest =
         ContainerProtos.DeleteContainerRequestProto.newBuilder();
-    deleteRequest.setName(client.getPipeline().getContainerName());
-    deleteRequest.setPipeline(client.getPipeline().getProtobufMessage());
+    deleteRequest.setContainerID(containerID);
     deleteRequest.setForceDelete(force);
     String id = client.getPipeline().getLeader().getUuidString();
     ContainerCommandRequestProto.Builder request =
@@ -291,14 +286,15 @@ public final class ContainerProtocolCalls  {
    * Close a container.
    *
    * @param client
+   * @param containerID
    * @param traceID
    * @throws IOException
    */
-  public static void closeContainer(XceiverClientSpi client, String traceID)
-      throws IOException {
+  public static void closeContainer(XceiverClientSpi client,
+      long containerID, String traceID) throws IOException {
     ContainerProtos.CloseContainerRequestProto.Builder closeRequest =
         ContainerProtos.CloseContainerRequestProto.newBuilder();
-    closeRequest.setPipeline(client.getPipeline().getProtobufMessage());
+    closeRequest.setContainerID(containerID);
 
     String id = client.getPipeline().getLeader().getUuidString();
     ContainerCommandRequestProto.Builder request =
@@ -320,11 +316,11 @@ public final class ContainerProtocolCalls  {
    * @throws IOException
    */
   public static ReadContainerResponseProto readContainer(
-      XceiverClientSpi client, String containerName,
+      XceiverClientSpi client, long containerID,
       String traceID) throws IOException {
     ReadContainerRequestProto.Builder readRequest =
         ReadContainerRequestProto.newBuilder();
-    readRequest.setName(containerName);
+    readRequest.setContainerID(containerID);
     readRequest.setPipeline(client.getPipeline().getProtobufMessage());
     String id = client.getPipeline().getLeader().getUuidString();
     ContainerCommandRequestProto.Builder request =
@@ -340,25 +336,23 @@ public final class ContainerProtocolCalls  {
   }
 
   /**
-   * Reads the data given the container name and key.
+   * Reads the data given the blockID
    *
    * @param client
-   * @param containerName - name of the container
-   * @param key - key
+   * @param blockID - ID of the block
    * @param traceID - trace ID
    * @return GetSmallFileResponseProto
    * @throws IOException
    */
   public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client,
-      String containerName, String key, String traceID) throws IOException {
+      BlockID blockID, String traceID) throws IOException {
     KeyData containerKeyData = KeyData
         .newBuilder()
-        .setContainerName(containerName)
-        .setName(key).build();
+        .setBlockID(blockID.getProtobuf())
+        .build();
 
     GetKeyRequestProto.Builder getKey = GetKeyRequestProto
         .newBuilder()
-        .setPipeline(client.getPipeline().getProtobufMessage())
         .setKeyData(containerKeyData);
     ContainerProtos.GetSmallFileRequestProto getSmallFileRequest =
         GetSmallFileRequestProto

+ 18 - 8
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/BlockGroup.java

@@ -17,9 +17,12 @@
 
 package org.apache.hadoop.ozone.common;
 
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
     .KeyBlocks;
 
+import java.util.ArrayList;
 import java.util.List;
 
 /**
@@ -28,13 +31,13 @@ import java.util.List;
 public final class BlockGroup {
 
   private String groupID;
-  private List<String> blockIDs;
-  private BlockGroup(String groupID, List<String> blockIDs) {
+  private List<BlockID> blockIDs;
+  private BlockGroup(String groupID, List<BlockID> blockIDs) {
     this.groupID = groupID;
     this.blockIDs = blockIDs;
   }
 
-  public List<String> getBlockIDList() {
+  public List<BlockID> getBlockIDList() {
     return blockIDs;
   }
 
@@ -43,8 +46,11 @@ public final class BlockGroup {
   }
 
   public KeyBlocks getProto() {
-    return KeyBlocks.newBuilder().setKey(groupID)
-        .addAllBlocks(blockIDs).build();
+    KeyBlocks.Builder kbb = KeyBlocks.newBuilder();
+    for (BlockID block : blockIDs) {
+      kbb.addBlocks(block.getProtobuf());
+    }
+    return kbb.setKey(groupID).build();
   }
 
   /**
@@ -53,8 +59,12 @@ public final class BlockGroup {
    * @return a group of blocks.
    */
   public static BlockGroup getFromProto(KeyBlocks proto) {
+    List<BlockID> blockIDs = new ArrayList<>();
+    for (HddsProtos.BlockID block : proto.getBlocksList()) {
+      blockIDs.add(new BlockID(block.getContainerID(), block.getLocalID()));
+    }
     return BlockGroup.newBuilder().setKeyName(proto.getKey())
-        .addAllBlockIDs(proto.getBlocksList()).build();
+        .addAllBlockIDs(blockIDs).build();
   }
 
   public static Builder newBuilder() {
@@ -67,14 +77,14 @@ public final class BlockGroup {
   public static class Builder {
 
     private String groupID;
-    private List<String> blockIDs;
+    private List<BlockID> blockIDs;
 
     public Builder setKeyName(String blockGroupID) {
       this.groupID = blockGroupID;
       return this;
     }
 
-    public Builder addAllBlockIDs(List<String> keyBlocks) {
+    public Builder addAllBlockIDs(List<BlockID> keyBlocks) {
       this.blockIDs = keyBlocks;
       return this;
     }

+ 7 - 6
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/DeleteBlockGroupResult.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.ozone.common;
 
+import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.scm.container.common.helpers.DeleteBlockResult;
 import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
     .DeleteScmBlockResult;
@@ -52,7 +53,7 @@ public class DeleteBlockGroupResult {
         new ArrayList<>(blockResultList.size());
     for (DeleteBlockResult result : blockResultList) {
       DeleteScmBlockResult proto = DeleteScmBlockResult.newBuilder()
-          .setKey(result.getKey())
+          .setBlockID(result.getBlockID().getProtobuf())
           .setResult(result.getResult()).build();
       resultProtoList.add(proto);
     }
@@ -63,8 +64,8 @@ public class DeleteBlockGroupResult {
       List<DeleteScmBlockResult> results) {
     List<DeleteBlockResult> protoResults = new ArrayList<>(results.size());
     for (DeleteScmBlockResult result : results) {
-      protoResults.add(new DeleteBlockResult(result.getKey(),
-          result.getResult()));
+      protoResults.add(new DeleteBlockResult(BlockID.getFromProtobuf(
+          result.getBlockID()), result.getResult()));
     }
     return protoResults;
   }
@@ -87,10 +88,10 @@ public class DeleteBlockGroupResult {
   /**
    * @return A list of deletion failed block IDs.
    */
-  public List<String> getFailedBlocks() {
-    List<String> failedBlocks = blockResultList.stream()
+  public List<BlockID> getFailedBlocks() {
+    List<BlockID> failedBlocks = blockResultList.stream()
         .filter(result -> result.getResult() != Result.success)
-        .map(DeleteBlockResult::getKey).collect(Collectors.toList());
+        .map(DeleteBlockResult::getBlockID).collect(Collectors.toList());
     return failedBlocks;
   }
 }

+ 23 - 18
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.container.common.helpers;
 
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.client.BlockID;
 
 import java.io.IOException;
 import java.util.Collections;
@@ -30,8 +31,7 @@ import java.util.TreeMap;
  * Helper class to convert Protobuf to Java classes.
  */
 public class KeyData {
-  private final String containerName;
-  private final String keyName;
+  private final BlockID blockID;
   private final Map<String, String> metadata;
 
   /**
@@ -44,12 +44,10 @@ public class KeyData {
   /**
    * Constructs a KeyData Object.
    *
-   * @param containerName
-   * @param keyName
+   * @param blockID
    */
-  public KeyData(String containerName, String keyName) {
-    this.containerName = containerName;
-    this.keyName = keyName;
+  public KeyData(BlockID blockID) {
+    this.blockID = blockID;
     this.metadata = new TreeMap<>();
   }
 
@@ -62,7 +60,7 @@ public class KeyData {
    */
   public static KeyData getFromProtoBuf(ContainerProtos.KeyData data) throws
       IOException {
-    KeyData keyData = new KeyData(data.getContainerName(), data.getName());
+    KeyData keyData = new KeyData(BlockID.getFromProtobuf(data.getBlockID()));
     for (int x = 0; x < data.getMetadataCount(); x++) {
       keyData.addMetadata(data.getMetadata(x).getKey(),
           data.getMetadata(x).getValue());
@@ -78,8 +76,7 @@ public class KeyData {
   public ContainerProtos.KeyData getProtoBufMessage() {
     ContainerProtos.KeyData.Builder builder =
         ContainerProtos.KeyData.newBuilder();
-    builder.setContainerName(this.containerName);
-    builder.setName(this.getKeyName());
+    builder.setBlockID(this.blockID.getProtobuf());
     builder.addAllChunks(this.chunks);
     for (Map.Entry<String, String> entry : metadata.entrySet()) {
       HddsProtos.KeyValue.Builder keyValBuilder =
@@ -135,19 +132,27 @@ public class KeyData {
   }
 
   /**
-   * Returns container Name.
-   * @return String.
+   * Returns container ID.
+   * @return long.
    */
-  public String getContainerName() {
-    return containerName;
+  public long getContainerID() {
+    return blockID.getContainerID();
   }
 
   /**
-   * Returns KeyName.
-   * @return String.
+   * Returns LocalID.
+   * @return long.
    */
-  public String getKeyName() {
-    return keyName;
+  public long getLocalID() {
+    return blockID.getLocalID();
+  }
+
+  /**
+   * Return Block ID.
+   * @return BlockID.
+   */
+  public BlockID getBlockID() {
+    return blockID;
   }
 
   /**

+ 1 - 37
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/ScmBlockLocationProtocolServerSideTranslatorPB.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.ozone.protocolPB;
 
-import com.google.common.collect.Sets;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -38,18 +37,11 @@ import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
     .DeleteScmKeyBlocksRequestProto;
 import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
     .DeleteScmKeyBlocksResponseProto;
-import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
-    .GetScmBlockLocationsRequestProto;
-import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
-    .GetScmBlockLocationsResponseProto;
-import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
-    .ScmLocatedBlockProto;
 import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Set;
 import java.util.stream.Collectors;
 
 /**
@@ -73,34 +65,6 @@ public final class ScmBlockLocationProtocolServerSideTranslatorPB
     this.impl = impl;
   }
 
-
-  @Override
-  public GetScmBlockLocationsResponseProto getScmBlockLocations(
-      RpcController controller, GetScmBlockLocationsRequestProto req)
-      throws ServiceException {
-    Set<String> keys = Sets.newLinkedHashSetWithExpectedSize(
-        req.getKeysCount());
-    for (String key : req.getKeysList()) {
-      keys.add(key);
-    }
-    final Set<AllocatedBlock> blocks;
-    try {
-      blocks = impl.getBlockLocations(keys);
-    } catch (IOException ex) {
-      throw new ServiceException(ex);
-    }
-    GetScmBlockLocationsResponseProto.Builder resp =
-        GetScmBlockLocationsResponseProto.newBuilder();
-    for (AllocatedBlock block: blocks) {
-      ScmLocatedBlockProto.Builder locatedBlock =
-          ScmLocatedBlockProto.newBuilder()
-              .setKey(block.getKey())
-              .setPipeline(block.getPipeline().getProtobufMessage());
-      resp.addLocatedBlocks(locatedBlock.build());
-    }
-    return resp.build();
-  }
-
   @Override
   public AllocateScmBlockResponseProto allocateScmBlock(
       RpcController controller, AllocateScmBlockRequestProto request)
@@ -112,7 +76,7 @@ public final class ScmBlockLocationProtocolServerSideTranslatorPB
       if (allocatedBlock != null) {
         return
             AllocateScmBlockResponseProto.newBuilder()
-                .setKey(allocatedBlock.getKey())
+                .setBlockID(allocatedBlock.getBlockID().getProtobuf())
                 .setPipeline(allocatedBlock.getPipeline().getProtobufMessage())
                 .setCreateContainer(allocatedBlock.getCreateContainer())
                 .setErrorCode(AllocateScmBlockResponseProto.Error.success)

+ 11 - 19
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java

@@ -23,7 +23,6 @@ import com.google.protobuf.ServiceException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdds.scm.ScmInfo;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
 import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -83,11 +82,10 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB
   public ContainerResponseProto allocateContainer(RpcController unused,
       ContainerRequestProto request) throws ServiceException {
     try {
-      Pipeline pipeline = impl.allocateContainer(request.getReplicationType(),
-          request.getReplicationFactor(), request.getContainerName(),
-          request.getOwner());
+      ContainerInfo container = impl.allocateContainer(request.getReplicationType(),
+          request.getReplicationFactor(), request.getOwner());
       return ContainerResponseProto.newBuilder()
-          .setPipeline(pipeline.getProtobufMessage())
+          .setContainerInfo(container.getProtobuf())
           .setErrorCode(ContainerResponseProto.Error.success)
           .build();
 
@@ -101,9 +99,9 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB
       RpcController controller, GetContainerRequestProto request)
       throws ServiceException {
     try {
-      Pipeline pipeline = impl.getContainer(request.getContainerName());
+      ContainerInfo container = impl.getContainer(request.getContainerID());
       return GetContainerResponseProto.newBuilder()
-          .setPipeline(pipeline.getProtobufMessage())
+          .setContainerInfo(container.getProtobuf())
           .build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -114,23 +112,17 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB
   public SCMListContainerResponseProto listContainer(RpcController controller,
       SCMListContainerRequestProto request) throws ServiceException {
     try {
-      String startName = null;
-      String prefixName = null;
+      long startContainerID = 0;
       int count = -1;
 
       // Arguments check.
-      if (request.hasPrefixName()) {
+      if (request.hasStartContainerID()) {
         // End container name is given.
-        prefixName = request.getPrefixName();
+        startContainerID = request.getStartContainerID();
       }
-      if (request.hasStartName()) {
-        // End container name is given.
-        startName = request.getStartName();
-      }
-
       count = request.getCount();
       List<ContainerInfo> containerList =
-          impl.listContainer(startName, prefixName, count);
+          impl.listContainer(startContainerID, count);
       SCMListContainerResponseProto.Builder builder =
           SCMListContainerResponseProto.newBuilder();
       for (ContainerInfo container : containerList) {
@@ -147,7 +139,7 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB
       RpcController controller, SCMDeleteContainerRequestProto request)
       throws ServiceException {
     try {
-      impl.deleteContainer(request.getContainerName());
+      impl.deleteContainer(request.getContainerID());
       return SCMDeleteContainerResponseProto.newBuilder().build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -178,7 +170,7 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB
       RpcController controller, ObjectStageChangeRequestProto request)
       throws ServiceException {
     try {
-      impl.notifyObjectStageChange(request.getType(), request.getName(),
+      impl.notifyObjectStageChange(request.getType(), request.getId(),
           request.getOp(), request.getStage());
       return ObjectStageChangeResponseProto.newBuilder().build();
     } catch (IOException e) {

+ 3 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java

@@ -76,7 +76,9 @@ public class LevelDBStore implements MetadataStore {
   }
 
   private void openDB(File dbPath, Options options) throws IOException {
-    dbPath.getParentFile().mkdirs();
+    if (dbPath.getParentFile().mkdirs()) {
+      LOG.debug("Db path {} created.", dbPath.getParentFile());
+    }
     db = JniDBFactory.factory.open(dbPath, options);
     if (LOG.isDebugEnabled()) {
       LOG.debug("LevelDB successfully opened");

+ 17 - 3
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataKeyFilters.java

@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.utils;
 
+import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ozone.OzoneConsts;
 
 /**
@@ -94,8 +94,8 @@ public final class MetadataKeyFilters {
       if (Strings.isNullOrEmpty(keyPrefix)) {
         accept = true;
       } else {
-        if (currentKey != null &&
-            DFSUtil.bytes2String(currentKey).startsWith(keyPrefix)) {
+        byte [] prefixBytes = keyPrefix.getBytes();
+        if (currentKey != null && prefixMatch(prefixBytes, currentKey)) {
           keysHinted++;
           accept = true;
         } else {
@@ -114,5 +114,19 @@ public final class MetadataKeyFilters {
     public int getKeysHintedNum() {
       return keysHinted;
     }
+
+    private boolean prefixMatch(byte[] prefix, byte[] key) {
+      Preconditions.checkNotNull(prefix);
+      Preconditions.checkNotNull(key);
+      if (key.length < prefix.length) {
+        return false;
+      }
+      for (int i = 0; i < prefix.length; i++) {
+        if (key[i] != prefix[i]) {
+          return false;
+        }
+      }
+      return true;
+    }
   }
 }

+ 1 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java

@@ -367,6 +367,7 @@ public class RocksDBStore implements MetadataStore {
   public void close() throws IOException {
     if (statMBeanName != null) {
       MBeans.unregister(statMBeanName);
+      statMBeanName = null;
     }
     if (db != null) {
       db.close();

+ 34 - 48
hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto

@@ -197,17 +197,15 @@ message ContainerCommandResponseProto {
 }
 
 message ContainerData {
-  required string name = 1;
+  required int64 containerID = 1;
   repeated KeyValue metadata = 2;
   optional string dbPath = 3;
   optional string containerPath = 4;
-  optional string hash = 6;
-  optional int64 bytesUsed = 7;
-  optional int64 size = 8;
-  optional int64 keyCount = 9;
-  //TODO: change required after we switch container ID from string to long
-  optional int64 containerID = 10;
-  optional LifeCycleState state = 11 [default = OPEN];
+  optional string hash = 5;
+  optional int64 bytesUsed = 6;
+  optional int64 size = 7;
+  optional int64 keyCount = 8;
+  optional LifeCycleState state = 9 [default = OPEN];
 }
 
 message ContainerMeta {
@@ -226,7 +224,7 @@ message  CreateContainerResponseProto {
 
 message  ReadContainerRequestProto {
   required Pipeline pipeline = 1;
-  required string name = 2;
+  required int64 containerID = 2;
 }
 
 message  ReadContainerResponseProto {
@@ -243,19 +241,16 @@ message  UpdateContainerResponseProto {
 }
 
 message  DeleteContainerRequestProto {
-  required Pipeline pipeline = 1;
-  required string name = 2;
-  optional bool forceDelete = 3 [default = false];
+  required int64 containerID = 1;
+  optional bool forceDelete = 2 [default = false];
 }
 
 message  DeleteContainerResponseProto {
 }
 
 message  ListContainerRequestProto {
-  required Pipeline pipeline = 1;
-  optional string prefix = 2;
-  required uint32 count = 3; // Max Results to return
-  optional string prevKey = 4;  // if this is not set query from start.
+  required int64 startContainerID = 1;
+  optional uint32 count = 2; // Max Results to return
 }
 
 message  ListContainerResponseProto {
@@ -263,34 +258,31 @@ message  ListContainerResponseProto {
 }
 
 message CloseContainerRequestProto {
-  required Pipeline pipeline = 1;
+  required int64 containerID = 1;
 }
 
 message CloseContainerResponseProto {
-  optional Pipeline pipeline = 1;
   optional string hash = 2;
+  optional int64 containerID = 3;
 }
 
 message KeyData {
-  required string containerName = 1;
-  required string name = 2;
-  optional int64 flags = 3; // for future use.
-  repeated KeyValue metadata = 4;
-  repeated ChunkInfo chunks = 5;
+  required BlockID blockID = 1;
+  optional int64 flags = 2; // for future use.
+  repeated KeyValue metadata = 3;
+  repeated ChunkInfo chunks = 4;
 }
 
 // Key Messages.
 message  PutKeyRequestProto {
-  required Pipeline pipeline = 1;
-  required KeyData keyData = 2;
+  required KeyData keyData = 1;
 }
 
 message  PutKeyResponseProto {
 }
 
 message  GetKeyRequestProto  {
-  required Pipeline pipeline = 1;
-  required KeyData keyData = 2;
+  required KeyData keyData = 1;
 }
 
 message  GetKeyResponseProto  {
@@ -299,17 +291,15 @@ message  GetKeyResponseProto  {
 
 
 message  DeleteKeyRequestProto {
-  required Pipeline pipeline = 1;
-  required string name = 2;
+  required BlockID blockID = 1;
 }
 
 message   DeleteKeyResponseProto {
 }
 
 message  ListKeyRequestProto {
-  required Pipeline pipeline = 1;
-  optional string prefix = 2; // if specified returns keys that match prefix.
-  required string prevKey = 3;
+  required int64 containerID = 1;
+  optional int64 startLocalID = 2;
   required uint32 count = 4;
 
 }
@@ -335,31 +325,28 @@ enum Stage {
 }
 
 message  WriteChunkRequestProto  {
-  required Pipeline pipeline = 1;
-  required string keyName = 2;
-  required ChunkInfo chunkData = 3;
-  optional bytes data = 4;
-  optional Stage stage = 5 [default = COMBINED];
+  required BlockID blockID = 1;
+  required ChunkInfo chunkData = 2;
+  optional bytes data = 3;
+  optional Stage stage = 4 [default = COMBINED];
 }
 
 message  WriteChunkResponseProto {
 }
 
 message  ReadChunkRequestProto  {
-  required Pipeline pipeline = 1;
-  required string keyName = 2;
-  required ChunkInfo chunkData = 3;
+  required BlockID blockID = 1;
+  required ChunkInfo chunkData = 2;
 }
 
 message  ReadChunkResponseProto {
-  required Pipeline pipeline = 1;
+  required BlockID blockID = 1;
   required ChunkInfo chunkData = 2;
   required bytes data = 3;
 }
 
 message  DeleteChunkRequestProto {
-  required Pipeline pipeline = 1;
-  required string keyName = 2;
+  required BlockID blockID = 1;
   required ChunkInfo chunkData = 3;
 }
 
@@ -367,10 +354,9 @@ message  DeleteChunkResponseProto {
 }
 
 message  ListChunkRequestProto {
-  required Pipeline pipeline = 1;
-  required string keyName = 2;
-  required string prevChunkName = 3;
-  required uint32 count = 4;
+  required BlockID blockID = 1;
+  required string prevChunkName = 2;
+  required uint32 count = 3;
 }
 
 message  ListChunkResponseProto {
@@ -400,7 +386,7 @@ message GetSmallFileResponseProto {
 }
 
 message CopyContainerRequestProto {
-  required string containerName = 1;
+  required int64 containerID = 1;
   required uint64 readOffset = 2;
   optional uint64 len = 3;
 }

+ 3 - 33
hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto

@@ -33,28 +33,6 @@ import "hdds.proto";
 
 
 // SCM Block protocol
-/**
- * keys - batch of block keys to find
- */
-message GetScmBlockLocationsRequestProto {
-  repeated string keys = 1;
-}
-
-/**
- * locatedBlocks - for each requested hash, nodes that currently host the
- *     container for that object key hash
- */
-message GetScmBlockLocationsResponseProto {
-  repeated ScmLocatedBlockProto locatedBlocks = 1;
-}
-
-/**
- * Holds the nodes that currently host the blocks for a key.
- */
-message ScmLocatedBlockProto {
-  required string key = 1;
-  required hadoop.hdds.Pipeline pipeline = 2;
-}
 
 /**
 * Request send to SCM asking allocate block of specified size.
@@ -84,7 +62,7 @@ message DeleteScmKeyBlocksRequestProto {
  */
 message KeyBlocks {
   required string key = 1;
-  repeated string blocks = 2;
+  repeated BlockID blocks = 2;
 }
 
 /**
@@ -112,7 +90,7 @@ message DeleteScmBlockResult {
     unknownFailure = 4;
   }
   required Result result = 1;
-  required string key = 2;
+  required BlockID blockID = 2;
 }
 
 /**
@@ -126,7 +104,7 @@ message AllocateScmBlockResponseProto {
     unknownFailure = 4;
   }
   required Error errorCode = 1;
-  required string key = 2;
+  required BlockID blockID = 2;
   required hadoop.hdds.Pipeline pipeline = 3;
   required bool createContainer = 4;
   optional string errorMessage = 5;
@@ -138,14 +116,6 @@ message AllocateScmBlockResponseProto {
  */
 service ScmBlockLocationProtocolService {
 
-  /**
-   * Find the set of nodes that currently host the block, as
-   * identified by the key.  This method supports batch lookup by
-   * passing multiple keys.
-   */
-  rpc getScmBlockLocations(GetScmBlockLocationsRequestProto)
-      returns (GetScmBlockLocationsResponseProto);
-
   /**
    * Creates a block entry in SCM.
    */

+ 7 - 9
hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto

@@ -35,7 +35,6 @@ import "hdds.proto";
 * Request send to SCM asking where the container should be created.
 */
 message ContainerRequestProto {
-  required string containerName = 1;
   // Ozone only support replciation of either 1 or 3.
   required ReplicationFactor replicationFactor = 2;
   required ReplicationType  replicationType = 3;
@@ -53,30 +52,29 @@ message ContainerResponseProto {
     errorContainerMissing = 3;
   }
   required Error errorCode = 1;
-  required Pipeline pipeline = 2;
+  required SCMContainerInfo containerInfo = 2;
   optional string errorMessage = 3;
 }
 
 message GetContainerRequestProto {
-  required string containerName = 1;
+  required int64 containerID = 1;
 }
 
 message GetContainerResponseProto {
-  required Pipeline pipeline = 1;
+  required SCMContainerInfo containerInfo = 1;
 }
 
 message SCMListContainerRequestProto {
   required uint32 count = 1;
-  optional string startName = 2;
-  optional string prefixName = 3;
-}
+  optional uint64 startContainerID = 2;
+ }
 
 message SCMListContainerResponseProto {
   repeated SCMContainerInfo containers = 1;
 }
 
 message SCMDeleteContainerRequestProto {
-  required string containerName = 1;
+  required int64 containerID = 1;
 }
 
 message SCMDeleteContainerResponseProto {
@@ -97,7 +95,7 @@ message ObjectStageChangeRequestProto {
     begin = 1;
     complete = 2;
   }
-  required string name = 1;
+  required int64 id = 1;
   required Type type = 2;
   required Op op= 3;
   required Stage stage = 4;

+ 9 - 4
hadoop-hdds/common/src/main/proto/hdds.proto

@@ -50,7 +50,6 @@ message PipelineChannel {
 // A pipeline is composed of PipelineChannel (Ratis/StandAlone) that back a
 // container.
 message Pipeline {
-    required string containerName = 1;
     required PipelineChannel pipelineChannel = 2;
 }
 
@@ -135,8 +134,7 @@ enum LifeCycleEvent {
 }
 
 message SCMContainerInfo {
-    // TODO : Remove the container name from pipeline.
-    required string containerName = 1;
+    required int64 containerID = 1;
     required LifeCycleState state = 2;
     required Pipeline pipeline = 3;
     // This is not total size of container, but space allocated by SCM for
@@ -146,7 +144,6 @@ message SCMContainerInfo {
     required uint64 numberOfKeys = 6;
     optional int64 stateEnterTime = 7;
     required string owner = 8;
-    required int64 containerID = 9;
 }
 
 message GetScmInfoRequestProto {
@@ -168,3 +165,11 @@ enum ReplicationFactor {
     ONE = 1;
     THREE = 3;
 }
+
+/**
+ * Block ID that uniquely identify a block by SCM.
+ */
+message BlockID {
+    required int64 containerID = 1;
+    required int64 localID = 2;
+}

+ 9 - 11
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java

@@ -21,7 +21,6 @@ import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
@@ -105,18 +104,17 @@ public final class ChunkUtils {
    * Validates chunk data and returns a file object to Chunk File that we are
    * expected to write data to.
    *
-   * @param pipeline - pipeline.
    * @param data - container data.
    * @param info - chunk info.
    * @return File
    * @throws StorageContainerException
    */
-  public static File validateChunk(Pipeline pipeline, ContainerData data,
+  public static File validateChunk(ContainerData data,
       ChunkInfo info) throws StorageContainerException {
 
     Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
 
-    File chunkFile = getChunkFile(pipeline, data, info);
+    File chunkFile = getChunkFile(data, info);
     if (ChunkUtils.isOverWriteRequested(chunkFile, info)) {
       if (!ChunkUtils.isOverWritePermitted(info)) {
         log.error("Rejecting write chunk request. Chunk overwrite " +
@@ -132,21 +130,21 @@ public final class ChunkUtils {
   /**
    * Validates that Path to chunk file exists.
    *
-   * @param pipeline - Container Info.
    * @param data - Container Data
    * @param info - Chunk info
    * @return - File.
    * @throws StorageContainerException
    */
-  public static File getChunkFile(Pipeline pipeline, ContainerData data,
+  public static File getChunkFile(ContainerData data,
       ChunkInfo info) throws StorageContainerException {
 
+    Preconditions.checkNotNull(data, "Container data can't be null");
     Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
-    if (data == null) {
-      log.error("Invalid container Name: {}", pipeline.getContainerName());
-      throw new StorageContainerException("Unable to find the container Name:" +
+    if (data.getContainerID() < 0) {
+      log.error("Invalid container id: {}", data.getContainerID());
+      throw new StorageContainerException("Unable to find the container id:" +
           " " +
-          pipeline.getContainerName(), CONTAINER_NOT_FOUND);
+          data.getContainerID(), CONTAINER_NOT_FOUND);
     }
 
     File dataDir = ContainerUtils.getDataDirectory(data).toFile();
@@ -335,7 +333,7 @@ public final class ChunkUtils {
         ContainerProtos.ReadChunkResponseProto.newBuilder();
     response.setChunkData(info.getProtoBufMessage());
     response.setData(ByteString.copyFrom(data));
-    response.setPipeline(msg.getReadChunk().getPipeline());
+    response.setBlockID(msg.getReadChunk().getBlockID());
 
     ContainerProtos.ContainerCommandResponseProto.Builder builder =
         ContainerUtils.getContainerResponse(msg, ContainerProtos.Result

+ 12 - 21
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerData.java

@@ -40,7 +40,6 @@ import java.util.concurrent.atomic.AtomicLong;
  */
 public class ContainerData {
 
-  private final String containerName;
   private final Map<String, String> metadata;
   private String dbPath;  // Path to Level DB Store.
   // Path to Physical file system where container and checksum are stored.
@@ -48,18 +47,18 @@ public class ContainerData {
   private String hash;
   private AtomicLong bytesUsed;
   private long maxSize;
-  private Long containerID;
+  private long containerID;
   private HddsProtos.LifeCycleState state;
 
   /**
    * Constructs a  ContainerData Object.
    *
-   * @param containerName - Name
+   * @param containerID - ID
+   * @param conf - Configuration
    */
-  public ContainerData(String containerName, Long containerID,
+  public ContainerData(long containerID,
       Configuration conf) {
     this.metadata = new TreeMap<>();
-    this.containerName = containerName;
     this.maxSize = conf.getLong(ScmConfigKeys.SCM_CONTAINER_CLIENT_MAX_SIZE_KEY,
         ScmConfigKeys.SCM_CONTAINER_CLIENT_MAX_SIZE_DEFAULT) * OzoneConsts.GB;
     this.bytesUsed =  new AtomicLong(0L);
@@ -76,7 +75,7 @@ public class ContainerData {
   public static ContainerData getFromProtBuf(
       ContainerProtos.ContainerData protoData, Configuration conf)
       throws IOException {
-    ContainerData data = new ContainerData(protoData.getName(),
+    ContainerData data = new ContainerData(
         protoData.getContainerID(), conf);
     for (int x = 0; x < protoData.getMetadataCount(); x++) {
       data.addMetadata(protoData.getMetadata(x).getKey(),
@@ -117,7 +116,6 @@ public class ContainerData {
   public ContainerProtos.ContainerData getProtoBufMessage() {
     ContainerProtos.ContainerData.Builder builder = ContainerProtos
         .ContainerData.newBuilder();
-    builder.setName(this.getContainerName());
     builder.setContainerID(this.getContainerID());
 
     if (this.getDBPath() != null) {
@@ -156,15 +154,6 @@ public class ContainerData {
     return builder.build();
   }
 
-  /**
-   * Returns the name of the container.
-   *
-   * @return - name
-   */
-  public String getContainerName() {
-    return containerName;
-  }
-
   /**
    * Adds metadata.
    */
@@ -231,9 +220,11 @@ public class ContainerData {
    *
    * @return String Name.
    */
-  public String getName() {
-    return getContainerName();
-  }
+    // TODO: check the ContainerCache class to see if we are using the ContainerID instead.
+   /*
+   public String getName() {
+    return getContainerID();
+  }*/
 
   /**
    * Get container file path.
@@ -255,7 +246,7 @@ public class ContainerData {
    * Get container ID.
    * @return - container ID.
    */
-  public synchronized Long getContainerID() {
+  public synchronized long getContainerID() {
     return containerID;
   }
 
@@ -284,7 +275,7 @@ public class ContainerData {
 
     // Some thing brain dead for now. name + Time stamp of when we get the close
     // container message.
-    setHash(DigestUtils.sha256Hex(this.getContainerName() +
+    setHash(DigestUtils.sha256Hex(this.getContainerID() +
         Long.toString(Time.monotonicNow())));
   }
 

+ 4 - 15
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java

@@ -26,7 +26,6 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolPro
  */
 public class ContainerReport {
   private static final int UNKNOWN = -1;
-  private final String containerName;
   private final String finalhash;
   private long size;
   private long keyCount;
@@ -51,11 +50,11 @@ public class ContainerReport {
   /**
    * Constructs the ContainerReport.
    *
-   * @param containerName - Container Name.
+   * @param containerID - Container ID.
    * @param finalhash - Final Hash.
    */
-  public ContainerReport(String containerName, String finalhash) {
-    this.containerName = containerName;
+  public ContainerReport(long containerID, String finalhash) {
+    this.containerID = containerID;
     this.finalhash = finalhash;
     this.size = UNKNOWN;
     this.keyCount = UNKNOWN;
@@ -74,7 +73,7 @@ public class ContainerReport {
    */
   public static ContainerReport getFromProtoBuf(ContainerInfo info) {
     Preconditions.checkNotNull(info);
-    ContainerReport report = new ContainerReport(info.getContainerName(),
+    ContainerReport report = new ContainerReport(info.getContainerID(),
         info.getFinalhash());
     if (info.hasSize()) {
       report.setSize(info.getSize());
@@ -102,15 +101,6 @@ public class ContainerReport {
     return report;
   }
 
-  /**
-   * Gets the container name.
-   *
-   * @return - Name
-   */
-  public String getContainerName() {
-    return containerName;
-  }
-
   /**
    * Returns the final signature for this container.
    *
@@ -203,7 +193,6 @@ public class ContainerReport {
    */
   public ContainerInfo getProtoBufMessage() {
     return ContainerInfo.newBuilder()
-        .setContainerName(this.getContainerName())
         .setKeyCount(this.getKeyCount())
         .setSize(this.getSize())
         .setUsed(this.getBytesUsed())

+ 10 - 4
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java

@@ -184,6 +184,12 @@ public final class ContainerUtils {
         removeExtension(containerFile.getName())).toString();
   }
 
+  public static long getContainerIDFromFile(File containerFile) {
+    Preconditions.checkNotNull(containerFile);
+    String containerID = getContainerNameFromFile(containerFile);
+    return Long.parseLong(containerID);
+  }
+
   /**
    * Verifies that this in indeed a new container.
    *
@@ -289,8 +295,8 @@ public final class ContainerUtils {
    */
   public static File getMetadataFile(ContainerData containerData,
       Path location) {
-    return location.resolve(containerData
-        .getContainerName().concat(CONTAINER_META))
+    return location.resolve(Long.toString(containerData
+        .getContainerID()).concat(CONTAINER_META))
         .toFile();
   }
 
@@ -303,8 +309,8 @@ public final class ContainerUtils {
    */
   public static File getContainerFile(ContainerData containerData,
       Path location) {
-    return location.resolve(containerData
-        .getContainerName().concat(CONTAINER_EXTENSION))
+    return location.resolve(Long.toString(containerData
+        .getContainerID()).concat(CONTAINER_EXTENSION))
         .toFile();
   }
 

+ 10 - 9
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DeletedContainerBlocksSummary.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.container.common.helpers;
 import com.google.common.collect.Maps;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
+import org.apache.hadoop.util.StringUtils;
 
 import java.util.List;
 import java.util.Map;
@@ -37,7 +38,7 @@ public final class DeletedContainerBlocksSummary {
   // value : the number of blocks need to be deleted in this container
   // if the message contains multiple entries for same block,
   // blocks will be merged
-  private final Map<String, Integer> blockSummary;
+  private final Map<Long, Integer> blockSummary;
   // total number of blocks in this message
   private int numOfBlocks;
 
@@ -47,14 +48,14 @@ public final class DeletedContainerBlocksSummary {
     blockSummary = Maps.newHashMap();
     blocks.forEach(entry -> {
       txSummary.put(entry.getTxID(), entry.getCount());
-      if (blockSummary.containsKey(entry.getContainerName())) {
-        blockSummary.put(entry.getContainerName(),
-            blockSummary.get(entry.getContainerName())
-                + entry.getBlockIDCount());
+      if (blockSummary.containsKey(entry.getContainerID())) {
+        blockSummary.put(entry.getContainerID(),
+            blockSummary.get(entry.getContainerID())
+                + entry.getLocalIDCount());
       } else {
-        blockSummary.put(entry.getContainerName(), entry.getBlockIDCount());
+        blockSummary.put(entry.getContainerID(), entry.getLocalIDCount());
       }
-      numOfBlocks += entry.getBlockIDCount();
+      numOfBlocks += entry.getLocalIDCount();
     });
   }
 
@@ -93,9 +94,9 @@ public final class DeletedContainerBlocksSummary {
           .append("TimesProceed=")
           .append(blks.getCount())
           .append(", ")
-          .append(blks.getContainerName())
+          .append(blks.getContainerID())
           .append(" : [")
-          .append(String.join(",", blks.getBlockIDList())).append("]")
+          .append(StringUtils.join(',', blks.getLocalIDList())).append("]")
           .append("\n");
     }
     return sb.toString();

+ 2 - 1
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/FileUtils.java

@@ -65,7 +65,8 @@ public final class FileUtils {
         ContainerProtos.ReadChunkResponseProto.newBuilder();
     readChunkresponse.setChunkData(info.getProtoBufMessage());
     readChunkresponse.setData(ByteString.copyFrom(data));
-    readChunkresponse.setPipeline(msg.getGetSmallFile().getKey().getPipeline());
+    readChunkresponse.setBlockID(msg.getGetSmallFile().getKey().
+        getKeyData().getBlockID());
 
     ContainerProtos.GetSmallFileResponseProto.Builder getSmallFile =
         ContainerProtos.GetSmallFileResponseProto.newBuilder();

+ 3 - 3
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java

@@ -63,11 +63,11 @@ public final class KeyUtils {
     ContainerCache cache = ContainerCache.getInstance(conf);
     Preconditions.checkNotNull(cache);
     try {
-      return cache.getDB(container.getContainerName(), container.getDBPath());
+      return cache.getDB(container.getContainerID(), container.getDBPath());
     } catch (IOException ex) {
       String message =
           String.format("Unable to open DB. DB Name: %s, Path: %s. ex: %s",
-          container.getContainerName(), container.getDBPath(), ex.getMessage());
+          container.getContainerID(), container.getDBPath(), ex.getMessage());
       throw new StorageContainerException(message, UNABLE_TO_READ_METADATA_DB);
     }
   }
@@ -83,7 +83,7 @@ public final class KeyUtils {
     Preconditions.checkNotNull(container);
     ContainerCache cache = ContainerCache.getInstance(conf);
     Preconditions.checkNotNull(cache);
-    cache.removeDB(container.getContainerName());
+    cache.removeDB(container.getContainerID());
   }
   /**
    * Shutdown all DB Handles.

+ 37 - 39
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkManagerImpl.java

@@ -19,11 +19,11 @@ package org.apache.hadoop.ozone.container.common.impl;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkUtils;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
@@ -66,13 +66,12 @@ public class ChunkManagerImpl implements ChunkManager {
   /**
    * writes a given chunk.
    *
-   * @param pipeline - Name and the set of machines that make this container.
-   * @param keyName - Name of the Key.
+   * @param blockID - ID of the block.
    * @param info - ChunkInfo.
    * @throws StorageContainerException
    */
   @Override
-  public void writeChunk(Pipeline pipeline, String keyName, ChunkInfo info,
+  public void writeChunk(BlockID blockID, ChunkInfo info,
       byte[] data, ContainerProtos.Stage stage)
       throws StorageContainerException {
     // we don't want container manager to go away while we are writing chunks.
@@ -80,13 +79,13 @@ public class ChunkManagerImpl implements ChunkManager {
 
     // TODO : Take keyManager Write lock here.
     try {
-      Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
-      String containerName = pipeline.getContainerName();
-      Preconditions.checkNotNull(containerName,
-          "Container name cannot be null");
+      Preconditions.checkNotNull(blockID, "Block ID cannot be null.");
+      long containerID = blockID.getContainerID();
+      Preconditions.checkState(containerID >= 0,
+          "Container ID cannot be negative");
       ContainerData container =
-          containerManager.readContainer(containerName);
-      File chunkFile = ChunkUtils.validateChunk(pipeline, container, info);
+          containerManager.readContainer(containerID);
+      File chunkFile = ChunkUtils.validateChunk(container, info);
       File tmpChunkFile = getTmpChunkFile(chunkFile, info);
 
       LOG.debug("writing chunk:{} chunk stage:{} chunk file:{} tmp chunk file",
@@ -96,16 +95,16 @@ public class ChunkManagerImpl implements ChunkManager {
         ChunkUtils.writeData(tmpChunkFile, info, data);
         break;
       case COMMIT_DATA:
-        commitChunk(tmpChunkFile, chunkFile, containerName, info.getLen());
+        commitChunk(tmpChunkFile, chunkFile, containerID, info.getLen());
         break;
       case COMBINED:
         // directly write to the chunk file
         long oldSize = chunkFile.length();
         ChunkUtils.writeData(chunkFile, info, data);
         long newSize = chunkFile.length();
-        containerManager.incrBytesUsed(containerName, newSize - oldSize);
-        containerManager.incrWriteCount(containerName);
-        containerManager.incrWriteBytes(containerName, info.getLen());
+        containerManager.incrBytesUsed(containerID, newSize - oldSize);
+        containerManager.incrWriteCount(containerID);
+        containerManager.incrWriteBytes(containerID, info.getLen());
         break;
       default:
         throw new IOException("Can not identify write operation.");
@@ -136,22 +135,21 @@ public class ChunkManagerImpl implements ChunkManager {
 
   // Commit the chunk by renaming the temporary chunk file to chunk file
   private void commitChunk(File tmpChunkFile, File chunkFile,
-      String containerName, long chunkLen) throws IOException {
+      long containerID, long chunkLen) throws IOException {
     long sizeDiff = tmpChunkFile.length() - chunkFile.length();
     // It is safe to replace here as the earlier chunk if existing should be
     // caught as part of validateChunk
     Files.move(tmpChunkFile.toPath(), chunkFile.toPath(),
         StandardCopyOption.REPLACE_EXISTING);
-    containerManager.incrBytesUsed(containerName, sizeDiff);
-    containerManager.incrWriteCount(containerName);
-    containerManager.incrWriteBytes(containerName, chunkLen);
+    containerManager.incrBytesUsed(containerID, sizeDiff);
+    containerManager.incrWriteCount(containerID);
+    containerManager.incrWriteBytes(containerID, chunkLen);
   }
 
   /**
    * reads the data defined by a chunk.
    *
-   * @param pipeline - container pipeline.
-   * @param keyName - Name of the Key
+   * @param blockID - ID of the block.
    * @param info - ChunkInfo.
    * @return byte array
    * @throws StorageContainerException
@@ -159,20 +157,20 @@ public class ChunkManagerImpl implements ChunkManager {
    * TODO: Explore if we need to do that for ozone.
    */
   @Override
-  public byte[] readChunk(Pipeline pipeline, String keyName, ChunkInfo info)
+  public byte[] readChunk(BlockID blockID, ChunkInfo info)
       throws StorageContainerException {
     containerManager.readLock();
     try {
-      Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
-      String containerName = pipeline.getContainerName();
-      Preconditions.checkNotNull(containerName,
-          "Container name cannot be null");
+      Preconditions.checkNotNull(blockID, "Block ID cannot be null.");
+      long containerID = blockID.getContainerID();
+      Preconditions.checkState(containerID >= 0,
+          "Container ID cannot be negative");
       ContainerData container =
-          containerManager.readContainer(containerName);
-      File chunkFile = ChunkUtils.getChunkFile(pipeline, container, info);
+          containerManager.readContainer(containerID);
+      File chunkFile = ChunkUtils.getChunkFile(container, info);
       ByteBuffer data =  ChunkUtils.readData(chunkFile, info);
-      containerManager.incrReadCount(containerName);
-      containerManager.incrReadBytes(containerName, chunkFile.length());
+      containerManager.incrReadCount(containerID);
+      containerManager.incrReadBytes(containerID, chunkFile.length());
       return data.array();
     } catch (ExecutionException | NoSuchAlgorithmException e) {
       LOG.error("read data failed. error: {}", e);
@@ -191,25 +189,25 @@ public class ChunkManagerImpl implements ChunkManager {
   /**
    * Deletes a given chunk.
    *
-   * @param pipeline - Pipeline.
-   * @param keyName - Key Name
+   * @param blockID - ID of the block.
    * @param info - Chunk Info
    * @throws StorageContainerException
    */
   @Override
-  public void deleteChunk(Pipeline pipeline, String keyName, ChunkInfo info)
+  public void deleteChunk(BlockID blockID, ChunkInfo info)
       throws StorageContainerException {
     containerManager.readLock();
     try {
-      Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
-      String containerName = pipeline.getContainerName();
-      Preconditions.checkNotNull(containerName,
-          "Container name cannot be null");
-      File chunkFile = ChunkUtils.getChunkFile(pipeline, containerManager
-          .readContainer(containerName), info);
+      Preconditions.checkNotNull(blockID, "Block ID cannot be null.");
+      long containerID = blockID.getContainerID();
+      Preconditions.checkState(containerID >= 0,
+          "Container ID cannot be negative");
+
+      File chunkFile = ChunkUtils.getChunkFile(containerManager
+          .readContainer(containerID), info);
       if ((info.getOffset() == 0) && (info.getLen() == chunkFile.length())) {
         FileUtil.fullyDelete(chunkFile);
-        containerManager.decrBytesUsed(containerName, chunkFile.length());
+        containerManager.decrBytesUsed(containerID, chunkFile.length());
       } else {
         LOG.error("Not Supported Operation. Trying to delete a " +
             "chunk that is in shared file. chunk info : " + info.toString());

+ 106 - 104
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java

@@ -24,7 +24,6 @@ import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
@@ -113,7 +112,9 @@ public class ContainerManagerImpl implements ContainerManager {
   static final Logger LOG =
       LoggerFactory.getLogger(ContainerManagerImpl.class);
 
-  private final ConcurrentSkipListMap<String, ContainerStatus>
+  // TODO: consider primitive collection like eclipse-collections
+  // to avoid autoboxing overhead
+  private final ConcurrentSkipListMap<Long, ContainerStatus>
       containerMap = new ConcurrentSkipListMap<>();
 
   // Use a non-fair RW lock for better throughput, we may revisit this decision
@@ -229,6 +230,7 @@ public class ContainerManagerImpl implements ContainerManager {
     Preconditions.checkNotNull(keyName,
         "Container Name  to container key mapping is null");
 
+    long containerID = Long.parseLong(keyName);
     try {
       String containerFileName = containerName.concat(CONTAINER_EXTENSION);
       String metaFileName = containerName.concat(CONTAINER_META);
@@ -249,7 +251,7 @@ public class ContainerManagerImpl implements ContainerManager {
         // when loading the info we get a null, this often means last time
         // SCM was ending up at some middle phase causing that the metadata
         // was not populated. Such containers are marked as inactive.
-        containerMap.put(keyName, new ContainerStatus(null));
+        containerMap.put(containerID, new ContainerStatus(null));
         return;
       }
       containerData = ContainerData.getFromProtBuf(containerDataProto, conf);
@@ -263,7 +265,7 @@ public class ContainerManagerImpl implements ContainerManager {
         // Hopefully SCM will ask us to delete this container and rebuild it.
         LOG.error("Invalid SHA found for container data. Name :{}"
             + "cowardly refusing to read invalid data", containerName);
-        containerMap.put(keyName, new ContainerStatus(null));
+        containerMap.put(containerID, new ContainerStatus(null));
         return;
       }
 
@@ -295,7 +297,7 @@ public class ContainerManagerImpl implements ContainerManager {
       }).sum();
       containerStatus.setBytesUsed(bytesUsed);
 
-      containerMap.put(keyName, containerStatus);
+      containerMap.put(containerID, containerStatus);
     } catch (IOException | NoSuchAlgorithmException ex) {
       LOG.error("read failed for file: {} ex: {}", containerName,
           ex.getMessage());
@@ -303,7 +305,7 @@ public class ContainerManagerImpl implements ContainerManager {
       // TODO : Add this file to a recovery Queue.
 
       // Remember that this container is busted and we cannot use it.
-      containerMap.put(keyName, new ContainerStatus(null));
+      containerMap.put(containerID, new ContainerStatus(null));
       throw new StorageContainerException("Unable to read container info",
           UNABLE_TO_READ_METADATA_DB);
     } finally {
@@ -316,18 +318,17 @@ public class ContainerManagerImpl implements ContainerManager {
   /**
    * Creates a container with the given name.
    *
-   * @param pipeline -- Nodes which make up this container.
    * @param containerData - Container Name and metadata.
    * @throws StorageContainerException - Exception
    */
   @Override
-  public void createContainer(Pipeline pipeline, ContainerData containerData)
+  public void createContainer(ContainerData containerData)
       throws StorageContainerException {
     Preconditions.checkNotNull(containerData, "Container data cannot be null");
     writeLock();
     try {
-      if (containerMap.containsKey(containerData.getName())) {
-        LOG.debug("container already exists. {}", containerData.getName());
+      if (containerMap.containsKey(containerData.getContainerID())) {
+        LOG.debug("container already exists. {}", containerData.getContainerID());
         throw new StorageContainerException("container already exists.",
             CONTAINER_EXISTS);
       }
@@ -399,7 +400,7 @@ public class ContainerManagerImpl implements ContainerManager {
           location);
       File metadataFile = ContainerUtils.getMetadataFile(containerData,
           location);
-      String containerName = containerData.getContainerName();
+      String containerName = Long.toString(containerData.getContainerID());
 
       if(!overwrite) {
         ContainerUtils.verifyIsNewContainer(containerFile, metadataFile);
@@ -446,7 +447,7 @@ public class ContainerManagerImpl implements ContainerManager {
 
       LOG.error("Creation of container failed. Name: {}, we might need to " +
               "cleanup partially created artifacts. ",
-          containerData.getContainerName(), ex);
+          containerData.getContainerID(), ex);
       throw new StorageContainerException("Container creation failed. ",
           ex, CONTAINER_INTERNAL_ERROR);
     } finally {
@@ -459,45 +460,45 @@ public class ContainerManagerImpl implements ContainerManager {
   /**
    * Deletes an existing container.
    *
-   * @param pipeline - nodes that make this container.
-   * @param containerName - name of the container.
+   * @param containerID - ID of the container.
    * @param forceDelete - whether this container should be deleted forcibly.
    * @throws StorageContainerException
    */
   @Override
-  public void deleteContainer(Pipeline pipeline, String containerName,
+  public void deleteContainer(long containerID,
       boolean forceDelete) throws StorageContainerException {
-    Preconditions.checkNotNull(containerName, "Container name cannot be null");
-    Preconditions.checkState(containerName.length() > 0,
-        "Container name length cannot be zero.");
+    Preconditions.checkState(containerID >= 0,
+        "Container ID cannot be negative.");
     writeLock();
     try {
-      if (isOpen(pipeline.getContainerName())) {
+      if (isOpen(containerID)) {
         throw new StorageContainerException(
             "Deleting an open container is not allowed.",
             UNCLOSED_CONTAINER_IO);
       }
 
-      ContainerStatus status = containerMap.get(containerName);
+      ContainerStatus status = containerMap.get(containerID);
       if (status == null) {
-        LOG.debug("No such container. Name: {}", containerName);
-        throw new StorageContainerException("No such container. Name : " +
-            containerName, CONTAINER_NOT_FOUND);
+        LOG.debug("No such container. ID: {}", containerID);
+        throw new StorageContainerException("No such container. ID : " +
+            containerID, CONTAINER_NOT_FOUND);
       }
       if (status.getContainer() == null) {
-        LOG.debug("Invalid container data. Name: {}", containerName);
+        LOG.debug("Invalid container data. ID: {}", containerID);
         throw new StorageContainerException("Invalid container data. Name : " +
-            containerName, CONTAINER_NOT_FOUND);
+            containerID, CONTAINER_NOT_FOUND);
       }
       ContainerUtils.removeContainer(status.getContainer(), conf, forceDelete);
-      containerMap.remove(containerName);
+      containerMap.remove(containerID);
     } catch (StorageContainerException e) {
       throw e;
     } catch (IOException e) {
       // TODO : An I/O error during delete can leave partial artifacts on the
       // disk. We will need the cleaner thread to cleanup this information.
-      LOG.error("Failed to cleanup container. Name: {}", containerName, e);
-      throw new StorageContainerException(containerName, e, IO_EXCEPTION);
+      String errMsg = String.format("Failed to cleanup container. ID: %d",
+          containerID);
+      LOG.error(errMsg, e);
+      throw new StorageContainerException(errMsg, e, IO_EXCEPTION);
     } finally {
       writeUnlock();
     }
@@ -511,25 +512,29 @@ public class ContainerManagerImpl implements ContainerManager {
    * time. It is possible that using this iteration you can miss certain
    * container from the listing.
    *
-   * @param prefix -  Return keys that match this prefix.
+   * @param startContainerID -  Return containers with ID >= startContainerID.
    * @param count - how many to return
-   * @param prevKey - Previous Key Value or empty String.
    * @param data - Actual containerData
    * @throws StorageContainerException
    */
   @Override
-  public void listContainer(String prefix, long count, String prevKey,
+  public void listContainer(long startContainerID, long count,
       List<ContainerData> data) throws StorageContainerException {
-    // TODO : Support list with Prefix and PrevKey
     Preconditions.checkNotNull(data,
         "Internal assertion: data cannot be null");
+    Preconditions.checkState(startContainerID >= 0,
+        "Start container ID cannot be negative");
+    Preconditions.checkState(count > 0,
+        "max number of containers returned " +
+            "must be positive");
+
     readLock();
     try {
-      ConcurrentNavigableMap<String, ContainerStatus> map;
-      if (prevKey == null || prevKey.isEmpty()) {
+      ConcurrentNavigableMap<Long, ContainerStatus> map;
+      if (startContainerID == 0) {
         map = containerMap.tailMap(containerMap.firstKey(), true);
       } else {
-        map = containerMap.tailMap(prevKey, false);
+        map = containerMap.tailMap(startContainerID, false);
       }
 
       int currentCount = 0;
@@ -549,24 +554,23 @@ public class ContainerManagerImpl implements ContainerManager {
   /**
    * Get metadata about a specific container.
    *
-   * @param containerName - Name of the container
+   * @param containerID - ID of the container
    * @return ContainerData - Container Data.
    * @throws StorageContainerException
    */
   @Override
-  public ContainerData readContainer(String containerName) throws
-      StorageContainerException {
-    Preconditions.checkNotNull(containerName, "Container name cannot be null");
-    Preconditions.checkState(containerName.length() > 0,
-        "Container name length cannot be zero.");
-    if (!containerMap.containsKey(containerName)) {
-      throw new StorageContainerException("Unable to find the container. Name: "
-          + containerName, CONTAINER_NOT_FOUND);
+  public ContainerData readContainer(long containerID)
+      throws StorageContainerException {
+    Preconditions.checkState(containerID >= 0,
+        "Container ID cannot be negative.");
+    if (!containerMap.containsKey(containerID)) {
+      throw new StorageContainerException("Unable to find the container. ID: "
+          + containerID, CONTAINER_NOT_FOUND);
     }
-    ContainerData cData = containerMap.get(containerName).getContainer();
+    ContainerData cData = containerMap.get(containerID).getContainer();
     if (cData == null) {
-      throw new StorageContainerException("Invalid container data. Name: "
-          + containerName, CONTAINER_INTERNAL_ERROR);
+      throw new StorageContainerException("Invalid container data. ID: "
+          + containerID, CONTAINER_INTERNAL_ERROR);
     }
     return cData;
   }
@@ -575,13 +579,13 @@ public class ContainerManagerImpl implements ContainerManager {
    * Closes a open container, if it is already closed or does not exist a
    * StorageContainerException is thrown.
    *
-   * @param containerName - Name of the container.
+   * @param containerID - ID of the container.
    * @throws StorageContainerException
    */
   @Override
-  public void closeContainer(String containerName)
+  public void closeContainer(long containerID)
       throws StorageContainerException, NoSuchAlgorithmException {
-    ContainerData containerData = readContainer(containerName);
+    ContainerData containerData = readContainer(containerID);
     containerData.closeContainer();
     writeContainerInfo(containerData, true);
     MetadataStore db = KeyUtils.getDB(containerData, conf);
@@ -602,15 +606,13 @@ public class ContainerManagerImpl implements ContainerManager {
     // issues.
 
     ContainerStatus status = new ContainerStatus(containerData);
-    containerMap.put(containerName, status);
+    containerMap.put(containerID, status);
   }
 
   @Override
-  public void updateContainer(Pipeline pipeline, String containerName,
-      ContainerData data, boolean forceUpdate)
-      throws StorageContainerException {
-    Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
-    Preconditions.checkNotNull(containerName, "Container name cannot be null");
+  public void updateContainer(long containerID, ContainerData data,
+      boolean forceUpdate) throws StorageContainerException {
+    Preconditions.checkState(containerID >= 0, "Container ID cannot be negative.");
     Preconditions.checkNotNull(data, "Container data cannot be null");
     FileOutputStream containerStream = null;
     DigestOutputStream dos = null;
@@ -618,9 +620,9 @@ public class ContainerManagerImpl implements ContainerManager {
     File containerFileBK = null, containerFile = null;
     boolean deleted = false;
 
-    if(!containerMap.containsKey(containerName)) {
+    if(!containerMap.containsKey(containerID)) {
       throw new StorageContainerException("Container doesn't exist. Name :"
-          + containerName, CONTAINER_NOT_FOUND);
+          + containerID, CONTAINER_NOT_FOUND);
     }
 
     try {
@@ -633,7 +635,7 @@ public class ContainerManagerImpl implements ContainerManager {
 
     try {
       Path location = locationManager.getContainerPath();
-      ContainerData orgData = containerMap.get(containerName).getContainer();
+      ContainerData orgData = containerMap.get(containerID).getContainer();
       if (orgData == null) {
         // updating a invalid container
         throw new StorageContainerException("Update a container with invalid" +
@@ -642,7 +644,7 @@ public class ContainerManagerImpl implements ContainerManager {
 
       if (!forceUpdate && !orgData.isOpen()) {
         throw new StorageContainerException(
-            "Update a closed container is not allowed. Name: " + containerName,
+            "Update a closed container is not allowed. ID: " + containerID,
             UNSUPPORTED_REQUEST);
       }
 
@@ -652,7 +654,7 @@ public class ContainerManagerImpl implements ContainerManager {
       if (!forceUpdate) {
         if (!containerFile.exists() || !containerFile.canWrite()) {
           throw new StorageContainerException(
-              "Container file not exists or corrupted. Name: " + containerName,
+              "Container file not exists or corrupted. ID: " + containerID,
               CONTAINER_INTERNAL_ERROR);
         }
 
@@ -672,7 +674,7 @@ public class ContainerManagerImpl implements ContainerManager {
 
       // Update the in-memory map
       ContainerStatus newStatus = new ContainerStatus(data);
-      containerMap.replace(containerName, newStatus);
+      containerMap.replace(containerID, newStatus);
     } catch (IOException e) {
       // Restore the container file from backup
       if(containerFileBK != null && containerFileBK.exists() && deleted) {
@@ -683,8 +685,8 @@ public class ContainerManagerImpl implements ContainerManager {
               CONTAINER_INTERNAL_ERROR);
         } else {
           throw new StorageContainerException(
-              "Failed to restore container data from the backup. Name: "
-                  + containerName, CONTAINER_INTERNAL_ERROR);
+              "Failed to restore container data from the backup. ID: "
+                  + containerID, CONTAINER_INTERNAL_ERROR);
         }
       } else {
         throw new StorageContainerException(
@@ -711,22 +713,22 @@ public class ContainerManagerImpl implements ContainerManager {
   /**
    * Checks if a container exists.
    *
-   * @param containerName - Name of the container.
+   * @param containerID - ID of the container.
    * @return true if the container is open false otherwise.
    * @throws StorageContainerException - Throws Exception if we are not able to
    *                                   find the container.
    */
   @Override
-  public boolean isOpen(String containerName) throws StorageContainerException {
-    final ContainerStatus status = containerMap.get(containerName);
+  public boolean isOpen(long containerID) throws StorageContainerException {
+    final ContainerStatus status = containerMap.get(containerID);
     if (status == null) {
       throw new StorageContainerException(
-          "Container status not found: " + containerName, CONTAINER_NOT_FOUND);
+          "Container status not found: " + containerID, CONTAINER_NOT_FOUND);
     }
     final ContainerData cData = status.getContainer();
     if (cData == null) {
       throw new StorageContainerException(
-          "Container not found: " + containerName, CONTAINER_NOT_FOUND);
+          "Container not found: " + containerID, CONTAINER_NOT_FOUND);
     }
     return cData.isOpen();
   }
@@ -746,7 +748,7 @@ public class ContainerManagerImpl implements ContainerManager {
 
 
   @VisibleForTesting
-  public ConcurrentSkipListMap<String, ContainerStatus> getContainerMap() {
+  public ConcurrentSkipListMap<Long, ContainerStatus> getContainerMap() {
     return containerMap;
   }
 
@@ -901,7 +903,7 @@ public class ContainerManagerImpl implements ContainerManager {
     for (ContainerStatus container: containers) {
       StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder =
           StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder();
-      ciBuilder.setContainerName(container.getContainer().getContainerName())
+      ciBuilder.setContainerID(container.getContainer().getContainerID())
           .setSize(container.getContainer().getMaxSize())
           .setUsed(container.getContainer().getBytesUsed())
           .setKeyCount(container.getContainer().getKeyCount())
@@ -966,7 +968,7 @@ public class ContainerManagerImpl implements ContainerManager {
   }
 
   @Override
-  public void incrPendingDeletionBlocks(int numBlocks, String containerId) {
+  public void incrPendingDeletionBlocks(int numBlocks, long containerId) {
     writeLock();
     try {
       ContainerStatus status = containerMap.get(containerId);
@@ -977,7 +979,7 @@ public class ContainerManagerImpl implements ContainerManager {
   }
 
   @Override
-  public void decrPendingDeletionBlocks(int numBlocks, String containerId) {
+  public void decrPendingDeletionBlocks(int numBlocks, long containerId) {
     writeLock();
     try {
       ContainerStatus status = containerMap.get(containerId);
@@ -990,35 +992,35 @@ public class ContainerManagerImpl implements ContainerManager {
   /**
    * Increase the read count of the container.
    *
-   * @param containerName - Name of the container.
+   * @param containerId - ID of the container.
    */
   @Override
-  public void incrReadCount(String containerName) {
-    ContainerStatus status = containerMap.get(containerName);
+  public void incrReadCount(long containerId) {
+    ContainerStatus status = containerMap.get(containerId);
     status.incrReadCount();
   }
 
-  public long getReadCount(String containerName) {
-    ContainerStatus status = containerMap.get(containerName);
+  public long getReadCount(long containerId) {
+    ContainerStatus status = containerMap.get(containerId);
     return status.getReadCount();
   }
 
   /**
    * Increse the read counter for bytes read from the container.
    *
-   * @param containerName - Name of the container.
+   * @param containerId - ID of the container.
    * @param readBytes     - bytes read from the container.
    */
   @Override
-  public void incrReadBytes(String containerName, long readBytes) {
-    ContainerStatus status = containerMap.get(containerName);
+  public void incrReadBytes(long containerId, long readBytes) {
+    ContainerStatus status = containerMap.get(containerId);
     status.incrReadBytes(readBytes);
   }
 
-  public long getReadBytes(String containerName) {
+  public long getReadBytes(long containerId) {
     readLock();
     try {
-      ContainerStatus status = containerMap.get(containerName);
+      ContainerStatus status = containerMap.get(containerId);
       return status.getReadBytes();
     } finally {
       readUnlock();
@@ -1028,76 +1030,76 @@ public class ContainerManagerImpl implements ContainerManager {
   /**
    * Increase the write count of the container.
    *
-   * @param containerName - Name of the container.
+   * @param containerId - Name of the container.
    */
   @Override
-  public void incrWriteCount(String containerName) {
-    ContainerStatus status = containerMap.get(containerName);
+  public void incrWriteCount(long containerId) {
+    ContainerStatus status = containerMap.get(containerId);
     status.incrWriteCount();
   }
 
-  public long getWriteCount(String containerName) {
-    ContainerStatus status = containerMap.get(containerName);
+  public long getWriteCount(long containerId) {
+    ContainerStatus status = containerMap.get(containerId);
     return status.getWriteCount();
   }
 
   /**
    * Increse the write counter for bytes write into the container.
    *
-   * @param containerName - Name of the container.
+   * @param containerId   - ID of the container.
    * @param writeBytes    - bytes write into the container.
    */
   @Override
-  public void incrWriteBytes(String containerName, long writeBytes) {
-    ContainerStatus status = containerMap.get(containerName);
+  public void incrWriteBytes(long containerId, long writeBytes) {
+    ContainerStatus status = containerMap.get(containerId);
     status.incrWriteBytes(writeBytes);
   }
 
-  public long getWriteBytes(String containerName) {
-    ContainerStatus status = containerMap.get(containerName);
+  public long getWriteBytes(long containerId) {
+    ContainerStatus status = containerMap.get(containerId);
     return status.getWriteBytes();
   }
 
   /**
    * Increase the bytes used by the container.
    *
-   * @param containerName - Name of the container.
+   * @param containerId   - ID of the container.
    * @param used          - additional bytes used by the container.
    * @return the current bytes used.
    */
   @Override
-  public long incrBytesUsed(String containerName, long used) {
-    ContainerStatus status = containerMap.get(containerName);
+  public long incrBytesUsed(long containerId, long used) {
+    ContainerStatus status = containerMap.get(containerId);
     return status.incrBytesUsed(used);
   }
 
   /**
    * Decrease the bytes used by the container.
    *
-   * @param containerName - Name of the container.
+   * @param containerId   - ID of the container.
    * @param used          - additional bytes reclaimed by the container.
    * @return the current bytes used.
    */
   @Override
-  public long decrBytesUsed(String containerName, long used) {
-    ContainerStatus status = containerMap.get(containerName);
+  public long decrBytesUsed(long containerId, long used) {
+    ContainerStatus status = containerMap.get(containerId);
     return status.decrBytesUsed(used);
   }
 
-  public long getBytesUsed(String containerName) {
-    ContainerStatus status = containerMap.get(containerName);
+  public long getBytesUsed(long containerId) {
+    ContainerStatus status = containerMap.get(containerId);
     return status.getBytesUsed();
   }
 
   /**
    * Get the number of keys in the container.
    *
-   * @param containerName - Name of the container.
+   * @param containerId - ID of the container.
    * @return the current key count.
    */
   @Override
-  public long getNumKeys(String containerName) {
-    ContainerStatus status = containerMap.get(containerName);
+  public long getNumKeys(long containerId) {
+    ContainerStatus status = containerMap.get(containerId);
     return status.getNumKeys();  }
 
   /**

+ 50 - 71
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.hdds.protocol.proto.ContainerProtos
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos
     .ContainerCommandResponseProto;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Type;
+import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkUtils;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
@@ -186,7 +187,7 @@ public class Dispatcher implements ContainerDispatcher {
     } catch (IOException ex) {
       LOG.warn("Container operation failed. " +
               "Container: {} Operation: {}  trace ID: {} Error: {}",
-          msg.getCreateContainer().getContainerData().getName(),
+          msg.getCreateContainer().getContainerData().getContainerID(),
           msg.getCmdType().name(),
           msg.getTraceID(),
           ex.toString(), ex);
@@ -230,7 +231,7 @@ public class Dispatcher implements ContainerDispatcher {
     } catch (IOException ex) {
       LOG.warn("Container operation failed. " +
               "Container: {} Operation: {}  trace ID: {} Error: {}",
-          msg.getCreateContainer().getContainerData().getName(),
+          msg.getCreateContainer().getContainerData().getContainerID(),
           msg.getCmdType().name(),
           msg.getTraceID(),
           ex.toString(), ex);
@@ -273,7 +274,7 @@ public class Dispatcher implements ContainerDispatcher {
     } catch (IOException ex) {
       LOG.warn("Container operation failed. " +
               "Container: {} Operation: {}  trace ID: {} Error: {}",
-          msg.getCreateContainer().getContainerData().getName(),
+          msg.getCreateContainer().getContainerData().getContainerID(),
           msg.getCmdType().name(),
           msg.getTraceID(),
           ex.toString(), ex);
@@ -318,17 +319,14 @@ public class Dispatcher implements ContainerDispatcher {
           msg.getTraceID());
       return ContainerUtils.malformedRequest(msg);
     }
-
-    Pipeline pipeline = Pipeline.getFromProtoBuf(
-        msg.getUpdateContainer().getPipeline());
-    String containerName = msg.getUpdateContainer()
-        .getContainerData().getName();
+    long containerID = msg.getUpdateContainer()
+        .getContainerData().getContainerID();
 
     ContainerData data = ContainerData.getFromProtBuf(
         msg.getUpdateContainer().getContainerData(), conf);
     boolean forceUpdate = msg.getUpdateContainer().getForceUpdate();
-    this.containerManager.updateContainer(
-        pipeline, containerName, data, forceUpdate);
+    this.containerManager.updateContainer(containerID,
+        data, forceUpdate);
     return ContainerUtils.getContainerResponse(msg);
   }
 
@@ -349,8 +347,9 @@ public class Dispatcher implements ContainerDispatcher {
       return ContainerUtils.malformedRequest(msg);
     }
 
-    String name = msg.getReadContainer().getName();
-    ContainerData container = this.containerManager.readContainer(name);
+    long containerID = msg.getReadContainer().getContainerID();
+    ContainerData container = this.containerManager.
+        readContainer(containerID);
     return ContainerUtils.getReadContainerResponse(msg, container);
   }
 
@@ -370,12 +369,9 @@ public class Dispatcher implements ContainerDispatcher {
       return ContainerUtils.malformedRequest(msg);
     }
 
-    Pipeline pipeline = Pipeline.getFromProtoBuf(
-        msg.getDeleteContainer().getPipeline());
-    Preconditions.checkNotNull(pipeline);
-    String name = msg.getDeleteContainer().getName();
+    long containerID = msg.getDeleteContainer().getContainerID();
     boolean forceDelete = msg.getDeleteContainer().getForceDelete();
-    this.containerManager.deleteContainer(pipeline, name, forceDelete);
+    this.containerManager.deleteContainer(containerID, forceDelete);
     return ContainerUtils.getContainerResponse(msg);
   }
 
@@ -401,7 +397,7 @@ public class Dispatcher implements ContainerDispatcher {
         msg.getCreateContainer().getPipeline());
     Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
 
-    this.containerManager.createContainer(pipeline, cData);
+    this.containerManager.createContainer(cData);
     return ContainerUtils.getContainerResponse(msg);
   }
 
@@ -420,14 +416,12 @@ public class Dispatcher implements ContainerDispatcher {
             msg.getTraceID());
         return ContainerUtils.malformedRequest(msg);
       }
-      Pipeline pipeline = Pipeline.getFromProtoBuf(msg.getCloseContainer()
-          .getPipeline());
-      Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
-      if (!this.containerManager.isOpen(pipeline.getContainerName())) {
+      long containerID = msg.getCloseContainer().getContainerID();
+      if (!this.containerManager.isOpen(containerID)) {
         throw new StorageContainerException("Attempting to close a closed " +
             "container.", CLOSED_CONTAINER_IO);
       }
-      this.containerManager.closeContainer(pipeline.getContainerName());
+      this.containerManager.closeContainer(containerID);
       return ContainerUtils.getContainerResponse(msg);
     } catch (NoSuchAlgorithmException e) {
       throw new StorageContainerException("No such Algorithm", e,
@@ -449,11 +443,9 @@ public class Dispatcher implements ContainerDispatcher {
           msg.getTraceID());
       return ContainerUtils.malformedRequest(msg);
     }
-    String keyName = msg.getWriteChunk().getKeyName();
-    Pipeline pipeline = Pipeline.getFromProtoBuf(
-        msg.getWriteChunk().getPipeline());
-    Preconditions.checkNotNull(pipeline);
-    if (!this.containerManager.isOpen(pipeline.getContainerName())) {
+    BlockID blockID = BlockID.getFromProtobuf(
+        msg.getWriteChunk().getBlockID());
+    if (!this.containerManager.isOpen(blockID.getContainerID())) {
       throw new StorageContainerException("Write to closed container.",
           CLOSED_CONTAINER_IO);
     }
@@ -469,7 +461,7 @@ public class Dispatcher implements ContainerDispatcher {
 
     }
     this.containerManager.getChunkManager()
-        .writeChunk(pipeline, keyName, chunkInfo,
+        .writeChunk(blockID, chunkInfo,
             data, msg.getWriteChunk().getStage());
 
     return ChunkUtils.getChunkResponse(msg);
@@ -489,17 +481,13 @@ public class Dispatcher implements ContainerDispatcher {
           msg.getTraceID());
       return ContainerUtils.malformedRequest(msg);
     }
-
-    String keyName = msg.getReadChunk().getKeyName();
-    Pipeline pipeline = Pipeline.getFromProtoBuf(
-        msg.getReadChunk().getPipeline());
-    Preconditions.checkNotNull(pipeline);
-
+    BlockID blockID = BlockID.getFromProtobuf(
+        msg.getReadChunk().getBlockID());
     ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(msg.getReadChunk()
         .getChunkData());
     Preconditions.checkNotNull(chunkInfo);
-    byte[] data = this.containerManager.getChunkManager().readChunk(pipeline,
-        keyName, chunkInfo);
+    byte[] data = this.containerManager.getChunkManager().
+        readChunk(blockID, chunkInfo);
     metrics.incContainerBytesStats(Type.ReadChunk, data.length);
     return ChunkUtils.getReadChunkResponse(msg, data, chunkInfo);
   }
@@ -519,11 +507,10 @@ public class Dispatcher implements ContainerDispatcher {
       return ContainerUtils.malformedRequest(msg);
     }
 
-    String keyName = msg.getDeleteChunk().getKeyName();
-    Pipeline pipeline = Pipeline.getFromProtoBuf(
-        msg.getDeleteChunk().getPipeline());
-    Preconditions.checkNotNull(pipeline);
-    if (!this.containerManager.isOpen(pipeline.getContainerName())) {
+    BlockID blockID = BlockID.getFromProtobuf(msg.getDeleteChunk()
+        .getBlockID());
+    long containerID = blockID.getContainerID();
+    if (!this.containerManager.isOpen(containerID)) {
       throw new StorageContainerException("Write to closed container.",
           CLOSED_CONTAINER_IO);
     }
@@ -531,7 +518,7 @@ public class Dispatcher implements ContainerDispatcher {
         .getChunkData());
     Preconditions.checkNotNull(chunkInfo);
 
-    this.containerManager.getChunkManager().deleteChunk(pipeline, keyName,
+    this.containerManager.getChunkManager().deleteChunk(blockID,
         chunkInfo);
     return ChunkUtils.getChunkResponse(msg);
   }
@@ -550,15 +537,16 @@ public class Dispatcher implements ContainerDispatcher {
           msg.getTraceID());
       return ContainerUtils.malformedRequest(msg);
     }
-    Pipeline pipeline = Pipeline.getFromProtoBuf(msg.getPutKey().getPipeline());
-    Preconditions.checkNotNull(pipeline);
-    if (!this.containerManager.isOpen(pipeline.getContainerName())) {
+    BlockID blockID = BlockID.getFromProtobuf(
+        msg.getPutKey().getKeyData().getBlockID());
+    long containerID = blockID.getContainerID();
+    if (!this.containerManager.isOpen(containerID)) {
       throw new StorageContainerException("Write to closed container.",
           CLOSED_CONTAINER_IO);
     }
     KeyData keyData = KeyData.getFromProtoBuf(msg.getPutKey().getKeyData());
     Preconditions.checkNotNull(keyData);
-    this.containerManager.getKeyManager().putKey(pipeline, keyData);
+    this.containerManager.getKeyManager().putKey(keyData);
     long numBytes = keyData.getProtoBufMessage().toByteArray().length;
     metrics.incContainerBytesStats(Type.PutKey, numBytes);
     return KeyUtils.getKeyResponse(msg);
@@ -601,17 +589,15 @@ public class Dispatcher implements ContainerDispatcher {
           msg.getTraceID());
       return ContainerUtils.malformedRequest(msg);
     }
-    Pipeline pipeline =
-        Pipeline.getFromProtoBuf(msg.getDeleteKey().getPipeline());
-    Preconditions.checkNotNull(pipeline);
-    if (!this.containerManager.isOpen(pipeline.getContainerName())) {
+    BlockID blockID = BlockID.getFromProtobuf(msg.getDeleteKey()
+        .getBlockID());
+    Preconditions.checkNotNull(blockID);
+    long containerID = blockID.getContainerID();
+    if (!this.containerManager.isOpen(containerID)) {
       throw new StorageContainerException("Write to closed container.",
           CLOSED_CONTAINER_IO);
     }
-    String keyName = msg.getDeleteKey().getName();
-    Preconditions.checkNotNull(keyName);
-    Preconditions.checkState(!keyName.isEmpty());
-    this.containerManager.getKeyManager().deleteKey(pipeline, keyName);
+    this.containerManager.getKeyManager().deleteKey(blockID);
     return KeyUtils.getKeyResponse(msg);
   }
 
@@ -632,12 +618,11 @@ public class Dispatcher implements ContainerDispatcher {
     }
     try {
 
-      Pipeline pipeline =
-          Pipeline.getFromProtoBuf(msg.getPutSmallFile()
-              .getKey().getPipeline());
+      BlockID blockID = BlockID.getFromProtobuf(msg.
+          getPutSmallFile().getKey().getKeyData().getBlockID());
+      long containerID = blockID.getContainerID();
 
-      Preconditions.checkNotNull(pipeline);
-      if (!this.containerManager.isOpen(pipeline.getContainerName())) {
+      if (!this.containerManager.isOpen(containerID)) {
         throw new StorageContainerException("Write to closed container.",
             CLOSED_CONTAINER_IO);
       }
@@ -648,12 +633,12 @@ public class Dispatcher implements ContainerDispatcher {
       byte[] data = msg.getPutSmallFile().getData().toByteArray();
 
       metrics.incContainerBytesStats(Type.PutSmallFile, data.length);
-      this.containerManager.getChunkManager().writeChunk(pipeline, keyData
-          .getKeyName(), chunkInfo, data, ContainerProtos.Stage.COMBINED);
+      this.containerManager.getChunkManager().writeChunk(blockID,
+          chunkInfo, data, ContainerProtos.Stage.COMBINED);
       List<ContainerProtos.ChunkInfo> chunks = new LinkedList<>();
       chunks.add(chunkInfo.getProtoBufMessage());
       keyData.setChunks(chunks);
-      this.containerManager.getKeyManager().putKey(pipeline, keyData);
+      this.containerManager.getKeyManager().putKey(keyData);
       return FileUtils.getPutFileResponse(msg);
     } catch (StorageContainerException e) {
       return ContainerUtils.logAndReturnError(LOG, e, msg);
@@ -680,12 +665,7 @@ public class Dispatcher implements ContainerDispatcher {
       return ContainerUtils.malformedRequest(msg);
     }
     try {
-      Pipeline pipeline =
-          Pipeline.getFromProtoBuf(msg.getGetSmallFile()
-              .getKey().getPipeline());
-
       long bytes = 0;
-      Preconditions.checkNotNull(pipeline);
       KeyData keyData = KeyData.getFromProtoBuf(msg.getGetSmallFile()
           .getKey().getKeyData());
       KeyData data = this.containerManager.getKeyManager().getKey(keyData);
@@ -694,9 +674,8 @@ public class Dispatcher implements ContainerDispatcher {
         bytes += chunk.getSerializedSize();
         ByteString current =
             ByteString.copyFrom(this.containerManager.getChunkManager()
-                .readChunk(
-                    pipeline, keyData.getKeyName(), ChunkInfo.getFromProtoBuf(
-                        chunk)));
+                .readChunk(keyData.getBlockID(),
+                    ChunkInfo.getFromProtoBuf(chunk)));
         dataBuf = dataBuf.concat(current);
         c = chunk;
       }

+ 32 - 34
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyManagerImpl.java

@@ -19,20 +19,18 @@
 package org.apache.hadoop.ozone.container.common.impl;
 
 import com.google.common.base.Preconditions;
+import com.google.common.primitives.Longs;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
+import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
 import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
 import org.apache.hadoop.ozone.container.common.interfaces.KeyManager;
 import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
-import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
-import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
 import org.apache.hadoop.utils.MetadataStore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -73,22 +71,21 @@ public class KeyManagerImpl implements KeyManager {
    * {@inheritDoc}
    */
   @Override
-  public void putKey(Pipeline pipeline, KeyData data) throws IOException {
+  public void putKey(KeyData data) throws IOException {
+    Preconditions.checkNotNull(data, "KeyData cannot be null for put operation.");
+    Preconditions.checkState(data.getContainerID() >= 0, "Container ID cannot be negative");
     containerManager.readLock();
     try {
       // We are not locking the key manager since LevelDb serializes all actions
       // against a single DB. We rely on DB level locking to avoid conflicts.
-      Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
-      String containerName = pipeline.getContainerName();
-      Preconditions.checkNotNull(containerName,
-          "Container name cannot be null");
-      ContainerData cData = containerManager.readContainer(containerName);
+      ContainerData cData = containerManager.readContainer(
+          data.getContainerID());
       MetadataStore db = KeyUtils.getDB(cData, conf);
 
       // This is a post condition that acts as a hint to the user.
       // Should never fail.
       Preconditions.checkNotNull(db, "DB cannot be null here");
-      db.put(data.getKeyName().getBytes(KeyUtils.ENCODING), data
+      db.put(Longs.toByteArray(data.getLocalID()), data
           .getProtoBufMessage().toByteArray());
     } finally {
       containerManager.readUnlock();
@@ -103,17 +100,17 @@ public class KeyManagerImpl implements KeyManager {
     containerManager.readLock();
     try {
       Preconditions.checkNotNull(data, "Key data cannot be null");
-      Preconditions.checkNotNull(data.getContainerName(),
+      Preconditions.checkNotNull(data.getContainerID(),
           "Container name cannot be null");
       ContainerData cData = containerManager.readContainer(data
-          .getContainerName());
+          .getContainerID());
       MetadataStore db = KeyUtils.getDB(cData, conf);
 
       // This is a post condition that acts as a hint to the user.
       // Should never fail.
       Preconditions.checkNotNull(db, "DB cannot be null here");
 
-      byte[] kData = db.get(data.getKeyName().getBytes(KeyUtils.ENCODING));
+      byte[] kData = db.get(Longs.toByteArray(data.getLocalID()));
       if (kData == null) {
         throw new StorageContainerException("Unable to find the key.",
             NO_SUCH_KEY);
@@ -130,15 +127,19 @@ public class KeyManagerImpl implements KeyManager {
    * {@inheritDoc}
    */
   @Override
-  public void deleteKey(Pipeline pipeline, String keyName)
+  public void deleteKey(BlockID blockID)
       throws IOException {
+    Preconditions.checkNotNull(blockID, "block ID cannot be null.");
+    Preconditions.checkState(blockID.getContainerID() >= 0,
+        "Container ID cannot be negative.");
+    Preconditions.checkState(blockID.getLocalID() >= 0,
+        "Local ID cannot be negative.");
+
     containerManager.readLock();
     try {
-      Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
-      String containerName = pipeline.getContainerName();
-      Preconditions.checkNotNull(containerName,
-          "Container name cannot be null");
-      ContainerData cData = containerManager.readContainer(containerName);
+
+      ContainerData cData = containerManager
+          .readContainer(blockID.getContainerID());
       MetadataStore db = KeyUtils.getDB(cData, conf);
 
       // This is a post condition that acts as a hint to the user.
@@ -149,12 +150,13 @@ public class KeyManagerImpl implements KeyManager {
       // to delete a key which might have just gotten inserted after
       // the get check.
 
-      byte[] kData = db.get(keyName.getBytes(KeyUtils.ENCODING));
+      byte[] kKey = Longs.toByteArray(blockID.getLocalID());
+      byte[] kData = db.get(kKey);
       if (kData == null) {
         throw new StorageContainerException("Unable to find the key.",
             NO_SUCH_KEY);
       }
-      db.delete(keyName.getBytes(KeyUtils.ENCODING));
+      db.delete(kKey);
     } finally {
       containerManager.readUnlock();
     }
@@ -165,26 +167,22 @@ public class KeyManagerImpl implements KeyManager {
    */
   @Override
   public List<KeyData> listKey(
-      Pipeline pipeline, String prefix, String startKey, int count)
+      long containerID, long startLocalID, int count)
       throws IOException {
-    Preconditions.checkNotNull(pipeline,
-        "Pipeline cannot be null.");
+    Preconditions.checkState(containerID >= 0, "Container ID cannot be negative");
+    Preconditions.checkState(startLocalID >= 0, "startLocal ID cannot be negative");
     Preconditions.checkArgument(count > 0,
         "Count must be a positive number.");
-    ContainerData cData = containerManager.readContainer(pipeline
-        .getContainerName());
+    ContainerData cData = containerManager.readContainer(containerID);
     MetadataStore db = KeyUtils.getDB(cData, conf);
 
-    List<KeyData> result = new ArrayList<KeyData>();
-    byte[] startKeyInBytes = startKey == null ? null :
-        DFSUtil.string2Bytes(startKey);
-    MetadataKeyFilter prefixFilter = new KeyPrefixFilter(prefix);
+    List<KeyData> result = new ArrayList<>();
+    byte[] startKeyInBytes = Longs.toByteArray(startLocalID);
     List<Map.Entry<byte[], byte[]>> range =
-        db.getSequentialRangeKVs(startKeyInBytes, count, prefixFilter);
+        db.getSequentialRangeKVs(startKeyInBytes, count, null);
     for (Map.Entry<byte[], byte[]> entry : range) {
-      String keyName = KeyUtils.getKeyName(entry.getKey());
       KeyData value = KeyUtils.getKeyData(entry.getValue());
-      KeyData data = new KeyData(value.getContainerName(), keyName);
+      KeyData data = new KeyData(value.getBlockID());
       result.add(data);
     }
     return result;

+ 2 - 2
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java

@@ -41,7 +41,7 @@ public class RandomContainerDeletionChoosingPolicy
 
   @Override
   public List<ContainerData> chooseContainerForBlockDeletion(int count,
-      Map<String, ContainerStatus> candidateContainers)
+      Map<Long, ContainerStatus> candidateContainers)
       throws StorageContainerException {
     Preconditions.checkNotNull(candidateContainers,
         "Internal assertion: candidate containers cannot be null");
@@ -58,7 +58,7 @@ public class RandomContainerDeletionChoosingPolicy
 
         LOG.debug("Select container {} for block deletion, "
             + "pending deletion blocks num: {}.",
-            entry.getContainer().getContainerName(),
+            entry.getContainer().getContainerID(),
             entry.getNumPendingDeletionBlocks());
       } else {
         break;

+ 2 - 2
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java

@@ -53,7 +53,7 @@ public class TopNOrderedContainerDeletionChoosingPolicy
 
   @Override
   public List<ContainerData> chooseContainerForBlockDeletion(int count,
-      Map<String, ContainerStatus> candidateContainers)
+      Map<Long, ContainerStatus> candidateContainers)
       throws StorageContainerException {
     Preconditions.checkNotNull(candidateContainers,
         "Internal assertion: candidate containers cannot be null");
@@ -74,7 +74,7 @@ public class TopNOrderedContainerDeletionChoosingPolicy
           LOG.debug(
               "Select container {} for block deletion, "
                   + "pending deletion blocks num: {}.",
-              entry.getContainer().getContainerName(),
+              entry.getContainer().getContainerID(),
               entry.getNumPendingDeletionBlocks());
         } else {
           LOG.debug("Stop looking for next container, there is no"

+ 8 - 11
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java

@@ -18,10 +18,10 @@
 
 package org.apache.hadoop.ozone.container.common.interfaces;
 
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
+import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 
 /**
@@ -32,20 +32,18 @@ public interface ChunkManager {
 
   /**
    * writes a given chunk.
-   * @param pipeline - Name and the set of machines that make this container.
-   * @param keyName - Name of the Key.
+   * @param blockID - ID of the block.
    * @param info - ChunkInfo.
    * @param stage - Chunk Stage write.
    * @throws StorageContainerException
    */
-  void writeChunk(Pipeline pipeline, String keyName,
-                  ChunkInfo info, byte[] data, ContainerProtos.Stage stage)
+  void writeChunk(BlockID blockID,
+      ChunkInfo info, byte[] data, ContainerProtos.Stage stage)
       throws StorageContainerException;
 
   /**
    * reads the data defined by a chunk.
-   * @param pipeline - container pipeline.
-   * @param keyName - Name of the Key
+   * @param blockID - ID of the block.
    * @param info - ChunkInfo.
    * @return  byte array
    * @throws StorageContainerException
@@ -53,17 +51,16 @@ public interface ChunkManager {
    * TODO: Right now we do not support partial reads and writes of chunks.
    * TODO: Explore if we need to do that for ozone.
    */
-  byte[] readChunk(Pipeline pipeline, String keyName, ChunkInfo info) throws
+  byte[] readChunk(BlockID blockID, ChunkInfo info) throws
       StorageContainerException;
 
   /**
    * Deletes a given chunk.
-   * @param pipeline  - Pipeline.
-   * @param keyName   - Key Name
+   * @param blockID - ID of the block.
    * @param info  - Chunk Info
    * @throws StorageContainerException
    */
-  void deleteChunk(Pipeline pipeline, String keyName, ChunkInfo info) throws
+  void deleteChunk(BlockID blockID, ChunkInfo info) throws
       StorageContainerException;
 
   // TODO : Support list operations.

+ 1 - 1
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java

@@ -41,6 +41,6 @@ public interface ContainerDeletionChoosingPolicy {
    * @throws StorageContainerException
    */
   List<ContainerData> chooseContainerForBlockDeletion(int count,
-      Map<String, ContainerStatus> candidateContainers)
+      Map<Long, ContainerStatus> candidateContainers)
       throws StorageContainerException;
 }

+ 35 - 41
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java

@@ -21,7 +21,6 @@ package org.apache.hadoop.ozone.container.common.interfaces;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
@@ -60,48 +59,43 @@ public interface ContainerManager extends RwLock {
   /**
    * Creates a container with the given name.
    *
-   * @param pipeline      -- Nodes which make up this container.
    * @param containerData - Container Name and metadata.
    * @throws StorageContainerException
    */
-  void createContainer(Pipeline pipeline, ContainerData containerData)
+  void createContainer(ContainerData containerData)
       throws StorageContainerException;
 
   /**
    * Deletes an existing container.
    *
-   * @param pipeline      - nodes that make this container.
-   * @param containerName - name of the container.
+   * @param containerID - ID of the container.
    * @param forceDelete   - whether this container should be deleted forcibly.
    * @throws StorageContainerException
    */
-  void deleteContainer(Pipeline pipeline, String containerName,
+  void deleteContainer(long containerID,
       boolean forceDelete) throws StorageContainerException;
 
   /**
    * Update an existing container.
    *
-   * @param pipeline container nodes
-   * @param containerName name of the container
+   * @param containerID ID of the container
    * @param data container data
    * @param forceUpdate if true, update container forcibly.
    * @throws StorageContainerException
    */
-  void updateContainer(Pipeline pipeline, String containerName,
-      ContainerData data, boolean forceUpdate) throws StorageContainerException;
+  void updateContainer(long containerID, ContainerData data,
+      boolean forceUpdate) throws StorageContainerException;
 
   /**
    * As simple interface for container Iterations.
    *
-   * @param prefix - Return only values matching this prefix
-   * @param count   - how many to return
-   * @param prevKey - Previous key - Server returns results from this point.
-   * @param data    - Actual containerData
+   * @param startContainerID -  Return containers with ID >= startContainerID.
+   * @param count - how many to return
+   * @param data - Actual containerData
    * @throws StorageContainerException
    */
-  void listContainer(String prefix, long count, String prevKey,
-                     List<ContainerData> data)
-      throws StorageContainerException;
+  void listContainer(long startContainerID, long count,
+      List<ContainerData> data) throws StorageContainerException;
 
   /**
    * Choose containers for block deletion.
@@ -115,30 +109,30 @@ public interface ContainerManager extends RwLock {
   /**
    * Get metadata about a specific container.
    *
-   * @param containerName - Name of the container
+   * @param containerID - ID of the container.
    * @return ContainerData - Container Data.
    * @throws StorageContainerException
    */
-  ContainerData readContainer(String containerName)
+  ContainerData readContainer(long containerID)
       throws StorageContainerException;
 
   /**
    * Closes a open container, if it is already closed or does not exist a
    * StorageContainerException is thrown.
-   * @param containerName - Name of the container.
+   * @param containerID - ID of the container.
    * @throws StorageContainerException
    */
-  void closeContainer(String containerName)
+  void closeContainer(long containerID)
       throws StorageContainerException, NoSuchAlgorithmException;
 
   /**
    * Checks if a container exists.
-   * @param containerName - Name of the container.
+   * @param containerID - ID of the container.
    * @return true if the container is open false otherwise.
    * @throws StorageContainerException  - Throws Exception if we are not
    * able to find the container.
    */
-  boolean isOpen(String containerName) throws StorageContainerException;
+  boolean isOpen(long containerID) throws StorageContainerException;
 
   /**
    * Supports clean shutdown of container.
@@ -203,7 +197,7 @@ public interface ContainerManager extends RwLock {
    * @param containerId
    *          container id
    */
-  void incrPendingDeletionBlocks(int numBlocks, String containerId);
+  void incrPendingDeletionBlocks(int numBlocks, long containerId);
 
   /**
    * Decrease pending deletion blocks count number of specified container.
@@ -213,64 +207,64 @@ public interface ContainerManager extends RwLock {
    * @param containerId
    *          container id
    */
-  void decrPendingDeletionBlocks(int numBlocks, String containerId);
+  void decrPendingDeletionBlocks(int numBlocks, long containerId);
 
   /**
    * Increase the read count of the container.
-   * @param containerName - Name of the container.
+   * @param containerId - ID of the container.
    */
-  void incrReadCount(String containerName);
+  void incrReadCount(long containerId);
 
   /**
    * Increse the read counter for bytes read from the container.
-   * @param containerName - Name of the container.
+   * @param containerId - ID of the container.
    * @param readBytes - bytes read from the container.
    */
-  void incrReadBytes(String containerName, long readBytes);
+  void incrReadBytes(long containerId, long readBytes);
 
 
   /**
    * Increase the write count of the container.
-   * @param containerName - Name of the container.
+   * @param containerId - ID of the container.
    */
-  void incrWriteCount(String containerName);
+  void incrWriteCount(long containerId);
 
   /**
    * Increase the write counter for bytes write into the container.
-   * @param containerName - Name of the container.
+   * @param containerId - ID of the container.
    * @param writeBytes - bytes write into the container.
    */
-  void incrWriteBytes(String containerName, long writeBytes);
+  void incrWriteBytes(long containerId, long writeBytes);
 
   /**
    * Increase the bytes used by the container.
-   * @param containerName - Name of the container.
+   * @param containerId - ID of the container.
    * @param used - additional bytes used by the container.
    * @return the current bytes used.
    */
-  long incrBytesUsed(String containerName, long used);
+  long incrBytesUsed(long containerId, long used);
 
   /**
    * Decrease the bytes used by the container.
-   * @param containerName - Name of the container.
+   * @param containerId - ID of the container.
    * @param used - additional bytes reclaimed by the container.
    * @return the current bytes used.
    */
-  long decrBytesUsed(String containerName, long used);
+  long decrBytesUsed(long containerId, long used);
 
   /**
    * Get the bytes used by the container.
-   * @param containerName - Name of the container.
+   * @param containerId - ID of the container.
    * @return the current bytes used by the container.
    */
-  long getBytesUsed(String containerName);
+  long getBytesUsed(long containerId);
 
   /**
    * Get the number of keys in the container.
-   * @param containerName - Name of the container.
+   * @param containerId - ID of the container.
    * @return the current key count.
    */
-  long getNumKeys(String containerName);
+  long getNumKeys(long containerId);
 
   /**
    * Get the container report state to send via HB to SCM.

+ 7 - 10
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/KeyManager.java

@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.ozone.container.common.interfaces;
 
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
+import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
 
 import java.io.IOException;
@@ -32,11 +32,10 @@ public interface KeyManager {
   /**
    * Puts or overwrites a key.
    *
-   * @param pipeline - Pipeline.
    * @param data     - Key Data.
    * @throws IOException
    */
-  void putKey(Pipeline pipeline, KeyData data) throws IOException;
+  void putKey(KeyData data) throws IOException;
 
   /**
    * Gets an existing key.
@@ -50,23 +49,21 @@ public interface KeyManager {
   /**
    * Deletes an existing Key.
    *
-   * @param pipeline - Pipeline.
-   * @param keyName  Key Data.
+   * @param blockID - ID of the block.
    * @throws StorageContainerException
    */
-  void deleteKey(Pipeline pipeline, String keyName)
+  void deleteKey(BlockID blockID)
       throws IOException;
 
   /**
    * List keys in a container.
    *
-   * @param pipeline - pipeline.
-   * @param prefix   - Prefix in needed.
-   * @param startKey  - Key to start from, EMPTY_STRING to begin.
+   * @param containerID - ID of the container.
+   * @param startLocalID  - Key to start from, 0 to begin.
    * @param count    - Number of keys to return.
    * @return List of Keys that match the criteria.
    */
-  List<KeyData> listKey(Pipeline pipeline, String prefix, String startKey,
+  List<KeyData> listKey(long containerID, long startLocalID,
       int count) throws IOException;
 
   /**

+ 4 - 4
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java

@@ -180,12 +180,12 @@ public class BlockDeletingService extends BackgroundService{
           meta.getSequentialRangeKVs(null, blockLimitPerTask, filter);
       if (toDeleteBlocks.isEmpty()) {
         LOG.debug("No under deletion block found in container : {}",
-            containerData.getContainerName());
+            containerData.getContainerID());
       }
 
       List<String> succeedBlocks = new LinkedList<>();
       LOG.debug("Container : {}, To-Delete blocks : {}",
-          containerData.getContainerName(), toDeleteBlocks.size());
+          containerData.getContainerID(), toDeleteBlocks.size());
       File dataDir = ContainerUtils.getDataDirectory(containerData).toFile();
       if (!dataDir.exists() || !dataDir.isDirectory()) {
         LOG.error("Invalid container data dir {} : "
@@ -220,11 +220,11 @@ public class BlockDeletingService extends BackgroundService{
       meta.writeBatch(batch);
       // update count of pending deletion blocks in in-memory container status
       containerManager.decrPendingDeletionBlocks(succeedBlocks.size(),
-          containerData.getContainerName());
+          containerData.getContainerID());
 
       if (!succeedBlocks.isEmpty()) {
         LOG.info("Container: {}, deleted blocks: {}, task elapsed time: {}ms",
-            containerData.getContainerName(), succeedBlocks.size(),
+            containerData.getContainerID(), succeedBlocks.size(),
             Time.monotonicNow() - startTime);
       }
       crr.addAll(succeedBlocks);

+ 5 - 4
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerHandler.java

@@ -58,19 +58,20 @@ public class CloseContainerHandler implements CommandHandler {
     LOG.debug("Processing Close Container command.");
     invocationCount++;
     long startTime = Time.monotonicNow();
-    String containerName = "UNKNOWN";
+    // TODO: define this as INVALID_CONTAINER_ID in HddsConsts.java (TBA)
+    long containerID = -1;
     try {
 
       SCMCloseContainerCmdResponseProto
           closeContainerProto =
           SCMCloseContainerCmdResponseProto
               .parseFrom(command.getProtoBufMessage());
-      containerName = closeContainerProto.getContainerName();
+      containerID = closeContainerProto.getContainerID();
 
-      container.getContainerManager().closeContainer(containerName);
+      container.getContainerManager().closeContainer(containerID);
 
     } catch (Exception e) {
-      LOG.error("Can't close container " + containerName, e);
+      LOG.error("Can't close container " + containerID, e);
     } finally {
       long endTime = Time.monotonicNow();
       totalTime += endTime - startTime;

+ 5 - 4
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java

@@ -16,6 +16,7 @@
  */
 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.hdfs.DFSUtil;
 import org.apache.hadoop.hdds.protocol.proto
@@ -108,7 +109,7 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
         txResultBuilder.setSuccess(true);
       } catch (IOException e) {
         LOG.warn("Failed to delete blocks for container={}, TXID={}",
-            entry.getContainerName(), entry.getTxID(), e);
+            entry.getContainerID(), entry.getTxID(), e);
         txResultBuilder.setSuccess(false);
       }
       resultBuilder.addResults(txResultBuilder.build());
@@ -150,7 +151,7 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
    */
   private void deleteContainerBlocks(DeletedBlocksTransaction delTX,
       Configuration config) throws IOException {
-    String containerId = delTX.getContainerName();
+    long containerId = delTX.getContainerID();
     ContainerData containerInfo = containerManager.readContainer(containerId);
     if (LOG.isDebugEnabled()) {
       LOG.debug("Processing Container : {}, DB path : {}", containerId,
@@ -159,9 +160,9 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
 
     int newDeletionBlocks = 0;
     MetadataStore containerDB = KeyUtils.getDB(containerInfo, config);
-    for (String blk : delTX.getBlockIDList()) {
+    for (Long blk : delTX.getLocalIDList()) {
       BatchOperation batch = new BatchOperation();
-      byte[] blkBytes = DFSUtil.string2Bytes(blk);
+      byte[] blkBytes = Longs.toByteArray(blk);
       byte[] blkInfo = containerDB.get(blkBytes);
       if (blkInfo != null) {
         // Found the block in container db,

+ 1 - 1
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java

@@ -170,7 +170,7 @@ public class HeartbeatEndpointTask
                 commandResponseProto.getCloseContainerProto());
         if (LOG.isDebugEnabled()) {
           LOG.debug("Received SCM container close request for container {}",
-              closeContainer.getContainerName());
+              closeContainer.getContainerID());
         }
         this.context.addCommand(closeContainer);
         break;

+ 10 - 11
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java

@@ -94,7 +94,7 @@ public class ContainerStateMachine extends BaseStateMachine {
   private ThreadPoolExecutor writeChunkExecutor;
   private final ConcurrentHashMap<Long, CompletableFuture<Message>>
       writeChunkFutureMap;
-  private final ConcurrentHashMap<String, CompletableFuture<Message>>
+  private final ConcurrentHashMap<Long, CompletableFuture<Message>>
       createContainerFutureMap;
 
   ContainerStateMachine(ContainerDispatcher dispatcher,
@@ -146,8 +146,7 @@ public class ContainerStateMachine extends BaseStateMachine {
       // create the log entry proto
       final WriteChunkRequestProto commitWriteChunkProto =
           WriteChunkRequestProto.newBuilder()
-              .setPipeline(write.getPipeline())
-              .setKeyName(write.getKeyName())
+              .setBlockID(write.getBlockID())
               .setChunkData(write.getChunkData())
               // skipping the data field as it is
               // already set in statemachine data proto
@@ -196,9 +195,9 @@ public class ContainerStateMachine extends BaseStateMachine {
   private CompletableFuture<Message> handleWriteChunk(
       ContainerCommandRequestProto requestProto, long entryIndex) {
     final WriteChunkRequestProto write = requestProto.getWriteChunk();
-    String containerName = write.getPipeline().getContainerName();
+    long containerID = write.getBlockID().getContainerID();
     CompletableFuture<Message> future =
-        createContainerFutureMap.get(containerName);
+        createContainerFutureMap.get(containerID);
     CompletableFuture<Message> writeChunkFuture;
     if (future != null) {
       writeChunkFuture = future.thenApplyAsync(
@@ -213,10 +212,10 @@ public class ContainerStateMachine extends BaseStateMachine {
 
   private CompletableFuture<Message> handleCreateContainer(
       ContainerCommandRequestProto requestProto) {
-    String containerName =
-        requestProto.getCreateContainer().getContainerData().getName();
+    long containerID =
+        requestProto.getCreateContainer().getContainerData().getContainerID();
     createContainerFutureMap.
-        computeIfAbsent(containerName, k -> new CompletableFuture<>());
+        computeIfAbsent(containerID, k -> new CompletableFuture<>());
     return CompletableFuture.completedFuture(() -> ByteString.EMPTY);
   }
 
@@ -270,9 +269,9 @@ public class ContainerStateMachine extends BaseStateMachine {
       } else {
         Message message = runCommand(requestProto);
         if (cmdType == ContainerProtos.Type.CreateContainer) {
-          String containerName =
-              requestProto.getCreateContainer().getContainerData().getName();
-          createContainerFutureMap.remove(containerName).complete(message);
+          long containerID =
+              requestProto.getCreateContainer().getContainerData().getContainerID();
+          createContainerFutureMap.remove(containerID).complete(message);
         }
         return CompletableFuture.completedFuture(message);
       }

+ 17 - 19
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java

@@ -69,15 +69,15 @@ public final class ContainerCache extends LRUMap {
   /**
    * Closes a db instance.
    *
-   * @param container - name of the container to be closed.
+   * @param containerID - ID of the container to be closed.
    * @param db - db instance to close.
    */
-  private void closeDB(String container, MetadataStore db) {
+  private void closeDB(long containerID, MetadataStore db) {
     if (db != null) {
       try {
         db.close();
       } catch (IOException e) {
-        LOG.error("Error closing DB. Container: " + container, e);
+        LOG.error("Error closing DB. Container: " + containerID, e);
       }
     }
   }
@@ -93,7 +93,7 @@ public final class ContainerCache extends LRUMap {
       while (iterator.hasNext()) {
         iterator.next();
         MetadataStore db = (MetadataStore) iterator.getValue();
-        closeDB(iterator.getKey().toString(), db);
+        closeDB(((Number)iterator.getKey()).longValue(), db);
       }
       // reset the cache
       cache.clear();
@@ -110,7 +110,7 @@ public final class ContainerCache extends LRUMap {
     lock.lock();
     try {
       MetadataStore db = (MetadataStore) entry.getValue();
-      closeDB(entry.getKey().toString(), db);
+      closeDB(((Number)entry.getKey()).longValue(), db);
     } finally {
       lock.unlock();
     }
@@ -120,28 +120,27 @@ public final class ContainerCache extends LRUMap {
   /**
    * Returns a DB handle if available, create the handler otherwise.
    *
-   * @param containerName - Name of the container.
+   * @param containerID - ID of the container.
    * @return MetadataStore.
    */
-  public MetadataStore getDB(String containerName, String containerDBPath)
+  public MetadataStore getDB(long containerID, String containerDBPath)
       throws IOException {
-    Preconditions.checkNotNull(containerName);
-    Preconditions.checkState(!containerName.isEmpty());
+    Preconditions.checkState(containerID >= 0, "Container ID cannot be negative.");
     lock.lock();
     try {
-      MetadataStore db = (MetadataStore) this.get(containerName);
+      MetadataStore db = (MetadataStore) this.get(containerID);
 
       if (db == null) {
         db = MetadataStoreBuilder.newBuilder()
             .setDbFile(new File(containerDBPath))
             .setCreateIfMissing(false)
             .build();
-        this.put(containerName, db);
+        this.put(containerID, db);
       }
       return db;
     } catch (Exception e) {
       LOG.error("Error opening DB. Container:{} ContainerPath:{}",
-          containerName, containerDBPath, e);
+          containerID, containerDBPath, e);
       throw e;
     } finally {
       lock.unlock();
@@ -151,16 +150,15 @@ public final class ContainerCache extends LRUMap {
   /**
    * Remove a DB handler from cache.
    *
-   * @param containerName - Name of the container.
+   * @param containerID - ID of the container.
    */
-  public void removeDB(String containerName) {
-    Preconditions.checkNotNull(containerName);
-    Preconditions.checkState(!containerName.isEmpty());
+  public void removeDB(long containerID) {
+    Preconditions.checkState(containerID >= 0, "Container ID cannot be negative.");
     lock.lock();
     try {
-      MetadataStore db = (MetadataStore)this.get(containerName);
-      closeDB(containerName, db);
-      this.remove(containerName);
+      MetadataStore db = (MetadataStore)this.get(containerID);
+      closeDB(containerID, db);
+      this.remove(containerID);
     } finally {
       lock.unlock();
     }

+ 7 - 7
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CloseContainerCommand.java

@@ -32,10 +32,10 @@ import static org.apache.hadoop.hdds.protocol.proto
 public class CloseContainerCommand
     extends SCMCommand<SCMCloseContainerCmdResponseProto> {
 
-  private String containerName;
+  private long containerID;
 
-  public CloseContainerCommand(String containerName) {
-    this.containerName = containerName;
+  public CloseContainerCommand(long containerID) {
+    this.containerID = containerID;
   }
 
   /**
@@ -60,17 +60,17 @@ public class CloseContainerCommand
 
   public SCMCloseContainerCmdResponseProto getProto() {
     return SCMCloseContainerCmdResponseProto.newBuilder()
-        .setContainerName(containerName).build();
+        .setContainerID(containerID).build();
   }
 
   public static CloseContainerCommand getFromProtobuf(
       SCMCloseContainerCmdResponseProto closeContainerProto) {
     Preconditions.checkNotNull(closeContainerProto);
-    return new CloseContainerCommand(closeContainerProto.getContainerName());
+    return new CloseContainerCommand(closeContainerProto.getContainerID());
 
   }
 
-  public String getContainerName() {
-    return containerName;
+  public long getContainerID() {
+    return containerID;
   }
 }

+ 5 - 4
hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto

@@ -87,7 +87,6 @@ message NodeContianerMapping {
 A container report contains the following information.
 */
 message ContainerInfo {
-  required string containerName = 1;
   optional string finalhash = 2;
   optional int64 size = 3;
   optional int64 used = 4;
@@ -102,10 +101,12 @@ message ContainerInfo {
 }
 
 // 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 string containerName = 2;
-  repeated string blockID = 3;
+  required int64 containerID = 2;
+  repeated int64 localID = 3;
   // the retry time of sending deleting command to datanode.
   required int32 count = 4;
 }
@@ -201,7 +202,7 @@ message SendContainerReportProto {
 This command asks the datanode to close a specific container.
 */
 message SCMCloseContainerCmdResponseProto {
-  required string containerName = 1;
+  required int64 containerID = 1;
 }
 
 /**

+ 1 - 1
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java

@@ -239,7 +239,7 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
 
       for (StorageContainerDatanodeProtocolProtos.ContainerInfo report:
           reports.getReportsList()) {
-        containers.put(report.getContainerName(), report);
+        containers.put(report.getContainerID(), report);
       }
     }
 

+ 2 - 10
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java

@@ -18,8 +18,8 @@
 package org.apache.hadoop.hdds.scm.block;
 
 import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.client.BlockID;
 
 import java.io.Closeable;
 import java.io.IOException;
@@ -42,14 +42,6 @@ public interface BlockManager extends Closeable {
   AllocatedBlock allocateBlock(long size, HddsProtos.ReplicationType type,
       HddsProtos.ReplicationFactor factor, String owner) throws IOException;
 
-  /**
-   *  Give the key to the block, get the pipeline info.
-   * @param key - key to the block.
-   * @return - Pipeline that used to access the block.
-   * @throws IOException
-   */
-  Pipeline getBlock(String key) throws IOException;
-
   /**
    * Deletes a list of blocks in an atomic operation. Internally, SCM
    * writes these blocks into a {@link DeletedBlockLog} and deletes them
@@ -60,7 +52,7 @@ public interface BlockManager extends Closeable {
    *                 a particular object key.
    * @throws IOException if exception happens, non of the blocks is deleted.
    */
-  void deleteBlocks(List<String> blockIDs) throws IOException;
+  void deleteBlocks(List<BlockID> blockIDs) throws IOException;
 
   /**
    * @return the block deletion transaction log maintained by SCM.

+ 32 - 139
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java

@@ -16,30 +16,25 @@
  */
 package org.apache.hadoop.hdds.scm.block;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.Mapping;
 import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.utils.BatchOperation;
-import org.apache.hadoop.utils.MetadataStore;
-import org.apache.hadoop.utils.MetadataStoreBuilder;
+import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.management.ObjectName;
-import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -53,11 +48,8 @@ import java.util.concurrent.locks.ReentrantLock;
 
 import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
     .CHILL_MODE_EXCEPTION;
-import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
-    .FAILED_TO_FIND_BLOCK;
 import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
     .INVALID_BLOCK_SIZE;
-import static org.apache.hadoop.hdds.server.ServerUtils.getOzoneMetaDirPath;
 import static org.apache.hadoop.ozone.OzoneConfigKeys
     .OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
 import static org.apache.hadoop.ozone.OzoneConfigKeys
@@ -66,7 +58,6 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys
     .OZONE_BLOCK_DELETING_SERVICE_TIMEOUT;
 import static org.apache.hadoop.ozone.OzoneConfigKeys
     .OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT;
-import static org.apache.hadoop.ozone.OzoneConsts.BLOCK_DB;
 
 /** Block Manager manages the block access for SCM. */
 public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
@@ -78,11 +69,9 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
 
   private final NodeManager nodeManager;
   private final Mapping containerManager;
-  private final MetadataStore blockStore;
 
   private final Lock lock;
   private final long containerSize;
-  private final long cacheSize;
 
   private final DeletedBlockLog deletedBlockLog;
   private final SCMBlockDeletingService blockDeletingService;
@@ -97,30 +86,17 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
    * @param conf - configuration.
    * @param nodeManager - node manager.
    * @param containerManager - container manager.
-   * @param cacheSizeMB - cache size for level db store.
    * @throws IOException
    */
   public BlockManagerImpl(final Configuration conf,
-      final NodeManager nodeManager, final Mapping containerManager,
-      final int cacheSizeMB) throws IOException {
+      final NodeManager nodeManager, final Mapping containerManager)
+      throws IOException {
     this.nodeManager = nodeManager;
     this.containerManager = containerManager;
-    this.cacheSize = cacheSizeMB;
 
     this.containerSize = OzoneConsts.GB * conf.getInt(
         ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
         ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
-    File metaDir = getOzoneMetaDirPath(conf);
-    String scmMetaDataDir = metaDir.getPath();
-
-    // Write the block key to container name mapping.
-    File blockContainerDbPath = new File(scmMetaDataDir, BLOCK_DB);
-    blockStore =
-        MetadataStoreBuilder.newBuilder()
-            .setConf(conf)
-            .setDbFile(blockContainerDbPath)
-            .setCacheSize(this.cacheSize * OzoneConsts.MB)
-            .build();
 
     this.containerProvisionBatchSize =
         conf.getInt(
@@ -181,12 +157,11 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
     lock.lock();
     try {
       for (int i = 0; i < count; i++) {
-        String containerName = UUID.randomUUID().toString();
         ContainerInfo containerInfo = null;
         try {
           // TODO: Fix this later when Ratis is made the Default.
           containerInfo = containerManager.allocateContainer(type, factor,
-              containerName, owner);
+              owner);
 
           if (containerInfo == null) {
             LOG.warn("Unable to allocate container.");
@@ -267,7 +242,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
                   size, owner, type, factor, HddsProtos.LifeCycleState
                       .ALLOCATED);
       if (containerInfo != null) {
-        containerManager.updateContainerState(containerInfo.getContainerName(),
+        containerManager.updateContainerState(containerInfo.getContainerID(),
             HddsProtos.LifeCycleEvent.CREATE);
         return newBlock(containerInfo, HddsProtos.LifeCycleState.ALLOCATED);
       }
@@ -297,7 +272,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
                   size, owner, type, factor, HddsProtos.LifeCycleState
                       .ALLOCATED);
       if (containerInfo != null) {
-        containerManager.updateContainerState(containerInfo.getContainerName(),
+        containerManager.updateContainerState(containerInfo.getContainerID(),
             HddsProtos.LifeCycleEvent.CREATE);
         return newBlock(containerInfo, HddsProtos.LifeCycleState.ALLOCATED);
       }
@@ -327,68 +302,27 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
       ContainerInfo containerInfo, HddsProtos.LifeCycleState state)
       throws IOException {
 
-    // TODO : Replace this with Block ID.
-    String blockKey = UUID.randomUUID().toString();
-    boolean createContainer = (state == HddsProtos.LifeCycleState.ALLOCATED);
-
-    AllocatedBlock.Builder abb =
-        new AllocatedBlock.Builder()
-            .setKey(blockKey)
-            // TODO : Use containerinfo instead of pipeline.
-            .setPipeline(containerInfo.getPipeline())
-            .setShouldCreateContainer(createContainer);
-    LOG.trace("New block allocated : {} Container ID: {}", blockKey,
-        containerInfo.toString());
-
     if (containerInfo.getPipeline().getMachines().size() == 0) {
       LOG.error("Pipeline Machine count is zero.");
       return null;
     }
 
-    // Persist this block info to the blockStore DB, so getBlock(key) can
-    // find which container the block lives.
-    // TODO : Remove this DB in future
-    // and make this a KSM operation. Category: SCALABILITY.
-    if (containerInfo.getPipeline().getMachines().size() > 0) {
-      blockStore.put(
-          DFSUtil.string2Bytes(blockKey),
-          DFSUtil.string2Bytes(containerInfo.getPipeline().getContainerName()));
-    }
-    return abb.build();
-  }
+    // TODO : Revisit this local ID allocation when HA is added.
+    // TODO: this does not work well if multiple allocation kicks in a tight
+    // loop.
+    long localID = Time.getUtcTime();
+    long containerID = containerInfo.getContainerID();
 
-  /**
-   * Given a block key, return the Pipeline information.
-   *
-   * @param key - block key assigned by SCM.
-   * @return Pipeline (list of DNs and leader) to access the block.
-   * @throws IOException
-   */
-  @Override
-  public Pipeline getBlock(final String key) throws IOException {
-    lock.lock();
-    try {
-      byte[] containerBytes = blockStore.get(DFSUtil.string2Bytes(key));
-      if (containerBytes == null) {
-        throw new SCMException(
-            "Specified block key does not exist. key : " + key,
-            FAILED_TO_FIND_BLOCK);
-      }
+    boolean createContainer = (state == HddsProtos.LifeCycleState.ALLOCATED);
 
-      String containerName = DFSUtil.bytes2String(containerBytes);
-      ContainerInfo containerInfo = containerManager.getContainer(
-          containerName);
-      if (containerInfo == null) {
-        LOG.debug("Container {} allocated by block service"
-            + "can't be found in SCM", containerName);
-        throw new SCMException(
-            "Unable to find container for the block",
-            SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
-      }
-      return containerInfo.getPipeline();
-    } finally {
-      lock.unlock();
-    }
+    AllocatedBlock.Builder abb =
+        new AllocatedBlock.Builder()
+            .setBlockID(new BlockID(containerID, localID))
+            .setPipeline(containerInfo.getPipeline())
+            .setShouldCreateContainer(createContainer);
+    LOG.trace("New block allocated : {} Container ID: {}", localID,
+        containerID);
+    return abb.build();
   }
 
   /**
@@ -403,40 +337,28 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
    * @throws IOException if exception happens, non of the blocks is deleted.
    */
   @Override
-  public void deleteBlocks(List<String> blockIDs) throws IOException {
+  public void deleteBlocks(List<BlockID> blockIDs) throws IOException {
     if (!nodeManager.isOutOfChillMode()) {
       throw new SCMException("Unable to delete block while in chill mode",
           CHILL_MODE_EXCEPTION);
     }
 
     lock.lock();
-    LOG.info("Deleting blocks {}", String.join(",", blockIDs));
-    Map<String, List<String>> containerBlocks = new HashMap<>();
-    BatchOperation batch = new BatchOperation();
-    BatchOperation rollbackBatch = new BatchOperation();
+    LOG.info("Deleting blocks {}", StringUtils.join(",", blockIDs));
+    Map<Long, List<Long>> containerBlocks = new HashMap<>();
     // TODO: track the block size info so that we can reclaim the container
     // TODO: used space when the block is deleted.
     try {
-      for (String blockKey : blockIDs) {
-        byte[] blockKeyBytes = DFSUtil.string2Bytes(blockKey);
-        byte[] containerBytes = blockStore.get(blockKeyBytes);
-        if (containerBytes == null) {
-          throw new SCMException(
-              "Specified block key does not exist. key : " + blockKey,
-              FAILED_TO_FIND_BLOCK);
-        }
-        batch.delete(blockKeyBytes);
-        rollbackBatch.put(blockKeyBytes, containerBytes);
-
+      for (BlockID block : blockIDs) {
         // Merge blocks to a container to blocks mapping,
         // prepare to persist this info to the deletedBlocksLog.
-        String containerName = DFSUtil.bytes2String(containerBytes);
-        if (containerBlocks.containsKey(containerName)) {
-          containerBlocks.get(containerName).add(blockKey);
+        long containerID = block.getContainerID();
+        if (containerBlocks.containsKey(containerID)) {
+          containerBlocks.get(containerID).add(block.getLocalID());
         } else {
-          List<String> item = new ArrayList<>();
-          item.add(blockKey);
-          containerBlocks.put(containerName, item);
+          List<Long> item = new ArrayList<>();
+          item.add(block.getLocalID());
+          containerBlocks.put(containerID, item);
         }
       }
 
@@ -445,34 +367,13 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
       // removed. If we write the log first, once log is written, the
       // async deleting service will start to scan and might be picking
       // up some blocks to do real deletions, that might cause data loss.
-      blockStore.writeBatch(batch);
       try {
         deletedBlockLog.addTransactions(containerBlocks);
       } catch (IOException e) {
-        try {
-          // If delLog update is failed, we need to rollback the changes.
-          blockStore.writeBatch(rollbackBatch);
-        } catch (IOException rollbackException) {
-          // This is a corner case. AddTX fails and rollback also fails,
-          // this will leave these blocks in inconsistent state. They were
-          // moved to pending deletion state in SCM DB but were not written
-          // into delLog so real deletions would not be done. Blocks become
-          // to be invisible from namespace but actual data are not removed.
-          // We log an error here so admin can manually check and fix such
-          // errors.
-          LOG.error(
-              "Blocks might be in inconsistent state because"
-                  + " they were moved to pending deletion state in SCM DB but"
-                  + " not written into delLog. Admin can manually add them"
-                  + " into delLog for deletions. Inconsistent block list: {}",
-              String.join(",", blockIDs),
-              e);
-          throw rollbackException;
-        }
         throw new IOException(
             "Skip writing the deleted blocks info to"
                 + " the delLog because addTransaction fails. Batch skipped: "
-                + String.join(",", blockIDs),
+                + StringUtils.join(",", blockIDs),
             e);
       }
       // TODO: Container report handling of the deleted blocks:
@@ -488,11 +389,6 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
     return this.deletedBlockLog;
   }
 
-  @VisibleForTesting
-  public String getDeletedKeyName(String key) {
-    return StringUtils.format(".Deleted/%s", key);
-  }
-
   /**
    * Close the resources for BlockManager.
    *
@@ -500,9 +396,6 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
    */
   @Override
   public void close() throws IOException {
-    if (blockStore != null) {
-      blockStore.close();
-    }
     if (deletedBlockLog != null) {
       deletedBlockLog.close();
     }

+ 3 - 3
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DatanodeDeletedBlockTransactions.java

@@ -56,7 +56,7 @@ public class DatanodeDeletedBlockTransactions {
   public void addTransaction(DeletedBlocksTransaction tx) throws IOException {
     ContainerInfo info = null;
     try {
-      info = mappingService.getContainer(tx.getContainerName());
+      info = mappingService.getContainer(tx.getContainerID());
     } catch (IOException e) {
       SCMBlockDeletingService.LOG.warn("Got container info error.", e);
     }
@@ -64,7 +64,7 @@ public class DatanodeDeletedBlockTransactions {
     if (info == null) {
       SCMBlockDeletingService.LOG.warn(
           "Container {} not found, continue to process next",
-          tx.getContainerName());
+          tx.getContainerID());
       return;
     }
 
@@ -75,7 +75,7 @@ public class DatanodeDeletedBlockTransactions {
         if (txs != null && txs.size() < maximumAllowedTXNum) {
           boolean hasContained = false;
           for (DeletedBlocksTransaction t : txs) {
-            if (t.getContainerName().equals(tx.getContainerName())) {
+            if (t.getContainerID() == tx.getContainerID()) {
               hasContained = true;
               break;
             }

+ 3 - 3
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLog.java

@@ -89,12 +89,12 @@ public interface DeletedBlockLog extends Closeable {
   /**
    * Creates a block deletion transaction and adds that into the log.
    *
-   * @param containerName - container name.
+   * @param containerID - container ID.
    * @param blocks - blocks that belong to the same container.
    *
    * @throws IOException
    */
-  void addTransaction(String containerName, List<String> blocks)
+  void addTransaction(long containerID, List<Long> blocks)
       throws IOException;
 
   /**
@@ -110,7 +110,7 @@ public interface DeletedBlockLog extends Closeable {
    * @param containerBlocksMap a map of containerBlocks.
    * @throws IOException
    */
-  void addTransactions(Map<String, List<String>> containerBlocksMap)
+  void addTransactions(Map<Long, List<Long>> containerBlocksMap)
       throws IOException;
 
   /**

+ 15 - 9
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java

@@ -190,8 +190,14 @@ public class DeletedBlockLogImpl implements DeletedBlockLog {
     try {
       for(Long txID : txIDs) {
         try {
+          byte [] deleteBlockBytes =
+              deletedStore.get(Longs.toByteArray(txID));
+          if (deleteBlockBytes == null) {
+            LOG.warn("Delete txID {} not found", txID);
+            continue;
+          }
           DeletedBlocksTransaction block = DeletedBlocksTransaction
-              .parseFrom(deletedStore.get(Longs.toByteArray(txID)));
+              .parseFrom(deleteBlockBytes);
           DeletedBlocksTransaction.Builder builder = block.toBuilder();
           int currentCount = block.getCount();
           if (currentCount > -1) {
@@ -216,11 +222,11 @@ public class DeletedBlockLogImpl implements DeletedBlockLog {
   }
 
   private DeletedBlocksTransaction constructNewTransaction(long txID,
-      String containerName, List<String> blocks) {
+      long containerID, List<Long> blocks) {
     return DeletedBlocksTransaction.newBuilder()
         .setTxID(txID)
-        .setContainerName(containerName)
-        .addAllBlockID(blocks)
+        .setContainerID(containerID)
+        .addAllLocalID(blocks)
         .setCount(0)
         .build();
   }
@@ -250,18 +256,18 @@ public class DeletedBlockLogImpl implements DeletedBlockLog {
   /**
    * {@inheritDoc}
    *
-   * @param containerName - container name.
+   * @param containerID - container ID.
    * @param blocks - blocks that belong to the same container.
    * @throws IOException
    */
   @Override
-  public void addTransaction(String containerName, List<String> blocks)
+  public void addTransaction(long containerID, List<Long> blocks)
       throws IOException {
     BatchOperation batch = new BatchOperation();
     lock.lock();
     try {
       DeletedBlocksTransaction tx = constructNewTransaction(lastTxID + 1,
-          containerName, blocks);
+          containerID, blocks);
       byte[] key = Longs.toByteArray(lastTxID + 1);
 
       batch.put(key, tx.toByteArray());
@@ -303,13 +309,13 @@ public class DeletedBlockLogImpl implements DeletedBlockLog {
    * @throws IOException
    */
   @Override
-  public void addTransactions(Map<String, List<String>> containerBlocksMap)
+  public void addTransactions(Map<Long, List<Long>> containerBlocksMap)
       throws IOException {
     BatchOperation batch = new BatchOperation();
     lock.lock();
     try {
       long currentLatestID = lastTxID;
-      for (Map.Entry<String, List<String>> entry :
+      for (Map.Entry<Long, List<Long>> entry :
           containerBlocksMap.entrySet()) {
         currentLatestID += 1;
         byte[] key = Longs.toByteArray(currentLatestID);

+ 32 - 48
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java

@@ -18,6 +18,7 @@ package org.apache.hadoop.hdds.scm.container;
 
 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.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.closer.ContainerCloser;
@@ -26,7 +27,6 @@ import org.apache.hadoop.hdds.scm.container.replication.ContainerSupervisor;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
@@ -38,8 +38,6 @@ import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.lease.Lease;
 import org.apache.hadoop.ozone.lease.LeaseException;
 import org.apache.hadoop.ozone.lease.LeaseManager;
-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.slf4j.Logger;
@@ -149,16 +147,15 @@ public class ContainerMapping implements Mapping {
    * {@inheritDoc}
    */
   @Override
-  public ContainerInfo getContainer(final String containerName) throws
+  public ContainerInfo getContainer(final long containerID) throws
       IOException {
     ContainerInfo containerInfo;
     lock.lock();
     try {
-      byte[] containerBytes = containerStore.get(containerName.getBytes(
-          encoding));
+      byte[] containerBytes = containerStore.get(Longs.toByteArray(containerID));
       if (containerBytes == null) {
         throw new SCMException(
-            "Specified key does not exist. key : " + containerName,
+            "Specified key does not exist. key : " + containerID,
             SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
       }
 
@@ -175,19 +172,18 @@ public class ContainerMapping implements Mapping {
    * {@inheritDoc}
    */
   @Override
-  public List<ContainerInfo> listContainer(String startName,
-      String prefixName, int count) throws IOException {
+  public List<ContainerInfo> listContainer(long startContainerID,
+      int count) throws IOException {
     List<ContainerInfo> containerList = new ArrayList<>();
     lock.lock();
     try {
       if (containerStore.isEmpty()) {
         throw new IOException("No container exists in current db");
       }
-      MetadataKeyFilter prefixFilter = new KeyPrefixFilter(prefixName);
-      byte[] startKey = startName == null ? null : DFSUtil.string2Bytes(
-          startName);
+      byte[] startKey = startContainerID <= 0 ? null :
+          Longs.toByteArray(startContainerID);
       List<Map.Entry<byte[], byte[]>> range =
-          containerStore.getSequentialRangeKVs(startKey, count, prefixFilter);
+          containerStore.getSequentialRangeKVs(startKey, count, null);
 
       // Transform the values into the pipelines.
       // TODO: filter by container state
@@ -209,7 +205,6 @@ public class ContainerMapping implements Mapping {
    * Allocates a new container.
    *
    * @param replicationFactor - replication factor of the container.
-   * @param containerName - Name of the container.
    * @param owner - The string name of the Service that owns this container.
    * @return - Pipeline that makes up this container.
    * @throws IOException - Exception
@@ -218,11 +213,8 @@ public class ContainerMapping implements Mapping {
   public ContainerInfo allocateContainer(
       ReplicationType type,
       ReplicationFactor replicationFactor,
-      final String containerName,
       String owner)
       throws IOException {
-    Preconditions.checkNotNull(containerName);
-    Preconditions.checkState(!containerName.isEmpty());
 
     ContainerInfo containerInfo;
     if (!nodeManager.isOutOfChillMode()) {
@@ -233,19 +225,12 @@ public class ContainerMapping implements Mapping {
 
     lock.lock();
     try {
-      byte[] containerBytes = containerStore.get(containerName.getBytes(
-          encoding));
-      if (containerBytes != null) {
-        throw new SCMException(
-            "Specified container already exists. key : " + containerName,
-            SCMException.ResultCodes.CONTAINER_EXISTS);
-      }
       containerInfo =
           containerStateManager.allocateContainer(
-              pipelineSelector, type, replicationFactor, containerName,
-              owner);
-      containerStore.put(
-          containerName.getBytes(encoding), containerInfo.getProtobuf()
+              pipelineSelector, type, replicationFactor, owner);
+
+      byte[] containerIDBytes = Longs.toByteArray(containerInfo.getContainerID());
+      containerStore.put(containerIDBytes, containerInfo.getProtobuf()
               .toByteArray());
     } finally {
       lock.unlock();
@@ -256,20 +241,20 @@ public class ContainerMapping implements Mapping {
   /**
    * Deletes a container from SCM.
    *
-   * @param containerName - Container name
+   * @param containerID - Container ID
    * @throws IOException if container doesn't exist or container store failed
    *                     to delete the
    *                     specified key.
    */
   @Override
-  public void deleteContainer(String containerName) throws IOException {
+  public void deleteContainer(long containerID) throws IOException {
     lock.lock();
     try {
-      byte[] dbKey = containerName.getBytes(encoding);
+      byte[] dbKey = Longs.toByteArray(containerID);
       byte[] containerBytes = containerStore.get(dbKey);
       if (containerBytes == null) {
         throw new SCMException(
-            "Failed to delete container " + containerName + ", reason : " +
+            "Failed to delete container " + containerID + ", reason : " +
                 "container doesn't exist.",
             SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
       }
@@ -284,17 +269,17 @@ public class ContainerMapping implements Mapping {
    */
   @Override
   public HddsProtos.LifeCycleState updateContainerState(
-      String containerName, HddsProtos.LifeCycleEvent event) throws
+      long containerID, HddsProtos.LifeCycleEvent event) throws
       IOException {
     ContainerInfo containerInfo;
     lock.lock();
     try {
-      byte[] dbKey = containerName.getBytes(encoding);
+      byte[] dbKey = Longs.toByteArray(containerID);
       byte[] containerBytes = containerStore.get(dbKey);
       if (containerBytes == null) {
         throw new SCMException(
             "Failed to update container state"
-                + containerName
+                + containerID
                 + ", reason : container doesn't exist.",
             SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
       }
@@ -310,7 +295,7 @@ public class ContainerMapping implements Mapping {
             containerLeaseManager.acquire(containerInfo);
         // Register callback to be executed in case of timeout
         containerLease.registerCallBack(() -> {
-          updateContainerState(containerName,
+          updateContainerState(containerID,
               HddsProtos.LifeCycleEvent.TIMEOUT);
           return null;
         });
@@ -388,7 +373,7 @@ public class ContainerMapping implements Mapping {
     containerSupervisor.handleContainerReport(reports);
     for (StorageContainerDatanodeProtocolProtos.ContainerInfo datanodeState :
         containerInfos) {
-      byte[] dbKey = datanodeState.getContainerNameBytes().toByteArray();
+      byte[] dbKey = Longs.toByteArray(datanodeState.getContainerID());
       lock.lock();
       try {
         byte[] containerBytes = containerStore.get(dbKey);
@@ -409,14 +394,14 @@ public class ContainerMapping implements Mapping {
           // If the container is closed, then state is already written to SCM
           // DB.TODO: So can we can write only once to DB.
           if (closeContainerIfNeeded(newState)) {
-            LOG.info("Closing the Container: {}", newState.getContainerName());
+            LOG.info("Closing the Container: {}", newState.getContainerID());
           }
         } else {
           // 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(),
-              datanodeState.getContainerName());
+              datanodeState.getContainerID());
         }
       } finally {
         lock.unlock();
@@ -436,7 +421,7 @@ public class ContainerMapping implements Mapping {
       HddsProtos.SCMContainerInfo knownState) {
     HddsProtos.SCMContainerInfo.Builder builder =
         HddsProtos.SCMContainerInfo.newBuilder();
-    builder.setContainerName(knownState.getContainerName());
+    builder.setContainerID(knownState.getContainerID());
     builder.setPipeline(knownState.getPipeline());
     // If used size is greater than allocated size, we will be updating
     // allocated size with used size. This update is done as a fallback
@@ -473,7 +458,7 @@ public class ContainerMapping implements Mapping {
     float containerUsedPercentage = 1.0f *
         newState.getUsedBytes() / this.size;
 
-    ContainerInfo scmInfo = getContainer(newState.getContainerName());
+    ContainerInfo scmInfo = getContainer(newState.getContainerID());
     if (containerUsedPercentage >= containerCloseThreshold
         && !isClosed(scmInfo)) {
       // We will call closer till get to the closed state.
@@ -488,13 +473,13 @@ public class ContainerMapping implements Mapping {
         // closed state from container reports. This state change should be
         // invoked once and only once.
         HddsProtos.LifeCycleState state = updateContainerState(
-            scmInfo.getContainerName(),
+            scmInfo.getContainerID(),
             HddsProtos.LifeCycleEvent.FINALIZE);
         if (state != HddsProtos.LifeCycleState.CLOSING) {
           LOG.error("Failed to close container {}, reason : Not able " +
                   "to " +
                   "update container state, current container state: {}.",
-              newState.getContainerName(), state);
+              newState.getContainerID(), state);
           return false;
         }
         return true;
@@ -561,11 +546,11 @@ public class ContainerMapping implements Mapping {
   @VisibleForTesting
   public void flushContainerInfo() throws IOException {
     List<ContainerInfo> containers = containerStateManager.getAllContainers();
-    List<String> failedContainers = new ArrayList<>();
+    List<Long> failedContainers = new ArrayList<>();
     for (ContainerInfo info : containers) {
       // even if some container updated failed, others can still proceed
       try {
-        byte[] dbKey = info.getContainerName().getBytes(encoding);
+        byte[] dbKey = Longs.toByteArray(info.getContainerID());
         byte[] containerBytes = containerStore.get(dbKey);
         // TODO : looks like when a container is deleted, the container is
         // removed from containerStore but not containerStateManager, so it can
@@ -577,7 +562,6 @@ public class ContainerMapping implements Mapping {
           ContainerInfo oldInfo = ContainerInfo.fromProtobuf(oldInfoProto);
           ContainerInfo newInfo = new ContainerInfo.Builder()
               .setAllocatedBytes(info.getAllocatedBytes())
-              .setContainerName(oldInfo.getContainerName())
               .setNumberOfKeys(oldInfo.getNumberOfKeys())
               .setOwner(oldInfo.getOwner())
               .setPipeline(oldInfo.getPipeline())
@@ -588,10 +572,10 @@ public class ContainerMapping implements Mapping {
         } else {
           LOG.debug("Container state manager has container {} but not found " +
                   "in container store, a deleted container?",
-              info.getContainerName());
+              info.getContainerID());
         }
       } catch (IOException ioe) {
-        failedContainers.add(info.getContainerName());
+        failedContainers.add(info.getContainerID());
       }
     }
     if (!failedContainers.isEmpty()) {

+ 4 - 8
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java

@@ -157,8 +157,7 @@ public class ContainerStateManager implements Closeable {
 
     List<ContainerInfo> containerList;
     try {
-      containerList = containerMapping.listContainer(null,
-          null, Integer.MAX_VALUE);
+      containerList = containerMapping.listContainer(0, Integer.MAX_VALUE);
 
       // if there are no container to load, let us return.
       if (containerList == null || containerList.size() == 0) {
@@ -280,24 +279,21 @@ public class ContainerStateManager implements Closeable {
    * @param selector -- Pipeline selector class.
    * @param type -- Replication type.
    * @param replicationFactor - Replication replicationFactor.
-   * @param containerName - Container Name.
    * @return Container Info.
    * @throws IOException  on Failure.
    */
   public ContainerInfo allocateContainer(PipelineSelector selector, HddsProtos
       .ReplicationType type, HddsProtos.ReplicationFactor replicationFactor,
-      final String containerName, String owner) throws
-      IOException {
+      String owner) throws IOException {
 
     Pipeline pipeline = selector.getReplicationPipeline(type,
-        replicationFactor, containerName);
+        replicationFactor);
 
     Preconditions.checkNotNull(pipeline, "Pipeline type=%s/"
         + "replication=%s couldn't be found for the new container. "
         + "Do you have enough nodes?", type, replicationFactor);
 
     ContainerInfo containerInfo = new ContainerInfo.Builder()
-        .setContainerName(containerName)
         .setState(HddsProtos.LifeCycleState.ALLOCATED)
         .setPipeline(pipeline)
         // This is bytes allocated for blocks inside container, not the
@@ -332,7 +328,7 @@ public class ContainerStateManager implements Closeable {
       String error = String.format("Failed to update container state %s, " +
               "reason: invalid state transition from state: %s upon " +
               "event: %s.",
-          info.getPipeline().getContainerName(), info.getState(), event);
+          info.getContainerID(), info.getState(), event);
       LOG.error(error);
       throw new SCMException(error, FAILED_TO_CHANGE_CONTAINER_STATE);
     }

+ 14 - 19
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java

@@ -31,62 +31,57 @@ import java.util.List;
  */
 public interface Mapping extends Closeable {
   /**
-   * Returns the ContainerInfo from the container name.
+   * Returns the ContainerInfo from the container ID.
    *
-   * @param containerName - Name
+   * @param containerID - ID of container.
    * @return - ContainerInfo such as creation state and the pipeline.
    * @throws IOException
    */
-  ContainerInfo getContainer(String containerName) throws IOException;
+  ContainerInfo getContainer(long containerID) throws IOException;
 
   /**
    * Returns containers under certain conditions.
-   * Search container names from start name(exclusive),
-   * and use prefix name to filter the result. The max
-   * size of the searching range cannot exceed the
+   * Search container IDs from start ID(exclusive),
+   * The max size of the searching range cannot exceed the
    * value of count.
    *
-   * @param startName start name, if null, start searching at the head.
-   * @param prefixName prefix name, if null, then filter is disabled.
-   * @param count count, if count < 0, the max size is unlimited.(
+   * @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)
+   *              value instead of being unlimited in case the db is very big.
    *
    * @return a list of container.
    * @throws IOException
    */
-  List<ContainerInfo> listContainer(String startName, String prefixName,
-      int count) throws IOException;
+  List<ContainerInfo> listContainer(long startContainerID, int count) throws IOException;
 
   /**
    * Allocates a new container for a given keyName and replication factor.
    *
    * @param replicationFactor - replication factor of the container.
-   * @param containerName - Name.
    * @param owner
    * @return - Container Info.
    * @throws IOException
    */
   ContainerInfo allocateContainer(HddsProtos.ReplicationType type,
-      HddsProtos.ReplicationFactor replicationFactor,
-      String containerName, String owner) throws IOException;
+      HddsProtos.ReplicationFactor replicationFactor, String owner) throws IOException;
 
   /**
    * Deletes a container from SCM.
    *
-   * @param containerName - Container Name
+   * @param containerID - Container ID
    * @throws IOException
    */
-  void deleteContainer(String containerName) throws IOException;
+  void deleteContainer(long containerID) throws IOException;
 
   /**
    * Update container state.
-   * @param containerName - Container Name
+   * @param containerID - Container ID
    * @param event - container life cycle event
    * @return - new container state
    * @throws IOException
    */
-  HddsProtos.LifeCycleState updateContainerState(String containerName,
+  HddsProtos.LifeCycleState updateContainerState(long containerID,
       HddsProtos.LifeCycleEvent event) throws IOException;
 
   /**

+ 8 - 8
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/closer/ContainerCloser.java

@@ -51,7 +51,7 @@ public class ContainerCloser {
   private static final long MULTIPLIER = 3L;
   private static final int CLEANUP_WATER_MARK = 1000;
   private final NodeManager nodeManager;
-  private final Map<String, Long> commandIssued;
+  private final Map<Long, Long> commandIssued;
   private final Configuration configuration;
   private final AtomicInteger mapCount;
   private final long reportInterval;
@@ -93,12 +93,12 @@ public class ContainerCloser {
    */
   public void close(HddsProtos.SCMContainerInfo info) {
 
-    if (commandIssued.containsKey(info.getContainerName())) {
+    if (commandIssued.containsKey(info.getContainerID())) {
       // We check if we issued a close command in last 3 * reportInterval secs.
-      long commandQueueTime = commandIssued.get(info.getContainerName());
+      long commandQueueTime = commandIssued.get(info.getContainerID());
       long currentTime = TimeUnit.MILLISECONDS.toSeconds(Time.monotonicNow());
       if (currentTime > commandQueueTime + (MULTIPLIER * reportInterval)) {
-        commandIssued.remove(info.getContainerName());
+        commandIssued.remove(info.getContainerID());
         mapCount.decrementAndGet();
       } else {
         // Ignore this request, since we just issued a close command. We
@@ -131,10 +131,10 @@ public class ContainerCloser {
         pipeline.getPipelineChannel().getMembersList()) {
       nodeManager.addDatanodeCommand(
           DatanodeDetails.getFromProtoBuf(datanodeDetails).getUuid(),
-          new CloseContainerCommand(info.getContainerName()));
+          new CloseContainerCommand(info.getContainerID()));
     }
-    if (!commandIssued.containsKey(info.getContainerName())) {
-      commandIssued.put(info.getContainerName(),
+    if (!commandIssued.containsKey(info.getContainerID())) {
+      commandIssued.put(info.getContainerID(),
           TimeUnit.MILLISECONDS.toSeconds(Time.monotonicNow()));
       mapCount.incrementAndGet();
     }
@@ -150,7 +150,7 @@ public class ContainerCloser {
       Runnable entryCleaner = () -> {
         LOG.debug("Starting close container Hash map cleaner.");
         try {
-          for (Map.Entry<String, Long> entry : commandIssued.entrySet()) {
+          for (Map.Entry<Long, Long> entry : commandIssued.entrySet()) {
             long commandQueueTime = entry.getValue();
             if (commandQueueTime + (MULTIPLIER * reportInterval) >
                 TimeUnit.MILLISECONDS.toSeconds(Time.monotonicNow())) {

+ 5 - 5
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/InProgressPool.java

@@ -61,7 +61,7 @@ public final class InProgressPool {
   private final NodeManager nodeManager;
   private final NodePoolManager poolManager;
   private final ExecutorService executorService;
-  private final Map<String, Integer> containerCountMap;
+  private final Map<Long, Integer> containerCountMap;
   private final Map<UUID, Boolean> processedNodeSet;
   private final long startTime;
   private ProgressStatus status;
@@ -258,12 +258,12 @@ public final class InProgressPool {
         for (ContainerInfo info : reports.getReportsList()) {
           containerProcessedCount.incrementAndGet();
           LOG.debug("Total Containers processed: {} Container Name: {}",
-              containerProcessedCount.get(), info.getContainerName());
+              containerProcessedCount.get(), info.getContainerID());
 
           // Update the container map with count + 1 if the key exists or
           // update the map with 1. Since this is a concurrentMap the
           // computation and update is atomic.
-          containerCountMap.merge(info.getContainerName(), 1, Integer::sum);
+          containerCountMap.merge(info.getContainerID(), 1, Integer::sum);
         }
       }
     };
@@ -275,8 +275,8 @@ public final class InProgressPool {
    * @param predicate -- Predicate to filter by
    * @return A list of map entries.
    */
-  public List<Map.Entry<String, Integer>> filterContainer(
-      Predicate<Map.Entry<String, Integer>> predicate) {
+  public List<Map.Entry<Long, Integer>> filterContainer(
+      Predicate<Map.Entry<Long, Integer>> predicate) {
     return containerCountMap.entrySet().stream()
         .filter(predicate).collect(Collectors.toList());
   }

+ 1 - 1
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodePoolManager.java

@@ -248,7 +248,7 @@ public final class SCMNodePoolManager implements NodePoolManager {
       throws SCMException {
     Preconditions.checkNotNull(datanodeDetails, "node is null");
     try {
-      byte[]  result = nodePoolStore.get(
+      byte[] result = nodePoolStore.get(
           datanodeDetails.getProtoBufMessage().toByteArray());
       return result == null ? null : DFSUtil.bytes2String(result);
     } catch (IOException e) {

+ 8 - 7
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java

@@ -50,11 +50,10 @@ public abstract class PipelineManager {
    * needed and based on the replication type in the request appropriate
    * Interface is invoked.
    *
-   * @param containerName Name of the container
    * @param replicationFactor - Replication Factor
    * @return a Pipeline.
    */
-  public synchronized final Pipeline getPipeline(String containerName,
+  public synchronized final Pipeline getPipeline(
       ReplicationFactor replicationFactor, ReplicationType replicationType)
       throws IOException {
     /**
@@ -74,15 +73,17 @@ public abstract class PipelineManager {
     PipelineChannel pipelineChannel =
         allocatePipelineChannel(replicationFactor);
     if (pipelineChannel != null) {
-      LOG.debug("created new pipelineChannel:{} for container:{}",
-          pipelineChannel.getName(), containerName);
+      LOG.debug("created new pipelineChannel:{} for container with " +
+              "replicationType:{} replicationFactor:{}",
+          pipelineChannel.getName(), replicationType, replicationFactor);
       activePipelineChannels.add(pipelineChannel);
     } else {
       pipelineChannel =
           findOpenPipelineChannel(replicationType, replicationFactor);
       if (pipelineChannel != null) {
-        LOG.debug("re-used pipelineChannel:{} for container:{}",
-            pipelineChannel.getName(), containerName);
+        LOG.debug("re-used pipelineChannel:{} for container with " +
+                "replicationType:{} replicationFactor:{}",
+            pipelineChannel.getName(), replicationType, replicationFactor);
       }
     }
     if (pipelineChannel == null) {
@@ -90,7 +91,7 @@ public abstract class PipelineManager {
               "free nodes or operational pipelineChannel.");
       return null;
     } else {
-      return new Pipeline(containerName, pipelineChannel);
+      return new Pipeline(pipelineChannel);
     }
   }
 

+ 4 - 4
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java

@@ -166,14 +166,14 @@ public class PipelineSelector {
    */
 
   public Pipeline getReplicationPipeline(ReplicationType replicationType,
-      HddsProtos.ReplicationFactor replicationFactor, String containerName)
+      HddsProtos.ReplicationFactor replicationFactor)
       throws IOException {
     PipelineManager manager = getPipelineManager(replicationType);
     Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
-    LOG.debug("Getting replication pipeline for {} : Replication {}",
-        containerName, replicationFactor.toString());
+    LOG.debug("Getting replication pipeline forReplicationType {} : ReplicationFactor {}",
+        replicationType.toString(), replicationFactor.toString());
     return manager.
-        getPipeline(containerName, replicationFactor, replicationType);
+        getPipeline(replicationFactor, replicationType);
   }
 
   /**

+ 1 - 1
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java

@@ -95,7 +95,7 @@ public class RatisManagerImpl extends PipelineManager {
               PipelineSelector.newPipelineFromNodes(newNodesList,
               LifeCycleState.OPEN, ReplicationType.RATIS, factor, conduitName);
           Pipeline pipeline =
-              new Pipeline("setup", pipelineChannel);
+              new Pipeline(pipelineChannel);
           try (XceiverClientRatis client =
               XceiverClientRatis.newXceiverClientRatis(pipeline, conf)) {
             client.createPipeline(pipeline.getPipelineName(), newNodesList);

+ 2 - 18
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java

@@ -29,7 +29,6 @@ import org.apache.hadoop.hdds.scm.HddsServerUtil;
 import org.apache.hadoop.hdds.scm.ScmInfo;
 import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
 import org.apache.hadoop.hdds.scm.container.common.helpers.DeleteBlockResult;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
 import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolPB;
@@ -37,6 +36,7 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ozone.common.BlockGroup;
+import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
 import org.apache.hadoop.ozone.protocolPB
     .ScmBlockLocationProtocolServerSideTranslatorPB;
@@ -46,9 +46,7 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
     .OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY;
@@ -138,20 +136,6 @@ public class SCMBlockProtocolServer implements ScmBlockLocationProtocol {
     getBlockRpcServer().join();
   }
 
-  @Override
-  public Set<AllocatedBlock> getBlockLocations(Set<String> keys) throws
-      IOException {
-    Set<AllocatedBlock> locatedBlocks = new HashSet<>();
-    for (String key : keys) {
-      Pipeline pipeline = scm.getScmBlockManager().getBlock(key);
-      AllocatedBlock block = new AllocatedBlock.Builder().setKey(key)
-          .setPipeline(pipeline).build();
-      locatedBlocks.add(block);
-    }
-    return locatedBlocks;
-
-  }
-
   @Override
   public AllocatedBlock allocateBlock(long size, HddsProtos.ReplicationType
       type, HddsProtos.ReplicationFactor factor, String owner) throws
@@ -202,7 +186,7 @@ public class SCMBlockProtocolServer implements ScmBlockLocationProtocol {
             .Result.unknownFailure;
       }
       List<DeleteBlockResult> blockResultList = new ArrayList<>();
-      for (String blockKey : keyBlocks.getBlockIDList()) {
+      for (BlockID blockKey : keyBlocks.getBlockIDList()) {
         blockResultList.add(new DeleteBlockResult(blockKey, resultCode));
       }
       results.add(new DeleteBlockGroupResult(keyBlocks.getGroupID(),

+ 26 - 21
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java

@@ -21,6 +21,7 @@
  */
 package org.apache.hadoop.hdds.scm.server;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.BlockingService;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -137,32 +138,31 @@ public class SCMClientProtocolServer implements
   }
 
   @Override
-  public Pipeline allocateContainer(HddsProtos.ReplicationType
-      replicationType, HddsProtos.ReplicationFactor factor, String
-      containerName, String owner) throws IOException {
-    scm.checkAdminAccess();
+  public ContainerInfo allocateContainer(HddsProtos.ReplicationType
+      replicationType, HddsProtos.ReplicationFactor factor,
+      String owner) throws IOException {
+    getScm().checkAdminAccess();
     return scm.getScmContainerManager()
-        .allocateContainer(replicationType, factor, containerName, owner)
-        .getPipeline();
+        .allocateContainer(replicationType, factor, owner);
   }
 
   @Override
-  public Pipeline getContainer(String containerName) throws IOException {
+  public ContainerInfo getContainer(long containerID) throws IOException {
     return scm.getScmContainerManager()
-        .getContainer(containerName).getPipeline();
+        .getContainer(containerID);
   }
 
   @Override
-  public List<ContainerInfo> listContainer(String startName,
-      String prefixName, int count) throws IOException {
-    return scm.getScmContainerManager()
-        .listContainer(startName, prefixName, count);
+  public List<ContainerInfo> listContainer(long startContainerID,
+      int count) throws IOException {
+    return scm.getScmContainerManager().
+        listContainer(startContainerID, count);
   }
 
   @Override
-  public void deleteContainer(String containerName) throws IOException {
-    scm.checkAdminAccess();
-    scm.getScmContainerManager().deleteContainer(containerName);
+  public void deleteContainer(long containerID) throws IOException {
+    getScm().checkAdminAccess();
+    scm.getScmContainerManager().deleteContainer(containerID);
 
   }
 
@@ -193,12 +193,12 @@ public class SCMClientProtocolServer implements
 
   @Override
   public void notifyObjectStageChange(StorageContainerLocationProtocolProtos
-      .ObjectStageChangeRequestProto.Type type, String name,
+      .ObjectStageChangeRequestProto.Type type, long id,
       StorageContainerLocationProtocolProtos.ObjectStageChangeRequestProto.Op
           op, StorageContainerLocationProtocolProtos
       .ObjectStageChangeRequestProto.Stage stage) throws IOException {
 
-    LOG.info("Object type {} name {} op {} new stage {}", type, name, op,
+    LOG.info("Object type {} id {} op {} new stage {}", type, id, op,
         stage);
     if (type == StorageContainerLocationProtocolProtos
         .ObjectStageChangeRequestProto.Type.container) {
@@ -206,10 +206,10 @@ public class SCMClientProtocolServer implements
           .ObjectStageChangeRequestProto.Op.create) {
         if (stage == StorageContainerLocationProtocolProtos
             .ObjectStageChangeRequestProto.Stage.begin) {
-          scm.getScmContainerManager().updateContainerState(name, HddsProtos
+          scm.getScmContainerManager().updateContainerState(id, HddsProtos
               .LifeCycleEvent.CREATE);
         } else {
-          scm.getScmContainerManager().updateContainerState(name, HddsProtos
+          scm.getScmContainerManager().updateContainerState(id, HddsProtos
               .LifeCycleEvent.CREATED);
         }
       } else {
@@ -217,10 +217,10 @@ public class SCMClientProtocolServer implements
             .ObjectStageChangeRequestProto.Op.close) {
           if (stage == StorageContainerLocationProtocolProtos
               .ObjectStageChangeRequestProto.Stage.begin) {
-            scm.getScmContainerManager().updateContainerState(name, HddsProtos
+            scm.getScmContainerManager().updateContainerState(id, HddsProtos
                 .LifeCycleEvent.FINALIZE);
           } else {
-            scm.getScmContainerManager().updateContainerState(name, HddsProtos
+            scm.getScmContainerManager().updateContainerState(id, HddsProtos
                 .LifeCycleEvent.CLOSE);
           }
         }
@@ -292,6 +292,11 @@ public class SCMClientProtocolServer implements
     return resultList;
   }
 
+  @VisibleForTesting
+  public StorageContainerManager getScm() {
+    return scm;
+  }
+
   /**
    * Query the System for Nodes.
    *

+ 4 - 5
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java

@@ -87,7 +87,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 final class StorageContainerManager extends ServiceRuntimeInfoImpl
+public class StorageContainerManager extends ServiceRuntimeInfoImpl
     implements SCMMXBean {
 
   private static final Logger LOG = LoggerFactory
@@ -168,8 +168,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
         cacheSize);
 
     scmBlockManager =
-        new BlockManagerImpl(conf, getScmNodeManager(), scmContainerManager,
-            cacheSize);
+        new BlockManagerImpl(conf, getScmNodeManager(), scmContainerManager);
 
     scmAdminUsernames = conf.getTrimmedStringCollection(OzoneConfigKeys
         .OZONE_ADMINISTRATORS);
@@ -459,9 +458,9 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
   }
 
   @VisibleForTesting
-  public ContainerInfo getContainerInfo(String containerName) throws
+  public ContainerInfo getContainerInfo(long containerID) throws
       IOException {
-    return scmContainerManager.getContainer(containerName);
+    return scmContainerManager.getContainer(containerID);
   }
 
   /**

+ 3 - 31
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java

@@ -23,7 +23,6 @@ import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.ContainerMapping;
 import org.apache.hadoop.hdds.scm.container.MockNodeManager;
 import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.common.SCMTestUtils;
@@ -40,7 +39,6 @@ import java.io.File;
 import java.io.IOException;
 import java.nio.file.Paths;
 import java.util.Collections;
-import java.util.UUID;
 
 import static org.apache.hadoop.ozone.OzoneConsts.GB;
 import static org.apache.hadoop.ozone.OzoneConsts.MB;
@@ -76,7 +74,7 @@ public class TestBlockManager {
     }
     nodeManager = new MockNodeManager(true, 10);
     mapping = new ContainerMapping(conf, nodeManager, 128);
-    blockManager = new BlockManagerImpl(conf, nodeManager, mapping, 128);
+    blockManager = new BlockManagerImpl(conf, nodeManager, mapping);
     if(conf.getBoolean(ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY,
         ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT)){
       factor = HddsProtos.ReplicationFactor.THREE;
@@ -106,33 +104,13 @@ public class TestBlockManager {
     Assert.assertNotNull(block);
   }
 
-  @Test
-  public void testGetAllocatedBlock() throws IOException {
-    AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
-        type, factor, containerOwner);
-    Assert.assertNotNull(block);
-    Pipeline pipeline = blockManager.getBlock(block.getKey());
-    Assert.assertEquals(pipeline.getLeader().getUuid(),
-        block.getPipeline().getLeader().getUuid());
-  }
-
   @Test
   public void testDeleteBlock() throws Exception {
     AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
         type, factor, containerOwner);
     Assert.assertNotNull(block);
-    blockManager.deleteBlocks(Collections.singletonList(block.getKey()));
-
-    // Deleted block can not be retrieved
-    thrown.expectMessage("Specified block key does not exist.");
-    blockManager.getBlock(block.getKey());
-
-    // Tombstone of the deleted block can be retrieved if it has not been
-    // cleaned yet.
-    String deletedKeyName = blockManager.getDeletedKeyName(block.getKey());
-    Pipeline pipeline = blockManager.getBlock(deletedKeyName);
-    Assert.assertEquals(pipeline.getLeader().getUuid(),
-        block.getPipeline().getLeader().getUuid());
+    blockManager.deleteBlocks(Collections.singletonList(
+        block.getBlockID()));
   }
 
   @Test
@@ -143,12 +121,6 @@ public class TestBlockManager {
         type, factor, containerOwner);
   }
 
-  @Test
-  public void testGetNoneExistentContainer() throws IOException {
-    String nonExistBlockKey = UUID.randomUUID().toString();
-    thrown.expectMessage("Specified block key does not exist.");
-    blockManager.getBlock(nonExistBlockKey);
-  }
 
   @Test
   public void testChillModeAllocateBlockFails() throws IOException {

+ 30 - 26
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java

@@ -82,17 +82,22 @@ public class TestDeletedBlockLog {
     FileUtils.deleteDirectory(testDir);
   }
 
-  private Map<String, List<String>> generateData(int dataSize) {
-    Map<String, List<String>> blockMap = new HashMap<>();
+  private Map<Long, List<Long>> generateData(int dataSize) {
+    Map<Long, List<Long>> blockMap = new HashMap<>();
     Random random = new Random(1);
+    int continerIDBase = random.nextInt(100);
+    int localIDBase = random.nextInt(1000);
     for (int i = 0; i < dataSize; i++) {
-      String containerName = "container-" + UUID.randomUUID().toString();
-      List<String> blocks = new ArrayList<>();
+      //String containerName = "container-" + UUID.randomUUID().toString();
+      long containerID = continerIDBase + i;
+      List<Long> blocks = new ArrayList<>();
       int blockSize = random.nextInt(30) + 1;
       for (int j = 0; j < blockSize; j++)  {
-        blocks.add("block-" + UUID.randomUUID().toString());
+        //blocks.add("block-" + UUID.randomUUID().toString());
+        long localID = localIDBase + j;
+        blocks.add(localID);
       }
-      blockMap.put(containerName, blocks);
+      blockMap.put(containerID, blocks);
     }
     return blockMap;
   }
@@ -104,7 +109,7 @@ public class TestDeletedBlockLog {
     Assert.assertEquals(0, blocks.size());
 
     // Creates 40 TX in the log.
-    for (Map.Entry<String, List<String>> entry : generateData(40).entrySet()){
+    for (Map.Entry<Long, List<Long>> entry : generateData(40).entrySet()){
       deletedBlockLog.addTransaction(entry.getKey(), entry.getValue());
     }
 
@@ -143,7 +148,7 @@ public class TestDeletedBlockLog {
     int maxRetry = conf.getInt(OZONE_SCM_BLOCK_DELETION_MAX_RETRY, 20);
 
     // Create 30 TXs in the log.
-    for (Map.Entry<String, List<String>> entry : generateData(30).entrySet()){
+    for (Map.Entry<Long, List<Long>> entry : generateData(30).entrySet()){
       deletedBlockLog.addTransaction(entry.getKey(), entry.getValue());
     }
 
@@ -172,7 +177,7 @@ public class TestDeletedBlockLog {
 
   @Test
   public void testCommitTransactions() throws Exception {
-    for (Map.Entry<String, List<String>> entry : generateData(50).entrySet()){
+    for (Map.Entry<Long, List<Long>> entry : generateData(50).entrySet()){
       deletedBlockLog.addTransaction(entry.getKey(), entry.getValue());
     }
     List<DeletedBlocksTransaction> blocks =
@@ -203,7 +208,7 @@ public class TestDeletedBlockLog {
     for (int i = 0; i < 100; i++) {
       int state = random.nextInt(4);
       if (state == 0) {
-        for (Map.Entry<String, List<String>> entry :
+        for (Map.Entry<Long, List<Long>> entry :
             generateData(10).entrySet()){
           deletedBlockLog.addTransaction(entry.getKey(), entry.getValue());
         }
@@ -234,7 +239,7 @@ public class TestDeletedBlockLog {
 
   @Test
   public void testPersistence() throws Exception {
-    for (Map.Entry<String, List<String>> entry : generateData(50).entrySet()){
+    for (Map.Entry<Long, List<Long>> entry : generateData(50).entrySet()){
       deletedBlockLog.addTransaction(entry.getKey(), entry.getValue());
     }
     // close db and reopen it again to make sure
@@ -257,10 +262,10 @@ public class TestDeletedBlockLog {
     int txNum = 10;
     int maximumAllowedTXNum = 5;
     List<DeletedBlocksTransaction> blocks = null;
-    List<String> containerNames = new LinkedList<>();
+    List<Long> containerIDs = new LinkedList<>();
 
     int count = 0;
-    String containerName = null;
+    long containerID = 0L;
     DatanodeDetails dnDd1 = DatanodeDetails.newBuilder()
         .setUuid(UUID.randomUUID().toString())
         .setIpAddress("127.0.0.1")
@@ -279,18 +284,18 @@ public class TestDeletedBlockLog {
         .build();
     Mapping mappingService = mock(ContainerMapping.class);
     // Creates {TXNum} TX in the log.
-    for (Map.Entry<String, List<String>> entry : generateData(txNum)
+    for (Map.Entry<Long, List<Long>> entry : generateData(txNum)
         .entrySet()) {
       count++;
-      containerName = entry.getKey();
-      containerNames.add(containerName);
-      deletedBlockLog.addTransaction(containerName, entry.getValue());
+      containerID = entry.getKey();
+      containerIDs.add(containerID);
+      deletedBlockLog.addTransaction(containerID, entry.getValue());
 
       // make TX[1-6] for datanode1; TX[7-10] for datanode2
       if (count <= (maximumAllowedTXNum + 1)) {
-        mockContainerInfo(mappingService, containerName, dnDd1);
+        mockContainerInfo(mappingService, containerID, dnDd1);
       } else {
-        mockContainerInfo(mappingService, containerName, dnId2);
+        mockContainerInfo(mappingService, containerID, dnId2);
       }
     }
 
@@ -325,7 +330,7 @@ public class TestDeletedBlockLog {
     DeletedBlocksTransaction.Builder builder =
         DeletedBlocksTransaction.newBuilder();
     builder.setTxID(11);
-    builder.setContainerName(containerName);
+    builder.setContainerID(containerID);
     builder.setCount(0);
     transactions.addTransaction(builder.build());
 
@@ -334,30 +339,29 @@ public class TestDeletedBlockLog {
         transactions.getDatanodeTransactions(dnId2.getUuid()).size());
 
     // Add new TX in dnID2, then dnID2 will reach maximum value.
-    containerName = "newContainer";
     builder = DeletedBlocksTransaction.newBuilder();
     builder.setTxID(12);
-    builder.setContainerName(containerName);
+    builder.setContainerID(containerID);
     builder.setCount(0);
-    mockContainerInfo(mappingService, containerName, dnId2);
+    mockContainerInfo(mappingService, containerID, dnId2);
     transactions.addTransaction(builder.build());
     // Since all node are full, then transactions is full.
     Assert.assertTrue(transactions.isFull());
   }
 
-  private void mockContainerInfo(Mapping mappingService, String containerName,
+  private void mockContainerInfo(Mapping mappingService, long containerID,
       DatanodeDetails dd) throws IOException {
     PipelineChannel pipelineChannel =
         new PipelineChannel("fake", LifeCycleState.OPEN,
             ReplicationType.STAND_ALONE, ReplicationFactor.ONE, "fake");
     pipelineChannel.addMember(dd);
-    Pipeline pipeline = new Pipeline(containerName, pipelineChannel);
+    Pipeline pipeline = new Pipeline(pipelineChannel);
 
     ContainerInfo.Builder builder = new ContainerInfo.Builder();
     builder.setPipeline(pipeline);
 
     ContainerInfo conatinerInfo = builder.build();
     Mockito.doReturn(conatinerInfo).when(mappingService)
-        .getContainer(containerName);
+        .getContainer(containerID);
   }
 }

+ 27 - 52
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java

@@ -45,6 +45,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.NavigableSet;
+import java.util.Random;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.UUID;
@@ -59,6 +60,7 @@ public class TestContainerMapping {
   private static File testDir;
   private static XceiverClientManager xceiverClientManager;
   private static String containerOwner = "OZONE";
+  private static Random random;
 
   private static final long TIMEOUT = 10000;
 
@@ -83,6 +85,7 @@ public class TestContainerMapping {
     nodeManager = new MockNodeManager(true, 10);
     mapping = new ContainerMapping(conf, nodeManager, 128);
     xceiverClientManager = new XceiverClientManager(conf);
+    random = new Random();
   }
 
   @AfterClass
@@ -103,7 +106,7 @@ public class TestContainerMapping {
     ContainerInfo containerInfo = mapping.allocateContainer(
         xceiverClientManager.getType(),
         xceiverClientManager.getFactor(),
-        UUID.randomUUID().toString(), containerOwner);
+        containerOwner);
     Assert.assertNotNull(containerInfo);
   }
 
@@ -120,7 +123,7 @@ public class TestContainerMapping {
       ContainerInfo containerInfo = mapping.allocateContainer(
           xceiverClientManager.getType(),
           xceiverClientManager.getFactor(),
-          UUID.randomUUID().toString(), containerOwner);
+          containerOwner);
 
       Assert.assertNotNull(containerInfo);
       Assert.assertNotNull(containerInfo.getPipeline());
@@ -132,59 +135,41 @@ public class TestContainerMapping {
 
   @Test
   public void testGetContainer() throws IOException {
-    String containerName = UUID.randomUUID().toString();
-    Pipeline pipeline = mapping.allocateContainer(
+    ContainerInfo containerInfo = mapping.allocateContainer(
         xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), containerName,
-        containerOwner).getPipeline();
+        xceiverClientManager.getFactor(),
+        containerOwner);
+    Pipeline pipeline  = containerInfo.getPipeline();
     Assert.assertNotNull(pipeline);
-    Pipeline newPipeline = mapping.getContainer(containerName).getPipeline();
+    Pipeline newPipeline = mapping.getContainer(
+        containerInfo.getContainerID()).getPipeline();
     Assert.assertEquals(pipeline.getLeader().getUuid(),
         newPipeline.getLeader().getUuid());
   }
 
-  @Test
-  public void testDuplicateAllocateContainerFails() throws IOException {
-    String containerName = UUID.randomUUID().toString();
-    Pipeline pipeline = mapping.allocateContainer(
-        xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), containerName,
-        containerOwner).getPipeline();
-    Assert.assertNotNull(pipeline);
-    thrown.expectMessage("Specified container already exists.");
-    mapping.allocateContainer(xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), containerName,
-        containerOwner);
-  }
-
   @Test
   public void testgetNoneExistentContainer() throws IOException {
-    String containerName = UUID.randomUUID().toString();
     thrown.expectMessage("Specified key does not exist.");
-    mapping.getContainer(containerName);
+    mapping.getContainer(random.nextLong());
   }
 
   @Test
   public void testChillModeAllocateContainerFails() throws IOException {
-    String containerName = UUID.randomUUID().toString();
     nodeManager.setChillmode(true);
     thrown.expectMessage("Unable to create container while in chill mode");
     mapping.allocateContainer(xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), containerName,
-        containerOwner);
+        xceiverClientManager.getFactor(), containerOwner);
   }
 
   @Test
   public void testContainerCreationLeaseTimeout() throws IOException,
       InterruptedException {
-    String containerName = UUID.randomUUID().toString();
     nodeManager.setChillmode(false);
     ContainerInfo containerInfo = mapping.allocateContainer(
         xceiverClientManager.getType(),
         xceiverClientManager.getFactor(),
-        containerName,
         containerOwner);
-    mapping.updateContainerState(containerInfo.getContainerName(),
+    mapping.updateContainerState(containerInfo.getContainerID(),
         HddsProtos.LifeCycleEvent.CREATE);
     Thread.sleep(TIMEOUT + 1000);
 
@@ -198,14 +183,13 @@ public class TestContainerMapping {
 
     thrown.expect(IOException.class);
     thrown.expectMessage("Lease Exception");
-    mapping.updateContainerState(containerInfo.getContainerName(),
+    mapping.updateContainerState(containerInfo.getContainerID(),
         HddsProtos.LifeCycleEvent.CREATED);
   }
 
   @Test
   public void testFullContainerReport() throws IOException {
-    String containerName = UUID.randomUUID().toString();
-    ContainerInfo info = createContainer(containerName);
+    ContainerInfo info = createContainer();
     DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
     ContainerReportsRequestProto.reportType reportType =
         ContainerReportsRequestProto.reportType.fullReport;
@@ -213,9 +197,7 @@ public class TestContainerMapping {
         new ArrayList<>();
     StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder =
         StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder();
-    ciBuilder.setContainerName(containerName)
-        //setting some random hash
-        .setFinalhash("e16cc9d6024365750ed8dbd194ea46d2")
+    ciBuilder.setFinalhash("e16cc9d6024365750ed8dbd194ea46d2")
         .setSize(5368709120L)
         .setUsed(2000000000L)
         .setKeyCount(100000000L)
@@ -234,15 +216,14 @@ public class TestContainerMapping {
 
     mapping.processContainerReports(crBuilder.build());
 
-    ContainerInfo updatedContainer = mapping.getContainer(containerName);
+    ContainerInfo updatedContainer = mapping.getContainer(info.getContainerID());
     Assert.assertEquals(100000000L, updatedContainer.getNumberOfKeys());
     Assert.assertEquals(2000000000L, updatedContainer.getUsedBytes());
   }
 
   @Test
   public void testContainerCloseWithContainerReport() throws IOException {
-    String containerName = UUID.randomUUID().toString();
-    ContainerInfo info = createContainer(containerName);
+    ContainerInfo info = createContainer();
     DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
     ContainerReportsRequestProto.reportType reportType =
         ContainerReportsRequestProto.reportType.fullReport;
@@ -251,9 +232,7 @@ public class TestContainerMapping {
 
     StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder =
         StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder();
-    ciBuilder.setContainerName(containerName)
-        //setting some random hash
-        .setFinalhash("7c45eb4d7ed5e0d2e89aaab7759de02e")
+    ciBuilder.setFinalhash("7c45eb4d7ed5e0d2e89aaab7759de02e")
         .setSize(5368709120L)
         .setUsed(5368705120L)
         .setKeyCount(500000000L)
@@ -272,7 +251,7 @@ public class TestContainerMapping {
 
     mapping.processContainerReports(crBuilder.build());
 
-    ContainerInfo updatedContainer = mapping.getContainer(containerName);
+    ContainerInfo updatedContainer = mapping.getContainer(info.getContainerID());
     Assert.assertEquals(500000000L, updatedContainer.getNumberOfKeys());
     Assert.assertEquals(5368705120L, updatedContainer.getUsedBytes());
     NavigableSet<ContainerID> pendingCloseContainers = mapping.getStateManager()
@@ -287,9 +266,8 @@ public class TestContainerMapping {
 
   @Test
   public void testCloseContainer() throws IOException {
-    String containerName = UUID.randomUUID().toString();
-    ContainerInfo info = createContainer(containerName);
-    mapping.updateContainerState(containerName,
+    ContainerInfo info = createContainer();
+    mapping.updateContainerState(info.getContainerID(),
         HddsProtos.LifeCycleEvent.FINALIZE);
     NavigableSet<ContainerID> pendingCloseContainers = mapping.getStateManager()
         .getMatchingContainerIDs(
@@ -298,7 +276,7 @@ public class TestContainerMapping {
             xceiverClientManager.getFactor(),
             HddsProtos.LifeCycleState.CLOSING);
     Assert.assertTrue(pendingCloseContainers.contains(info.containerID()));
-    mapping.updateContainerState(containerName,
+    mapping.updateContainerState(info.getContainerID(),
         HddsProtos.LifeCycleEvent.CLOSE);
     NavigableSet<ContainerID> closeContainers = mapping.getStateManager()
         .getMatchingContainerIDs(
@@ -311,21 +289,18 @@ public class TestContainerMapping {
 
   /**
    * Creates a container with the given name in ContainerMapping.
-   * @param containerName
-   *          Name of the container
    * @throws IOException
    */
-  private ContainerInfo createContainer(String containerName)
+  private ContainerInfo createContainer()
       throws IOException {
     nodeManager.setChillmode(false);
     ContainerInfo containerInfo = mapping.allocateContainer(
         xceiverClientManager.getType(),
         xceiverClientManager.getFactor(),
-        containerName,
         containerOwner);
-    mapping.updateContainerState(containerInfo.getContainerName(),
+    mapping.updateContainerState(containerInfo.getContainerID(),
         HddsProtos.LifeCycleEvent.CREATE);
-    mapping.updateContainerState(containerInfo.getContainerName(),
+    mapping.updateContainerState(containerInfo.getContainerID(),
         HddsProtos.LifeCycleEvent.CREATED);
     return containerInfo;
   }

+ 10 - 15
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java

@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hdds.scm.container.closer;
 
-import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.scm.TestUtils;
@@ -92,15 +91,13 @@ public class TestContainerCloser {
 
   @Test
   public void testClose() throws IOException {
-    String containerName = "container-" + RandomStringUtils.randomNumeric(5);
-
     ContainerInfo info = mapping.allocateContainer(
         HddsProtos.ReplicationType.STAND_ALONE,
-        HddsProtos.ReplicationFactor.ONE, containerName, "ozone");
+        HddsProtos.ReplicationFactor.ONE, "ozone");
 
     //Execute these state transitions so that we can close the container.
-    mapping.updateContainerState(containerName, CREATE);
-    mapping.updateContainerState(containerName, CREATED);
+    mapping.updateContainerState(info.getContainerID(), CREATE);
+    mapping.updateContainerState(info.getContainerID(), CREATED);
     long currentCount = mapping.getCloser().getCloseCount();
     long runCount = mapping.getCloser().getThreadRunCount();
 
@@ -120,7 +117,7 @@ public class TestContainerCloser {
     long newUsed = (long) (size * 0.91f);
     sendContainerReport(info, newUsed);
 
-    // with only one container the  cleaner thread should not run.
+    // with only one container the cleaner thread should not run.
     Assert.assertEquals(runCount, mapping.getCloser().getThreadRunCount());
 
     // and close count will be one.
@@ -140,14 +137,13 @@ public class TestContainerCloser {
 
     configuration.setTimeDuration(OZONE_CONTAINER_REPORT_INTERVAL, 1,
         TimeUnit.SECONDS);
-    String containerName = "container-" + RandomStringUtils.randomNumeric(5);
 
     ContainerInfo info = mapping.allocateContainer(
         HddsProtos.ReplicationType.STAND_ALONE,
-        HddsProtos.ReplicationFactor.ONE, containerName, "ozone");
+        HddsProtos.ReplicationFactor.ONE, "ozone");
 
     //Execute these state transitions so that we can close the container.
-    mapping.updateContainerState(containerName, CREATE);
+    mapping.updateContainerState(info.getContainerID(), CREATE);
 
     long currentCount = mapping.getCloser().getCloseCount();
     long runCount = mapping.getCloser().getThreadRunCount();
@@ -187,12 +183,11 @@ public class TestContainerCloser {
     long runCount = mapping.getCloser().getThreadRunCount();
 
     for (int x = 0; x < ContainerCloser.getCleanupWaterMark() + 10; x++) {
-      String containerName = "container-" + RandomStringUtils.randomNumeric(7);
       ContainerInfo info = mapping.allocateContainer(
           HddsProtos.ReplicationType.STAND_ALONE,
-          HddsProtos.ReplicationFactor.ONE, containerName, "ozone");
-      mapping.updateContainerState(containerName, CREATE);
-      mapping.updateContainerState(containerName, CREATED);
+          HddsProtos.ReplicationFactor.ONE, "ozone");
+      mapping.updateContainerState(info.getContainerID(), CREATE);
+      mapping.updateContainerState(info.getContainerID(), CREATED);
       sendContainerReport(info, 5 * GIGABYTE);
     }
 
@@ -210,7 +205,7 @@ public class TestContainerCloser {
 
     StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder =
         StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder();
-    ciBuilder.setContainerName(info.getContainerName())
+    ciBuilder.setContainerID(info.getContainerID())
         .setFinalhash("e16cc9d6024365750ed8dbd194ea46d2")
         .setSize(size)
         .setUsed(used)

+ 4 - 5
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.TestUtils;
 import org.apache.hadoop.hdds.scm.XceiverClientManager;
 import org.apache.hadoop.hdds.scm.container.ContainerMapping;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.container.placement.algorithms
     .ContainerPlacementPolicy;
@@ -160,13 +161,11 @@ public class TestContainerPlacement {
 
       assertTrue(nodeManager.isOutOfChillMode());
 
-      String container1 = UUID.randomUUID().toString();
-      Pipeline pipeline1 = containerManager.allocateContainer(
+      ContainerInfo containerInfo = containerManager.allocateContainer(
           xceiverClientManager.getType(),
-          xceiverClientManager.getFactor(), container1, "OZONE")
-          .getPipeline();
+          xceiverClientManager.getFactor(), "OZONE");
       assertEquals(xceiverClientManager.getFactor().getNumber(),
-          pipeline1.getMachines().size());
+          containerInfo.getPipeline().getMachines().size());
     } finally {
       IOUtils.closeQuietly(containerManager);
       IOUtils.closeQuietly(nodeManager);

+ 4 - 3
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java

@@ -17,6 +17,7 @@
 package org.apache.hadoop.ozone.container.common;
 
 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.scm.TestUtils;
@@ -362,7 +363,7 @@ public class TestEndPoint {
    * @return
    */
   ContainerReport getRandomContainerReport() {
-    return new ContainerReport(UUID.randomUUID().toString(),
+    return new ContainerReport(RandomUtils.nextLong(),
         DigestUtils.sha256Hex("Random"));
   }
 
@@ -436,7 +437,8 @@ public class TestEndPoint {
         reportsBuilder = StorageContainerDatanodeProtocolProtos
         .ContainerReportsRequestProto.newBuilder();
     for (int x = 0; x < count; x++) {
-      ContainerReport report = new ContainerReport(UUID.randomUUID().toString(),
+      long containerID = RandomUtils.nextLong();
+      ContainerReport report = new ContainerReport(containerID,
             DigestUtils.sha256Hex("Simulated"));
       report.setKeyCount(1000);
       report.setSize(OzoneConsts.GB * 5);
@@ -445,7 +447,6 @@ public class TestEndPoint {
       report.setReadBytes(OzoneConsts.GB * 1);
       report.setWriteCount(50);
       report.setWriteBytes(OzoneConsts.GB * 2);
-      report.setContainerID(1);
 
       reportsBuilder.addReports(report.getProtoBufMessage());
     }

+ 17 - 15
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerSupervisor.java

@@ -149,17 +149,17 @@ public class TestContainerSupervisor {
    */
   public void testDetectSingleContainerReplica() throws TimeoutException,
       InterruptedException {
-    String singleNodeContainer = "SingleNodeContainer";
-    String threeNodeContainer = "ThreeNodeContainer";
+    long singleNodeContainerID = 9001;
+    long threeNodeContainerID = 9003;
     InProgressPool ppool = containerSupervisor.getInProcessPoolList().get(0);
     // Only single datanode reporting that "SingleNodeContainer" exists.
     List<ContainerReportsRequestProto> clist =
-        datanodeStateManager.getContainerReport(singleNodeContainer,
+        datanodeStateManager.getContainerReport(singleNodeContainerID,
             ppool.getPool().getPoolName(), 1);
     ppool.handleContainerReport(clist.get(0));
 
     // Three nodes are going to report that ThreeNodeContainer  exists.
-    clist = datanodeStateManager.getContainerReport(threeNodeContainer,
+    clist = datanodeStateManager.getContainerReport(threeNodeContainerID,
         ppool.getPool().getPoolName(), 3);
 
     for (ContainerReportsRequestProto reportsProto : clist) {
@@ -169,9 +169,10 @@ public class TestContainerSupervisor {
         200, 1000);
     ppool.setDoneProcessing();
 
-    List<Map.Entry<String, Integer>> containers = ppool.filterContainer(p -> p
+    List<Map.Entry<Long, Integer>> containers = ppool.filterContainer(p -> p
         .getValue() == 1);
-    Assert.assertEquals(singleNodeContainer, containers.get(0).getKey());
+    Assert.assertEquals(singleNodeContainerID,
+        containers.get(0).getKey().longValue());
     int count = containers.get(0).getValue();
     Assert.assertEquals(1L, count);
   }
@@ -184,24 +185,24 @@ public class TestContainerSupervisor {
    */
   public void testDetectOverReplica() throws TimeoutException,
       InterruptedException {
-    String normalContainer = "NormalContainer";
-    String overReplicated = "OverReplicatedContainer";
-    String wayOverReplicated = "WayOverReplicated";
+    long normalContainerID = 9000;
+    long overReplicatedContainerID = 9001;
+    long wayOverReplicatedContainerID = 9002;
     InProgressPool ppool = containerSupervisor.getInProcessPoolList().get(0);
 
     List<ContainerReportsRequestProto> clist =
-        datanodeStateManager.getContainerReport(normalContainer,
+        datanodeStateManager.getContainerReport(normalContainerID,
             ppool.getPool().getPoolName(), 3);
     ppool.handleContainerReport(clist.get(0));
 
-    clist = datanodeStateManager.getContainerReport(overReplicated,
+    clist = datanodeStateManager.getContainerReport(overReplicatedContainerID,
         ppool.getPool().getPoolName(), 4);
 
     for (ContainerReportsRequestProto reportsProto : clist) {
       ppool.handleContainerReport(reportsProto);
     }
 
-    clist = datanodeStateManager.getContainerReport(wayOverReplicated,
+    clist = datanodeStateManager.getContainerReport(wayOverReplicatedContainerID,
         ppool.getPool().getPoolName(), 7);
 
     for (ContainerReportsRequestProto reportsProto : clist) {
@@ -215,7 +216,7 @@ public class TestContainerSupervisor {
         200, 1000);
     ppool.setDoneProcessing();
 
-    List<Map.Entry<String, Integer>> containers = ppool.filterContainer(p -> p
+    List<Map.Entry<Long, Integer>> containers = ppool.filterContainer(p -> p
         .getValue() > 3);
     Assert.assertEquals(2, containers.size());
   }
@@ -255,14 +256,15 @@ public class TestContainerSupervisor {
               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("NewContainer1",
+          datanodeStateManager.getContainerReport(newContainerID,
               "PoolNew", 1);
       containerSupervisor.handleContainerReport(clist.get(0));
       GenericTestUtils.waitFor(() ->
-          inProgressLog.getOutput().contains("NewContainer1") && inProgressLog
+          inProgressLog.getOutput().contains(Long.toString(newContainerID)) && inProgressLog
               .getOutput().contains(id.getUuidString()),
           200, 10 * 1000);
     } finally {

+ 6 - 5
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationDatanodeStateManager.java

@@ -16,6 +16,7 @@
  */
 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;
@@ -56,13 +57,13 @@ public class ReplicationDatanodeStateManager {
 
   /**
    * Get Container Report as if it is from a datanode in the cluster.
-   * @param containerName - Container Name.
+   * @param containerID - Container ID.
    * @param poolName - Pool Name.
    * @param dataNodeCount - Datanode Count.
    * @return List of Container Reports.
    */
   public List<ContainerReportsRequestProto> getContainerReport(
-      String containerName, String poolName, int dataNodeCount) {
+      long containerID, String poolName, int dataNodeCount) {
     List<ContainerReportsRequestProto> containerList = new LinkedList<>();
     List<DatanodeDetails> nodesInPool = poolManager.getNodes(poolName);
 
@@ -75,7 +76,6 @@ public class ReplicationDatanodeStateManager {
           "required container reports");
     }
 
-    int containerID = 1;
     while (containerList.size() < dataNodeCount && nodesInPool.size() > 0) {
       DatanodeDetails id = nodesInPool.get(r.nextInt(nodesInPool.size()));
       nodesInPool.remove(id);
@@ -83,8 +83,9 @@ public class ReplicationDatanodeStateManager {
       // We return container reports only for nodes that are healthy.
       if (nodeManager.getNodeState(id) == HEALTHY) {
         ContainerInfo info = ContainerInfo.newBuilder()
-            .setContainerName(containerName)
-            .setFinalhash(DigestUtils.sha256Hex(containerName))
+            .setContainerID(containerID)
+            .setFinalhash(DigestUtils.sha256Hex(
+                Longs.toByteArray(containerID)))
             .setContainerID(containerID)
             .build();
         ContainerReportsRequestProto containerReport =

+ 14 - 12
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java

@@ -24,7 +24,7 @@ import org.apache.commons.cli.Options;
 import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler;
 import org.apache.hadoop.hdds.scm.cli.SCMCLI;
 import org.apache.hadoop.hdds.scm.client.ScmClient;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 
 import java.io.IOException;
 
@@ -34,30 +34,32 @@ import java.io.IOException;
 public class CloseContainerHandler extends OzoneCommandHandler {
 
   public static final String CONTAINER_CLOSE = "close";
-  public static final String OPT_CONTAINER_NAME = "c";
+  public static final String OPT_CONTAINER_ID = "c";
 
   @Override
   public void execute(CommandLine cmd) throws IOException {
     if (!cmd.hasOption(CONTAINER_CLOSE)) {
       throw new IOException("Expecting container close");
     }
-    if (!cmd.hasOption(OPT_CONTAINER_NAME)) {
+    if (!cmd.hasOption(OPT_CONTAINER_ID)) {
       displayHelp();
       if (!cmd.hasOption(SCMCLI.HELP_OP)) {
-        throw new IOException("Expecting container name");
+        throw new IOException("Expecting container id");
       } else {
         return;
       }
     }
-    String containerName = cmd.getOptionValue(OPT_CONTAINER_NAME);
+    String containerID = cmd.getOptionValue(OPT_CONTAINER_ID);
 
-    Pipeline pipeline = getScmClient().getContainer(containerName);
-    if (pipeline == null) {
+    ContainerInfo container = getScmClient().
+        getContainer(Long.parseLong(containerID));
+    if (container == null) {
       throw new IOException("Cannot close an non-exist container "
-          + containerName);
+          + containerID);
     }
-    logOut("Closing container : %s.", containerName);
-    getScmClient().closeContainer(pipeline);
+    logOut("Closing container : %s.", containerID);
+    getScmClient().closeContainer(container.getContainerID(),
+        container.getPipeline());
     logOut("Container closed.");
   }
 
@@ -72,8 +74,8 @@ public class CloseContainerHandler extends OzoneCommandHandler {
   }
 
   public static void addOptions(Options options) {
-    Option containerNameOpt = new Option(OPT_CONTAINER_NAME,
-        true, "Specify container name");
+    Option containerNameOpt = new Option(OPT_CONTAINER_ID,
+        true, "Specify container ID");
     options.addOption(containerNameOpt);
   }
 

+ 0 - 1
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommandHandler.java

@@ -119,7 +119,6 @@ public class ContainerCommandHandler extends OzoneCommandHandler {
   public static void addOptions(Options options) {
     addCommandsOption(options);
     // for create container options.
-    CreateContainerHandler.addOptions(options);
     DeleteContainerHandler.addOptions(options);
     InfoContainerHandler.addOptions(options);
     ListContainerHandler.addOptions(options);

+ 2 - 20
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateContainerHandler.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.hdds.scm.cli.container;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler;
 import org.apache.hadoop.hdds.scm.client.ScmClient;
@@ -35,7 +34,6 @@ import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP;
 public class CreateContainerHandler extends OzoneCommandHandler {
 
   public static final String CONTAINER_CREATE = "create";
-  public static final String OPT_CONTAINER_NAME = "c";
   public static final String CONTAINER_OWNER = "OZONE";
   // TODO Support an optional -p <pipelineID> option to create
   // container on given datanodes.
@@ -49,33 +47,17 @@ public class CreateContainerHandler extends OzoneCommandHandler {
     if (!cmd.hasOption(CONTAINER_CREATE)) {
       throw new IOException("Expecting container create");
     }
-    if (!cmd.hasOption(OPT_CONTAINER_NAME)) {
-      displayHelp();
-      if (!cmd.hasOption(HELP_OP)) {
-        throw new IOException("Expecting container name");
-      } else {
-        return;
-      }
-    }
-    String containerName = cmd.getOptionValue(OPT_CONTAINER_NAME);
 
-    logOut("Creating container : %s.", containerName);
-    getScmClient().createContainer(containerName, CONTAINER_OWNER);
+    logOut("Creating container...");
+    getScmClient().createContainer(CONTAINER_OWNER);
     logOut("Container created.");
   }
 
   @Override
   public void displayHelp() {
     Options options = new Options();
-    addOptions(options);
     HelpFormatter helpFormatter = new HelpFormatter();
     helpFormatter.printHelp(CMD_WIDTH, "hdfs scm -container -create <option>",
         "where <option> is", options, "");
   }
-
-  public static void addOptions(Options options) {
-    Option containerNameOpt = new Option(OPT_CONTAINER_NAME,
-        true, "Specify container name");
-    options.addOption(containerNameOpt);
-  }
 }

+ 14 - 12
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java

@@ -25,7 +25,7 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler;
 import org.apache.hadoop.hdds.scm.client.ScmClient;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 
 import java.io.IOException;
 
@@ -39,7 +39,7 @@ public class DeleteContainerHandler extends OzoneCommandHandler {
 
   protected static final String CONTAINER_DELETE = "delete";
   protected static final String OPT_FORCE = "f";
-  protected static final String OPT_CONTAINER_NAME = "c";
+  protected static final String OPT_CONTAINER_ID = "c";
 
   public DeleteContainerHandler(ScmClient scmClient) {
     super(scmClient);
@@ -49,7 +49,7 @@ public class DeleteContainerHandler extends OzoneCommandHandler {
   public void execute(CommandLine cmd) throws IOException {
     Preconditions.checkArgument(cmd.hasOption(CONTAINER_DELETE),
         "Expecting command delete");
-    if (!cmd.hasOption(OPT_CONTAINER_NAME)) {
+    if (!cmd.hasOption(OPT_CONTAINER_ID)) {
       displayHelp();
       if (!cmd.hasOption(HELP_OP)) {
         throw new IOException("Expecting container name");
@@ -58,17 +58,19 @@ public class DeleteContainerHandler extends OzoneCommandHandler {
       }
     }
 
-    String containerName = cmd.getOptionValue(OPT_CONTAINER_NAME);
+    String containerID = cmd.getOptionValue(OPT_CONTAINER_ID);
 
-    Pipeline pipeline = getScmClient().getContainer(containerName);
-    if (pipeline == null) {
+    ContainerInfo container = getScmClient().getContainer(
+        Long.parseLong(containerID));
+    if (container == null) {
       throw new IOException("Cannot delete an non-exist container "
-          + containerName);
+          + containerID);
     }
 
-    logOut("Deleting container : %s.", containerName);
-    getScmClient().deleteContainer(pipeline, cmd.hasOption(OPT_FORCE));
-    logOut("Container %s deleted.", containerName);
+    logOut("Deleting container : %s.", containerID);
+    getScmClient().deleteContainer(container.getContainerID(),
+        container.getPipeline(), cmd.hasOption(OPT_FORCE));
+    logOut("Container %s deleted.", containerID);
   }
 
   @Override
@@ -85,8 +87,8 @@ public class DeleteContainerHandler extends OzoneCommandHandler {
         false,
         "forcibly delete a container");
     options.addOption(forceOpt);
-    Option containerNameOpt = new Option(OPT_CONTAINER_NAME,
-        true, "Specify container name");
+    Option containerNameOpt = new Option(OPT_CONTAINER_ID,
+        true, "Specify container id");
     options.addOption(containerNameOpt);
   }
 }

+ 17 - 14
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java

@@ -24,7 +24,7 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler;
 import org.apache.hadoop.hdds.scm.client.ScmClient;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.ContainerData;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -41,7 +41,7 @@ import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP;
 public class InfoContainerHandler extends OzoneCommandHandler {
 
   public static final String CONTAINER_INFO = "info";
-  protected static final String OPT_CONTAINER_NAME = "c";
+  protected static final String OPT_CONTAINER_ID = "c";
 
   /**
    * Constructs a handler object.
@@ -57,7 +57,7 @@ public class InfoContainerHandler extends OzoneCommandHandler {
     if (!cmd.hasOption(CONTAINER_INFO)) {
       throw new IOException("Expecting container info");
     }
-    if (!cmd.hasOption(OPT_CONTAINER_NAME)) {
+    if (!cmd.hasOption(OPT_CONTAINER_ID)) {
       displayHelp();
       if (!cmd.hasOption(HELP_OP)) {
         throw new IOException("Expecting container name");
@@ -65,16 +65,17 @@ public class InfoContainerHandler extends OzoneCommandHandler {
         return;
       }
     }
-    String containerName = cmd.getOptionValue(OPT_CONTAINER_NAME);
-    Pipeline pipeline = getScmClient().getContainer(containerName);
-    Preconditions.checkNotNull(pipeline, "Pipeline cannot be null");
+    String containerID = cmd.getOptionValue(OPT_CONTAINER_ID);
+    ContainerInfo container = getScmClient().
+        getContainer(Long.parseLong(containerID));
+    Preconditions.checkNotNull(container, "Container cannot be null");
 
     ContainerData containerData =
-        getScmClient().readContainer(pipeline);
+        getScmClient().readContainer(container.getContainerID(),
+            container.getPipeline());
 
     // Print container report info.
-    logOut("Container Name: %s",
-        containerData.getName());
+    logOut("Container id: %s", containerID);
     String openStatus =
         containerData.getState() == HddsProtos.LifeCycleState.OPEN ? "OPEN" :
             "CLOSED";
@@ -91,8 +92,10 @@ public class InfoContainerHandler extends OzoneCommandHandler {
     logOut("Container Metadata: {%s}", metadataStr);
 
     // Print pipeline of an existing container.
-    logOut("LeaderID: %s", pipeline.getLeader().getHostName());
-    String machinesStr = pipeline.getMachines().stream().map(
+    logOut("LeaderID: %s", container.getPipeline()
+        .getLeader().getHostName());
+    String machinesStr = container.getPipeline()
+        .getMachines().stream().map(
         DatanodeDetails::getHostName).collect(Collectors.joining(","));
     logOut("Datanodes: [%s]", machinesStr);
   }
@@ -107,8 +110,8 @@ public class InfoContainerHandler extends OzoneCommandHandler {
   }
 
   public static void addOptions(Options options) {
-    Option containerNameOpt = new Option(OPT_CONTAINER_NAME,
-        true, "Specify container name");
-    options.addOption(containerNameOpt);
+    Option containerIdOpt = new Option(OPT_CONTAINER_ID,
+        true, "Specify container id");
+    options.addOption(containerIdOpt);
   }
 }

+ 4 - 8
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListContainerHandler.java

@@ -40,7 +40,6 @@ public class ListContainerHandler extends OzoneCommandHandler {
 
   public static final String CONTAINER_LIST = "list";
   public static final String OPT_START_CONTAINER = "start";
-  public static final String OPT_PREFIX_CONTAINER = "prefix";
   public static final String OPT_COUNT = "count";
 
   /**
@@ -71,8 +70,7 @@ public class ListContainerHandler extends OzoneCommandHandler {
       }
     }
 
-    String startName = cmd.getOptionValue(OPT_START_CONTAINER);
-    String prefixName = cmd.getOptionValue(OPT_PREFIX_CONTAINER);
+    String startID = cmd.getOptionValue(OPT_START_CONTAINER);
     int count = 0;
 
     if (cmd.hasOption(OPT_COUNT)) {
@@ -84,7 +82,8 @@ public class ListContainerHandler extends OzoneCommandHandler {
     }
 
     List<ContainerInfo> containerList =
-        getScmClient().listContainer(startName, prefixName, count);
+        getScmClient().listContainer(
+            Long.parseLong(startID), count);
 
     // Output data list
     for (ContainerInfo container : containerList) {
@@ -109,13 +108,10 @@ public class ListContainerHandler extends OzoneCommandHandler {
 
   public static void addOptions(Options options) {
     Option startContainerOpt = new Option(OPT_START_CONTAINER,
-        true, "Specify start container name");
-    Option endContainerOpt = new Option(OPT_PREFIX_CONTAINER,
-        true, "Specify prefix container name");
+        true, "Specify start container id");
     Option countOpt = new Option(OPT_COUNT, true,
         "Specify count number, required");
     options.addOption(countOpt);
     options.addOption(startContainerOpt);
-    options.addOption(endContainerOpt);
   }
 }

+ 13 - 12
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java

@@ -21,11 +21,12 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.fs.Seekable;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
+import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
 import org.apache.hadoop.hdds.scm.XceiverClientManager;
 import org.apache.hadoop.hdds.scm.XceiverClientSpi;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdds.scm.storage.ChunkInputStream;
 import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls;
@@ -260,7 +261,7 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
           storageContainerLocationClient, String requestId)
       throws IOException {
     long length = 0;
-    String containerKey;
+    long containerKey;
     ChunkGroupInputStream groupInputStream = new ChunkGroupInputStream();
     groupInputStream.key = keyInfo.getKeyName();
     List<KsmKeyLocationInfo> keyLocationInfos =
@@ -268,20 +269,20 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
     groupInputStream.streamOffset = new long[keyLocationInfos.size()];
     for (int i = 0; i < keyLocationInfos.size(); i++) {
       KsmKeyLocationInfo ksmKeyLocationInfo = keyLocationInfos.get(i);
-      String containerName = ksmKeyLocationInfo.getContainerName();
-      Pipeline pipeline =
-          storageContainerLocationClient.getContainer(containerName);
+      BlockID blockID = ksmKeyLocationInfo.getBlockID();
+      long containerID = blockID.getContainerID();
+      ContainerInfo container =
+          storageContainerLocationClient.getContainer(containerID);
       XceiverClientSpi xceiverClient =
-          xceiverClientManager.acquireClient(pipeline);
+          xceiverClientManager.acquireClient(container.getPipeline(), containerID);
       boolean success = false;
-      containerKey = ksmKeyLocationInfo.getBlockID();
+      containerKey = ksmKeyLocationInfo.getLocalID();
       try {
         LOG.debug("get key accessing {} {}",
-            xceiverClient.getPipeline().getContainerName(), containerKey);
+            containerID, containerKey);
         groupInputStream.streamOffset[i] = length;
-        ContainerProtos.KeyData containerKeyData = OzoneContainerTranslation
-            .containerKeyDataForRead(
-                xceiverClient.getPipeline().getContainerName(), containerKey);
+          ContainerProtos.KeyData containerKeyData = OzoneContainerTranslation
+            .containerKeyDataForRead(blockID);
         ContainerProtos.GetKeyResponseProto response = ContainerProtocolCalls
             .getKey(xceiverClient, containerKeyData, requestId);
         List<ContainerProtos.ChunkInfo> chunks =
@@ -291,7 +292,7 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
         }
         success = true;
         ChunkInputStream inputStream = new ChunkInputStream(
-            containerKey, xceiverClientManager, xceiverClient,
+            ksmKeyLocationInfo.getBlockID(), xceiverClientManager, xceiverClient,
             chunks, requestId);
         groupInputStream.addStream(inputStream,
             ksmKeyLocationInfo.getLength());

+ 20 - 15
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java

@@ -21,6 +21,8 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.Result;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
+import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfoGroup;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
@@ -32,7 +34,6 @@ import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession;
 import org.apache.hadoop.ozone.ksm.protocolPB.KeySpaceManagerProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdds.scm.XceiverClientManager;
 import org.apache.hadoop.hdds.scm.XceiverClientSpi;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.hdds.scm.protocolPB
@@ -162,30 +163,34 @@ public class ChunkGroupOutputStream extends OutputStream {
 
   private void checkKeyLocationInfo(KsmKeyLocationInfo subKeyInfo)
       throws IOException {
-    String containerKey = subKeyInfo.getBlockID();
-    String containerName = subKeyInfo.getContainerName();
-    Pipeline pipeline = scmClient.getContainer(containerName);
+    ContainerInfo container = scmClient.getContainer(
+        subKeyInfo.getContainerID());
     XceiverClientSpi xceiverClient =
-        xceiverClientManager.acquireClient(pipeline);
+        xceiverClientManager.acquireClient(container.getPipeline(),
+            container.getContainerID());
     // create container if needed
     if (subKeyInfo.getShouldCreateContainer()) {
       try {
-        ContainerProtocolCalls.createContainer(xceiverClient, requestID);
+        ContainerProtocolCalls.createContainer(xceiverClient,
+            container.getContainerID(), requestID);
         scmClient.notifyObjectStageChange(
             ObjectStageChangeRequestProto.Type.container,
-            containerName, ObjectStageChangeRequestProto.Op.create,
+            subKeyInfo.getContainerID(),
+            ObjectStageChangeRequestProto.Op.create,
             ObjectStageChangeRequestProto.Stage.complete);
       } catch (StorageContainerException ex) {
         if (ex.getResult().equals(Result.CONTAINER_EXISTS)) {
           //container already exist, this should never happen
-          LOG.debug("Container {} already exists.", containerName);
+          LOG.debug("Container {} already exists.",
+              container.getContainerID());
         } else {
-          LOG.error("Container creation failed for {}.", containerName, ex);
+          LOG.error("Container creation failed for {}.",
+              container.getContainerID(), ex);
           throw ex;
         }
       }
     }
-    streamEntries.add(new ChunkOutputStreamEntry(containerKey,
+    streamEntries.add(new ChunkOutputStreamEntry(subKeyInfo.getBlockID(),
         keyArgs.getKeyName(), xceiverClientManager, xceiverClient, requestID,
         chunkSize, subKeyInfo.getLength()));
   }
@@ -390,7 +395,7 @@ public class ChunkGroupOutputStream extends OutputStream {
 
   private static class ChunkOutputStreamEntry extends OutputStream {
     private OutputStream outputStream;
-    private final String containerKey;
+    private final BlockID blockID;
     private final String key;
     private final XceiverClientManager xceiverClientManager;
     private final XceiverClientSpi xceiverClient;
@@ -401,12 +406,12 @@ public class ChunkGroupOutputStream extends OutputStream {
     // the current position of this stream 0 <= currentPosition < length
     private long currentPosition;
 
-    ChunkOutputStreamEntry(String containerKey, String key,
+    ChunkOutputStreamEntry(BlockID blockID, String key,
         XceiverClientManager xceiverClientManager,
         XceiverClientSpi xceiverClient, String requestId, int chunkSize,
         long length) {
       this.outputStream = null;
-      this.containerKey = containerKey;
+      this.blockID = blockID;
       this.key = key;
       this.xceiverClientManager = xceiverClientManager;
       this.xceiverClient = xceiverClient;
@@ -424,7 +429,7 @@ public class ChunkGroupOutputStream extends OutputStream {
      */
     ChunkOutputStreamEntry(OutputStream outputStream, long length) {
       this.outputStream = outputStream;
-      this.containerKey = null;
+      this.blockID = null;
       this.key = null;
       this.xceiverClientManager = null;
       this.xceiverClient = null;
@@ -445,7 +450,7 @@ public class ChunkGroupOutputStream extends OutputStream {
 
     private synchronized void checkStream() {
       if (this.outputStream == null) {
-        this.outputStream = new ChunkOutputStream(containerKey,
+        this.outputStream = new ChunkOutputStream(blockID,
             key, xceiverClientManager, xceiverClient,
             requestId, chunkSize);
       }

+ 5 - 6
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneContainerTranslation.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.ozone.client.io;
 
 
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.KeyData;
+import org.apache.hadoop.hdds.client.BlockID;
+
 
 /**
  * This class contains methods that define the translation between the Ozone
@@ -30,16 +32,13 @@ final class OzoneContainerTranslation {
   /**
    * Creates key data intended for reading a container key.
    *
-   * @param containerName container name
-   * @param containerKey container key
+   * @param blockID - ID of the block.
    * @return KeyData intended for reading the container key
    */
-  public static KeyData containerKeyDataForRead(String containerName,
-      String containerKey) {
+  public static KeyData containerKeyDataForRead(BlockID blockID) {
     return KeyData
         .newBuilder()
-        .setContainerName(containerName)
-        .setName(containerKey)
+        .setBlockID(blockID.getProtobuf())
         .build();
   }
 

+ 17 - 24
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfo.java

@@ -16,6 +16,7 @@
  */
 package org.apache.hadoop.ozone.ksm.helpers;
 
+import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyLocation;
 
 /**
@@ -23,9 +24,7 @@ import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyL
  * into a number of subkeys. This class represents one such subkey instance.
  */
 public final class KsmKeyLocationInfo {
-  private final String containerName;
-  // name of the block id SCM assigned for the key
-  private final String blockID;
+  private final BlockID blockID;
   private final boolean shouldCreateContainer;
   // the id of this subkey in all the subkeys.
   private final long length;
@@ -33,10 +32,8 @@ public final class KsmKeyLocationInfo {
   // the version number indicating when this block was added
   private long createVersion;
 
-  private KsmKeyLocationInfo(String containerName,
-      String blockID, boolean shouldCreateContainer,
+  private KsmKeyLocationInfo(BlockID blockID, boolean shouldCreateContainer,
       long length, long offset) {
-    this.containerName = containerName;
     this.blockID = blockID;
     this.shouldCreateContainer = shouldCreateContainer;
     this.length = length;
@@ -51,12 +48,16 @@ public final class KsmKeyLocationInfo {
     return createVersion;
   }
 
-  public String getContainerName() {
-    return containerName;
+  public BlockID getBlockID() {
+    return blockID;
   }
 
-  public String getBlockID() {
-    return blockID;
+  public long getContainerID() {
+    return blockID.getContainerID();
+  }
+
+  public long getLocalID() {
+    return blockID.getLocalID();
   }
 
   public boolean getShouldCreateContainer() {
@@ -75,19 +76,13 @@ public final class KsmKeyLocationInfo {
    * Builder of KsmKeyLocationInfo.
    */
   public static class Builder {
-    private String containerName;
-    private String blockID;
+    private BlockID blockID;
     private boolean shouldCreateContainer;
     private long length;
     private long offset;
 
-    public Builder setContainerName(String container) {
-      this.containerName = container;
-      return this;
-    }
-
-    public Builder setBlockID(String block) {
-      this.blockID = block;
+    public Builder setBlockID(BlockID blockId) {
+      this.blockID = blockId;
       return this;
     }
 
@@ -107,15 +102,14 @@ public final class KsmKeyLocationInfo {
     }
 
     public KsmKeyLocationInfo build() {
-      return new KsmKeyLocationInfo(containerName, blockID,
+      return new KsmKeyLocationInfo(blockID,
           shouldCreateContainer, length, offset);
     }
   }
 
   public KeyLocation getProtobuf() {
     return KeyLocation.newBuilder()
-        .setContainerName(containerName)
-        .setBlockID(blockID)
+        .setBlockID(blockID.getProtobuf())
         .setShouldCreateContainer(shouldCreateContainer)
         .setLength(length)
         .setOffset(offset)
@@ -125,8 +119,7 @@ public final class KsmKeyLocationInfo {
 
   public static KsmKeyLocationInfo getFromProtobuf(KeyLocation keyLocation) {
     KsmKeyLocationInfo info = new KsmKeyLocationInfo(
-        keyLocation.getContainerName(),
-        keyLocation.getBlockID(),
+        BlockID.getFromProtobuf(keyLocation.getBlockID()),
         keyLocation.getShouldCreateContainer(),
         keyLocation.getLength(),
         keyLocation.getOffset());

+ 1 - 1
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyLocationInfoGroup.java

@@ -111,7 +111,7 @@ public class KsmKeyLocationInfoGroup {
     StringBuilder sb = new StringBuilder();
     sb.append("version:").append(version).append(" ");
     for (KsmKeyLocationInfo kli : locationList) {
-      sb.append(kli.getBlockID()).append(" || ");
+      sb.append(kli.getLocalID()).append(" || ");
     }
     return sb.toString();
   }

+ 6 - 2
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/handlers/UserArgs.java

@@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.Request;
 import javax.ws.rs.core.UriInfo;
+import java.util.Arrays;
 
 /**
  * UserArgs is used to package caller info
@@ -118,7 +119,8 @@ public class UserArgs {
    * @return String[]
    */
   public String[] getGroups() {
-    return this.groups;
+    return groups != null ?
+        Arrays.copyOf(groups, groups.length) : null;
   }
 
   /**
@@ -127,7 +129,9 @@ public class UserArgs {
    * @param groups list of groups
    */
   public void setGroups(String[] groups) {
-    this.groups = groups;
+    if (groups != null) {
+      this.groups = Arrays.copyOf(groups, groups.length);
+    }
   }
 
   /**

+ 5 - 6
hadoop-ozone/common/src/main/proto/KeySpaceManagerProtocol.proto

@@ -230,13 +230,12 @@ message KeyArgs {
 }
 
 message KeyLocation {
-    required string blockID = 1;
-    required string containerName = 2;
-    required bool shouldCreateContainer = 3;
-    required uint64 offset = 4;
-    required uint64 length = 5;
+    required hadoop.hdds.BlockID blockID = 1;
+    required bool shouldCreateContainer = 2;
+    required uint64 offset = 3;
+    required uint64 length = 4;
     // indicated at which version this block gets created.
-    optional uint64 createVersion = 6;
+    optional uint64 createVersion = 5;
 }
 
 message KeyLocationList {

+ 64 - 60
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManager.java

@@ -16,7 +16,7 @@
  */
 package org.apache.hadoop.hdds.scm.container;
 
-import org.apache.commons.lang.RandomStringUtils;
+import com.google.common.primitives.Longs;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -31,6 +31,8 @@ import org.junit.Test;
 
 import java.io.IOException;
 import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.NavigableSet;
 import java.util.Random;
 
@@ -69,15 +71,14 @@ public class TestContainerStateManager {
   @Test
   public void testAllocateContainer() throws IOException {
     // Allocate a container and verify the container info
-    String container1 = "container" + RandomStringUtils.randomNumeric(5);
-    scm.getClientProtocolServer().allocateContainer(
+    ContainerInfo container1 = scm.getClientProtocolServer().allocateContainer(
         xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), container1, containerOwner);
+        xceiverClientManager.getFactor(), containerOwner);
     ContainerInfo info = containerStateManager
         .getMatchingContainer(OzoneConsts.GB * 3, containerOwner,
             xceiverClientManager.getType(), xceiverClientManager.getFactor(),
             HddsProtos.LifeCycleState.ALLOCATED);
-    Assert.assertEquals(container1, info.getContainerName());
+    Assert.assertEquals(container1.getContainerID(), info.getContainerID());
     Assert.assertEquals(OzoneConsts.GB * 3, info.getAllocatedBytes());
     Assert.assertEquals(containerOwner, info.getOwner());
     Assert.assertEquals(xceiverClientManager.getType(),
@@ -87,28 +88,31 @@ public class TestContainerStateManager {
     Assert.assertEquals(HddsProtos.LifeCycleState.ALLOCATED, info.getState());
 
     // Check there are two containers in ALLOCATED state after allocation
-    String container2 = "container" + RandomStringUtils.randomNumeric(5);
-    scm.getClientProtocolServer().allocateContainer(
+    ContainerInfo container2 = scm.getClientProtocolServer().allocateContainer(
         xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), container2, containerOwner);
+        xceiverClientManager.getFactor(), containerOwner);
     int numContainers = containerStateManager
         .getMatchingContainerIDs(containerOwner,
             xceiverClientManager.getType(), xceiverClientManager.getFactor(),
             HddsProtos.LifeCycleState.ALLOCATED).size();
+    Assert.assertNotEquals(container1.getContainerID(), container2.getContainerID());
     Assert.assertEquals(2, numContainers);
   }
 
   @Test
   public void testContainerStateManagerRestart() throws IOException {
     // Allocate 5 containers in ALLOCATED state and 5 in CREATING state
-    String cname = "container" + RandomStringUtils.randomNumeric(5);
+
+    List<ContainerInfo> containers = new ArrayList<>();
     for (int i = 0; i < 10; i++) {
-      scm.getClientProtocolServer().allocateContainer(
+      ContainerInfo container = scm.getClientProtocolServer().allocateContainer(
           xceiverClientManager.getType(),
-          xceiverClientManager.getFactor(), cname + i, containerOwner);
+          xceiverClientManager.getFactor(), containerOwner);
+      containers.add(container);
       if (i >= 5) {
         scm.getScmContainerManager()
-            .updateContainerState(cname + i, HddsProtos.LifeCycleEvent.CREATE);
+            .updateContainerState(container.getContainerID(),
+                HddsProtos.LifeCycleEvent.CREATE);
       }
     }
 
@@ -117,48 +121,46 @@ public class TestContainerStateManager {
     ContainerStateManager stateManager =
         new ContainerStateManager(conf, scmContainerMapping
         );
-    int containers = stateManager
+    int matchCount = stateManager
         .getMatchingContainerIDs(containerOwner,
             xceiverClientManager.getType(), xceiverClientManager.getFactor(),
             HddsProtos.LifeCycleState.ALLOCATED).size();
-    Assert.assertEquals(5, containers);
-    containers = stateManager.getMatchingContainerIDs(containerOwner,
+    Assert.assertEquals(5, matchCount);
+    matchCount = stateManager.getMatchingContainerIDs(containerOwner,
         xceiverClientManager.getType(), xceiverClientManager.getFactor(),
         HddsProtos.LifeCycleState.CREATING).size();
-    Assert.assertEquals(5, containers);
+    Assert.assertEquals(5, matchCount);
   }
 
   @Test
   public void testGetMatchingContainer() throws IOException {
-    String container1 = "container-01234";
-    scm.getClientProtocolServer().allocateContainer(
-        xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), container1, containerOwner);
-    scmContainerMapping.updateContainerState(container1,
+    ContainerInfo container1 = scm.getClientProtocolServer().
+        allocateContainer(xceiverClientManager.getType(),
+        xceiverClientManager.getFactor(), containerOwner);
+    scmContainerMapping.updateContainerState(container1.getContainerID(),
         HddsProtos.LifeCycleEvent.CREATE);
-    scmContainerMapping.updateContainerState(container1,
+    scmContainerMapping.updateContainerState(container1.getContainerID(),
         HddsProtos.LifeCycleEvent.CREATED);
 
-    String container2 = "container-56789";
-    scm.getClientProtocolServer().allocateContainer(
-        xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), container2, containerOwner);
+    ContainerInfo container2 = scm.getClientProtocolServer().
+        allocateContainer(xceiverClientManager.getType(),
+        xceiverClientManager.getFactor(), containerOwner);
 
     ContainerInfo info = containerStateManager
         .getMatchingContainer(OzoneConsts.GB * 3, containerOwner,
             xceiverClientManager.getType(), xceiverClientManager.getFactor(),
             HddsProtos.LifeCycleState.OPEN);
-    Assert.assertEquals(container1, info.getContainerName());
+    Assert.assertEquals(container1.getContainerID(), info.getContainerID());
 
     info = containerStateManager
         .getMatchingContainer(OzoneConsts.GB * 3, containerOwner,
             xceiverClientManager.getType(), xceiverClientManager.getFactor(),
             HddsProtos.LifeCycleState.ALLOCATED);
-    Assert.assertEquals(container2, info.getContainerName());
+    Assert.assertEquals(container2.getContainerID(), info.getContainerID());
 
-    scmContainerMapping.updateContainerState(container2,
+    scmContainerMapping.updateContainerState(container2.getContainerID(),
         HddsProtos.LifeCycleEvent.CREATE);
-    scmContainerMapping.updateContainerState(container2,
+    scmContainerMapping.updateContainerState(container2.getContainerID(),
         HddsProtos.LifeCycleEvent.CREATED);
 
     // space has already been allocated in container1, now container 2 should
@@ -167,7 +169,7 @@ public class TestContainerStateManager {
         .getMatchingContainer(OzoneConsts.GB * 3, containerOwner,
             xceiverClientManager.getType(), xceiverClientManager.getFactor(),
             HddsProtos.LifeCycleState.OPEN);
-    Assert.assertEquals(container2, info.getContainerName());
+    Assert.assertEquals(container2.getContainerID(), info.getContainerID());
   }
 
   @Test
@@ -181,23 +183,22 @@ public class TestContainerStateManager {
 
     // Allocate container1 and update its state from ALLOCATED -> CREATING ->
     // OPEN -> CLOSING -> CLOSED -> DELETING -> DELETED
-    String container1 = "container" + RandomStringUtils.randomNumeric(5);
-    scm.getClientProtocolServer().allocateContainer(
+    ContainerInfo container1 = scm.getClientProtocolServer().allocateContainer(
         xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), container1, containerOwner);
+        xceiverClientManager.getFactor(), containerOwner);
     containers = containerStateManager.getMatchingContainerIDs(containerOwner,
         xceiverClientManager.getType(), xceiverClientManager.getFactor(),
         HddsProtos.LifeCycleState.ALLOCATED).size();
     Assert.assertEquals(1, containers);
 
-    scmContainerMapping.updateContainerState(container1,
+    scmContainerMapping.updateContainerState(container1.getContainerID(),
         HddsProtos.LifeCycleEvent.CREATE);
     containers = containerStateManager.getMatchingContainerIDs(containerOwner,
         xceiverClientManager.getType(), xceiverClientManager.getFactor(),
         HddsProtos.LifeCycleState.CREATING).size();
     Assert.assertEquals(1, containers);
 
-    scmContainerMapping.updateContainerState(container1,
+    scmContainerMapping.updateContainerState(container1.getContainerID(),
         HddsProtos.LifeCycleEvent.CREATED);
     containers = containerStateManager.getMatchingContainerIDs(containerOwner,
         xceiverClientManager.getType(), xceiverClientManager.getFactor(),
@@ -205,28 +206,32 @@ public class TestContainerStateManager {
     Assert.assertEquals(1, containers);
 
     scmContainerMapping
-        .updateContainerState(container1, HddsProtos.LifeCycleEvent.FINALIZE);
+        .updateContainerState(container1.getContainerID(),
+            HddsProtos.LifeCycleEvent.FINALIZE);
     containers = containerStateManager.getMatchingContainerIDs(containerOwner,
         xceiverClientManager.getType(), xceiverClientManager.getFactor(),
         HddsProtos.LifeCycleState.CLOSING).size();
     Assert.assertEquals(1, containers);
 
     scmContainerMapping
-        .updateContainerState(container1, HddsProtos.LifeCycleEvent.CLOSE);
+        .updateContainerState(container1.getContainerID(),
+            HddsProtos.LifeCycleEvent.CLOSE);
     containers = containerStateManager.getMatchingContainerIDs(containerOwner,
         xceiverClientManager.getType(), xceiverClientManager.getFactor(),
         HddsProtos.LifeCycleState.CLOSED).size();
     Assert.assertEquals(1, containers);
 
     scmContainerMapping
-        .updateContainerState(container1, HddsProtos.LifeCycleEvent.DELETE);
+        .updateContainerState(container1.getContainerID(),
+            HddsProtos.LifeCycleEvent.DELETE);
     containers = containerStateManager.getMatchingContainerIDs(containerOwner,
         xceiverClientManager.getType(), xceiverClientManager.getFactor(),
         HddsProtos.LifeCycleState.DELETING).size();
     Assert.assertEquals(1, containers);
 
     scmContainerMapping
-        .updateContainerState(container1, HddsProtos.LifeCycleEvent.CLEANUP);
+        .updateContainerState(container1.getContainerID(),
+            HddsProtos.LifeCycleEvent.CLEANUP);
     containers = containerStateManager.getMatchingContainerIDs(containerOwner,
         xceiverClientManager.getType(), xceiverClientManager.getFactor(),
         HddsProtos.LifeCycleState.DELETED).size();
@@ -234,14 +239,14 @@ public class TestContainerStateManager {
 
     // Allocate container1 and update its state from ALLOCATED -> CREATING ->
     // DELETING
-    String container2 = "container" + RandomStringUtils.randomNumeric(5);
-    scm.getClientProtocolServer().allocateContainer(
+    ContainerInfo container2 = scm.getClientProtocolServer().allocateContainer(
         xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), container2, containerOwner);
-    scmContainerMapping.updateContainerState(container2,
+        xceiverClientManager.getFactor(), containerOwner);
+    scmContainerMapping.updateContainerState(container2.getContainerID(),
         HddsProtos.LifeCycleEvent.CREATE);
     scmContainerMapping
-        .updateContainerState(container2, HddsProtos.LifeCycleEvent.TIMEOUT);
+        .updateContainerState(container2.getContainerID(),
+            HddsProtos.LifeCycleEvent.TIMEOUT);
     containers = containerStateManager.getMatchingContainerIDs(containerOwner,
         xceiverClientManager.getType(), xceiverClientManager.getFactor(),
         HddsProtos.LifeCycleState.DELETING).size();
@@ -249,18 +254,18 @@ public class TestContainerStateManager {
 
     // Allocate container1 and update its state from ALLOCATED -> CREATING ->
     // OPEN -> CLOSING -> CLOSED
-    String container3 = "container" + RandomStringUtils.randomNumeric(5);
-    scm.getClientProtocolServer().allocateContainer(
+    ContainerInfo container3 = scm.getClientProtocolServer().allocateContainer(
         xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), container3, containerOwner);
-    scmContainerMapping.updateContainerState(container3,
+        xceiverClientManager.getFactor(), containerOwner);
+    scmContainerMapping.updateContainerState(container3.getContainerID(),
         HddsProtos.LifeCycleEvent.CREATE);
-    scmContainerMapping.updateContainerState(container3,
+    scmContainerMapping.updateContainerState(container3.getContainerID(),
         HddsProtos.LifeCycleEvent.CREATED);
-    scmContainerMapping.updateContainerState(container3,
+    scmContainerMapping.updateContainerState(container3.getContainerID(),
         HddsProtos.LifeCycleEvent.FINALIZE);
     scmContainerMapping
-        .updateContainerState(container3, HddsProtos.LifeCycleEvent.CLOSE);
+        .updateContainerState(container3.getContainerID(),
+            HddsProtos.LifeCycleEvent.CLOSE);
     containers = containerStateManager.getMatchingContainerIDs(containerOwner,
         xceiverClientManager.getType(), xceiverClientManager.getFactor(),
         HddsProtos.LifeCycleState.CLOSED).size();
@@ -269,13 +274,12 @@ public class TestContainerStateManager {
 
   @Test
   public void testUpdatingAllocatedBytes() throws Exception {
-    String container1 = "container" + RandomStringUtils.randomNumeric(5);
-    scm.getClientProtocolServer().allocateContainer(
+    ContainerInfo container1 = scm.getClientProtocolServer().allocateContainer(
         xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), container1, containerOwner);
-    scmContainerMapping.updateContainerState(container1,
+        xceiverClientManager.getFactor(), containerOwner);
+    scmContainerMapping.updateContainerState(container1.getContainerID(),
         HddsProtos.LifeCycleEvent.CREATE);
-    scmContainerMapping.updateContainerState(container1,
+    scmContainerMapping.updateContainerState(container1.getContainerID(),
         HddsProtos.LifeCycleEvent.CREATED);
 
     Random ran = new Random();
@@ -288,7 +292,7 @@ public class TestContainerStateManager {
           .getMatchingContainer(size, containerOwner,
               xceiverClientManager.getType(), xceiverClientManager.getFactor(),
               HddsProtos.LifeCycleState.OPEN);
-      Assert.assertEquals(container1, info.getContainerName());
+      Assert.assertEquals(container1.getContainerID(), info.getContainerID());
 
       ContainerMapping containerMapping =
           (ContainerMapping)scmContainerMapping;
@@ -296,10 +300,10 @@ public class TestContainerStateManager {
       // to disk
       containerMapping.flushContainerInfo();
 
-      Charset utf8 = Charset.forName("UTF-8");
       // the persisted value should always be equal to allocated size.
       byte[] containerBytes =
-          containerMapping.getContainerStore().get(container1.getBytes(utf8));
+          containerMapping.getContainerStore().get(
+              Longs.toByteArray(container1.getContainerID()));
       HddsProtos.SCMContainerInfo infoProto =
           HddsProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes);
       ContainerInfo currentInfo = ContainerInfo.fromProtobuf(infoProto);

+ 6 - 4
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.ozone;
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -77,11 +78,12 @@ public class TestContainerOperations {
    */
   @Test
   public void testCreate() throws Exception {
-    Pipeline pipeline0 = storageClient.createContainer(HddsProtos
+    ContainerInfo container = storageClient.createContainer(HddsProtos
         .ReplicationType.STAND_ALONE, HddsProtos.ReplicationFactor
-        .ONE, "container0", "OZONE");
-    assertEquals("container0", pipeline0.getContainerName());
-
+        .ONE, "OZONE");
+    assertEquals(container.getContainerID(),
+        storageClient.getContainer(container.getContainerID()).
+            getContainerID());
   }
 
 }

+ 1 - 2
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java

@@ -91,7 +91,6 @@ public class TestMiniOzoneCluster {
     assertEquals(numberOfNodes, datanodes.size());
     for(HddsDatanodeService dn : datanodes) {
       // Create a single member pipe line
-      String containerName = OzoneUtils.getRequestID();
       DatanodeDetails datanodeDetails = dn.getDatanodeDetails();
       final PipelineChannel pipelineChannel =
           new PipelineChannel(datanodeDetails.getUuidString(),
@@ -99,7 +98,7 @@ public class TestMiniOzoneCluster {
               HddsProtos.ReplicationType.STAND_ALONE,
               HddsProtos.ReplicationFactor.ONE, "test");
       pipelineChannel.addMember(datanodeDetails);
-      Pipeline pipeline = new Pipeline(containerName, pipelineChannel);
+      Pipeline pipeline = new Pipeline(pipelineChannel);
 
       // Verify client is able to connect to the container
       try (XceiverClient client = new XceiverClient(pipeline, conf)){

+ 33 - 31
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java

@@ -20,10 +20,13 @@ package org.apache.hadoop.ozone;
 import static org.junit.Assert.fail;
 import java.io.IOException;
 
-import org.apache.commons.lang.RandomStringUtils;
+import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
+import org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer;
 import org.apache.hadoop.hdds.scm.server.SCMStorage;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.ozone.container.ContainerTestHelper;
 import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -38,7 +41,6 @@ import org.apache.hadoop.hdds.scm.block.DeletedBlockLog;
 import org.apache.hadoop.hdds.scm.block.SCMBlockDeletingService;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.XceiverClientManager;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.ScmInfo;
 import org.junit.Rule;
 import org.junit.Assert;
@@ -113,7 +115,8 @@ public class TestStorageContainerManager {
           .thenReturn(fakeUser);
 
       try {
-        mockScm.getClientProtocolServer().deleteContainer("container1");
+        mockScm.getClientProtocolServer().deleteContainer(
+            ContainerTestHelper.getTestContainerID());
         fail("Operation should fail, expecting an IOException here.");
       } catch (Exception e) {
         if (expectPermissionDenied) {
@@ -127,35 +130,34 @@ public class TestStorageContainerManager {
       }
 
       try {
-        Pipeline pipeLine2 = mockScm.getClientProtocolServer()
+        ContainerInfo container2 = mockScm.getClientProtocolServer()
             .allocateContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, "container2", "OZONE");
+            HddsProtos.ReplicationFactor.ONE,  "OZONE");
         if (expectPermissionDenied) {
           fail("Operation should fail, expecting an IOException here.");
         } else {
-          Assert.assertEquals("container2", pipeLine2.getContainerName());
+          Assert.assertEquals(1, container2.getPipeline().getMachines().size());
         }
       } catch (Exception e) {
         verifyPermissionDeniedException(e, fakeUser);
       }
 
       try {
-        Pipeline pipeLine3 = mockScm.getClientProtocolServer()
+        ContainerInfo container3 = mockScm.getClientProtocolServer()
             .allocateContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, "container3", "OZONE");
-
+            HddsProtos.ReplicationFactor.ONE, "OZONE");
         if (expectPermissionDenied) {
           fail("Operation should fail, expecting an IOException here.");
         } else {
-          Assert.assertEquals("container3", pipeLine3.getContainerName());
-          Assert.assertEquals(1, pipeLine3.getMachines().size());
+          Assert.assertEquals(1, container3.getPipeline().getMachines().size());
         }
       } catch (Exception e) {
         verifyPermissionDeniedException(e, fakeUser);
       }
 
       try {
-        mockScm.getClientProtocolServer().getContainer("container4");
+        mockScm.getClientProtocolServer().getContainer(
+            ContainerTestHelper.getTestContainerID());
         fail("Operation should fail, expecting an IOException here.");
       } catch (Exception e) {
         if (expectPermissionDenied) {
@@ -210,9 +212,9 @@ public class TestStorageContainerManager {
           new TestStorageContainerManagerHelper(cluster, conf);
       Map<String, KsmKeyInfo> keyLocations = helper.createKeys(numKeys, 4096);
 
-      Map<String, List<String>> containerBlocks = createDeleteTXLog(delLog,
+      Map<Long, List<Long>> containerBlocks = createDeleteTXLog(delLog,
           keyLocations, helper);
-      Set<String> containerNames = containerBlocks.keySet();
+      Set<Long> containerIDs = containerBlocks.keySet();
 
       // Verify a few TX gets created in the TX log.
       Assert.assertTrue(delLog.getNumOfValidTransactions() > 0);
@@ -229,16 +231,16 @@ public class TestStorageContainerManager {
           return false;
         }
       }, 1000, 10000);
-      Assert.assertTrue(helper.getAllBlocks(containerNames).isEmpty());
+      Assert.assertTrue(helper.getAllBlocks(containerIDs).isEmpty());
 
       // Continue the work, add some TXs that with known container names,
       // but unknown block IDs.
-      for (String containerName : containerBlocks.keySet()) {
+      for (Long containerID : containerBlocks.keySet()) {
         // Add 2 TXs per container.
-        delLog.addTransaction(containerName,
-            Collections.singletonList(RandomStringUtils.randomAlphabetic(5)));
-        delLog.addTransaction(containerName,
-            Collections.singletonList(RandomStringUtils.randomAlphabetic(5)));
+        delLog.addTransaction(containerID,
+            Collections.singletonList(RandomUtils.nextLong()));
+        delLog.addTransaction(containerID,
+            Collections.singletonList(RandomUtils.nextLong()));
       }
 
       // Verify a few TX gets created in the TX log.
@@ -319,16 +321,16 @@ public class TestStorageContainerManager {
     }, 500, 10000);
   }
 
-  private Map<String, List<String>> createDeleteTXLog(DeletedBlockLog delLog,
+  private Map<Long, List<Long>> createDeleteTXLog(DeletedBlockLog delLog,
       Map<String, KsmKeyInfo> keyLocations,
       TestStorageContainerManagerHelper helper) throws IOException {
     // These keys will be written into a bunch of containers,
     // gets a set of container names, verify container containerBlocks
     // on datanodes.
-    Set<String> containerNames = new HashSet<>();
+    Set<Long> containerNames = new HashSet<>();
     for (Map.Entry<String, KsmKeyInfo> entry : keyLocations.entrySet()) {
       entry.getValue().getLatestVersionLocations().getLocationList()
-          .forEach(loc -> containerNames.add(loc.getContainerName()));
+          .forEach(loc -> containerNames.add(loc.getContainerID()));
     }
 
     // Total number of containerBlocks of these containers should be equal to
@@ -342,22 +344,22 @@ public class TestStorageContainerManager {
         helper.getAllBlocks(containerNames).size());
 
     // Create a deletion TX for each key.
-    Map<String, List<String>> containerBlocks = Maps.newHashMap();
+    Map<Long, List<Long>> containerBlocks = Maps.newHashMap();
     for (KsmKeyInfo info : keyLocations.values()) {
       List<KsmKeyLocationInfo> list =
           info.getLatestVersionLocations().getLocationList();
       list.forEach(location -> {
-        if (containerBlocks.containsKey(location.getContainerName())) {
-          containerBlocks.get(location.getContainerName())
-              .add(location.getBlockID());
+        if (containerBlocks.containsKey(location.getContainerID())) {
+          containerBlocks.get(location.getContainerID())
+              .add(location.getBlockID().getLocalID());
         } else {
-          List<String> blks = Lists.newArrayList();
-          blks.add(location.getBlockID());
-          containerBlocks.put(location.getContainerName(), blks);
+          List<Long> blks = Lists.newArrayList();
+          blks.add(location.getBlockID().getLocalID());
+          containerBlocks.put(location.getContainerID(), blks);
         }
       });
     }
-    for (Map.Entry<String, List<String>> tx : containerBlocks.entrySet()) {
+    for (Map.Entry<Long, List<Long>> tx : containerBlocks.entrySet()) {
       delLog.addTransaction(tx.getKey(), tx.getValue());
     }
 

+ 17 - 16
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java

@@ -22,6 +22,7 @@ import com.google.common.collect.Sets;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@@ -36,7 +37,6 @@ import org.apache.hadoop.ozone.web.handlers.UserArgs;
 import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
 import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
 import org.apache.hadoop.utils.MetadataStore;
@@ -114,10 +114,10 @@ public class TestStorageContainerManagerHelper {
     return keyLocationMap;
   }
 
-  public List<String> getPendingDeletionBlocks(String containerName)
+  public List<String> getPendingDeletionBlocks(Long containerID)
       throws IOException {
     List<String> pendingDeletionBlocks = Lists.newArrayList();
-    MetadataStore meta = getContainerMetadata(containerName);
+    MetadataStore meta = getContainerMetadata(containerID);
     KeyPrefixFilter filter =
         new KeyPrefixFilter(OzoneConsts.DELETING_KEY_PREFIX);
     List<Map.Entry<byte[], byte[]>> kvs = meta
@@ -130,18 +130,18 @@ public class TestStorageContainerManagerHelper {
     return pendingDeletionBlocks;
   }
 
-  public List<String> getAllBlocks(Set<String> containerNames)
+  public List<Long> getAllBlocks(Set<Long> containerIDs)
       throws IOException {
-    List<String> allBlocks = Lists.newArrayList();
-    for (String containerName : containerNames) {
-      allBlocks.addAll(getAllBlocks(containerName));
+    List<Long> allBlocks = Lists.newArrayList();
+    for (Long containerID : containerIDs) {
+      allBlocks.addAll(getAllBlocks(containerID));
     }
     return allBlocks;
   }
 
-  public List<String> getAllBlocks(String containerName) throws IOException {
-    List<String> allBlocks = Lists.newArrayList();
-    MetadataStore meta = getContainerMetadata(containerName);
+  public List<Long> getAllBlocks(Long containeID) throws IOException {
+    List<Long> allBlocks = Lists.newArrayList();
+    MetadataStore meta = getContainerMetadata(containeID);
     MetadataKeyFilter filter =
         (preKey, currentKey, nextKey) -> !DFSUtil.bytes2String(currentKey)
             .startsWith(OzoneConsts.DELETING_KEY_PREFIX);
@@ -149,20 +149,21 @@ public class TestStorageContainerManagerHelper {
         meta.getRangeKVs(null, Integer.MAX_VALUE, filter);
     kvs.forEach(entry -> {
       String key = DFSUtil.bytes2String(entry.getKey());
-      allBlocks.add(key.replace(OzoneConsts.DELETING_KEY_PREFIX, ""));
+      key.replace(OzoneConsts.DELETING_KEY_PREFIX, "");
+      allBlocks.add(Long.parseLong(key));
     });
     return allBlocks;
   }
 
-  private MetadataStore getContainerMetadata(String containerName)
+  private MetadataStore getContainerMetadata(Long containerID)
       throws IOException {
-    Pipeline pipeline = cluster.getStorageContainerManager()
-        .getClientProtocolServer().getContainer(containerName);
-    DatanodeDetails leadDN = pipeline.getLeader();
+    ContainerInfo container = cluster.getStorageContainerManager()
+        .getClientProtocolServer().getContainer(containerID);
+    DatanodeDetails leadDN = container.getPipeline().getLeader();
     OzoneContainer containerServer =
         getContainerServerByDatanodeUuid(leadDN.getUuidString());
     ContainerData containerData = containerServer.getContainerManager()
-        .readContainer(containerName);
+        .readContainer(containerID);
     return KeyUtils.getDB(containerData, conf);
   }
 

+ 5 - 5
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.client.rpc;
 
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -45,7 +46,6 @@ import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.protocolPB.
     StorageContainerLocationProtocolClientSideTranslatorPB;
 import org.apache.hadoop.util.Time;
@@ -388,10 +388,10 @@ public class TestOzoneRpcClient {
     KsmKeyInfo keyInfo = keySpaceManager.lookupKey(keyArgs);
     for (KsmKeyLocationInfo info:
         keyInfo.getLatestVersionLocations().getLocationList()) {
-      Pipeline pipeline =
-          storageContainerLocationClient.getContainer(info.getContainerName());
-      if ((pipeline.getFactor() != replicationFactor) ||
-          (pipeline.getType() != replicationType)) {
+      ContainerInfo container =
+          storageContainerLocationClient.getContainer(info.getContainerID());
+      if ((container.getPipeline().getFactor() != replicationFactor) ||
+          (container.getPipeline().getType() != replicationType)) {
         return false;
       }
     }

+ 93 - 87
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java

@@ -21,12 +21,14 @@ package org.apache.hadoop.ozone.container;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
 import org.apache.commons.codec.binary.Hex;
+import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos
     .ContainerCommandRequestProto;
 import org.apache.hadoop.hdds.protocol.proto.ContainerProtos
     .ContainerCommandResponseProto;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
@@ -39,6 +41,7 @@ import org.apache.hadoop.ozone.container.common.helpers.KeyData;
 import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineChannel;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Time;
 import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -79,9 +82,9 @@ public final class ContainerTestHelper {
    * @return Pipeline with single node in it.
    * @throws IOException
    */
-  public static Pipeline createSingleNodePipeline(String containerName) throws
+  public static Pipeline createSingleNodePipeline() throws
       IOException {
-    return createPipeline(containerName, 1);
+    return createPipeline(1);
   }
 
   public static String createLocalAddress() throws IOException {
@@ -111,19 +114,18 @@ public final class ContainerTestHelper {
    * @return Pipeline with single node in it.
    * @throws IOException
    */
-  public static Pipeline createPipeline(String containerName, int numNodes)
+  public static Pipeline createPipeline(int numNodes)
       throws IOException {
     Preconditions.checkArgument(numNodes >= 1);
     final List<DatanodeDetails> ids = new ArrayList<>(numNodes);
     for(int i = 0; i < numNodes; i++) {
       ids.add(createDatanodeDetails());
     }
-    return createPipeline(containerName, ids);
+    return createPipeline(ids);
   }
 
   public static Pipeline createPipeline(
-      String containerName, Iterable<DatanodeDetails> ids)
-      throws IOException {
+      Iterable<DatanodeDetails> ids) throws IOException {
     Objects.requireNonNull(ids, "ids == null");
     final Iterator<DatanodeDetails> i = ids.iterator();
     Preconditions.checkArgument(i.hasNext());
@@ -136,21 +138,21 @@ public final class ContainerTestHelper {
     for(; i.hasNext();) {
       pipelineChannel.addMember(i.next());
     }
-    return new Pipeline(containerName, pipelineChannel);
+    return new Pipeline(pipelineChannel);
   }
 
   /**
    * Creates a ChunkInfo for testing.
    *
-   * @param keyName - Name of the key
+   * @param keyID - ID of the key
    * @param seqNo - Chunk number.
    * @return ChunkInfo
    * @throws IOException
    */
-  public static ChunkInfo getChunk(String keyName, int seqNo, long offset,
+  public static ChunkInfo getChunk(long keyID, int seqNo, long offset,
       long len) throws IOException {
 
-    ChunkInfo info = new ChunkInfo(String.format("%s.data.%d", keyName,
+    ChunkInfo info = new ChunkInfo(String.format("%d.data.%d", keyID,
         seqNo), offset, len);
     return info;
   }
@@ -185,29 +187,27 @@ public final class ContainerTestHelper {
    * Returns a writeChunk Request.
    *
    * @param pipeline - A set of machines where this container lives.
-   * @param containerName - Name of the container.
-   * @param keyName - Name of the Key this chunk is part of.
+   * @param blockID - Block ID of the chunk.
    * @param datalen - Length of data.
    * @return ContainerCommandRequestProto
    * @throws IOException
    * @throws NoSuchAlgorithmException
    */
   public static ContainerCommandRequestProto getWriteChunkRequest(
-      Pipeline pipeline, String containerName, String keyName, int datalen)
+      Pipeline pipeline, BlockID blockID, int datalen)
       throws IOException, NoSuchAlgorithmException {
-    LOG.trace("writeChunk {} (key={}) to pipeline=",
-        datalen, keyName, pipeline);
+    LOG.trace("writeChunk {} (blockID={}) to pipeline=",
+        datalen, blockID, pipeline);
     ContainerProtos.WriteChunkRequestProto.Builder writeRequest =
         ContainerProtos.WriteChunkRequestProto
             .newBuilder();
 
     Pipeline newPipeline =
-        new Pipeline(containerName, pipeline.getPipelineChannel());
-    writeRequest.setPipeline(newPipeline.getProtobufMessage());
-    writeRequest.setKeyName(keyName);
+        new Pipeline(pipeline.getPipelineChannel());
+    writeRequest.setBlockID(blockID.getProtobuf());
 
     byte[] data = getData(datalen);
-    ChunkInfo info = getChunk(keyName, 0, 0, datalen);
+    ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
     setDataChecksum(info, data);
 
     writeRequest.setChunkData(info.getProtoBufMessage());
@@ -227,29 +227,26 @@ public final class ContainerTestHelper {
    * Returns PutSmallFile Request that we can send to the container.
    *
    * @param pipeline - Pipeline
-   * @param containerName - ContainerName.
-   * @param keyName - KeyName
+   * @param blockID - Block ID of the small file.
    * @param dataLen - Number of bytes in the data
    * @return ContainerCommandRequestProto
    */
   public static ContainerCommandRequestProto getWriteSmallFileRequest(
-      Pipeline pipeline, String containerName, String keyName, int dataLen)
+      Pipeline pipeline, BlockID blockID, int dataLen)
       throws Exception {
     ContainerProtos.PutSmallFileRequestProto.Builder smallFileRequest =
         ContainerProtos.PutSmallFileRequestProto.newBuilder();
     Pipeline newPipeline =
-        new Pipeline(containerName, pipeline.getPipelineChannel());
+        new Pipeline(pipeline.getPipelineChannel());
     byte[] data = getData(dataLen);
-    ChunkInfo info = getChunk(keyName, 0, 0, dataLen);
+    ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, dataLen);
     setDataChecksum(info, data);
 
 
     ContainerProtos.PutKeyRequestProto.Builder putRequest =
         ContainerProtos.PutKeyRequestProto.newBuilder();
 
-    putRequest.setPipeline(newPipeline.getProtobufMessage());
-    KeyData keyData = new KeyData(containerName, keyName);
-
+    KeyData keyData = new KeyData(blockID);
     List<ContainerProtos.ChunkInfo> newList = new LinkedList<>();
     newList.add(info.getProtoBufMessage());
     keyData.setChunks(newList);
@@ -270,12 +267,11 @@ public final class ContainerTestHelper {
 
 
   public static ContainerCommandRequestProto getReadSmallFileRequest(
-      ContainerProtos.PutKeyRequestProto putKey)
+      Pipeline pipeline, ContainerProtos.PutKeyRequestProto putKey)
       throws Exception {
     ContainerProtos.GetSmallFileRequestProto.Builder smallFileRequest =
         ContainerProtos.GetSmallFileRequestProto.newBuilder();
-    Pipeline pipeline = Pipeline.getFromProtoBuf(putKey.getPipeline());
-    ContainerCommandRequestProto getKey = getKeyRequest(putKey);
+    ContainerCommandRequestProto getKey = getKeyRequest(pipeline, putKey);
     smallFileRequest.setKey(getKey.getGetKey());
 
     ContainerCommandRequestProto.Builder request =
@@ -290,23 +286,21 @@ public final class ContainerTestHelper {
   /**
    * Returns a read Request.
    *
+   * @param pipeline pipeline.
    * @param request writeChunkRequest.
    * @return Request.
    * @throws IOException
    * @throws NoSuchAlgorithmException
    */
   public static ContainerCommandRequestProto getReadChunkRequest(
-      ContainerProtos.WriteChunkRequestProto request)
+      Pipeline pipeline, ContainerProtos.WriteChunkRequestProto request)
       throws IOException, NoSuchAlgorithmException {
-    LOG.trace("readChunk key={} from pipeline={}",
-        request.getKeyName(), request.getPipeline());
+    LOG.trace("readChunk blockID={} from pipeline={}",
+        request.getBlockID(), pipeline);
 
     ContainerProtos.ReadChunkRequestProto.Builder readRequest =
         ContainerProtos.ReadChunkRequestProto.newBuilder();
-    Pipeline pipeline = Pipeline.getFromProtoBuf(request.getPipeline());
-    readRequest.setPipeline(request.getPipeline());
-
-    readRequest.setKeyName(request.getKeyName());
+    readRequest.setBlockID(request.getBlockID());
     readRequest.setChunkData(request.getChunkData());
 
     ContainerCommandRequestProto.Builder newRequest =
@@ -321,25 +315,25 @@ public final class ContainerTestHelper {
   /**
    * Returns a delete Request.
    *
+   * @param pipeline pipeline.
    * @param writeRequest - write request
    * @return request
    * @throws IOException
    * @throws NoSuchAlgorithmException
    */
   public static ContainerCommandRequestProto getDeleteChunkRequest(
-      ContainerProtos.WriteChunkRequestProto writeRequest)
+      Pipeline pipeline, ContainerProtos.WriteChunkRequestProto writeRequest)
       throws
       IOException, NoSuchAlgorithmException {
-    LOG.trace("deleteChunk key={} from pipeline={}",
-        writeRequest.getKeyName(), writeRequest.getPipeline());
-    Pipeline pipeline = Pipeline.getFromProtoBuf(writeRequest.getPipeline());
+    LOG.trace("deleteChunk blockID={} from pipeline={}",
+        writeRequest.getBlockID(), pipeline);
+
     ContainerProtos.DeleteChunkRequestProto.Builder deleteRequest =
         ContainerProtos.DeleteChunkRequestProto
             .newBuilder();
 
-    deleteRequest.setPipeline(writeRequest.getPipeline());
     deleteRequest.setChunkData(writeRequest.getChunkData());
-    deleteRequest.setKeyName(writeRequest.getKeyName());
+    deleteRequest.setBlockID(writeRequest.getBlockID());
 
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
@@ -357,19 +351,17 @@ public final class ContainerTestHelper {
    * @return ContainerCommandRequestProto.
    */
   public static ContainerCommandRequestProto getCreateContainerRequest(
-      String containerName, Pipeline pipeline) throws IOException {
-    LOG.trace("addContainer: {}", containerName);
+      long containerID, Pipeline pipeline) throws IOException {
+    LOG.trace("addContainer: {}", containerID);
 
     ContainerProtos.CreateContainerRequestProto.Builder createRequest =
         ContainerProtos.CreateContainerRequestProto
             .newBuilder();
     ContainerProtos.ContainerData.Builder containerData = ContainerProtos
         .ContainerData.newBuilder();
-    containerData.setName(containerName);
-    createRequest.setPipeline(
-        ContainerTestHelper.createSingleNodePipeline(containerName)
-            .getProtobufMessage());
+    containerData.setContainerID(containerID);
     createRequest.setContainerData(containerData.build());
+    createRequest.setPipeline(pipeline.getProtobufMessage());
 
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
@@ -386,18 +378,18 @@ public final class ContainerTestHelper {
    * Creates a container data based on the given meta data,
    * and request to update an existing container with it.
    *
-   * @param containerName
+   * @param containerID
    * @param metaData
    * @return
    * @throws IOException
    */
   public static ContainerCommandRequestProto getUpdateContainerRequest(
-      String containerName, Map<String, String> metaData) throws IOException {
+      long containerID, Map<String, String> metaData) throws IOException {
     ContainerProtos.UpdateContainerRequestProto.Builder updateRequestBuilder =
         ContainerProtos.UpdateContainerRequestProto.newBuilder();
     ContainerProtos.ContainerData.Builder containerData = ContainerProtos
         .ContainerData.newBuilder();
-    containerData.setName(containerName);
+    containerData.setContainerID(containerID);
     String[] keys = metaData.keySet().toArray(new String[]{});
     for(int i=0; i<keys.length; i++) {
       KeyValue.Builder kvBuilder = KeyValue.newBuilder();
@@ -406,7 +398,7 @@ public final class ContainerTestHelper {
       containerData.addMetadata(i, kvBuilder.build());
     }
     Pipeline pipeline =
-        ContainerTestHelper.createSingleNodePipeline(containerName);
+        ContainerTestHelper.createSingleNodePipeline();
     updateRequestBuilder.setPipeline(pipeline.getProtobufMessage());
     updateRequestBuilder.setContainerData(containerData.build());
 
@@ -440,22 +432,20 @@ public final class ContainerTestHelper {
 
   /**
    * Returns the PutKeyRequest for test purpose.
-   *
+   * @param pipeline - pipeline.
    * @param writeRequest - Write Chunk Request.
    * @return - Request
    */
   public static ContainerCommandRequestProto getPutKeyRequest(
-      ContainerProtos.WriteChunkRequestProto writeRequest) {
+      Pipeline pipeline, ContainerProtos.WriteChunkRequestProto writeRequest) {
     LOG.trace("putKey: {} to pipeline={}",
-        writeRequest.getKeyName(), writeRequest.getPipeline());
+        writeRequest.getBlockID());
 
-    Pipeline pipeline = Pipeline.getFromProtoBuf(writeRequest.getPipeline());
     ContainerProtos.PutKeyRequestProto.Builder putRequest =
         ContainerProtos.PutKeyRequestProto.newBuilder();
 
-    putRequest.setPipeline(writeRequest.getPipeline());
-    KeyData keyData = new KeyData(writeRequest.getPipeline().getContainerName(),
-        writeRequest.getKeyName());
+    KeyData keyData = new KeyData(
+        BlockID.getFromProtobuf(writeRequest.getBlockID()));
     List<ContainerProtos.ChunkInfo> newList = new LinkedList<>();
     newList.add(writeRequest.getChunkData());
     keyData.setChunks(newList);
@@ -472,24 +462,22 @@ public final class ContainerTestHelper {
 
   /**
    * Gets a GetKeyRequest for test purpose.
-   *
+   * @param  pipeline - pipeline
    * @param putKeyRequest - putKeyRequest.
    * @return - Request
+   * immediately.
    */
   public static ContainerCommandRequestProto getKeyRequest(
-      ContainerProtos.PutKeyRequestProto putKeyRequest) {
-    LOG.trace("getKey: name={} from pipeline={}",
-        putKeyRequest.getKeyData().getName(), putKeyRequest.getPipeline());
-    Pipeline pipeline = Pipeline.getFromProtoBuf(putKeyRequest.getPipeline());
+      Pipeline pipeline, ContainerProtos.PutKeyRequestProto putKeyRequest) {
+    HddsProtos.BlockID blockID = putKeyRequest.getKeyData().getBlockID();
+    LOG.trace("getKey: blockID={}", blockID);
 
     ContainerProtos.GetKeyRequestProto.Builder getRequest =
         ContainerProtos.GetKeyRequestProto.newBuilder();
     ContainerProtos.KeyData.Builder keyData = ContainerProtos.KeyData
         .newBuilder();
-    keyData.setContainerName(putKeyRequest.getPipeline().getContainerName());
-    keyData.setName(putKeyRequest.getKeyData().getName());
+    keyData.setBlockID(blockID);
     getRequest.setKeyData(keyData);
-    getRequest.setPipeline(putKeyRequest.getPipeline());
 
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
@@ -517,18 +505,17 @@ public final class ContainerTestHelper {
   }
 
   /**
+   * @param pipeline - pipeline.
    * @param putKeyRequest - putKeyRequest.
    * @return - Request
    */
   public static ContainerCommandRequestProto getDeleteKeyRequest(
-      ContainerProtos.PutKeyRequestProto putKeyRequest) {
-    LOG.trace("deleteKey: name={} from pipeline={}",
-        putKeyRequest.getKeyData().getName(), putKeyRequest.getPipeline());
-    Pipeline pipeline = Pipeline.getFromProtoBuf(putKeyRequest.getPipeline());
+      Pipeline pipeline, ContainerProtos.PutKeyRequestProto putKeyRequest) {
+    LOG.trace("deleteKey: name={}",
+        putKeyRequest.getKeyData().getBlockID());
     ContainerProtos.DeleteKeyRequestProto.Builder delRequest =
         ContainerProtos.DeleteKeyRequestProto.newBuilder();
-    delRequest.setPipeline(putKeyRequest.getPipeline());
-    delRequest.setName(putKeyRequest.getKeyData().getName());
+    delRequest.setBlockID(putKeyRequest.getKeyData().getBlockID());
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.DeleteKey);
@@ -541,14 +528,14 @@ public final class ContainerTestHelper {
   /**
    * Returns a close container request.
    * @param pipeline - pipeline
+   * @param containerID - ID of the container.
    * @return ContainerCommandRequestProto.
    */
   public static ContainerCommandRequestProto getCloseContainer(
-      Pipeline pipeline) {
-    Preconditions.checkNotNull(pipeline);
+      Pipeline pipeline, long containerID) {
     ContainerProtos.CloseContainerRequestProto closeRequest =
-        ContainerProtos.CloseContainerRequestProto.newBuilder().setPipeline(
-            pipeline.getProtobufMessage()).build();
+        ContainerProtos.CloseContainerRequestProto.newBuilder().
+            setContainerID(containerID).build();
     ContainerProtos.ContainerCommandRequestProto cmd =
         ContainerCommandRequestProto.newBuilder().setCmdType(ContainerProtos
             .Type.CloseContainer).setCloseContainer(closeRequest)
@@ -562,19 +549,19 @@ public final class ContainerTestHelper {
   /**
    * Returns a simple request without traceId.
    * @param pipeline - pipeline
+   * @param containerID - ID of the container.
    * @return ContainerCommandRequestProto without traceId.
    */
   public static ContainerCommandRequestProto getRequestWithoutTraceId(
-          Pipeline pipeline) {
+      Pipeline pipeline, long containerID) {
     Preconditions.checkNotNull(pipeline);
     ContainerProtos.CloseContainerRequestProto closeRequest =
-            ContainerProtos.CloseContainerRequestProto.newBuilder().setPipeline(
-                    pipeline.getProtobufMessage()).build();
+            ContainerProtos.CloseContainerRequestProto.newBuilder().
+                setContainerID(containerID).build();
     ContainerProtos.ContainerCommandRequestProto cmd =
             ContainerCommandRequestProto.newBuilder().setCmdType(ContainerProtos
                     .Type.CloseContainer).setCloseContainer(closeRequest)
-                    .setDatanodeUuid(
-                        pipeline.getLeader().getUuidString())
+                    .setDatanodeUuid(pipeline.getLeader().getUuidString())
                     .build();
     return cmd;
   }
@@ -585,12 +572,12 @@ public final class ContainerTestHelper {
    * @return ContainerCommandRequestProto.
    */
   public static ContainerCommandRequestProto getDeleteContainer(
-      Pipeline pipeline, boolean forceDelete) {
+      Pipeline pipeline, long containerID, boolean forceDelete) {
     Preconditions.checkNotNull(pipeline);
     ContainerProtos.DeleteContainerRequestProto deleteRequest =
-        ContainerProtos.DeleteContainerRequestProto.newBuilder().setName(
-            pipeline.getContainerName()).setPipeline(
-            pipeline.getProtobufMessage()).setForceDelete(forceDelete).build();
+        ContainerProtos.DeleteContainerRequestProto.newBuilder().
+            setContainerID(containerID).
+            setForceDelete(forceDelete).build();
     return ContainerCommandRequestProto.newBuilder()
         .setCmdType(ContainerProtos.Type.DeleteContainer)
         .setDeleteContainer(deleteRequest)
@@ -598,4 +585,23 @@ public final class ContainerTestHelper {
         .setDatanodeUuid(pipeline.getLeader().getUuidString())
         .build();
   }
+
+  private static void sleep(long milliseconds) {
+    try {
+      Thread.sleep(milliseconds);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  public static BlockID getTestBlockID(long containerID) {
+    // Add 2ms delay so that localID based on UtcTime
+    // won't collide.
+    sleep(2);
+    return new BlockID(containerID, Time.getUtcTime());
+  }
+
+  public static long getTestContainerID() {
+    return Time.getUtcTime();
+  }
 }

Beberapa file tidak ditampilkan karena terlalu banyak file yang berubah dalam diff ini