Browse Source

HDFS-12387. Ozone: Support Ratis as a first class replication mechanism. Contributed by Anu Engineer.

Anu Engineer 7 năm trước cách đây
mục cha
commit
019fb091b7
50 tập tin đã thay đổi với 2148 bổ sung936 xóa
  1. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
  2. 5 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
  3. 9 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
  4. 4 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
  5. 14 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
  6. 5 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
  7. 30 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyArgs.java
  8. 4 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java
  9. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/ScmConfigKeys.java
  10. 25 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClient.java
  11. 3 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientManager.java
  12. 70 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientRatis.java
  13. 25 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientSpi.java
  14. 109 41
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/client/ContainerOperationClient.java
  15. 97 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/BlockContainerInfo.java
  16. 117 18
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/ContainerInfo.java
  17. 111 23
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/Pipeline.java
  18. 7 8
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/protocol/ScmBlockLocationProtocol.java
  19. 14 16
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java
  20. 2 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/KeySpaceManagerProtocol.proto
  21. 16 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/Ozone.proto
  22. 3 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ScmBlockLocationProtocol.proto
  23. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/StorageContainerLocationProtocol.proto
  24. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
  25. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
  26. 26 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeyManagerImpl.java
  27. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/KeySpaceManagerProtocolServerSideTranslatorPB.java
  28. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/ScmBlockLocationProtocolServerSideTranslatorPB.java
  29. 15 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java
  30. 8 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockManager.java
  31. 220 300
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockManagerImpl.java
  32. 122 155
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerMapping.java
  33. 472 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerStateManager.java
  34. 9 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/Mapping.java
  35. 29 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/pipelines/PipelineSelector.java
  36. 151 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/pipelines/ratis/RatisManagerImpl.java
  37. 12 27
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/pipelines/standalone/StandaloneManagerImpl.java
  38. 223 197
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/tools/Corona.java
  39. 17 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
  40. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/ozone-default.xml
  41. 21 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
  42. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java
  43. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestMultipleContainerReadWrite.java
  44. 17 38
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java
  45. 23 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/block/TestBlockManager.java
  46. 11 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/container/TestContainerMapping.java
  47. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/node/TestContainerPlacement.java
  48. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/tools/TestCorona.java
  49. 40 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
  50. 37 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java

@@ -93,9 +93,9 @@ public final class OzoneConfigKeys {
       "ozone.container.cache.size";
   public static final int OZONE_CONTAINER_CACHE_DEFAULT = 1024;
 
-  public static final String OZONE_SCM_BLOCK_SIZE_KEY =
-      "ozone.scm.block.size";
-  public static final long OZONE_SCM_BLOCK_SIZE_DEFAULT = 256 * OzoneConsts.MB;
+  public static final String OZONE_SCM_BLOCK_SIZE_IN_MB =
+      "ozone.scm.block.size.in.mb";
+  public static final long OZONE_SCM_BLOCK_SIZE_DEFAULT = 256;
 
   /**
    * Ozone administrator users delimited by comma.

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 
 import java.io.IOException;
 import java.util.Iterator;
@@ -195,10 +196,12 @@ public class OzoneBucket {
    * @return OzoneOutputStream to which the data has to be written.
    * @throws IOException
    */
-  public OzoneOutputStream createKey(String key, long size)throws IOException {
+  public OzoneOutputStream createKey(String key, long size, OzoneProtos
+      .ReplicationType type, OzoneProtos.ReplicationFactor factor)
+      throws IOException {
     Preconditions.checkNotNull(proxy, "Client proxy is not set.");
     Preconditions.checkNotNull(key);
-    return proxy.createKey(volumeName, name, key, size);
+    return proxy.createKey(volumeName, name, key, size, type, factor);
   }
 
   /**

+ 9 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java

@@ -276,10 +276,10 @@ public class ChunkGroupOutputStream extends OutputStream {
     // case we should revert the above allocateKey to KSM.
     // check index as sanity check
     int index = 0;
-    String containerKey;
+    String blockID;
     ChunkGroupOutputStream groupOutputStream = new ChunkGroupOutputStream();
     for (KsmKeyLocationInfo subKeyInfo : keyInfo.getKeyLocationList()) {
-      containerKey = subKeyInfo.getBlockID();
+      blockID = subKeyInfo.getBlockID();
 
       Preconditions.checkArgument(index++ == subKeyInfo.getIndex());
       String containerName = subKeyInfo.getContainerName();
@@ -290,8 +290,13 @@ public class ChunkGroupOutputStream extends OutputStream {
       // create container if needed
       if (subKeyInfo.getShouldCreateContainer()) {
         try {
-          // Block manager sets the container creation stage begin.
+          storageContainerLocationClient.notifyObjectCreationStage(
+              NotifyObjectCreationStageRequestProto.Type.container,
+              containerName,
+              NotifyObjectCreationStageRequestProto.Stage.begin);
+
           ContainerProtocolCalls.createContainer(xceiverClient, requestId);
+
           storageContainerLocationClient.notifyObjectCreationStage(
               NotifyObjectCreationStageRequestProto.Type.container,
               containerName,
@@ -307,7 +312,7 @@ public class ChunkGroupOutputStream extends OutputStream {
         }
       }
 
-      groupOutputStream.addStream(containerKey, keyInfo.getKeyName(),
+      groupOutputStream.addStream(blockID, keyInfo.getKeyName(),
           xceiverClientManager, xceiverClient, requestId, chunkSize,
           subKeyInfo.getLength());
     }

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.VolumeArgs;
 import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 
 import java.io.IOException;
 import java.util.List;
@@ -257,7 +258,9 @@ public interface ClientProtocol {
    *
    */
   OzoneOutputStream createKey(String volumeName, String bucketName,
-                              String keyName, long size)
+                              String keyName, long size,
+                              OzoneProtos.ReplicationType type,
+                              OzoneProtos.ReplicationFactor factor)
       throws IOException;
 
   /**

+ 14 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 
 import java.io.IOException;
 import java.util.List;
@@ -171,10 +172,20 @@ public class RestClient implements ClientProtocol {
     throw new UnsupportedOperationException("Not yet implemented.");
   }
 
+  /**
+   * Writes a key in an existing bucket.
+   *
+   * @param volumeName Name of the Volume
+   * @param bucketName Name of the Bucket
+   * @param keyName Name of the Key
+   * @param size Size of the data
+   * @param type
+   * @param factor @return {@link OzoneOutputStream}
+   */
   @Override
-  public OzoneOutputStream createKey(
-      String volumeName, String bucketName, String keyName, long size)
-      throws IOException {
+  public OzoneOutputStream createKey(String volumeName, String bucketName,
+      String keyName, long size, OzoneProtos.ReplicationType type,
+      OzoneProtos.ReplicationFactor factor) throws IOException {
     throw new UnsupportedOperationException("Not yet implemented.");
   }
 

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java

@@ -51,6 +51,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
 import org.apache.hadoop.scm.ScmConfigKeys;
 import org.apache.hadoop.scm.XceiverClientManager;
@@ -430,7 +431,8 @@ public class RpcClient implements ClientProtocol {
 
   @Override
   public OzoneOutputStream createKey(
-      String volumeName, String bucketName, String keyName, long size)
+      String volumeName, String bucketName, String keyName, long size,
+      OzoneProtos.ReplicationType type, OzoneProtos.ReplicationFactor factor)
       throws IOException {
     String requestId = UUID.randomUUID().toString();
     KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
@@ -438,6 +440,8 @@ public class RpcClient implements ClientProtocol {
         .setBucketName(bucketName)
         .setKeyName(keyName)
         .setDataSize(size)
+        .setType(type)
+        .setFactor(factor)
         .build();
 
     KsmKeyInfo keyInfo = keySpaceManagerClient.allocateKey(keyArgs);

+ 30 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmKeyArgs.java

@@ -16,6 +16,8 @@
  * limitations under the License.
  */
 package org.apache.hadoop.ozone.ksm.helpers;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationType;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationFactor;
 
 /**
  * Args for key. Client use this to specify key's attributes on  key creation
@@ -25,15 +27,26 @@ public final class KsmKeyArgs {
   private final String volumeName;
   private final String bucketName;
   private final String keyName;
-
   private final long dataSize;
+  private final ReplicationType type;
+  private final ReplicationFactor factor;
 
   private KsmKeyArgs(String volumeName, String bucketName, String keyName,
-      long dataSize) {
+      long dataSize, ReplicationType type, ReplicationFactor factor) {
     this.volumeName = volumeName;
     this.bucketName = bucketName;
     this.keyName = keyName;
     this.dataSize = dataSize;
+    this.type = type;
+    this.factor = factor;
+  }
+
+  public ReplicationType getType() {
+    return type;
+  }
+
+  public ReplicationFactor getFactor() {
+    return factor;
   }
 
   public String getVolumeName() {
@@ -60,6 +73,9 @@ public final class KsmKeyArgs {
     private String bucketName;
     private String keyName;
     private long dataSize;
+    private ReplicationType type;
+    private ReplicationFactor factor;
+
 
     public Builder setVolumeName(String volume) {
       this.volumeName = volume;
@@ -81,8 +97,19 @@ public final class KsmKeyArgs {
       return this;
     }
 
+    public Builder setType(ReplicationType type) {
+      this.type = type;
+      return this;
+    }
+
+    public Builder  setFactor(ReplicationFactor factor) {
+      this.factor = factor;
+      return this;
+    }
+
     public KsmKeyArgs build() {
-      return new KsmKeyArgs(volumeName, bucketName, keyName, dataSize);
+      return new KsmKeyArgs(volumeName, bucketName, keyName, dataSize,
+          type, factor);
     }
   }
 }

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java

@@ -511,7 +511,10 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
         .setVolumeName(args.getVolumeName())
         .setBucketName(args.getBucketName())
         .setKeyName(args.getKeyName())
-        .setDataSize(args.getDataSize()).build();
+        .setDataSize(args.getDataSize())
+        .setType(args.getType())
+        .setFactor(args.getFactor())
+        .build();
     req.setKeyArgs(keyArgs);
 
     final LocateKeyResponse resp;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/ScmConfigKeys.java

@@ -186,7 +186,7 @@ public final class ScmConfigKeys {
 
   public static final String OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE =
       "ozone.scm.container.provision_batch_size";
-  public static final int OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE_DEFAULT = 5;
+  public static final int OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE_DEFAULT = 20;
 
   public static final String OZONE_SCM_CONTAINER_DELETION_CHOOSING_POLICY =
       "ozone.scm.container.deletion-choosing.policy";

+ 25 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClient.java

@@ -34,11 +34,13 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
+import java.util.List;
 
 /**
  * A Client for the storageContainer protocol.
@@ -154,4 +156,27 @@ public class XceiverClient extends XceiverClientSpi {
         channelFuture.channel().pipeline().get(XceiverClientHandler.class);
     return handler.sendCommandAsync(request);
   }
+
+  /**
+   * Create a pipeline.
+   *
+   * @param pipelineID - Name of the pipeline.
+   * @param datanodes - Datanodes
+   */
+  @Override
+  public void createPipeline(String pipelineID, List<DatanodeID> datanodes)
+      throws IOException {
+    // For stand alone pipeline, there is no notion called setup pipeline.
+    return;
+  }
+
+  /**
+   * Returns pipeline Type.
+   *
+   * @return - Stand Alone as the type.
+   */
+  @Override
+  public OzoneProtos.ReplicationType getPipelineType() {
+    return OzoneProtos.ReplicationType.STAND_ALONE;
+  }
 }

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientManager.java

@@ -42,6 +42,8 @@ import static org.apache.hadoop.scm.ScmConfigKeys
     .SCM_CONTAINER_CLIENT_MAX_SIZE_KEY;
 import static org.apache.hadoop.scm.ScmConfigKeys
     .SCM_CONTAINER_CLIENT_MAX_SIZE_DEFAULT;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneProtos
+    .ReplicationType.RATIS;
 
 /**
  * XceiverClientManager is responsible for the lifecycle of XceiverClient
@@ -146,7 +148,7 @@ public class XceiverClientManager implements Closeable {
           new Callable<XceiverClientSpi>() {
           @Override
           public XceiverClientSpi call() throws Exception {
-            XceiverClientSpi client = useRatis ?
+            XceiverClientSpi client = pipeline.getType() == RATIS ?
                     XceiverClientRatis.newXceiverClientRatis(pipeline, conf)
                     : new XceiverClient(pipeline, conf);
             client.connect();

+ 70 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientRatis.java

@@ -20,14 +20,15 @@ package org.apache.hadoop.scm;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
-    .ContainerCommandRequestProto;
-import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
-    .ContainerCommandResponseProto;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandRequestProto;
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandResponseProto;
 import org.apache.ratis.RatisHelper;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.rpc.RpcType;
 import org.apache.ratis.rpc.SupportedRpcType;
 import org.apache.ratis.shaded.com.google.protobuf.ByteString;
@@ -36,6 +37,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.Objects;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
@@ -68,6 +70,70 @@ public final class XceiverClientRatis extends XceiverClientSpi {
     this.rpcType = rpcType;
   }
 
+  /**
+   *  {@inheritDoc}
+   */
+  public void createPipeline(String clusterId, List<DatanodeID> datanodes)
+      throws IOException {
+    final RaftPeer[] newPeers = datanodes.stream().map(RatisHelper::toRaftPeer)
+        .toArray(RaftPeer[]::new);
+    reinitialize(datanodes, newPeers);
+  }
+
+  /**
+   * Returns Ratis as pipeline Type.
+   * @return - Ratis
+   */
+  @Override
+  public OzoneProtos.ReplicationType getPipelineType() {
+    return OzoneProtos.ReplicationType.RATIS;
+  }
+
+  private void reinitialize(List<DatanodeID> datanodes, RaftPeer[] newPeers)
+      throws IOException {
+    if (datanodes.isEmpty()) {
+      return;
+    }
+
+    IOException exception = null;
+    for (DatanodeID d : datanodes) {
+      try {
+        reinitialize(d, newPeers);
+      } catch (IOException ioe) {
+        if (exception == null) {
+          exception = new IOException(
+              "Failed to reinitialize some of the RaftPeer(s)", ioe);
+        } else {
+          exception.addSuppressed(ioe);
+        }
+      }
+    }
+    if (exception != null) {
+      throw exception;
+    }
+  }
+
+  /**
+   * Adds a new peers to the Ratis Ring.
+   * @param datanode - new datanode
+   * @param newPeers - Raft machines
+   * @throws IOException - on Failure.
+   */
+  private void reinitialize(DatanodeID datanode, RaftPeer[] newPeers)
+      throws IOException {
+    final RaftPeer p = RatisHelper.toRaftPeer(datanode);
+    try (RaftClient client = RatisHelper.newRaftClient(rpcType, p)) {
+      client.reinitialize(newPeers, p.getId());
+    } catch (IOException ioe) {
+      LOG.error("Failed to reinitialize RaftPeer:{} datanode: {}  ",
+          p, datanode, ioe);
+      throw new IOException("Failed to reinitialize RaftPeer " + p
+          + "(datanode=" + datanode + ")", ioe);
+    }
+  }
+
+
+
   @Override
   public Pipeline getPipeline() {
     return pipeline;

+ 25 - 6
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClientSpi.java

@@ -19,14 +19,17 @@
 package org.apache.hadoop.scm;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hdfs.ozone.protocol.proto
-    .ContainerProtos.ContainerCommandRequestProto;
-import org.apache.hadoop.hdfs.ozone.protocol.proto
-    .ContainerProtos.ContainerCommandResponseProto;
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
+    .ContainerCommandRequestProto;
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
+    .ContainerCommandResponseProto;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -99,12 +102,28 @@ public abstract class XceiverClientSpi implements Closeable {
 
   /**
    * Sends a given command to server gets a waitable future back.
+   *
    * @param request Request
    * @return Response to the command
    * @throws IOException
    */
-  public abstract CompletableFuture<ContainerCommandResponseProto> sendCommandAsync(
-      ContainerCommandRequestProto request)
+  public abstract CompletableFuture<ContainerCommandResponseProto>
+      sendCommandAsync(ContainerCommandRequestProto request)
       throws IOException, ExecutionException, InterruptedException;
 
+  /**
+   * Create a pipeline.
+   *
+   * @param pipelineID - Name of the pipeline.
+   * @param datanodes - Datanodes
+   */
+  public abstract void createPipeline(String pipelineID,
+      List<DatanodeID> datanodes) throws IOException;
+
+  /**
+   * Returns pipeline Type.
+   *
+   * @return - {Stand_Alone, Ratis or Chained}
+   */
+  public abstract OzoneProtos.ReplicationType getPipelineType();
 }

+ 109 - 41
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/client/ContainerOperationClient.java

@@ -17,14 +17,16 @@
  */
 package org.apache.hadoop.scm.client;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerData;
 import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ReadContainerResponseProto;
 import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolProtos.NotifyObjectCreationStageRequestProto;
-import org.apache.hadoop.scm.XceiverClientSpi;
-import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
 import org.apache.hadoop.scm.XceiverClientManager;
+import org.apache.hadoop.scm.XceiverClientSpi;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.scm.protocolPB
+    .StorageContainerLocationProtocolClientSideTranslatorPB;
 import org.apache.hadoop.scm.storage.ContainerProtocolCalls;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -34,6 +36,9 @@ import java.util.EnumSet;
 import java.util.List;
 import java.util.UUID;
 
+import static org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleState.ALLOCATED;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleState.OPEN;
+
 /**
  * This class provides the client-facing APIs of container operations.
  */
@@ -84,23 +89,18 @@ public class ContainerOperationClient implements ScmClient {
           storageContainerLocationClient.allocateContainer(
               xceiverClientManager.getType(),
               xceiverClientManager.getFactor(), containerId);
-
       client = xceiverClientManager.acquireClient(pipeline);
-      String traceID = UUID.randomUUID().toString();
-      storageContainerLocationClient.notifyObjectCreationStage(
-          NotifyObjectCreationStageRequestProto.Type.container,
-          containerId,
-          NotifyObjectCreationStageRequestProto.Stage.begin);
-      ContainerProtocolCalls.createContainer(client, traceID);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Created container " + containerId
-            + " leader:" + pipeline.getLeader()
-            + " machines:" + pipeline.getMachines());
+
+      // Allocated State means that SCM has allocated this pipeline in its
+      // namespace. The client needs to create the pipeline on the machines
+      // which was choosen by the SCM.
+      Preconditions.checkState(pipeline.getLifeCycleState() == ALLOCATED ||
+          pipeline.getLifeCycleState() == OPEN, "Unexpected pipeline state");
+      if (pipeline.getLifeCycleState() == ALLOCATED) {
+        createPipeline(client, pipeline);
       }
-      storageContainerLocationClient.notifyObjectCreationStage(
-          NotifyObjectCreationStageRequestProto.Type.container,
-          containerId,
-          NotifyObjectCreationStageRequestProto.Stage.complete);
+      // TODO : Container Client State needs to be updated.
+      createContainer(containerId, client, pipeline);
       return pipeline;
     } finally {
       if (client != null) {
@@ -109,6 +109,76 @@ 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.
+   * @throws IOException
+   */
+  public void createContainer(String containerId, XceiverClientSpi client,
+      Pipeline pipeline) throws IOException {
+    String traceID = UUID.randomUUID().toString();
+    storageContainerLocationClient.notifyObjectCreationStage(
+        NotifyObjectCreationStageRequestProto.Type.container,
+        containerId,
+        NotifyObjectCreationStageRequestProto.Stage.begin);
+    ContainerProtocolCalls.createContainer(client, traceID);
+    storageContainerLocationClient.notifyObjectCreationStage(
+        NotifyObjectCreationStageRequestProto.Type.container,
+        containerId,
+        NotifyObjectCreationStageRequestProto.Stage.complete);
+
+    // Let us log this info after we let SCM know that we have completed the
+    // creation state.
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Created container " + containerId
+          + " leader:" + pipeline.getLeader()
+          + " machines:" + pipeline.getMachines());
+    }
+  }
+
+  /**
+   * Creates a pipeline over the machines choosen by the SCM.
+   *
+   * @param client - Client
+   * @param pipeline - pipeline to be createdon Datanodes.
+   * @throws IOException
+   */
+  private void createPipeline(XceiverClientSpi client, Pipeline pipeline)
+      throws IOException {
+
+    Preconditions.checkNotNull(pipeline.getPipelineName(), "Pipeline " +
+        "name cannot be null when client create flag is set.");
+
+    // Pipeline creation is a three step process.
+    //
+    // 1. Notify SCM that this client is doing a create pipeline on
+    // datanodes.
+    //
+    // 2. Talk to Datanodes to create the pipeline.
+    //
+    // 3. update SCM that pipeline creation was successful.
+    storageContainerLocationClient.notifyObjectCreationStage(
+        NotifyObjectCreationStageRequestProto.Type.pipeline,
+        pipeline.getPipelineName(),
+        NotifyObjectCreationStageRequestProto.Stage.begin);
+
+    client.createPipeline(pipeline.getPipelineName(),
+        pipeline.getMachines());
+
+    storageContainerLocationClient.notifyObjectCreationStage(
+        NotifyObjectCreationStageRequestProto.Type.pipeline,
+        pipeline.getPipelineName(),
+        NotifyObjectCreationStageRequestProto.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.
+    LOG.debug("Pipeline creation successful. Pipeline: {}",
+        pipeline.toString());
+  }
+
   /**
    * @inheritDoc
    */
@@ -122,24 +192,18 @@ public class ContainerOperationClient implements ScmClient {
       Pipeline pipeline =
           storageContainerLocationClient.allocateContainer(type, factor,
               containerId);
+      client = xceiverClientManager.acquireClient(pipeline);
+
+      // Allocated State means that SCM has allocated this pipeline in its
+      // namespace. The client needs to create the pipeline on the machines
+      // which was choosen by the SCM.
+      if (pipeline.getLifeCycleState() == ALLOCATED) {
+        createPipeline(client, pipeline);
+      }
+
       // connect to pipeline leader and allocate container on leader datanode.
       client = xceiverClientManager.acquireClient(pipeline);
-      String traceID = UUID.randomUUID().toString();
-      storageContainerLocationClient.notifyObjectCreationStage(
-          NotifyObjectCreationStageRequestProto.Type.container,
-          containerId,
-          NotifyObjectCreationStageRequestProto.Stage.begin);
-
-      ContainerProtocolCalls.createContainer(client, traceID);
-      LOG.info("Created container " + containerId +
-          " leader:" + pipeline.getLeader() +
-          " machines:" + pipeline.getMachines() +
-          " replication factor:" + factor);
-
-      storageContainerLocationClient.notifyObjectCreationStage(
-          NotifyObjectCreationStageRequestProto.Type.container,
-          containerId,
-          NotifyObjectCreationStageRequestProto.Stage.complete);
+      createContainer(containerId, client, pipeline);
       return pipeline;
     } finally {
       if (client != null) {
@@ -192,10 +256,12 @@ public class ContainerOperationClient implements ScmClient {
       ContainerProtocolCalls.deleteContainer(client, force, traceID);
       storageContainerLocationClient
           .deleteContainer(pipeline.getContainerName());
-      LOG.info("Deleted container {}, leader: {}, machines: {} ",
-          pipeline.getContainerName(),
-          pipeline.getLeader(),
-          pipeline.getMachines());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Deleted container {}, leader: {}, machines: {} ",
+            pipeline.getContainerName(),
+            pipeline.getLeader(),
+            pipeline.getMachines());
+      }
     } finally {
       if (client != null) {
         xceiverClientManager.releaseClient(client);
@@ -231,10 +297,12 @@ public class ContainerOperationClient implements ScmClient {
       ReadContainerResponseProto response =
           ContainerProtocolCalls.readContainer(client,
               pipeline.getContainerName(), traceID);
-      LOG.info("Read container {}, leader: {}, machines: {} ",
-          pipeline.getContainerName(),
-          pipeline.getLeader(),
-          pipeline.getMachines());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Read container {}, leader: {}, machines: {} ",
+            pipeline.getContainerName(),
+            pipeline.getLeader(),
+            pipeline.getMachines());
+      }
       return response.getContainerData();
     } finally {
       if (client != null) {

+ 97 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/BlockContainerInfo.java

@@ -1,4 +1,4 @@
-/*
+  /*
  * 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
@@ -18,15 +18,26 @@
 
 package org.apache.hadoop.scm.container.common.helpers;
 
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.hadoop.util.Time;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
 /**
- * Class wraps container + allocated info for containers managed by block svc.
+ * Manages Block Information inside a container.
  */
-public class BlockContainerInfo extends ContainerInfo{
+public class BlockContainerInfo extends ContainerInfo
+    implements Comparator<BlockContainerInfo>,
+    Comparable<BlockContainerInfo>, Serializable {
   private long allocated;
+  private long lastUsed; // last used time
 
   public BlockContainerInfo(ContainerInfo container, long used) {
     super(container);
     this.allocated = used;
+    this.lastUsed = Time.monotonicNow();
   }
 
   public long addAllocated(long size) {
@@ -42,4 +53,87 @@ public class BlockContainerInfo extends ContainerInfo{
   public long getAllocated() {
     return this.allocated;
   }
+
+  /**
+   * Gets the last used time from SCM's perspective.
+   * @return time in milliseconds.
+   */
+  public long getLastUsed() {
+    return lastUsed;
+  }
+
+  /**
+   * Sets the last used time from SCM's perspective.
+   * @param lastUsed time in milliseconds.
+   */
+  public void setLastUsed(long lastUsed) {
+    this.lastUsed = lastUsed;
+  }
+
+  /**
+   * Compares its two arguments for order.  Returns a negative integer, zero, or
+   * a positive integer as the first argument is less than, equal to, or greater
+   * than the second.<p>
+   *
+   * @param o1 the first object to be compared.
+   * @param o2 the second object to be compared.
+   * @return a negative integer, zero, or a positive integer as the first
+   * argument is less than, equal to, or greater than the second.
+   * @throws NullPointerException if an argument is null and this comparator
+   *                              does not permit null arguments
+   * @throws ClassCastException   if the arguments' types prevent them from
+   *                              being compared by this comparator.
+   */
+  @Override
+  public int compare(BlockContainerInfo o1, BlockContainerInfo o2) {
+    return Long.compare(o1.getLastUsed(), o2.getLastUsed());
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    return new EqualsBuilder()
+        .appendSuper(super.equals(o))
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 2017)
+        .appendSuper(super.hashCode())
+        .toHashCode();
+  }
+
+  @Override
+  public String toString() {
+
+    return "BlockContainerInfo{" +
+        "allocated=" + allocated +
+        ", lastUsed=" + lastUsed +
+        ", ContainerInfo=" + super.toString() + '}';
+  }
+
+  /**
+   * Compares this object with the specified object for order.  Returns a
+   * negative integer, zero, or a positive integer as this object is less than,
+   * equal to, or greater than the specified object.
+   *
+   * @param o the object to be compared.
+   * @return a negative integer, zero, or a positive integer as this object is
+   * less than, equal to, or greater than the specified object.
+   * @throws NullPointerException if the specified object is null
+   * @throws ClassCastException   if the specified object's type prevents it
+   *                              from being compared to this object.
+   */
+  @Override
+  public int compareTo(BlockContainerInfo o) {
+    return this.compare(this, o);
+  }
 }

+ 117 - 18
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/ContainerInfo.java

@@ -18,33 +18,71 @@
 
 package org.apache.hadoop.scm.container.common.helpers;
 
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
 import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 import org.apache.hadoop.util.Time;
 
-/**
- * Class wraps ozone container info.
- */
+/** Class wraps ozone container info. */
 public class ContainerInfo {
   private OzoneProtos.LifeCycleState state;
   private Pipeline pipeline;
   // The wall-clock ms since the epoch at which the current state enters.
   private long stateEnterTime;
-
-  ContainerInfo(OzoneProtos.LifeCycleState state, Pipeline pipeline,
-      long stateEnterTime) {
+  private OzoneProtos.Owner owner;
+  private String containerName;
+
+  ContainerInfo(
+      final String containerName,
+      OzoneProtos.LifeCycleState state,
+      Pipeline pipeline,
+      long stateEnterTime,
+      OzoneProtos.Owner owner) {
+    this.containerName = containerName;
     this.pipeline = pipeline;
     this.state = state;
     this.stateEnterTime = stateEnterTime;
+    this.owner = owner;
   }
 
   public ContainerInfo(ContainerInfo container) {
     this.pipeline = container.getPipeline();
     this.state = container.getState();
     this.stateEnterTime = container.getStateEnterTime();
+    this.owner = container.getOwner();
+  }
+
+  /**
+   * Needed for serialization findbugs.
+   */
+  public ContainerInfo() {
+  }
+
+  public static ContainerInfo fromProtobuf(OzoneProtos.SCMContainerInfo info) {
+    ContainerInfo.Builder builder = new ContainerInfo.Builder();
+    builder.setPipeline(Pipeline.getFromProtoBuf(info.getPipeline()));
+    builder.setState(info.getState());
+    builder.setStateEnterTime(info.getStateEnterTime());
+    builder.setOwner(info.getOwner());
+    builder.setContainerName(info.getContainerName());
+    return builder.build();
+  }
+
+  public String getContainerName() {
+    return containerName;
+  }
+
+  public void setContainerName(String containerName) {
+    this.containerName = containerName;
+  }
+
+  public OzoneProtos.LifeCycleState getState() {
+    return state;
   }
 
   /**
    * Update the current container state and state enter time to now.
+   *
    * @param state
    */
   public void setState(OzoneProtos.LifeCycleState state) {
@@ -52,10 +90,6 @@ public class ContainerInfo {
     this.stateEnterTime = Time.monotonicNow();
   }
 
-  public OzoneProtos.LifeCycleState getState() {
-    return state;
-  }
-
   public long getStateEnterTime() {
     return stateEnterTime;
   }
@@ -70,16 +104,68 @@ public class ContainerInfo {
     builder.setPipeline(getPipeline().getProtobufMessage());
     builder.setState(state);
     builder.setStateEnterTime(stateEnterTime);
+
+    if (getOwner() != null) {
+      builder.setOwner(getOwner());
+    }
+    builder.setContainerName(getContainerName());
     return builder.build();
   }
 
-  public static ContainerInfo fromProtobuf(
-      OzoneProtos.SCMContainerInfo info) {
-    ContainerInfo.Builder builder = new ContainerInfo.Builder();
-    builder.setPipeline(Pipeline.getFromProtoBuf(info.getPipeline()));
-    builder.setState(info.getState());
-    builder.setStateEnterTime(info.getStateEnterTime());
-    return builder.build();
+  public OzoneProtos.Owner getOwner() {
+    return owner;
+  }
+
+  public void setOwner(OzoneProtos.Owner owner) {
+    this.owner = owner;
+  }
+
+  @Override
+  public String toString() {
+    return "ContainerInfo{"
+        + "state=" + state
+        + ", pipeline=" + pipeline
+        + ", stateEnterTime=" + stateEnterTime
+        + ", owner=" + owner
+        + ", containerName='" + containerName
+        + '}';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    ContainerInfo that = (ContainerInfo) o;
+
+    return new EqualsBuilder()
+        .append(state, that.state)
+        .append(pipeline.getContainerName(), that.pipeline.getContainerName())
+
+        // TODO : Fix this later. If we add these factors some tests fail.
+        // So Commenting this to continue and will enforce this with
+        // Changes in pipeline where we remove Container Name to
+        // SCMContainerinfo from Pipline.
+        // .append(pipeline.getFactor(), that.pipeline.getFactor())
+        // .append(pipeline.getType(), that.pipeline.getType())
+        .append(owner, that.owner)
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(11, 811)
+        .append(state)
+        .append(pipeline.getContainerName())
+        .append(pipeline.getFactor())
+        .append(pipeline.getType())
+        .append(owner)
+        .toHashCode();
   }
 
   /** Builder class for ContainerInfo. */
@@ -87,6 +173,8 @@ public class ContainerInfo {
     private OzoneProtos.LifeCycleState state;
     private Pipeline pipeline;
     private long stateEnterTime;
+    private OzoneProtos.Owner owner;
+    private String containerName;
 
     public Builder setState(OzoneProtos.LifeCycleState lifeCycleState) {
       this.state = lifeCycleState;
@@ -103,8 +191,19 @@ public class ContainerInfo {
       return this;
     }
 
+    public Builder setOwner(OzoneProtos.Owner owner) {
+      this.owner = owner;
+      return this;
+    }
+
+    public Builder setContainerName(String containerName) {
+      this.containerName = containerName;
+      return this;
+    }
+
     public ContainerInfo build() {
-      return new ContainerInfo(state, pipeline, stateEnterTime);
+      return new
+          ContainerInfo(containerName, state, pipeline, stateEnterTime, owner);
     }
   }
 }

+ 111 - 23
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/Pipeline.java

@@ -43,16 +43,6 @@ import java.util.TreeMap;
  * A pipeline represents the group of machines over which a container lives.
  */
 public class Pipeline {
-  private String containerName;
-  private String leaderID;
-  private Map<String, DatanodeID> datanodes;
-  /**
-   * Allows you to maintain private data on pipelines.
-   * This is not serialized via protobuf, just allows us to maintain some
-   * private data.
-   */
-  private byte[] data;
-
   static final String PIPELINE_INFO = "PIPELINE_INFO_FILTER";
   private static final ObjectWriter WRITER;
 
@@ -69,10 +59,18 @@ public class Pipeline {
     WRITER = mapper.writer(filters);
   }
 
-  @JsonFilter(PIPELINE_INFO)
-  class MixIn {
-  }
-
+  private String containerName;
+  private String leaderID;
+  private Map<String, DatanodeID> datanodes;
+  private OzoneProtos.LifeCycleState lifeCycleState;
+  private OzoneProtos.ReplicationType type;
+  private OzoneProtos.ReplicationFactor factor;
+  private String pipelineName;
+  /**
+   * Allows you to maintain private data on pipelines. This is not serialized
+   * via protobuf, just allows us to maintain some private data.
+   */
+  private byte[] data;
   /**
    * Constructs a new pipeline data structure.
    *
@@ -98,9 +96,22 @@ public class Pipeline {
     }
 
     newPipeline.setContainerName(pipeline.getContainerName());
+    newPipeline.setLifeCycleState(pipeline.getState());
+    newPipeline.setType(pipeline.getType());
+    newPipeline.setFactor(pipeline.getFactor());
+    if (pipeline.hasPipelineName()) {
+      newPipeline.setPipelineName(pipeline.getPipelineName());
+    }
     return newPipeline;
   }
 
+  public OzoneProtos.ReplicationFactor getFactor() {
+    return factor;
+  }
+
+  public void setFactor(OzoneProtos.ReplicationFactor factor) {
+    this.factor = factor;
+  }
 
   /**
    * Adds a member to the pipeline.
@@ -167,6 +178,17 @@ public class Pipeline {
     }
     builder.setLeaderID(leaderID);
     builder.setContainerName(this.containerName);
+
+    if (this.getLifeCycleState() != null) {
+      builder.setState(this.getLifeCycleState());
+    }
+    if (this.getType() != null) {
+      builder.setType(this.getType());
+    }
+
+    if (this.getFactor() != null) {
+      builder.setFactor(this.getFactor());
+    }
     return builder.build();
   }
 
@@ -188,8 +210,22 @@ public class Pipeline {
     this.containerName = containerName;
   }
 
+  /**
+   * Returns private data that is set on this pipeline.
+   *
+   * @return blob, the user can interpret it any way they like.
+   */
+  public byte[] getData() {
+    if (this.data != null) {
+      return Arrays.copyOf(this.data, this.data.length);
+    } else {
+      return null;
+    }
+  }
+
   /**
    * Set private data on pipeline.
+   *
    * @param data -- private data.
    */
   public void setData(byte[] data) {
@@ -199,16 +235,57 @@ public class Pipeline {
   }
 
   /**
-   * Returns private data that is set on this pipeline.
+   * Gets the State of the pipeline.
    *
-   * @return blob, the user can interpret it any way they like.
+   * @return - LifeCycleStates.
    */
-  public byte[] getData() {
-    if (this.data != null) {
-      return Arrays.copyOf(this.data, this.data.length);
-    } else {
-      return null;
-    }
+  public OzoneProtos.LifeCycleState getLifeCycleState() {
+    return lifeCycleState;
+  }
+
+  /**
+   * Sets the lifecycleState.
+   *
+   * @param lifeCycleStates - Enum
+   */
+  public void setLifeCycleState(OzoneProtos.LifeCycleState lifeCycleStates) {
+    this.lifeCycleState = lifeCycleStates;
+  }
+
+  /**
+   * Gets the pipeline Name.
+   *
+   * @return - Name of the pipeline
+   */
+  public String getPipelineName() {
+    return pipelineName;
+  }
+
+  /**
+   * Sets the pipeline name.
+   *
+   * @param pipelineName - Sets the name.
+   */
+  public void setPipelineName(String pipelineName) {
+    this.pipelineName = pipelineName;
+  }
+
+  /**
+   * Returns the type.
+   *
+   * @return type - Standalone, Ratis, Chained.
+   */
+  public OzoneProtos.ReplicationType getType() {
+    return type;
+  }
+
+  /**
+   * Sets the type of this pipeline.
+   *
+   * @param type - Standalone, Ratis, Chained.
+   */
+  public void setType(OzoneProtos.ReplicationType type) {
+    this.type = type;
   }
 
   @Override
@@ -216,8 +293,15 @@ public class Pipeline {
     final StringBuilder b = new StringBuilder(getClass().getSimpleName())
         .append("[");
     datanodes.keySet().stream()
-        .forEach(id -> b.append(id.endsWith(leaderID)? "*" + id : id));
+        .forEach(id -> b.append(id.endsWith(leaderID) ? "*" + id : id));
     b.append("] container:").append(containerName);
+    b.append(" name:").append(getPipelineName());
+    if (getType() != null) {
+      b.append(" type:").append(getType().toString());
+    }
+    if (getLifeCycleState() != null) {
+      b.append(" State:").append(getLifeCycleState().toString());
+    }
     return b.toString();
   }
 
@@ -230,4 +314,8 @@ public class Pipeline {
   public String toJsonString() throws IOException {
     return WRITER.writeValueAsString(this);
   }
+
+  @JsonFilter(PIPELINE_INFO)
+  class MixIn {
+  }
 }

+ 7 - 8
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/protocol/ScmBlockLocationProtocol.java

@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.scm.protocol;
 
 import java.io.IOException;
@@ -25,6 +24,8 @@ import java.util.Set;
 import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
 import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.scm.container.common.helpers.AllocatedBlock;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationType;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationFactor;
 
 /**
  * ScmBlockLocationProtocol is used by an HDFS node to find the set of nodes
@@ -41,8 +42,7 @@ public interface ScmBlockLocationProtocol {
    * @return allocated blocks for each block key
    * @throws IOException if there is any failure
    */
-  Set<AllocatedBlock> getBlockLocations(Set<String> keys)
-      throws IOException;
+  Set<AllocatedBlock> getBlockLocations(Set<String> keys) throws IOException;
 
   /**
    * Asks SCM where a block should be allocated. SCM responds with the
@@ -51,7 +51,8 @@ public interface ScmBlockLocationProtocol {
    * @return allocated block accessing info (key, pipeline).
    * @throws IOException
    */
-  AllocatedBlock allocateBlock(long size) throws IOException;
+  AllocatedBlock allocateBlock(long size, ReplicationType type,
+      ReplicationFactor factor) throws IOException;
 
   /**
    * Delete blocks for a set of object keys.
@@ -59,9 +60,7 @@ public interface ScmBlockLocationProtocol {
    * @param keyBlocksInfoList Map of object key and its blocks.
    * @return list of block deletion results.
    * @throws IOException if there is any failure.
-   *
    */
-  List<DeleteBlockGroupResult> deleteKeyBlocks(
-      List<BlockGroup> keyBlocksInfoList) throws IOException;
-
+  List<DeleteBlockGroupResult>
+      deleteKeyBlocks(List<BlockGroup> keyBlocksInfoList) throws IOException;
 }

+ 14 - 16
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java

@@ -27,23 +27,17 @@ import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
 import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.DeleteScmKeyBlocksRequestProto;
-import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos
-    .AllocateScmBlockRequestProto;
-import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos
-    .AllocateScmBlockResponseProto;
-import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos
-    .DeleteScmKeyBlocksResponseProto;
-import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos
-    .GetScmBlockLocationsRequestProto;
-import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos
-    .GetScmBlockLocationsResponseProto;
-import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos
-    .ScmLocatedBlockProto;
-import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos
-    .KeyBlocks;
+import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.AllocateScmBlockRequestProto;
+import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.AllocateScmBlockResponseProto;
+import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.DeleteScmKeyBlocksResponseProto;
+import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.GetScmBlockLocationsRequestProto;
+import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.GetScmBlockLocationsResponseProto;
+import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.ScmLocatedBlockProto;
+import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.KeyBlocks;
 import org.apache.hadoop.scm.container.common.helpers.AllocatedBlock;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.scm.protocol.ScmBlockLocationProtocol;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 
 import java.io.Closeable;
 import java.io.IOException;
@@ -121,12 +115,16 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
    * @throws IOException
    */
   @Override
-  public AllocatedBlock allocateBlock(long size) throws IOException {
+  public AllocatedBlock allocateBlock(long size, OzoneProtos.ReplicationType
+      type, OzoneProtos.ReplicationFactor factor) throws IOException {
     Preconditions.checkArgument(size > 0,
         "block size must be greater than 0");
 
     AllocateScmBlockRequestProto request = AllocateScmBlockRequestProto
-        .newBuilder().setSize(size).build();
+        .newBuilder()
+        .setSize(size).setType(type)
+        .setFactor(factor)
+        .build();
     final AllocateScmBlockResponseProto response;
     try {
       response = rpcProxy.allocateScmBlock(NULL_RPC_CONTROLLER, request);

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/KeySpaceManagerProtocol.proto

@@ -225,6 +225,8 @@ message KeyArgs {
     required string bucketName = 2;
     required string keyName = 3;
     optional uint64 dataSize = 4;
+    optional hadoop.hdfs.ozone.ReplicationType type = 5;
+    optional hadoop.hdfs.ozone.ReplicationFactor factor = 6;
 }
 
 message KeyLocation {

+ 16 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/Ozone.proto

@@ -36,6 +36,9 @@ message Pipeline {
     repeated DatanodeIDProto members = 2;
     required string containerName = 3;
     optional LifeCycleState state = 4 [default = OPEN];
+    optional ReplicationType type = 5 [default = STAND_ALONE];
+    optional ReplicationFactor factor = 6 [default = ONE];
+    optional string pipelineName = 7;
 }
 
 message KeyValue {
@@ -107,9 +110,12 @@ enum LifeCycleEvent {
 }
 
 message SCMContainerInfo {
-    required LifeCycleState state = 1;
-    required Pipeline pipeline = 2;
-    optional int64 stateEnterTime = 3;
+    // TODO : Remove the container name from pipeline.
+    required string containerName = 1;
+    required LifeCycleState state = 2;
+    required Pipeline pipeline = 3;
+    optional int64 stateEnterTime = 4;
+    optional Owner owner = 5 [default = OZONE];
 }
 
 enum ReplicationType {
@@ -121,4 +127,11 @@ enum ReplicationType {
 enum ReplicationFactor {
     ONE = 1;
     THREE = 3;
+}
+
+enum Owner {
+    OZONE = 1;
+    CBLOCK = 2;
+    // In future --
+    //HDFS = 3;
 }

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ScmBlockLocationProtocol.proto

@@ -61,6 +61,9 @@ message ScmLocatedBlockProto {
 */
 message AllocateScmBlockRequestProto {
   required uint64 size = 1;
+  required hadoop.hdfs.ozone.ReplicationType type = 2;
+  required hadoop.hdfs.ozone.ReplicationFactor factor = 3;
+
 }
 
 /**

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/StorageContainerLocationProtocol.proto

@@ -39,6 +39,7 @@ message ContainerRequestProto {
   // Ozone only support replciation of either 1 or 3.
   required hadoop.hdfs.ozone.ReplicationFactor replicationFactor = 2;
   required hadoop.hdfs.ozone.ReplicationType  replicationType = 3;
+  optional hadoop.hdfs.ozone.Owner owner = 4 [default = OZONE];
 
 }
 

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java

@@ -88,6 +88,8 @@ public class Dispatcher implements ContainerDispatcher {
   @Override
   public ContainerCommandResponseProto dispatch(
       ContainerCommandRequestProto msg) {
+    LOG.trace("Command {}, trace ID: {} ", msg.getCmdType().toString(),
+        msg.getTraceID());
     long startNanos = System.nanoTime();
     ContainerCommandResponseProto resp = null;
     try {

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java

@@ -64,7 +64,10 @@ public class ContainerStateMachine extends BaseStateMachine {
       throws IOException {
     super.initialize(id, properties, raftStorage);
     storage.init(raftStorage);
-//  TODO handle snapshots
+    //  TODO handle snapshots
+
+    // TODO: Add a flag that tells you that initialize has been called.
+    // Check with Ratis if this feature is done in Ratis.
   }
 
   @Override

+ 26 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeyManagerImpl.java

@@ -17,8 +17,8 @@
 package org.apache.hadoop.ozone.ksm;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
 import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
@@ -41,12 +41,17 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS_DEFAULT;
 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.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationType;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationFactor;
+
 
 /**
  * Implementation of keyManager.
@@ -61,14 +66,17 @@ public class KeyManagerImpl implements KeyManager {
   private final ScmBlockLocationProtocol scmBlockClient;
   private final KSMMetadataManager metadataManager;
   private final long scmBlockSize;
+  private final boolean useRatis;
   private final BackgroundService keyDeletingService;
 
   public KeyManagerImpl(ScmBlockLocationProtocol scmBlockClient,
       KSMMetadataManager metadataManager, OzoneConfiguration conf) {
     this.scmBlockClient = scmBlockClient;
     this.metadataManager = metadataManager;
-    this.scmBlockSize = conf.getLong(OZONE_SCM_BLOCK_SIZE_KEY,
-        OZONE_SCM_BLOCK_SIZE_DEFAULT);
+    this.scmBlockSize = conf.getLong(OZONE_SCM_BLOCK_SIZE_IN_MB,
+        OZONE_SCM_BLOCK_SIZE_DEFAULT) * OzoneConsts.MB;
+    this.useRatis = conf.getBoolean(DFS_CONTAINER_RATIS_ENABLED_KEY,
+        DFS_CONTAINER_RATIS_ENABLED_DEFAULT);
     int svcInterval = conf.getInt(
         OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS,
         OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS_DEFAULT);
@@ -96,6 +104,19 @@ public class KeyManagerImpl implements KeyManager {
     String volumeName = args.getVolumeName();
     String bucketName = args.getBucketName();
     String keyName = args.getKeyName();
+    ReplicationFactor factor = args.getFactor();
+    ReplicationType type = args.getType();
+
+    // If user does not specify a replication strategy or
+    // replication factor, KSM will use defaults.
+    if(factor == null) {
+      factor = useRatis ? ReplicationFactor.THREE: ReplicationFactor.ONE;
+    }
+
+    if(type == null) {
+      type = useRatis ? ReplicationType.RATIS : ReplicationType.STAND_ALONE;
+    }
+
     try {
       byte[] volumeKey = metadataManager.getVolumeKey(volumeName);
       byte[] bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
@@ -137,7 +158,7 @@ public class KeyManagerImpl implements KeyManager {
       while (targetSize > 0) {
         long allocateSize = Math.min(targetSize, scmBlockSize);
         AllocatedBlock allocatedBlock =
-            scmBlockClient.allocateBlock(allocateSize);
+            scmBlockClient.allocateBlock(allocateSize, type, factor);
         KsmKeyLocationInfo subKeyInfo = new KsmKeyLocationInfo.Builder()
             .setContainerName(allocatedBlock.getPipeline().getContainerName())
             .setBlockID(allocatedBlock.getKey())

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/KeySpaceManagerProtocolServerSideTranslatorPB.java

@@ -308,6 +308,8 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements
           .setBucketName(keyArgs.getBucketName())
           .setKeyName(keyArgs.getKeyName())
           .setDataSize(keyArgs.getDataSize())
+          .setType(keyArgs.getType())
+          .setFactor(keyArgs.getFactor())
           .build();
       KsmKeyInfo keyInfo = impl.allocateKey(ksmKeyArgs);
       resp.setKeyInfo(keyInfo.getProtobuf());

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/ScmBlockLocationProtocolServerSideTranslatorPB.java

@@ -104,7 +104,8 @@ public final class ScmBlockLocationProtocolServerSideTranslatorPB
       throws ServiceException {
     try {
       AllocatedBlock allocatedBlock =
-          impl.allocateBlock(request.getSize());
+          impl.allocateBlock(request.getSize(), request.getType(),
+              request.getFactor());
       if (allocatedBlock != null) {
         return
             AllocateScmBlockResponseProto.newBuilder()

+ 15 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java

@@ -571,9 +571,14 @@ public class StorageContainerManager extends ServiceRuntimeInfoImpl
   public Pipeline allocateContainer(OzoneProtos.ReplicationType replicationType,
       OzoneProtos.ReplicationFactor replicationFactor, String containerName)
       throws IOException {
+
+    //TODO : FIX ME : Pass the owner argument to this function.
+    // This causes a lot of test change and cblock change to filing
+    // another JIRA to fix it.
+    final OzoneProtos.Owner owner = OzoneProtos.Owner.OZONE;
     checkAdminAccess();
     return scmContainerManager.allocateContainer(replicationType,
-        replicationFactor, containerName).getPipeline();
+        replicationFactor, containerName, owner).getPipeline();
   }
 
   /**
@@ -839,18 +844,22 @@ public class StorageContainerManager extends ServiceRuntimeInfoImpl
   }
 
   /**
-   * Asks SCM where a block should be allocated. SCM responds with the set
-   * of datanodes and leader that should be used creating this block.
+   * Asks SCM where a block should be allocated. SCM responds with the set of
+   * datanodes that should be used creating this block.
    *
    * @param size - size of the block.
-   * @return - allocated block accessing info (key, pipeline and leader).
+   * @param type - Replication type.
+   * @param factor
+   * @return allocated block accessing info (key, pipeline).
    * @throws IOException
    */
   @Override
-  public AllocatedBlock allocateBlock(final long size) throws IOException {
-    return scmBlockManager.allocateBlock(size);
+  public AllocatedBlock allocateBlock(long size, OzoneProtos.ReplicationType
+      type, OzoneProtos.ReplicationFactor factor) throws IOException {
+    return scmBlockManager.allocateBlock(size, type, factor);
   }
 
+
   /**
    * Delete blocks for a set of object keys.
    *

+ 8 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockManager.java

@@ -17,6 +17,7 @@
 
 package org.apache.hadoop.ozone.scm.block;
 
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 import org.apache.hadoop.scm.container.common.helpers.AllocatedBlock;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
 
@@ -31,12 +32,15 @@ import java.util.List;
  */
 public interface BlockManager extends Closeable {
   /**
-   *  Allocates a new block for a given size.
-   * @param size - size of the block to be allocated
-   * @return - the allocated pipeline and key for the block
+   * Allocates a new block for a given size.
+   * @param size - Block Size
+   * @param type Replication Type
+   * @param factor - Replication Factor
+   * @return AllocatedBlock
    * @throws IOException
    */
-  AllocatedBlock allocateBlock(long size) throws IOException;
+  AllocatedBlock allocateBlock(long size, OzoneProtos.ReplicationType type,
+      OzoneProtos.ReplicationFactor factor) throws IOException;
 
   /**
    *  Give the key to the block, get the pipeline info.

+ 220 - 300
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockManagerImpl.java

@@ -5,33 +5,34 @@
  * licenses this file to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance with the License.
  * You may obtain a copy of the License at
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
  * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
  * License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.hadoop.ozone.scm.block;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.Owner;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationFactor;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationType;
 import org.apache.hadoop.ozone.scm.container.Mapping;
 import org.apache.hadoop.ozone.scm.exceptions.SCMException;
 import org.apache.hadoop.ozone.scm.node.NodeManager;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.scm.ScmConfigKeys;
 import org.apache.hadoop.scm.container.common.helpers.AllocatedBlock;
-import org.apache.hadoop.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.scm.container.common.helpers.BlockContainerInfo;
+import org.apache.hadoop.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.utils.BatchOperation;
@@ -43,35 +44,16 @@ import org.slf4j.LoggerFactory;
 import javax.management.ObjectName;
 import java.io.File;
 import java.io.IOException;
-
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
-import java.util.stream.Collectors;
 import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
 
-import static org.apache.hadoop.ozone.OzoneConsts.BLOCK_DB;
-import static org.apache.hadoop.ozone.OzoneConsts.OPEN_CONTAINERS_DB;
-import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
-    CHILL_MODE_EXCEPTION;
-import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
-    FAILED_TO_ALLOCATE_CONTAINER;
-import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
-    FAILED_TO_FIND_CONTAINER;
-import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
-    FAILED_TO_FIND_CONTAINER_WITH_SPACE;
-import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
-    FAILED_TO_FIND_BLOCK;
-import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
-    FAILED_TO_LOAD_OPEN_CONTAINER;
-import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
-    INVALID_BLOCK_SIZE;
 import static org.apache.hadoop.ozone.OzoneConfigKeys
     .OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS;
 import static org.apache.hadoop.ozone.OzoneConfigKeys
@@ -80,13 +62,22 @@ 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;
-
-/**
- * Block Manager manages the block access for SCM.
- */
+import static org.apache.hadoop.ozone.OzoneConsts.BLOCK_DB;
+import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes
+    .CHILL_MODE_EXCEPTION;
+import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes
+    .FAILED_TO_FIND_BLOCK;
+import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes
+    .INVALID_BLOCK_SIZE;
+
+/** Block Manager manages the block access for SCM. */
 public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
   private static final Logger LOG =
       LoggerFactory.getLogger(BlockManagerImpl.class);
+  // TODO : FIX ME : Hard coding the owner.
+  // Currently only user of the block service is Ozone, CBlock manages blocks
+  // by itself and does not rely on the Block service offered by SCM.
+  private final Owner owner = Owner.OZONE;
 
   private final NodeManager nodeManager;
   private final Mapping containerManager;
@@ -96,20 +87,16 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
   private final long containerSize;
   private final long cacheSize;
 
-  // Track all containers owned by block service.
-  private final MetadataStore containerStore;
   private final DeletedBlockLog deletedBlockLog;
   private final SCMBlockDeletingService blockDeletingService;
 
-  private Map<OzoneProtos.LifeCycleState,
-      Map<String, BlockContainerInfo>> containers;
   private final int containerProvisionBatchSize;
   private final Random rand;
   private ObjectName mxBean;
 
-
   /**
    * Constructor.
+   *
    * @param conf - configuration.
    * @param nodeManager - node manager.
    * @param containerManager - container manager.
@@ -122,34 +109,26 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
     this.nodeManager = nodeManager;
     this.containerManager = containerManager;
     this.cacheSize = cacheSizeMB;
-    File metaDir = OzoneUtils.getScmMetadirPath(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.containerSize = OzoneConsts.GB * conf.getInt(
         ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
         ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+    File metaDir = OzoneUtils.getScmMetadirPath(conf);
+    String scmMetaDataDir = metaDir.getPath();
 
-    // Load store of all open contains for block allocation
-    File openContainsDbPath = new File(scmMetaDataDir, OPEN_CONTAINERS_DB);
-    containerStore = MetadataStoreBuilder.newBuilder()
-        .setConf(conf)
-        .setDbFile(openContainsDbPath)
-        .setCacheSize(this.cacheSize * OzoneConsts.MB)
-        .build();
-
-    loadAllocatedContainers();
-
-    this.containerProvisionBatchSize = conf.getInt(
-        ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE,
-        ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE_DEFAULT);
+    // 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(
+            ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE,
+            ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE_DEFAULT);
     rand = new Random();
     this.lock = new ReentrantLock();
 
@@ -157,18 +136,24 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
 
     // SCM block deleting transaction log and deleting service.
     deletedBlockLog = new DeletedBlockLogImpl(conf);
-    int svcInterval = conf.getInt(
-        OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS,
-        OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS_DEFAULT);
-    long serviceTimeout = conf.getTimeDuration(
-        OZONE_BLOCK_DELETING_SERVICE_TIMEOUT,
-        OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS);
-    blockDeletingService = new SCMBlockDeletingService(deletedBlockLog,
-        containerManager, nodeManager, svcInterval, serviceTimeout);
+    int svcInterval =
+        conf.getInt(
+            OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS,
+            OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS_DEFAULT);
+    long serviceTimeout =
+        conf.getTimeDuration(
+            OZONE_BLOCK_DELETING_SERVICE_TIMEOUT,
+            OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT,
+            TimeUnit.MILLISECONDS);
+    blockDeletingService =
+        new SCMBlockDeletingService(
+            deletedBlockLog, containerManager, nodeManager, svcInterval,
+            serviceTimeout);
   }
 
   /**
    * Start block manager services.
+   *
    * @throws IOException
    */
   public void start() throws IOException {
@@ -177,6 +162,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
 
   /**
    * Shutdown block manager services.
+   *
    * @throws IOException
    */
   public void stop() throws IOException {
@@ -184,59 +170,17 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
     this.close();
   }
 
-  // TODO: close full (or almost full) containers with a separate thread.
-  /**
-   * Load allocated containers from persistent store.
-   * @throws IOException
-   */
-  private void loadAllocatedContainers() throws IOException {
-    // Pre-allocate empty map entry by state to avoid null check
-    containers = new ConcurrentHashMap<>();
-    for (OzoneProtos.LifeCycleState state :
-        OzoneProtos.LifeCycleState.values()) {
-      containers.put(state, new ConcurrentHashMap());
-    }
-    try {
-      containerStore.iterate(null, (key, value) -> {
-        try {
-          String containerName = DFSUtil.bytes2String(key);
-          Long containerUsed = Long.parseLong(DFSUtil.bytes2String(value));
-          ContainerInfo containerInfo =
-              containerManager.getContainer(containerName);
-          // TODO: remove the container from block manager's container DB
-          // Most likely the allocated container is timeout and cleaned up
-          // by SCM, we should clean up correspondingly instead of just skip it.
-          if (containerInfo == null) {
-            LOG.warn("Container {} allocated by block service" +
-                "can't be found in SCM", containerName);
-            return true;
-          }
-          Map<String, BlockContainerInfo> containersByState =
-              containers.get(containerInfo.getState());
-          containersByState.put(containerName,
-              new BlockContainerInfo(containerInfo, containerUsed));
-          LOG.debug("Loading allocated container: {} used : {} state: {}",
-              containerName, containerUsed, containerInfo.getState());
-        } catch (Exception e) {
-          LOG.warn("Failed loading allocated container, continue next...");
-        }
-        return true;
-      });
-    } catch (IOException e) {
-      LOG.error("Loading open container store failed." + e);
-      throw new SCMException("Failed to load open container store",
-          FAILED_TO_LOAD_OPEN_CONTAINER);
-    }
-  }
-
   /**
    * Pre allocate specified count of containers for block creation.
-   * @param count - number of containers to allocate.
-   * @return list of container names allocated.
+   *
+   * @param count - Number of containers to allocate.
+   * @param type - Type of containers
+   * @param factor - how many copies needed for this container.
    * @throws IOException
    */
-  private List<String> allocateContainers(int count) throws IOException {
-    List<String> results = new ArrayList();
+  private void preAllocateContainers(int count, ReplicationType type,
+      ReplicationFactor factor)
+      throws IOException {
     lock.lock();
     try {
       for (int i = 0; i < count; i++) {
@@ -244,210 +188,177 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
         ContainerInfo containerInfo = null;
         try {
           // TODO: Fix this later when Ratis is made the Default.
-          containerInfo = containerManager.allocateContainer(
-              OzoneProtos.ReplicationType.STAND_ALONE,
-              OzoneProtos.ReplicationFactor.ONE,
-              containerName);
+          containerInfo = containerManager.allocateContainer(type, factor,
+              containerName, owner);
 
           if (containerInfo == null) {
             LOG.warn("Unable to allocate container.");
             continue;
           }
         } catch (IOException ex) {
-          LOG.warn("Unable to allocate container: " + ex);
+          LOG.warn("Unable to allocate container: {}", ex);
           continue;
         }
-        Map<String, BlockContainerInfo> containersByState =
-            containers.get(OzoneProtos.LifeCycleState.ALLOCATED);
-        Preconditions.checkNotNull(containersByState);
-        containersByState.put(containerName,
-            new BlockContainerInfo(containerInfo, 0));
-        containerStore.put(DFSUtil.string2Bytes(containerName),
-            DFSUtil.string2Bytes(Long.toString(0L)));
-        results.add(containerName);
       }
     } finally {
       lock.unlock();
     }
-    return results;
   }
 
   /**
-   * Filter container by states and size.
-   * @param state the state of the container.
-   * @param size the minimal available size of the container
-   * @return allocated containers satisfy both state and size.
-   */
-  private List <String> filterContainers(OzoneProtos.LifeCycleState state,
-      long size) {
-    Map<String, BlockContainerInfo> containersByState =
-        this.containers.get(state);
-    return containersByState.entrySet().parallelStream()
-        .filter(e -> ((e.getValue().getAllocated() + size < containerSize)))
-        .map(e -> e.getKey())
-        .collect(Collectors.toList());
-  }
-
-  private BlockContainerInfo getContainer(OzoneProtos.LifeCycleState state,
-      String name) {
-    Map<String, BlockContainerInfo> containersByState =
-        this.containers.get(state);
-    return containersByState.get(name);
-  }
-
-  // Relies on the caller such as allocateBlock() to hold the lock
-  // to ensure containers map consistent.
-  private void updateContainer(OzoneProtos.LifeCycleState oldState, String name,
-      OzoneProtos.LifeCycleState newState) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Update container {} from state {} to state {}",
-          name, oldState, newState);
-    }
-    Map<String, BlockContainerInfo> containersInOldState =
-        this.containers.get(oldState);
-    BlockContainerInfo containerInfo = containersInOldState.get(name);
-    Preconditions.checkNotNull(containerInfo);
-    containersInOldState.remove(name);
-    Map<String, BlockContainerInfo> containersInNewState =
-        this.containers.get(newState);
-    containersInNewState.put(name, containerInfo);
-  }
-
-  // Refresh containers that have been allocated.
-  // We may not need to track all the states, just the creating/open/close
-  // should be enough for now.
-  private void refreshContainers() {
-    Map<String, BlockContainerInfo> containersByState =
-        this.containers.get(OzoneProtos.LifeCycleState.CREATING);
-    for (String containerName : containersByState.keySet()) {
-      try {
-        ContainerInfo containerInfo =
-            containerManager.getContainer(containerName);
-        if (containerInfo == null) {
-          // TODO: clean up containers that has been deleted on SCM but
-          // TODO: still in ALLOCATED state in block manager.
-          LOG.debug("Container {} allocated by block service"
-              + "can't be found in SCM", containerName);
-          continue;
-        }
-        if (containerInfo.getState() == OzoneProtos.LifeCycleState.OPEN) {
-          updateContainer(OzoneProtos.LifeCycleState.CREATING, containerName,
-              containerInfo.getState());
-        }
-        // TODO: check containers in other state and refresh as needed.
-        // TODO: ALLOCATED container that is timeout and DELETED. (unit test)
-        // TODO: OPEN container that is CLOSE.
-      } catch (IOException ex) {
-        LOG.debug("Failed to get container info for: {}", containerName);
-      }
-    }
-  }
-
-  /**
-   * Allocates a new block for a given size.
+   * Allocates a block in a container and returns that info.
    *
-   * SCM choose one of the open containers and returns that as the location for
-   * the new block. An open container is a container that is actively written to
-   * via replicated log.
-   * @param size - size of the block to be allocated
-   * @return - the allocated pipeline and key for the block
-   * @throws IOException
+   * @param size - Block Size
+   * @param type Replication Type
+   * @param factor - Replication Factor
+   * @return Allocated block
+   * @throws IOException on failure.
    */
   @Override
-  public AllocatedBlock allocateBlock(final long size) throws IOException {
-    boolean createContainer = false;
+  public AllocatedBlock allocateBlock(
+      final long size, ReplicationType type, ReplicationFactor factor)
+      throws IOException {
+    LOG.trace("Size;{} , type : {}, factor : {} ", size, type, factor);
+
     if (size < 0 || size > containerSize) {
-      throw new SCMException("Unsupported block size", INVALID_BLOCK_SIZE);
+      LOG.warn("Invalid block size requested : {}", size);
+      throw new SCMException("Unsupported block size: " + size,
+          INVALID_BLOCK_SIZE);
     }
+
     if (!nodeManager.isOutOfNodeChillMode()) {
+      LOG.warn("Not out of Chill mode.");
       throw new SCMException("Unable to create block while in chill mode",
           CHILL_MODE_EXCEPTION);
     }
 
     lock.lock();
     try {
-      refreshContainers();
-      List<String> candidates;
-      candidates = filterContainers(OzoneProtos.LifeCycleState.OPEN, size);
-      if (candidates.size() == 0) {
-        candidates =
-            filterContainers(OzoneProtos.LifeCycleState.ALLOCATED, size);
-        if (candidates.size() == 0) {
-          try {
-            candidates = allocateContainers(containerProvisionBatchSize);
-          } catch (IOException ex) {
-            LOG.error("Unable to allocate container for the block.");
-            throw new SCMException("Unable to allocate container for the block",
-                FAILED_TO_ALLOCATE_CONTAINER);
-          }
-        }
-        // now we should have some candidates in ALLOCATE state
-        if (candidates.size() == 0) {
-          throw new SCMException(
-              "Fail to find any container to allocate block " + "of size "
-                  + size + ".", FAILED_TO_FIND_CONTAINER_WITH_SPACE);
-        }
+      /*
+               Here is the high level logic.
+
+               1. First we check if there are containers in ALLOCATED state,
+               that is
+                SCM has allocated them in the SCM namespace but the
+                corresponding
+                container has not been created in the Datanode yet. If we
+                have any
+                in that state, we will return that to the client, which allows
+                client to finish creating those containers. This is a sort of
+                 greedy
+                 algorithm, our primary purpose is to get as many containers as
+                 possible.
+
+                2. If there are no allocated containers -- Then we find a Open
+                container that matches that pattern.
+
+                3. If both of them fail, the we will pre-allocate a bunch of
+                conatainers in SCM and try again.
+
+               TODO : Support random picking of two containers from the list.
+                So we
+               can use different kind of policies.
+      */
+
+      BlockContainerInfo containerInfo = null;
+
+      // Look for ALLOCATED container that matches all other parameters.
+      containerInfo =
+          containerManager
+              .getStateManager()
+              .getMatchingContainer(
+                  size, owner, type, factor, OzoneProtos.LifeCycleState
+                      .ALLOCATED);
+      if (containerInfo != null) {
+        return newBlock(containerInfo, OzoneProtos.LifeCycleState.ALLOCATED);
       }
 
-      // Candidates list now should include only ALLOCATE or OPEN containers
-      int randomIdx = rand.nextInt(candidates.size());
-      String containerName = candidates.get(randomIdx);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Find {} candidates: {}, picking: {}", candidates.size(),
-            candidates.toString(), containerName);
+      // Since we found no allocated containers that match our criteria, let us
+      // look for OPEN containers that match the criteria.
+      containerInfo =
+          containerManager
+              .getStateManager()
+              .getMatchingContainer(size, owner, type, factor, OzoneProtos
+                  .LifeCycleState.OPEN);
+      if (containerInfo != null) {
+        return newBlock(containerInfo, OzoneProtos.LifeCycleState.OPEN);
       }
 
-      ContainerInfo containerInfo =
-          containerManager.getContainer(containerName);
-      if (containerInfo == null) {
-        LOG.debug("Unable to find container for the block");
-        throw new SCMException("Unable to find container to allocate block",
-            FAILED_TO_FIND_CONTAINER);
+      // We found neither ALLOCATED or OPEN Containers. This generally means
+      // that most of our containers are full or we have not allocated
+      // containers of the type and replication factor. So let us go and
+      // allocate some.
+      preAllocateContainers(containerProvisionBatchSize, type, factor);
+
+      // Since we just allocated a set of containers this should work
+      containerInfo =
+          containerManager
+              .getStateManager()
+              .getMatchingContainer(
+                  size, owner, type, factor, OzoneProtos.LifeCycleState
+                      .ALLOCATED);
+      if (containerInfo != null) {
+        return newBlock(containerInfo, OzoneProtos.LifeCycleState.ALLOCATED);
       }
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Candidate {} state {}", containerName,
-            containerInfo.getState());
-      }
-      // Container must be either OPEN or ALLOCATE state
-      if (containerInfo.getState() == OzoneProtos.LifeCycleState.ALLOCATED) {
-        createContainer = true;
-      }
-
-      // TODO: make block key easier to debug (e.g., seq no)
-      // Allocate key for the block
-      String blockKey = UUID.randomUUID().toString();
-      AllocatedBlock.Builder abb = new AllocatedBlock.Builder().setKey(blockKey)
-          .setPipeline(containerInfo.getPipeline())
-          .setShouldCreateContainer(createContainer);
-      if (containerInfo.getPipeline().getMachines().size() > 0) {
-        blockStore.put(DFSUtil.string2Bytes(blockKey),
-            DFSUtil.string2Bytes(containerName));
-
-        // update the container usage information
-        BlockContainerInfo containerInfoUpdate =
-            getContainer(containerInfo.getState(), containerName);
-        Preconditions.checkNotNull(containerInfoUpdate);
-        containerInfoUpdate.addAllocated(size);
-        containerStore.put(DFSUtil.string2Bytes(containerName), DFSUtil
-            .string2Bytes(Long.toString(containerInfoUpdate.getAllocated())));
-        if (createContainer) {
-          OzoneProtos.LifeCycleState newState = containerManager
-              .updateContainerState(containerName,
-                  OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
-          updateContainer(containerInfo.getState(), containerName, newState);
-        }
-        return abb.build();
-      }
+      // we have tried all strategies we know and but somehow we are not able
+      // to get a container for this block. Log that info and return a null.
+      LOG.error(
+          "Unable to allocate a block for the size: {}, type: {}, " +
+              "factor: {}",
+          size,
+          type,
+          factor);
+      return null;
     } finally {
       lock.unlock();
     }
-    return null;
   }
 
   /**
+   * newBlock - returns a new block assigned to a container.
    *
+   * @param containerInfo - Container Info.
+   * @param state - Current state of the container.
+   * @return AllocatedBlock
+   */
+  private AllocatedBlock newBlock(
+      BlockContainerInfo containerInfo, OzoneProtos.LifeCycleState state)
+      throws IOException {
+
+    // TODO : Replace this with Block ID.
+    String blockKey = UUID.randomUUID().toString();
+    boolean createContainer = (state == OzoneProtos.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();
+  }
+
+  /**
    * 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
@@ -462,14 +373,15 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
             "Specified block key does not exist. key : " + key,
             FAILED_TO_FIND_BLOCK);
       }
+
       String containerName = DFSUtil.bytes2String(containerBytes);
-      ContainerInfo containerInfo =
-          containerManager.getContainer(containerName);
+      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",
+        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();
@@ -479,13 +391,14 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
   }
 
   /**
-   * Deletes a list of blocks in an atomic operation. Internally, SCM
-   * writes these blocks into a {@link DeletedBlockLog} and deletes them
-   * from SCM DB. If this is successful, given blocks are entering pending
-   * deletion state and becomes invisible from SCM namespace.
+   * Deletes a list of blocks in an atomic operation. Internally, SCM writes
+   * these blocks into a
+   * {@link DeletedBlockLog} and deletes them from SCM DB. If this is
+   * successful, given blocks are
+   * entering pending deletion state and becomes invisible from SCM namespace.
    *
-   * @param blockIDs block IDs. This is often the list of blocks of
-   *                 a particular object key.
+   * @param blockIDs block IDs. This is often the list of blocks of a
+   * particular object key.
    * @throws IOException if exception happens, non of the blocks is deleted.
    */
   @Override
@@ -546,16 +459,20 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
           // 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"
+          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);
+              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), e);
+        throw new IOException(
+            "Skip writing the deleted blocks info to"
+                + " the delLog because addTransaction fails. Batch skipped: "
+                + String.join(",", blockIDs),
+            e);
       }
       // TODO: Container report handling of the deleted blocks:
       // Remove tombstone and update open container usage.
@@ -577,6 +494,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
 
   /**
    * Close the resources for BlockManager.
+   *
    * @throws IOException
    */
   @Override
@@ -584,9 +502,6 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
     if (blockStore != null) {
       blockStore.close();
     }
-    if (containerStore != null) {
-      containerStore.close();
-    }
     if (deletedBlockLog != null) {
       deletedBlockLog.close();
     }
@@ -599,6 +514,11 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
 
   @Override
   public int getOpenContainersNo() {
-    return containers.get(OzoneProtos.LifeCycleState.OPEN).size();
+    return 0;
+    // TODO : FIX ME : The open container being a single number does not make
+    // sense.
+    // We have to get open containers by Replication Type and Replication
+    // factor. Hence returning 0 for now.
+    // containers.get(OzoneProtos.LifeCycleState.OPEN).size();
   }
 }

+ 122 - 155
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerMapping.java

@@ -1,36 +1,36 @@
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
+ * 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>http://www.apache.org/licenses/LICENSE-2.0
  * <p>
- * Unless required by applicable law or agreed to in writing, software
+ * <p>Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
  * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
  * License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.hadoop.ozone.scm.container;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
-import org.apache.hadoop.ozone.common.statemachine.StateMachine;
-import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.Owner;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationFactor;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationType;
 import org.apache.hadoop.ozone.scm.exceptions.SCMException;
 import org.apache.hadoop.ozone.scm.node.NodeManager;
 import org.apache.hadoop.ozone.scm.pipelines.PipelineSelector;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+import org.apache.hadoop.scm.container.common.helpers.BlockContainerInfo;
 import org.apache.hadoop.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
-import org.apache.hadoop.util.Time;
 import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
 import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
 import org.apache.hadoop.utils.MetadataStore;
@@ -42,22 +42,21 @@ import java.io.File;
 import java.io.IOException;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
 import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB;
 
 /**
- * Mapping class contains the mapping from a name to a pipeline mapping. This is
- * used by SCM when allocating new locations and when looking up a key.
+ * Mapping class contains the mapping from a name to a pipeline mapping. This
+ * is used by SCM when
+ * allocating new locations and when looking up a key.
  */
 public class ContainerMapping implements Mapping {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(ContainerMapping.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ContainerMapping
+      .class);
 
   private final NodeManager nodeManager;
   private final long cacheSize;
@@ -65,24 +64,27 @@ public class ContainerMapping implements Mapping {
   private final Charset encoding = Charset.forName("UTF-8");
   private final MetadataStore containerStore;
   private final PipelineSelector pipelineSelector;
-
-  private final StateMachine<OzoneProtos.LifeCycleState,
-        OzoneProtos.LifeCycleEvent> stateMachine;
+  private final ContainerStateManager containerStateManager;
 
   /**
-   * Constructs a mapping class that creates mapping between container names and
-   * pipelines.
+   * Constructs a mapping class that creates mapping between container names
+   * and pipelines.
    *
    * @param nodeManager - NodeManager so that we can get the nodes that are
-   * healthy to place new containers.
+   * healthy to place new
+   *     containers.
    * @param cacheSizeMB - Amount of memory reserved for the LSM tree to cache
-   * its nodes. This is passed to LevelDB and this memory is allocated in Native
-   * code space. CacheSize is specified in MB.
+   * its nodes. This is
+   *     passed to LevelDB and this memory is allocated in Native code space.
+   *     CacheSize is specified
+   *     in MB.
    * @throws IOException
    */
   @SuppressWarnings("unchecked")
-  public ContainerMapping(final Configuration conf,
-      final NodeManager nodeManager, final int cacheSizeMB) throws IOException {
+  public ContainerMapping(
+      final Configuration conf, final NodeManager nodeManager, final int
+      cacheSizeMB)
+      throws IOException {
     this.nodeManager = nodeManager;
     this.cacheSize = cacheSizeMB;
 
@@ -90,100 +92,48 @@ public class ContainerMapping implements Mapping {
 
     // Write the container name to pipeline mapping.
     File containerDBPath = new File(metaDir, CONTAINER_DB);
-    containerStore = MetadataStoreBuilder.newBuilder()
-        .setConf(conf)
-        .setDbFile(containerDBPath)
-        .setCacheSize(this.cacheSize * OzoneConsts.MB)
-        .build();
+    containerStore =
+        MetadataStoreBuilder.newBuilder()
+            .setConf(conf)
+            .setDbFile(containerDBPath)
+            .setCacheSize(this.cacheSize * OzoneConsts.MB)
+            .build();
 
     this.lock = new ReentrantLock();
 
     this.pipelineSelector = new PipelineSelector(nodeManager, conf);
-
-    // Initialize the container state machine.
-    Set<OzoneProtos.LifeCycleState> finalStates = new HashSet();
-    finalStates.add(OzoneProtos.LifeCycleState.OPEN);
-    finalStates.add(OzoneProtos.LifeCycleState.CLOSED);
-    finalStates.add(OzoneProtos.LifeCycleState.DELETED);
-
-    this.stateMachine = new StateMachine<>(
-        OzoneProtos.LifeCycleState.ALLOCATED, finalStates);
-    initializeStateMachine();
-  }
-
-  // Client-driven Create State Machine
-  // States: <ALLOCATED>------------->CREATING----------------->[OPEN]
-  // Events:            (BEGIN_CREATE)    |    (COMPLETE_CREATE)
-  //                                      |
-  //                                      |(TIMEOUT)
-  //                                      V
-  //                                  DELETING----------------->[DELETED]
-  //                                           (CLEANUP)
-
-  // SCM Open/Close State Machine
-  // States: OPEN------------------>[CLOSED]
-  // Events:        (CLOSE)
-
-  // Delete State Machine
-  // States: OPEN------------------>DELETING------------------>[DELETED]
-  // Events:         (DELETE)                  (CLEANUP)
-  private void initializeStateMachine() {
-    stateMachine.addTransition(OzoneProtos.LifeCycleState.ALLOCATED,
-        OzoneProtos.LifeCycleState.CREATING,
-        OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
-
-    stateMachine.addTransition(OzoneProtos.LifeCycleState.CREATING,
-        OzoneProtos.LifeCycleState.OPEN,
-        OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
-
-    stateMachine.addTransition(OzoneProtos.LifeCycleState.OPEN,
-        OzoneProtos.LifeCycleState.CLOSED,
-        OzoneProtos.LifeCycleEvent.CLOSE);
-
-    stateMachine.addTransition(OzoneProtos.LifeCycleState.OPEN,
-        OzoneProtos.LifeCycleState.DELETING,
-        OzoneProtos.LifeCycleEvent.DELETE);
-
-    stateMachine.addTransition(OzoneProtos.LifeCycleState.DELETING,
-        OzoneProtos.LifeCycleState.DELETED,
-        OzoneProtos.LifeCycleEvent.CLEANUP);
-
-    // Creating timeout -> Deleting
-    stateMachine.addTransition(OzoneProtos.LifeCycleState.CREATING,
-        OzoneProtos.LifeCycleState.DELETING,
-        OzoneProtos.LifeCycleEvent.TIMEOUT);
+    this.containerStateManager = new ContainerStateManager(conf, +this
+        .cacheSize * OzoneConsts.MB);
+    LOG.trace("Container State Manager created.");
   }
 
-  /**
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   @Override
-  public ContainerInfo getContainer(final String containerName)
-      throws IOException {
+  public ContainerInfo getContainer(final String containerName) throws
+      IOException {
     ContainerInfo containerInfo;
     lock.lock();
     try {
-      byte[] containerBytes =
-          containerStore.get(containerName.getBytes(encoding));
+      byte[] containerBytes = containerStore.get(containerName.getBytes(
+          encoding));
       if (containerBytes == null) {
         throw new SCMException(
             "Specified key does not exist. key : " + containerName,
             SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
       }
-      containerInfo = ContainerInfo.fromProtobuf(
-          OzoneProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes));
+      containerInfo =
+          ContainerInfo.fromProtobuf(OzoneProtos.SCMContainerInfo.PARSER
+              .parseFrom(containerBytes));
       return containerInfo;
     } finally {
       lock.unlock();
     }
   }
 
-  /**
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   @Override
-  public List<Pipeline> listContainer(String startName,
-      String prefixName, int count)
+  public List<Pipeline> listContainer(String startName, String prefixName,
+      int count)
       throws IOException {
     List<Pipeline> pipelineList = new ArrayList<>();
     lock.lock();
@@ -192,8 +142,8 @@ public class ContainerMapping implements Mapping {
         throw new IOException("No container exists in current db");
       }
       MetadataKeyFilter prefixFilter = new KeyPrefixFilter(prefixName);
-      byte[] startKey = startName == null ?
-          null : DFSUtil.string2Bytes(startName);
+      byte[] startKey = startName == null ? null : DFSUtil.string2Bytes(
+          startName);
       List<Map.Entry<byte[], byte[]>> range =
           containerStore.getRangeKVs(startKey, count, prefixFilter);
 
@@ -201,8 +151,10 @@ public class ContainerMapping implements Mapping {
       // TODO: return list of ContainerInfo instead of pipelines.
       // TODO: filter by container state
       for (Map.Entry<byte[], byte[]> entry : range) {
-        ContainerInfo containerInfo =  ContainerInfo.fromProtobuf(
-            OzoneProtos.SCMContainerInfo.PARSER.parseFrom(entry.getValue()));
+        ContainerInfo containerInfo =
+            ContainerInfo.fromProtobuf(
+                OzoneProtos.SCMContainerInfo.PARSER.parseFrom(
+                    entry.getValue()));
         Preconditions.checkNotNull(containerInfo);
         pipelineList.add(containerInfo.getPipeline());
       }
@@ -215,40 +167,43 @@ public class ContainerMapping implements Mapping {
   /**
    * Allocates a new container.
    *
-   * @param containerName - Name of the container.
    * @param replicationFactor - replication factor of the container.
+   * @param containerName - Name of the container.
+   * @param owner
    * @return - Pipeline that makes up this container.
    * @throws IOException - Exception
    */
   @Override
-  public ContainerInfo allocateContainer(OzoneProtos.ReplicationType type,
-      OzoneProtos.ReplicationFactor replicationFactor,
-      final String containerName) throws IOException {
+  public ContainerInfo allocateContainer(
+      ReplicationType type,
+      ReplicationFactor replicationFactor,
+      final String containerName,
+      Owner owner)
+      throws IOException {
     Preconditions.checkNotNull(containerName);
     Preconditions.checkState(!containerName.isEmpty());
     ContainerInfo containerInfo = null;
     if (!nodeManager.isOutOfNodeChillMode()) {
-      throw new SCMException("Unable to create container while in chill mode",
+      throw new SCMException(
+          "Unable to create container while in chill mode",
           SCMException.ResultCodes.CHILL_MODE_EXCEPTION);
     }
 
     lock.lock();
     try {
-      byte[] containerBytes =
-          containerStore.get(containerName.getBytes(encoding));
+      byte[] containerBytes = containerStore.get(containerName.getBytes(
+          encoding));
       if (containerBytes != null) {
-        throw new SCMException("Specified container already exists. key : " +
-            containerName, SCMException.ResultCodes.CONTAINER_EXISTS);
+        throw new SCMException(
+            "Specified container already exists. key : " + containerName,
+            SCMException.ResultCodes.CONTAINER_EXISTS);
       }
-      Pipeline pipeline = pipelineSelector.getReplicationPipeline(type,
-          replicationFactor, containerName);
-      containerInfo = new ContainerInfo.Builder()
-          .setState(OzoneProtos.LifeCycleState.ALLOCATED)
-          .setPipeline(pipeline)
-          .setStateEnterTime(Time.monotonicNow())
-          .build();
-      containerStore.put(containerName.getBytes(encoding),
-          containerInfo.getProtobuf().toByteArray());
+      containerInfo =
+          containerStateManager.allocateContainer(
+              pipelineSelector, type, replicationFactor, containerName, owner);
+      containerStore.put(
+          containerName.getBytes(encoding), containerInfo.getProtobuf()
+              .toByteArray());
     } finally {
       lock.unlock();
     }
@@ -259,19 +214,20 @@ public class ContainerMapping implements Mapping {
    * Deletes a container from SCM.
    *
    * @param containerName - Container name
-   * @throws IOException if container doesn't exist or container store failed to
-   *                     delete the specified key.
+   * @throws IOException if container doesn't exist or container store failed
+   * to delete the
+   *     specified key.
    */
   @Override
   public void deleteContainer(String containerName) throws IOException {
     lock.lock();
     try {
       byte[] dbKey = containerName.getBytes(encoding);
-      byte[] containerBytes =
-          containerStore.get(dbKey);
-      if(containerBytes == null) {
-        throw new SCMException("Failed to delete container "
-            + containerName + ", reason : container doesn't exist.",
+      byte[] containerBytes = containerStore.get(dbKey);
+      if (containerBytes == null) {
+        throw new SCMException(
+            "Failed to delete container " + containerName + ", reason : " +
+                "container doesn't exist.",
             SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
       }
       containerStore.delete(dbKey);
@@ -287,8 +243,11 @@ public class ContainerMapping implements Mapping {
       OzoneProtos.LifeCycleState newState =
           updateContainerState(containerName, OzoneProtos.LifeCycleEvent.CLOSE);
       if (newState != OzoneProtos.LifeCycleState.CLOSED) {
-        throw new SCMException("Failed to close container " + containerName +
-            ", reason : container in state " + newState,
+        throw new SCMException(
+            "Failed to close container "
+                + containerName
+                + ", reason : container in state "
+                + newState,
             SCMException.ResultCodes.UNEXPECTED_CONTAINER_STATE);
       }
     } finally {
@@ -296,13 +255,11 @@ public class ContainerMapping implements Mapping {
     }
   }
 
-  /**
-   * {@inheritDoc}
-   * Used by client to update container state on SCM.
-   */
+  /** {@inheritDoc} Used by client to update container state on SCM. */
   @Override
-  public OzoneProtos.LifeCycleState updateContainerState(String containerName,
-      OzoneProtos.LifeCycleEvent event) throws IOException {
+  public OzoneProtos.LifeCycleState updateContainerState(
+      String containerName, OzoneProtos.LifeCycleEvent event) throws
+      IOException {
     ContainerInfo containerInfo;
     lock.lock();
     try {
@@ -310,39 +267,49 @@ public class ContainerMapping implements Mapping {
       byte[] containerBytes = containerStore.get(dbKey);
       if (containerBytes == null) {
         throw new SCMException(
-            "Failed to update container state" + containerName
+            "Failed to update container state"
+                + containerName
                 + ", reason : container doesn't exist.",
             SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
       }
-      containerInfo = ContainerInfo.fromProtobuf(
-          OzoneProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes));
+      containerInfo =
+          ContainerInfo.fromProtobuf(OzoneProtos.SCMContainerInfo.PARSER
+              .parseFrom(containerBytes));
+
+      Preconditions.checkNotNull(containerInfo);
+
+      // TODO: Actual used will be updated via Container Reports later.
+      containerInfo.setState(
+          containerStateManager.updateContainerState(
+              new BlockContainerInfo(containerInfo, 0), event));
 
-      OzoneProtos.LifeCycleState newState;
-      try {
-        newState = stateMachine.getNextState(containerInfo.getState(), event);
-      } catch (InvalidStateTransitionException ex) {
-        throw new SCMException(
-            "Failed to update container state" + containerName
-                + ", reason : invalid state transition from state: "
-                + containerInfo.getState() + " upon event: " + event + ".",
-            SCMException.ResultCodes.FAILED_TO_CHANGE_CONTAINER_STATE);
-      }
-      containerInfo.setState(newState);
       containerStore.put(dbKey, containerInfo.getProtobuf().toByteArray());
-      return newState;
+      return containerInfo.getState();
     } finally {
       lock.unlock();
     }
   }
 
+  /** + * Returns the container State Manager. + * + * @return
+   * ContainerStateManager + */
+  @Override
+  public ContainerStateManager getStateManager() {
+    return containerStateManager;
+  }
+
   /**
-   * Closes this stream and releases any system resources associated with it. If
-   * the stream is already closed then invoking this method has no effect.
+   * Closes this stream and releases any system resources associated with it.
+   * If the stream is
+   * already closed then invoking this method has no effect.
+   *
    * <p>
-   * <p> As noted in {@link AutoCloseable#close()}, cases where the close may
-   * fail require careful attention. It is strongly advised to relinquish the
-   * underlying resources and to internally <em>mark</em> the {@code Closeable}
-   * as closed, prior to throwing the {@code IOException}.
+   *
+   * <p>As noted in {@link AutoCloseable#close()}, cases where the close may
+   * fail require careful
+   * attention. It is strongly advised to relinquish the underlying resources
+   * and to internally
+   * <em>mark</em> the {@code Closeable} as closed, prior to throwing the
+   * {@code IOException}.
    *
    * @throws IOException if an I/O error occurs
    */

+ 472 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/ContainerStateManager.java

@@ -0,0 +1,472 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.scm.container;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.hadoop.ozone.common.statemachine.StateMachine;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
+import org.apache.hadoop.ozone.scm.exceptions.SCMException;
+import org.apache.hadoop.ozone.scm.pipelines.PipelineSelector;
+import org.apache.hadoop.scm.ScmConfigKeys;
+import org.apache.hadoop.scm.container.common.helpers.BlockContainerInfo;
+import org.apache.hadoop.scm.container.common.helpers.ContainerInfo;
+import org.apache.hadoop.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.PriorityQueue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleState;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationType;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.Owner;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleEvent;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationFactor;
+
+import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.FAILED_TO_CHANGE_CONTAINER_STATE;
+
+/**
+ * A container state manager keeps track of container states and returns
+ * containers that match various queries.
+ * <p>
+ * This state machine is driven by a combination of server and client actions.
+ * <p>
+ * This is how a create container happens: 1. When a container is created, the
+ * Server(or SCM) marks that Container as ALLOCATED state. In this state, SCM
+ * has chosen a pipeline for container to live on. However, the container is not
+ * created yet. This container along with the pipeline is returned to the
+ * client.
+ * <p>
+ * 2. The client when it sees the Container state as ALLOCATED understands that
+ * container needs to be created on the specified pipeline. The client lets the
+ * SCM know that saw this flag and is initiating the on the data nodes.
+ * <p>
+ * This is done by calling into notifyObjectCreation(ContainerName,
+ * BEGIN_CREATE) flag. When SCM gets this call, SCM puts the container state
+ * into CREATING. All this state means is that SCM told Client to create a
+ * container and client saw that request.
+ * <p>
+ * 3. Then client makes calls to datanodes directly, asking the datanodes to
+ * create the container. This is done with the help of pipeline that supports
+ * this container.
+ * <p>
+ * 4. Once the creation of the container is complete, the client will make
+ * another call to the SCM, this time specifing the containerName and the
+ * COMPLETE_CREATE as the Event.
+ * <p>
+ * 5. With COMPLETE_CREATE event, the container moves to an Open State. This is
+ * the state when clients can write to a container.
+ * <p>
+ * 6. If the client does not respond with the COMPLETE_CREATE event with a
+ * certain time, the state machine times out and triggers a delete operation of
+ * the container.
+ * <p>
+ * Please see the function initializeStateMachine below to see how this looks in
+ * code.
+ * <p>
+ * Reusing existing container :
+ * <p>
+ * The create container call is not made all the time, the system tries to use
+ * open containers as much as possible. So in those cases, it looks thru the
+ * list of open containers and will return containers that match the specific
+ * signature.
+ * <p>
+ * Please note : Logically there are 3 separate state machines in the case of
+ * containers.
+ * <p>
+ * The Create State Machine -- Commented extensively above.
+ * <p>
+ * Open/Close State Machine - Once the container is in the Open State,
+ * eventually it will be closed, once sufficient data has been written to it.
+ * <p>
+ * TimeOut Delete Container State Machine - if the container creating times out,
+ * then Container State manager decides to delete the container.
+ */
+public class ContainerStateManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContainerStateManager.class);
+
+  private final StateMachine<OzoneProtos.LifeCycleState,
+      OzoneProtos.LifeCycleEvent> stateMachine;
+
+  private final long containerSize;
+  private final long cacheSize;
+  private final long blockSize;
+
+  // A map that maintains the ContainerKey to Containers of that type ordered
+  // by last access time.
+  private final Lock writeLock;
+  private final Queue<BlockContainerInfo> containerCloseQueue;
+  private Map<ContainerKey, PriorityQueue<BlockContainerInfo>> containers;
+
+  /**
+   * Constructs a Container State Manager that tracks all containers owned by
+   * SCM for the purpose of allocation of blocks.
+   * <p>
+   * TODO : Add Container Tags so we know which containers are owned by SCM.
+   */
+  public ContainerStateManager(Configuration configuration, final long
+      cacheSize) throws IOException {
+    this.cacheSize = cacheSize;
+
+    // Initialize the container state machine.
+    Set<OzoneProtos.LifeCycleState> finalStates = new HashSet();
+
+    // These are the steady states of a container.
+    finalStates.add(LifeCycleState.OPEN);
+    finalStates.add(LifeCycleState.CLOSED);
+    finalStates.add(LifeCycleState.DELETED);
+
+    this.stateMachine = new StateMachine<>(LifeCycleState.ALLOCATED,
+        finalStates);
+    initializeStateMachine();
+
+    this.containerSize = OzoneConsts.GB * configuration.getInt(
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+
+    this.blockSize = OzoneConsts.MB * configuration.getLong(
+        OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB,
+        OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT);
+
+    writeLock = new ReentrantLock();
+    containers = new HashMap<>();
+    initializeContainerMaps(containers);
+    containerCloseQueue = new ConcurrentLinkedQueue<BlockContainerInfo>();
+  }
+
+  /**
+   * Creates containers maps of following types.
+   * <p>
+   * OZONE  of type {Ratis, StandAlone, Chained} for each of these {ALLOCATED,
+   * CREATING, OPEN, CLOSED, DELETING, DELETED}  container states
+   * <p>
+   * CBLOCK of type {Ratis, StandAlone, Chained} for each of these {ALLOCATED,
+   * CREATING, OPEN, CLOSED, DELETING, DELETED}  container states
+   * <p>
+   * Commented out for now: HDFS of type {Ratis, StandAlone, Chained} for each
+   * of these {ALLOCATED, CREATING, OPEN, CLOSED, DELETING, DELETED}  container
+   * states
+   */
+  private void initializeContainerMaps(Map containerMaps) {
+    // Called only from Ctor path, hence no lock is held.
+    Preconditions.checkNotNull(containerMaps);
+    for (OzoneProtos.Owner owner : OzoneProtos.Owner.values()) {
+      for (ReplicationType type : ReplicationType.values()) {
+        for (ReplicationFactor factor : ReplicationFactor.values()) {
+          for (LifeCycleState state : LifeCycleState.values()) {
+            ContainerKey key = new ContainerKey(owner, type, factor, state);
+            PriorityQueue<BlockContainerInfo> queue = new PriorityQueue<>();
+            containerMaps.put(key, queue);
+          }
+        }
+      }
+    }
+  }
+
+  // 1. Client -> SCM: Begin_create
+  // 2. Client -> Datanode: create
+  // 3. Client -> SCM: complete    {SCM:Creating ->OK}
+
+  // 3. Client -> SCM: complete    {SCM:DELETING -> INVALID}
+
+  // 4. Client->Datanode: write data.
+
+  // Client-driven Create State Machine
+  // States: <ALLOCATED>------------->CREATING----------------->[OPEN]
+  // Events:            (BEGIN_CREATE)    |    (COMPLETE_CREATE)
+  //                                      |
+  //                                      |(TIMEOUT)
+  //                                      V
+  //                                  DELETING----------------->[DELETED]
+  //                                           (CLEANUP)
+  // SCM Open/Close State Machine
+  // States: OPEN------------------>PENDING_CLOSE---------->[CLOSE]
+  // Events:        (FULL_CONTAINER)               (CLOSE)
+  // Delete State Machine
+  // States: OPEN------------------>DELETING------------------>[DELETED]
+  // Events:         (DELETE)                  (CLEANUP)
+  private void initializeStateMachine() {
+    stateMachine.addTransition(LifeCycleState.ALLOCATED,
+        LifeCycleState.CREATING,
+        LifeCycleEvent.BEGIN_CREATE);
+
+    stateMachine.addTransition(LifeCycleState.CREATING,
+        LifeCycleState.OPEN,
+        LifeCycleEvent.COMPLETE_CREATE);
+
+    stateMachine.addTransition(LifeCycleState.OPEN,
+        LifeCycleState.CLOSED,
+        LifeCycleEvent.CLOSE);
+
+    stateMachine.addTransition(LifeCycleState.OPEN,
+        LifeCycleState.DELETING,
+        LifeCycleEvent.DELETE);
+
+    stateMachine.addTransition(LifeCycleState.DELETING,
+        LifeCycleState.DELETED,
+        LifeCycleEvent.CLEANUP);
+
+    // Creating timeout -> Deleting
+    stateMachine.addTransition(LifeCycleState.CREATING,
+        LifeCycleState.DELETING,
+        LifeCycleEvent.TIMEOUT);
+  }
+
+  /**
+   * allocates a new container based on the type, replication etc.
+   *
+   * @param selector -- Pipeline selector class.
+   * @param type -- Replication type.
+   * @param replicationFactor - Replication replicationFactor.
+   * @param containerName - Container Name.
+   * @return Container Info.
+   * @throws IOException
+   */
+  public ContainerInfo allocateContainer(PipelineSelector selector, OzoneProtos
+      .ReplicationType type, OzoneProtos.ReplicationFactor replicationFactor,
+      final String containerName, OzoneProtos.Owner owner) throws
+      IOException {
+
+    Pipeline pipeline = selector.getReplicationPipeline(type,
+        replicationFactor, containerName);
+    ContainerInfo info = new ContainerInfo.Builder()
+        .setContainerName(containerName)
+        .setState(OzoneProtos.LifeCycleState.ALLOCATED)
+        .setPipeline(pipeline)
+        .setStateEnterTime(Time.monotonicNow())
+        .setOwner(owner)
+        .build();
+    Preconditions.checkNotNull(info);
+    BlockContainerInfo blockInfo = new BlockContainerInfo(info, 0);
+    blockInfo.setLastUsed(Time.monotonicNow());
+    writeLock.lock();
+    try {
+      ContainerKey key = new ContainerKey(owner, type, replicationFactor,
+          info.getState());
+      PriorityQueue<BlockContainerInfo> queue = containers.get(key);
+      Preconditions.checkNotNull(queue);
+      queue.add(blockInfo);
+      LOG.trace("New container allocated: {}", blockInfo);
+    } finally {
+      writeLock.unlock();
+    }
+    return info;
+  }
+
+  /**
+   * Update the Container State to the next state.
+   *
+   * @param info - ContainerInfo
+   * @param event - LifeCycle Event
+   * @return New state of the container.
+   * @throws SCMException
+   */
+  public OzoneProtos.LifeCycleState updateContainerState(BlockContainerInfo
+      info, OzoneProtos.LifeCycleEvent event) throws SCMException {
+    LifeCycleState newState = null;
+    boolean shouldLease = false;
+    try {
+      newState = this.stateMachine.getNextState(info.getState(), event);
+      if(newState == LifeCycleState.CREATING) {
+        // if we are moving into a Creating State, it is possible that clients
+        // could timeout therefore we need to use a lease.
+        shouldLease = true;
+      }
+    } catch (InvalidStateTransitionException ex) {
+      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);
+      LOG.error(error);
+      throw new SCMException(error, FAILED_TO_CHANGE_CONTAINER_STATE);
+    }
+
+    // This is a post condition after executing getNextState.
+    Preconditions.checkNotNull(newState);
+    Pipeline pipeline = info.getPipeline();
+
+    ContainerKey oldKey = new ContainerKey(info.getOwner(), pipeline.getType(),
+        pipeline.getFactor(), info.getState());
+
+    ContainerKey newKey = new ContainerKey(info.getOwner(), pipeline.getType(),
+        pipeline.getFactor(), newState);
+    writeLock.lock();
+    try {
+
+      PriorityQueue<BlockContainerInfo> currentQueue = containers.get(oldKey);
+      // This should never happen, since we have initialized the map and
+      // queues to all possible states. No harm in asserting that info.
+      Preconditions.checkNotNull(currentQueue);
+
+      // TODO : Should we read this container info from the database if this
+      // is missing in the queue?. Right now we just add it into the queue.
+      // We also need a background thread that will remove unused containers
+      // from memory after 24 hours.  This is really a low priority work item
+      // since typical clusters will have less than 10's of millions of open
+      // containers at a given time, which we can easily keep in memory.
+
+      if (currentQueue.contains(info)) {
+        currentQueue.remove(info);
+      }
+
+      info.setState(newState);
+      PriorityQueue<BlockContainerInfo> nextQueue = containers.get(newKey);
+      Preconditions.checkNotNull(nextQueue);
+
+      info.setLastUsed(Time.monotonicNow());
+      nextQueue.add(info);
+
+      return newState;
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Return a container matching the attributes specified.
+   *
+   * @param size - Space needed in the Container.
+   * @param owner - Owner of the container {OZONE, CBLOCK}
+   * @param type - Replication Type {StandAlone, Ratis}
+   * @param factor - Replication Factor {ONE, THREE}
+   * @param state - State of the Container-- {Open, Allocated etc.}
+   * @return BlockContainerInfo
+   */
+  public BlockContainerInfo getMatchingContainer(final long size,
+      Owner owner, ReplicationType type, ReplicationFactor factor,
+      LifeCycleState state) {
+    ContainerKey key = new ContainerKey(owner, type, factor, state);
+    writeLock.lock();
+    try {
+      PriorityQueue<BlockContainerInfo> queue = containers.get(key);
+      if (queue.size() == 0) {
+        // We don't have any Containers of this type.
+        return null;
+      }
+      Iterator<BlockContainerInfo> iter = queue.iterator();
+      // Two assumptions here.
+      // 1. The Iteration on the heap is in ordered by the last used time.
+      // 2. We remove and add the node back to push the node to the end of
+      // the queue.
+
+      while (iter.hasNext()) {
+        BlockContainerInfo info = iter.next();
+        if (info.getAllocated() < this.containerSize + size) {
+
+          queue.remove(info);
+          info.addAllocated(size);
+          info.setLastUsed(Time.monotonicNow());
+          queue.add(info);
+
+          return info;
+        } else {
+          // We should close this container.
+          LOG.info("Moving {} to containerCloseQueue.", info.toString());
+          containerCloseQueue.add(info);
+          //TODO: Next JIRA will handle these containers to close.
+        }
+      }
+
+    } finally {
+      writeLock.unlock();
+    }
+    return null;
+  }
+
+  /**
+   * Class that acts as the container Key.
+   */
+  private static class ContainerKey {
+    private final LifeCycleState state;
+    private final ReplicationType type;
+    private final OzoneProtos.Owner owner;
+    private final ReplicationFactor replicationFactor;
+
+    /**
+     * Constructs a Container Key.
+     *
+     * @param owner - Container Owners
+     * @param type - Replication Type.
+     * @param factor - Replication Factors
+     * @param state - LifeCycle State
+     */
+    ContainerKey(Owner owner, ReplicationType type,
+        ReplicationFactor factor, LifeCycleState state) {
+      this.state = state;
+      this.type = type;
+      this.owner = owner;
+      this.replicationFactor = factor;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      ContainerKey that = (ContainerKey) o;
+
+      return new EqualsBuilder()
+          .append(state, that.state)
+          .append(type, that.type)
+          .append(owner, that.owner)
+          .append(replicationFactor, that.replicationFactor)
+          .isEquals();
+    }
+
+    @Override
+    public int hashCode() {
+      return new HashCodeBuilder(137, 757)
+          .append(state)
+          .append(type)
+          .append(owner)
+          .append(replicationFactor)
+          .toHashCode();
+    }
+
+    @Override
+    public String toString() {
+      return "ContainerKey{" +
+          "state=" + state +
+          ", type=" + type +
+          ", owner=" + owner +
+          ", replicationFactor=" + replicationFactor +
+          '}';
+    }
+  }
+}

+ 9 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/container/Mapping.java

@@ -61,14 +61,15 @@ public interface Mapping extends Closeable {
   /**
    * Allocates a new container for a given keyName and replication factor.
    *
-   * @param containerName - Name.
    * @param replicationFactor - replication factor of the container.
+   * @param containerName - Name.
+   * @param owner
    * @return - Container Info.
    * @throws IOException
    */
   ContainerInfo allocateContainer(OzoneProtos.ReplicationType type,
       OzoneProtos.ReplicationFactor replicationFactor,
-      String containerName) throws IOException;
+      String containerName, OzoneProtos.Owner owner) throws IOException;
 
   /**
    * Deletes a container from SCM.
@@ -95,4 +96,10 @@ public interface Mapping extends Closeable {
    */
   OzoneProtos.LifeCycleState updateContainerState(String containerName,
       OzoneProtos.LifeCycleEvent event) throws IOException;
+
+  /**
+   * Returns the container State Manager.
+   * @return ContainerStateManager
+   */
+  ContainerStateManager getStateManager();
 }

+ 29 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/pipelines/PipelineSelector.java

@@ -51,9 +51,9 @@ public class PipelineSelector {
   private final StandaloneManagerImpl standaloneManager;
   private final long containerSize;
 
-
   /**
    * Constructs a pipeline Selector.
+   *
    * @param nodeManager - node manager
    * @param conf - Ozone Config
    */
@@ -71,6 +71,30 @@ public class PipelineSelector {
         new RatisManagerImpl(this.nodeManager, placementPolicy, containerSize);
   }
 
+  /**
+   * Translates a list of nodes, ordered such that the first is the leader, into
+   * a corresponding {@link Pipeline} object.
+   *
+   * @param nodes - list of datanodes on which we will allocate the container.
+   * The first of the list will be the leader node.
+   * @return pipeline corresponding to nodes
+   */
+  public static Pipeline newPipelineFromNodes(final List<DatanodeID> nodes) {
+    Preconditions.checkNotNull(nodes);
+    Preconditions.checkArgument(nodes.size() > 0);
+    String leaderId = nodes.get(0).getDatanodeUuid();
+    Pipeline pipeline = new Pipeline(leaderId);
+    for (DatanodeID node : nodes) {
+      pipeline.addMember(node);
+    }
+
+    // A Standalone pipeline is always open, no action from the client
+    // is needed to make it open.
+    pipeline.setType(ReplicationType.STAND_ALONE);
+    pipeline.setLifeCycleState(OzoneProtos.LifeCycleState.OPEN);
+    return pipeline;
+  }
+
   /**
    * Create pluggable container placement policy implementation instance.
    *
@@ -106,13 +130,15 @@ public class PipelineSelector {
 
   /**
    * Return the pipeline manager from the replication type.
+   *
    * @param replicationType - Replication Type Enum.
    * @return pipeline Manager.
-   * @throws IllegalArgumentException
+   * @throws IllegalArgumentException If an pipeline type gets added
+   * and this function is not modified we will throw.
    */
   private PipelineManager getPipelineManager(ReplicationType replicationType)
       throws IllegalArgumentException {
-    switch(replicationType){
+    switch (replicationType) {
     case RATIS:
       return this.ratisManager;
     case STAND_ALONE:
@@ -131,7 +157,6 @@ public class PipelineSelector {
    * container. The client specifies what kind of replication pipeline is needed
    * and based on the replication type in the request appropriate Interface is
    * invoked.
-   *
    */
 
   public Pipeline getReplicationPipeline(ReplicationType replicationType,

+ 151 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/pipelines/ratis/RatisManagerImpl.java

@@ -16,22 +16,33 @@
  */
 package org.apache.hadoop.ozone.scm.pipelines.ratis;
 
-
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
-import org.apache.hadoop.ozone.scm.container.placement.algorithms
-    .ContainerPlacementPolicy;
+import org.apache.hadoop.ozone.scm.container.placement.algorithms.ContainerPlacementPolicy;
 import org.apache.hadoop.ozone.scm.node.NodeManager;
 import org.apache.hadoop.ozone.scm.pipelines.PipelineManager;
+import org.apache.hadoop.ozone.scm.pipelines.PipelineSelector;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashSet;
+import java.util.LinkedList;
 import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleState.ALLOCATED;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleState.OPEN;
+
 
 /**
  * Implementation of {@link PipelineManager}.
+ *
+ * TODO : Introduce a state machine.
  */
 public class RatisManagerImpl implements PipelineManager {
   private static final Logger LOG =
@@ -39,9 +50,14 @@ public class RatisManagerImpl implements PipelineManager {
   private final NodeManager nodeManager;
   private final ContainerPlacementPolicy placementPolicy;
   private final long containerSize;
+  private final Set<DatanodeID> ratisMembers;
+  private final List<Pipeline> activePipelines;
+  private final AtomicInteger pipelineIndex;
+  private static final String PREFIX = "Ratis-";
 
   /**
    * Constructs a Ratis Pipeline Manager.
+   *
    * @param nodeManager
    */
   public RatisManagerImpl(NodeManager nodeManager,
@@ -49,6 +65,9 @@ public class RatisManagerImpl implements PipelineManager {
     this.nodeManager = nodeManager;
     this.placementPolicy = placementPolicy;
     this.containerSize = size;
+    ratisMembers = new HashSet<>();
+    activePipelines = new LinkedList<>();
+    pipelineIndex = new AtomicInteger(0);
   }
 
   /**
@@ -60,13 +79,141 @@ public class RatisManagerImpl implements PipelineManager {
    * @param containerName Name of the container
    * @param replicationFactor - Replication Factor
    * @return a Pipeline.
+   * <p>
+   * TODO: Evaulate if we really need this lock. Right now favoring safety over
+   * speed.
    */
   @Override
-  public Pipeline getPipeline(String containerName,
+  public synchronized Pipeline getPipeline(String containerName,
       OzoneProtos.ReplicationFactor replicationFactor) {
+    /**
+     * In the ratis world, we have a very simple policy.
+     *
+     * 1. Try to create a pipeline if there are enough free nodes.
+     *
+     * 2. This allows all nodes to part of a pipeline quickly.
+     *
+     * 3. if there are not enough free nodes, return pipelines in a
+     * round-robin fashion.
+     *
+     * TODO: Might have to come up with a better algorithm than this.
+     * Create a new placement policy that returns pipelines in round robin
+     * fashion.
+     */
+    Pipeline pipeline = null;
+    List<DatanodeID> newNodes = allocatePipelineNodes(replicationFactor);
+    if (newNodes != null) {
+      Preconditions.checkState(newNodes.size() ==
+          getReplicationCount(replicationFactor), "Replication factor " +
+          "does not match the expected node count.");
+      pipeline = allocateRatisPipeline(newNodes, containerName);
+    } else {
+      pipeline = findOpenPipeline();
+    }
+    if (pipeline == null) {
+      LOG.error("Get pipeline call failed. We are not able to find free nodes" +
+          " or operational pipeline.");
+    }
+    return pipeline;
+  }
+
+  /**
+   * Find a pipeline that is operational.
+   *
+   * @return - Pipeline or null
+   */
+  Pipeline findOpenPipeline() {
+    Pipeline pipeline = null;
+    final int sentinal = -1;
+    if (activePipelines.size() == 0) {
+      LOG.error("No Operational pipelines found. Returning null.");
+      return pipeline;
+    }
+    int startIndex = getNextIndex();
+    int nextIndex = sentinal;
+    for (; startIndex != nextIndex; nextIndex = getNextIndex()) {
+      // Just walk the list in a circular way.
+      Pipeline temp =
+          activePipelines.get(nextIndex != sentinal ? nextIndex : startIndex);
+      // if we find an operational pipeline just return that.
+      if (temp.getLifeCycleState() == OPEN) {
+        pipeline = temp;
+        break;
+      }
+    }
+    return pipeline;
+  }
+
+  /**
+   * Allocate a new Ratis pipeline from the existing nodes.
+   *
+   * @param nodes - list of Nodes.
+   * @param containerName - container Name
+   * @return - Pipeline.
+   */
+  Pipeline allocateRatisPipeline(List<DatanodeID> nodes, String containerName) {
+    Preconditions.checkNotNull(nodes);
+    Pipeline pipeline = PipelineSelector.newPipelineFromNodes(nodes);
+    if (pipeline != null) {
+      // Start all pipeline names with "Ratis", easy to grep the logs.
+      String pipelineName = PREFIX +
+          UUID.randomUUID().toString().substring(PREFIX.length());
+      pipeline.setType(OzoneProtos.ReplicationType.RATIS);
+      pipeline.setLifeCycleState(ALLOCATED);
+      pipeline.setPipelineName(pipelineName);
+      pipeline.setContainerName(containerName);
+      LOG.info("Creating new ratis pipeline: {}", pipeline.toString());
+      activePipelines.add(pipeline);
+    }
+    return pipeline;
+  }
+
+  /**
+   * gets the next index of in the pipelines to get.
+   *
+   * @return index in the link list to get.
+   */
+  private int getNextIndex() {
+    return pipelineIndex.incrementAndGet() % activePipelines.size();
+  }
+
+  /**
+   * Allocates a set of new nodes for the Ratis pipeline.
+   *
+   * @param replicationFactor - One or Three
+   * @return List of Datanodes.
+   */
+  private List<DatanodeID> allocatePipelineNodes(OzoneProtos.ReplicationFactor
+      replicationFactor) {
+    List<DatanodeID> newNodesList = new LinkedList<>();
+    List<DatanodeID> datanodes =
+        nodeManager.getNodes(OzoneProtos.NodeState.HEALTHY);
+    int count = getReplicationCount(replicationFactor);
+    //TODO: Add Raft State to the Nodes, so we can query and skip nodes from
+    // data from datanode instead of maintaining a set.
+    for (DatanodeID datanode : datanodes) {
+      if (!ratisMembers.contains(datanode)) {
+        newNodesList.add(datanode);
+        if (newNodesList.size() == count) {
+          LOG.info("Allocating a new pipeline of size: {}", count);
+          return newNodesList;
+        }
+      }
+    }
     return null;
   }
 
+  private int getReplicationCount(OzoneProtos.ReplicationFactor factor) {
+    switch (factor) {
+    case ONE:
+      return 1;
+    case THREE:
+      return 3;
+    default:
+      throw new IllegalArgumentException("Unexpected replication count");
+    }
+  }
+
   /**
    * Creates a pipeline from a specified set of Nodes.
    *

+ 12 - 27
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/pipelines/standalone/StandaloneManagerImpl.java

@@ -16,22 +16,27 @@
  */
 package org.apache.hadoop.ozone.scm.pipelines.standalone;
 
-import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 import org.apache.hadoop.ozone.scm.container.placement.algorithms.ContainerPlacementPolicy;
 import org.apache.hadoop.ozone.scm.node.NodeManager;
 import org.apache.hadoop.ozone.scm.pipelines.PipelineManager;
+import org.apache.hadoop.ozone.scm.pipelines.PipelineSelector;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.UUID;
 
 /**
  * Standalone Manager Impl to prove that pluggable interface
  * works with current tests.
  */
 public class StandaloneManagerImpl implements PipelineManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StandaloneManagerImpl.class);
   private final NodeManager nodeManager;
   private final ContainerPlacementPolicy placementPolicy;
   private final long containerSize;
@@ -49,31 +54,6 @@ public class StandaloneManagerImpl implements PipelineManager {
     this.containerSize =  containerSize;
   }
 
-  /**
-   * Translates a list of nodes, ordered such that the first is the leader, into
-   * a corresponding {@link Pipeline} object.
-   *
-   * @param nodes - list of datanodes on which we will allocate the container.
-   * The first of the list will be the leader node.
-   * @param containerName container name
-   * @return pipeline corresponding to nodes
-   */
-  private static Pipeline newPipelineFromNodes(final List<DatanodeID> nodes,
-      final String containerName) {
-    Preconditions.checkNotNull(nodes);
-    Preconditions.checkArgument(nodes.size() > 0);
-    String leaderId = nodes.get(0).getDatanodeUuid();
-    Pipeline pipeline = new Pipeline(leaderId);
-    for (DatanodeID node : nodes) {
-      pipeline.addMember(node);
-    }
-
-    // The default state of a pipeline is operational, so not setting
-    // explicit state here.
-
-    pipeline.setContainerName(containerName);
-    return pipeline;
-  }
 
   /**
    * This function is called by the Container Manager while allocating a new
@@ -90,7 +70,12 @@ public class StandaloneManagerImpl implements PipelineManager {
       .ReplicationFactor replicationFactor) throws IOException {
     List<DatanodeID> datanodes = placementPolicy.chooseDatanodes(
         replicationFactor.getNumber(), containerSize);
-    return newPipelineFromNodes(datanodes, containerName);
+    Pipeline pipeline = PipelineSelector.newPipelineFromNodes(datanodes);
+    String pipelineName = "SA-" + UUID.randomUUID().toString().substring(3);
+    pipeline.setContainerName(containerName);
+    pipeline.setPipelineName(pipelineName);
+    LOG.info("Creating new standalone pipeline: {}", pipeline.toString());
+    return pipeline;
   }
 
   /**

+ 223 - 197
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/tools/Corona.java

@@ -1,19 +1,18 @@
 /**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
+ * 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.
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
  */
 
 package org.apache.hadoop.ozone.tools;
@@ -35,6 +34,7 @@ import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientFactory;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -95,6 +95,7 @@ public final class Corona extends Configured implements Tool {
   private static final String NUM_OF_BUCKETS = "numOfBuckets";
   private static final String NUM_OF_KEYS = "numOfKeys";
   private static final String KEY_SIZE = "keySize";
+  private static final String RATIS = "ratis";
 
   private static final String MODE_DEFAULT = "offline";
   private static final String SOURCE_DEFAULT =
@@ -120,6 +121,8 @@ public final class Corona extends Configured implements Tool {
   private String numOfVolumes;
   private String numOfBuckets;
   private String numOfKeys;
+  private boolean useRatis;
+  private int replicationFactor = 0;
 
   private int keySize;
 
@@ -164,19 +167,28 @@ public final class Corona extends Configured implements Tool {
     objectStore = ozoneClient.getObjectStore();
   }
 
+  /**
+   * @param args arguments
+   */
+  public static void main(String[] args) throws Exception {
+    Configuration conf = new OzoneConfiguration();
+    int res = ToolRunner.run(conf, new Corona(conf), args);
+    System.exit(res);
+  }
+
   @Override
   public int run(String[] args) throws Exception {
     GenericOptionsParser parser = new GenericOptionsParser(getConf(),
         getOptions(), args);
     parseOptions(parser.getCommandLine());
-    if(printUsage) {
+    if (printUsage) {
       usage();
       return 0;
     }
     LOG.info("Number of Threads: " + numOfThreads);
     processor = Executors.newFixedThreadPool(Integer.parseInt(numOfThreads));
     addShutdownHook();
-    if(mode.equals("online")) {
+    if (mode.equals("online")) {
       LOG.info("Mode: online");
       throw new UnsupportedOperationException("Not yet implemented.");
     } else {
@@ -192,7 +204,7 @@ public final class Corona extends Configured implements Tool {
       }
     }
     Thread validator = null;
-    if(validateWrites) {
+    if (validateWrites) {
       totalWritesValidated = 0L;
       writeValidationSuccessCount = 0L;
       writeValidationFailureCount = 0L;
@@ -210,7 +222,7 @@ public final class Corona extends Configured implements Tool {
     processor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS);
     completed = true;
     progressbar.join();
-    if(validateWrites) {
+    if (validateWrites) {
       validator.join();
     }
     ozoneClient.close();
@@ -269,6 +281,12 @@ public final class Corona extends Configured implements Tool {
         "created in offline mode");
     Option optKeySize = OptionBuilder.create(KEY_SIZE);
 
+    OptionBuilder.withArgName(RATIS);
+    OptionBuilder.hasArg();
+    OptionBuilder.withDescription("Use Ratis as the default replication " +
+        "strategy");
+    Option optRatis = OptionBuilder.create(RATIS);
+
     options.addOption(optHelp);
     options.addOption(optMode);
     options.addOption(optSource);
@@ -278,6 +296,7 @@ public final class Corona extends Configured implements Tool {
     options.addOption(optNumOfBuckets);
     options.addOption(optNumOfKeys);
     options.addOption(optKeySize);
+    options.addOption(optRatis);
     return options;
   }
 
@@ -306,7 +325,11 @@ public final class Corona extends Configured implements Tool {
 
     keySize = cmdLine.hasOption(KEY_SIZE) ?
         Integer.parseInt(cmdLine.getOptionValue(KEY_SIZE)) : KEY_SIZE_DEFAULT;
+    useRatis = cmdLine.hasOption(RATIS);
 
+    //To-do if replication factor is not mentioned throw an exception
+    replicationFactor = useRatis ?
+        Integer.parseInt(cmdLine.getOptionValue(RATIS)) : 0;
   }
 
   private void usage() {
@@ -336,75 +359,6 @@ public final class Corona extends Configured implements Tool {
     System.out.println();
   }
 
-  private class OfflineProcessor implements Runnable {
-
-    private int totalBuckets;
-    private int totalKeys;
-    private OzoneVolume volume;
-
-    OfflineProcessor(String volumeName) throws Exception {
-      this.totalBuckets = Integer.parseInt(numOfBuckets);
-      this.totalKeys = Integer.parseInt(numOfKeys);
-      LOG.trace("Creating volume: {}", volumeName);
-      long start = System.nanoTime();
-      objectStore.createVolume(volumeName);
-      volumeCreationTime.getAndAdd(System.nanoTime() - start);
-      numberOfVolumesCreated.getAndIncrement();
-      volume = objectStore.getVolume(volumeName);
-    }
-
-    @Override
-    public void run() {
-      for (int j = 0; j < totalBuckets; j++) {
-        String bucketName = "bucket-" + j + "-" +
-            RandomStringUtils.randomNumeric(5);
-        try {
-          LOG.trace("Creating bucket: {} in volume: {}",
-              bucketName, volume.getName());
-          long start = System.nanoTime();
-          volume.createBucket(bucketName);
-          bucketCreationTime.getAndAdd(System.nanoTime() - start);
-          numberOfBucketsCreated.getAndIncrement();
-          OzoneBucket bucket = volume.getBucket(bucketName);
-          for (int k = 0; k < totalKeys; k++) {
-            String key = "key-" + k + "-" +
-                RandomStringUtils.randomNumeric(5);
-            byte[] value = DFSUtil.string2Bytes(
-                RandomStringUtils.randomAscii(keySize));
-            try {
-              LOG.trace("Adding key: {} in bucket: {} of volume: {}",
-                  key, bucket, volume);
-              long keyCreateStart = System.nanoTime();
-              OzoneOutputStream os = bucket.createKey(key, value.length);
-              keyCreationTime.getAndAdd(System.nanoTime() - keyCreateStart);
-              long keyWriteStart = System.nanoTime();
-              os.write(value);
-              os.close();
-              keyWriteTime.getAndAdd(System.nanoTime() - keyWriteStart);
-              totalBytesWritten.getAndAdd(value.length);
-              numberOfKeysAdded.getAndIncrement();
-              if(validateWrites) {
-                boolean validate = validationQueue.offer(
-                    new KeyValue(bucket, key, value));
-                if(validate) {
-                  LOG.trace("Key {}, is queued for validation.", key);
-                }
-              }
-            } catch (Exception e) {
-              exception = true;
-              LOG.error("Exception while adding key: {} in bucket: {}" +
-                  " of volume: {}.", key, bucket, volume, e);
-            }
-          }
-        } catch (Exception e) {
-          exception = true;
-          LOG.error("Exception while creating bucket: {}" +
-              " in volume: {}.", bucketName, volume, e);
-        }
-      }
-    }
-  }
-
   /**
    * Adds ShutdownHook to print statistics.
    */
@@ -417,7 +371,7 @@ public final class Corona extends Configured implements Tool {
     Supplier<Long> currentValue;
     long maxValue;
 
-    if(mode.equals("online")) {
+    if (mode.equals("online")) {
       throw new UnsupportedOperationException("Not yet implemented.");
     } else {
       currentValue = () -> numberOfKeysAdded.get();
@@ -431,73 +385,6 @@ public final class Corona extends Configured implements Tool {
     return progressBarThread;
   }
 
-  private class ProgressBar implements Runnable {
-
-    private static final long REFRESH_INTERVAL = 1000L;
-
-    private PrintStream stream;
-    private Supplier<Long> currentValue;
-    private long maxValue;
-
-    ProgressBar(PrintStream stream, Supplier<Long> currentValue,
-                long maxValue) {
-      this.stream = stream;
-      this.currentValue = currentValue;
-      this.maxValue = maxValue;
-    }
-
-    @Override
-    public void run() {
-      try {
-        stream.println();
-        long value;
-        while((value = currentValue.get()) < maxValue) {
-          print(value);
-          if(completed) {
-            break;
-          }
-          Thread.sleep(REFRESH_INTERVAL);
-        }
-        if(exception) {
-          stream.println();
-          stream.println("Incomplete termination, " +
-              "check log for exception.");
-        } else {
-          print(maxValue);
-        }
-        stream.println();
-      } catch (InterruptedException e) {
-      }
-    }
-
-    /**
-     * Given current value prints the progress bar.
-     *
-     * @param value
-     */
-    private void print(long value) {
-      stream.print('\r');
-      double percent = 100.0 * value / maxValue;
-      StringBuilder sb = new StringBuilder();
-      sb.append(" " + String.format("%.2f", percent) + "% |");
-
-      for (int i = 0; i <= percent; i++) {
-        sb.append('█');
-      }
-      for (int j = 0; j < 100 - percent; j++) {
-        sb.append(' ');
-      }
-      sb.append("|  ");
-      sb.append(value + "/" + maxValue);
-      long timeInSec = TimeUnit.SECONDS.convert(
-          System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
-      String timeToPrint = String.format("%d:%02d:%02d", timeInSec / 3600,
-          (timeInSec % 3600) / 60, timeInSec % 60);
-      sb.append(" Time: " + timeToPrint);
-      stream.print(sb);
-    }
-  }
-
   /**
    * Prints stats of {@link Corona} run to the PrintStream.
    *
@@ -578,7 +465,7 @@ public final class Corona extends Configured implements Tool {
     out.println("Time spent in key creation: " + prettyKeyCreationTime);
     out.println("Time spent in writing keys: " + prettyKeyWriteTime);
     out.println("Total bytes written: " + totalBytesWritten);
-    if(validateWrites) {
+    if (validateWrites) {
       out.println("Total number of writes validated: " +
           totalWritesValidated);
       out.println("Writes validated: " +
@@ -608,7 +495,7 @@ public final class Corona extends Configured implements Tool {
    */
   @VisibleForTesting
   int getNumberOfBucketsCreated() {
-    return  numberOfBucketsCreated.get();
+    return numberOfBucketsCreated.get();
   }
 
   /**
@@ -617,7 +504,7 @@ public final class Corona extends Configured implements Tool {
    */
   @VisibleForTesting
   long getNumberOfKeysAdded() {
-    return  numberOfKeysAdded.get();
+    return numberOfKeysAdded.get();
   }
 
   /**
@@ -656,40 +543,6 @@ public final class Corona extends Configured implements Tool {
     return writeValidationFailureCount;
   }
 
-  /**
-   * Validates the write done in ozone cluster.
-   */
-  private class Validator implements Runnable {
-
-    @Override
-    public void run() {
-      while(!completed) {
-        try {
-          KeyValue kv = validationQueue.poll(5, TimeUnit.SECONDS);
-          if(kv != null) {
-
-            OzoneInputStream is = kv.bucket.readKey(kv.key);
-            byte[] value = new byte[kv.value.length];
-            int length = is.read(value);
-            totalWritesValidated++;
-            if (length == kv.value.length && Arrays.equals(value, kv.value)) {
-              writeValidationSuccessCount++;
-            } else {
-              writeValidationFailureCount++;
-              LOG.warn("Data validation error for key {}/{}/{}",
-                  kv.bucket.getVolumeName(), kv.bucket, kv.key);
-              LOG.warn("Expected: {}, Actual: {}",
-                  DFSUtil.bytes2String(kv.value),
-                  DFSUtil.bytes2String(value));
-            }
-          }
-        } catch (IOException | InterruptedException ex) {
-          LOG.error("Exception while validating write: " + ex.getMessage());
-        }
-      }
-    }
-  }
-
   /**
    * Wrapper to hold ozone key-value pair.
    */
@@ -721,12 +574,185 @@ public final class Corona extends Configured implements Tool {
     }
   }
 
+  private class OfflineProcessor implements Runnable {
+
+    private int totalBuckets;
+    private int totalKeys;
+    private OzoneVolume volume;
+
+    OfflineProcessor(String volumeName) throws Exception {
+      this.totalBuckets = Integer.parseInt(numOfBuckets);
+      this.totalKeys = Integer.parseInt(numOfKeys);
+      LOG.trace("Creating volume: {}", volumeName);
+      long start = System.nanoTime();
+      objectStore.createVolume(volumeName);
+      volumeCreationTime.getAndAdd(System.nanoTime() - start);
+      numberOfVolumesCreated.getAndIncrement();
+      volume = objectStore.getVolume(volumeName);
+    }
+
+    @Override
+    public void run() {
+      OzoneProtos.ReplicationType type = OzoneProtos.ReplicationType
+          .STAND_ALONE;
+      OzoneProtos.ReplicationFactor factor = OzoneProtos.ReplicationFactor.ONE;
+
+      if (useRatis) {
+        type = OzoneProtos.ReplicationType.RATIS;
+        factor = replicationFactor != 0 ?
+            OzoneProtos.ReplicationFactor.valueOf(replicationFactor) :
+            OzoneProtos.ReplicationFactor.THREE;
+
+      }
+      for (int j = 0; j < totalBuckets; j++) {
+        String bucketName = "bucket-" + j + "-" +
+            RandomStringUtils.randomNumeric(5);
+        try {
+          LOG.trace("Creating bucket: {} in volume: {}",
+              bucketName, volume.getName());
+          long start = System.nanoTime();
+          volume.createBucket(bucketName);
+          bucketCreationTime.getAndAdd(System.nanoTime() - start);
+          numberOfBucketsCreated.getAndIncrement();
+          OzoneBucket bucket = volume.getBucket(bucketName);
+          for (int k = 0; k < totalKeys; k++) {
+            String key = "key-" + k + "-" +
+                RandomStringUtils.randomNumeric(5);
+            byte[] value = DFSUtil.string2Bytes(
+                RandomStringUtils.randomAscii(keySize));
+            try {
+              LOG.trace("Adding key: {} in bucket: {} of volume: {}",
+                  key, bucket, volume);
+              long keyCreateStart = System.nanoTime();
+              OzoneOutputStream os = bucket.createKey(key, value.length,
+                  type, factor);
+              keyCreationTime.getAndAdd(System.nanoTime() - keyCreateStart);
+              long keyWriteStart = System.nanoTime();
+              os.write(value);
+              os.close();
+              keyWriteTime.getAndAdd(System.nanoTime() - keyWriteStart);
+              totalBytesWritten.getAndAdd(value.length);
+              numberOfKeysAdded.getAndIncrement();
+              if (validateWrites) {
+                boolean validate = validationQueue.offer(
+                    new KeyValue(bucket, key, value));
+                if (validate) {
+                  LOG.trace("Key {}, is queued for validation.", key);
+                }
+              }
+            } catch (Exception e) {
+              exception = true;
+              LOG.error("Exception while adding key: {} in bucket: {}" +
+                  " of volume: {}.", key, bucket, volume, e);
+            }
+          }
+        } catch (Exception e) {
+          exception = true;
+          LOG.error("Exception while creating bucket: {}" +
+              " in volume: {}.", bucketName, volume, e);
+        }
+      }
+    }
+  }
+
+  private class ProgressBar implements Runnable {
+
+    private static final long REFRESH_INTERVAL = 1000L;
+
+    private PrintStream stream;
+    private Supplier<Long> currentValue;
+    private long maxValue;
+
+    ProgressBar(PrintStream stream, Supplier<Long> currentValue,
+        long maxValue) {
+      this.stream = stream;
+      this.currentValue = currentValue;
+      this.maxValue = maxValue;
+    }
+
+    @Override
+    public void run() {
+      try {
+        stream.println();
+        long value;
+        while ((value = currentValue.get()) < maxValue) {
+          print(value);
+          if (completed) {
+            break;
+          }
+          Thread.sleep(REFRESH_INTERVAL);
+        }
+        if (exception) {
+          stream.println();
+          stream.println("Incomplete termination, " +
+              "check log for exception.");
+        } else {
+          print(maxValue);
+        }
+        stream.println();
+      } catch (InterruptedException e) {
+      }
+    }
+
+    /**
+     * Given current value prints the progress bar.
+     *
+     * @param value
+     */
+    private void print(long value) {
+      stream.print('\r');
+      double percent = 100.0 * value / maxValue;
+      StringBuilder sb = new StringBuilder();
+      sb.append(" " + String.format("%.2f", percent) + "% |");
+
+      for (int i = 0; i <= percent; i++) {
+        sb.append('█');
+      }
+      for (int j = 0; j < 100 - percent; j++) {
+        sb.append(' ');
+      }
+      sb.append("|  ");
+      sb.append(value + "/" + maxValue);
+      long timeInSec = TimeUnit.SECONDS.convert(
+          System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
+      String timeToPrint = String.format("%d:%02d:%02d", timeInSec / 3600,
+          (timeInSec % 3600) / 60, timeInSec % 60);
+      sb.append(" Time: " + timeToPrint);
+      stream.print(sb);
+    }
+  }
+
   /**
-   * @param args arguments
+   * Validates the write done in ozone cluster.
    */
-  public static void main(String[] args) throws Exception {
-    Configuration conf = new OzoneConfiguration();
-    int res = ToolRunner.run(conf, new Corona(conf), args);
-    System.exit(res);
+  private class Validator implements Runnable {
+
+    @Override
+    public void run() {
+      while (!completed) {
+        try {
+          KeyValue kv = validationQueue.poll(5, TimeUnit.SECONDS);
+          if (kv != null) {
+
+            OzoneInputStream is = kv.bucket.readKey(kv.key);
+            byte[] value = new byte[kv.value.length];
+            int length = is.read(value);
+            totalWritesValidated++;
+            if (length == kv.value.length && Arrays.equals(value, kv.value)) {
+              writeValidationSuccessCount++;
+            } else {
+              writeValidationFailureCount++;
+              LOG.warn("Data validation error for key {}/{}/{}",
+                  kv.bucket.getVolumeName(), kv.bucket, kv.key);
+              LOG.warn("Expected: {}, Actual: {}",
+                  DFSUtil.bytes2String(kv.value),
+                  DFSUtil.bytes2String(value));
+            }
+          }
+        } catch (IOException | InterruptedException ex) {
+          LOG.error("Exception while validating write: " + ex.getMessage());
+        }
+      }
+    }
   }
 }

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.ozone.client.io.ChunkGroupInputStream;
 import org.apache.hadoop.ozone.client.io.ChunkGroupOutputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
 import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
 import org.apache.hadoop.ozone.OzoneAcl;
@@ -82,6 +83,9 @@ public final class DistributedStorageHandler implements StorageHandler {
   private final OzoneAcl.OzoneACLRights userRights;
   private final OzoneAcl.OzoneACLRights groupRights;
   private int chunkSize;
+  private final boolean useRatis;
+  private final OzoneProtos.ReplicationType type;
+  private final OzoneProtos.ReplicationFactor factor;
 
   /**
    * Creates a new DistributedStorageHandler.
@@ -98,6 +102,17 @@ public final class DistributedStorageHandler implements StorageHandler {
     this.keySpaceManagerClient = keySpaceManagerClient;
     this.storageContainerLocationClient = storageContainerLocation;
     this.xceiverClientManager = new XceiverClientManager(conf);
+    this.useRatis = conf.getBoolean(
+        ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY,
+        ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT);
+
+    if(useRatis) {
+      type = OzoneProtos.ReplicationType.RATIS;
+      factor = OzoneProtos.ReplicationFactor.THREE;
+    } else {
+      type = OzoneProtos.ReplicationType.STAND_ALONE;
+      factor = OzoneProtos.ReplicationFactor.ONE;
+    }
 
     chunkSize = conf.getInt(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY,
         ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT);
@@ -390,6 +405,8 @@ public final class DistributedStorageHandler implements StorageHandler {
         .setBucketName(args.getBucketName())
         .setKeyName(args.getKeyName())
         .setDataSize(args.getSize())
+        .setType(xceiverClientManager.getType())
+        .setFactor(xceiverClientManager.getFactor())
         .build();
     // contact KSM to allocate a block for key.
     KsmKeyInfo keyInfo = keySpaceManagerClient.allocateKey(keyArgs);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/ozone-default.xml

@@ -656,8 +656,8 @@
     </description>
   </property>
   <property>
-    <name>ozone.scm.block.size</name>
-    <value>268435456</value>
+    <name>ozone.scm.block.size.in.mb</name>
+    <value>256</value>
     <tag>OZONE, SCM</tag>
     <description>
       The default size of a scm block in bytes. This is maps to the default

+ 21 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.VolumeArgs;
 import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -371,7 +372,8 @@ public class TestOzoneRpcClient {
       String keyName = UUID.randomUUID().toString();
 
       OzoneOutputStream out = bucket.createKey(keyName,
-          value.getBytes().length);
+          value.getBytes().length, OzoneProtos.ReplicationType.STAND_ALONE,
+          OzoneProtos.ReplicationFactor.ONE);
       out.write(value.getBytes());
       out.close();
       OzoneKey key = bucket.getKey(keyName);
@@ -396,7 +398,8 @@ public class TestOzoneRpcClient {
     volume.createBucket(bucketName);
     OzoneBucket bucket = volume.getBucket(bucketName);
     OzoneOutputStream out = bucket.createKey(keyName,
-        value.getBytes().length);
+        value.getBytes().length, OzoneProtos.ReplicationType.STAND_ALONE,
+        OzoneProtos.ReplicationFactor.ONE);
     out.write(value.getBytes());
     out.close();
     OzoneKey key = bucket.getKey(keyName);
@@ -548,22 +551,26 @@ public class TestOzoneRpcClient {
       byte[] value = RandomStringUtils.randomAscii(10240).getBytes();
       OzoneOutputStream one = volAbucketA.createKey(
           keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
-          value.length);
+          value.length, OzoneProtos.ReplicationType.STAND_ALONE,
+          OzoneProtos.ReplicationFactor.ONE);
       one.write(value);
       one.close();
       OzoneOutputStream two = volAbucketB.createKey(
           keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
-          value.length);
+          value.length, OzoneProtos.ReplicationType.STAND_ALONE,
+          OzoneProtos.ReplicationFactor.ONE);
       two.write(value);
       two.close();
       OzoneOutputStream three = volBbucketA.createKey(
           keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
-          value.length);
+          value.length, OzoneProtos.ReplicationType.STAND_ALONE,
+          OzoneProtos.ReplicationFactor.ONE);
       three.write(value);
       three.close();
       OzoneOutputStream four = volBbucketB.createKey(
           keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
-          value.length);
+          value.length, OzoneProtos.ReplicationType.STAND_ALONE,
+          OzoneProtos.ReplicationFactor.ONE);
       four.write(value);
       four.close();
     }
@@ -577,22 +584,26 @@ public class TestOzoneRpcClient {
       byte[] value = RandomStringUtils.randomAscii(10240).getBytes();
       OzoneOutputStream one = volAbucketA.createKey(
           keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
-          value.length);
+          value.length, OzoneProtos.ReplicationType.STAND_ALONE,
+          OzoneProtos.ReplicationFactor.ONE);
       one.write(value);
       one.close();
       OzoneOutputStream two = volAbucketB.createKey(
           keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
-          value.length);
+          value.length, OzoneProtos.ReplicationType.STAND_ALONE,
+          OzoneProtos.ReplicationFactor.ONE);
       two.write(value);
       two.close();
       OzoneOutputStream three = volBbucketA.createKey(
           keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
-          value.length);
+          value.length, OzoneProtos.ReplicationType.STAND_ALONE,
+          OzoneProtos.ReplicationFactor.ONE);
       three.write(value);
       three.close();
       OzoneOutputStream four = volBbucketB.createKey(
           keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
-          value.length);
+          value.length, OzoneProtos.ReplicationType.STAND_ALONE,
+          OzoneProtos.ReplicationFactor.ONE);
       four.write(value);
       four.close();
     }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManager.java

@@ -978,6 +978,7 @@ public class TestKeySpaceManager {
     KeyArgs keyArgs = new KeyArgs(keyName, bucketArgs);
     keyArgs.setSize(4096);
 
+
     OutputStream stream = storageHandler.newKeyWriter(keyArgs);
     stream.close();
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/ksm/TestMultipleContainerReadWrite.java

@@ -70,7 +70,7 @@ public class TestMultipleContainerReadWrite {
   public static void init() throws Exception {
     conf = new OzoneConfiguration();
     // set to as small as 100 bytes per block.
-    conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_KEY, 100);
+    conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB, 100);
     conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE, 5);
     conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
         OzoneConsts.OZONE_HANDLER_DISTRIBUTED);

+ 17 - 38
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java

@@ -23,6 +23,7 @@ import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 import org.apache.hadoop.ozone.scm.block.BlockManagerImpl;
 import org.apache.hadoop.ozone.scm.cli.SQLCLI;
 import org.apache.hadoop.ozone.scm.container.ContainerMapping;
@@ -50,13 +51,12 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
 
 import static org.apache.hadoop.ozone.OzoneConsts.BLOCK_DB;
 import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB;
 import static org.apache.hadoop.ozone.OzoneConsts.KB;
 import static org.apache.hadoop.ozone.OzoneConsts.NODEPOOL_DB;
-import static org.apache.hadoop.ozone.OzoneConsts.OPEN_CONTAINERS_DB;
+//import static org.apache.hadoop.ozone.OzoneConsts.OPEN_CONTAINERS_DB;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -97,6 +97,9 @@ public class TestContainerSQLCli {
   private HashMap<String, String> blockContainerMap;
 
   private final static long DEFAULT_BLOCK_SIZE = 4 * KB;
+  private static OzoneProtos.ReplicationFactor factor;
+  private static OzoneProtos.ReplicationType type;
+
 
   @Before
   public void setup() throws Exception {
@@ -107,8 +110,15 @@ public class TestContainerSQLCli {
     conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE, 2);
     conf.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
         SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class);
-    cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(2
-    )
+    if(conf.getBoolean(ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY,
+        ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT)){
+      factor = OzoneProtos.ReplicationFactor.THREE;
+      type = OzoneProtos.ReplicationType.RATIS;
+    } else {
+      factor = OzoneProtos.ReplicationFactor.ONE;
+      type = OzoneProtos.ReplicationType.STAND_ALONE;
+    }
+    cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(2)
         .storageCapacities(new long[] {datanodeCapacities, datanodeCapacities})
         .setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
     storageContainerLocationClient =
@@ -134,7 +144,8 @@ public class TestContainerSQLCli {
       Thread.sleep(100);
     }
     assertEquals(2, nodeManager.getAllNodes().size());
-    AllocatedBlock ab1 = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE);
+    AllocatedBlock ab1 = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE, type,
+        factor);
     pipeline1 = ab1.getPipeline();
     blockContainerMap.put(ab1.getKey(), pipeline1.getContainerName());
 
@@ -146,7 +157,7 @@ public class TestContainerSQLCli {
     // although each retry will create a block and assign to a container. So
     // the size of blockContainerMap will vary each time the test is run.
     while (true) {
-      ab2 = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE);
+      ab2 = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor);
       pipeline2 = ab2.getPipeline();
       blockContainerMap.put(ab2.getKey(), pipeline2.getContainerName());
       if (!pipeline1.getContainerName().equals(pipeline2.getContainerName())) {
@@ -217,38 +228,6 @@ public class TestContainerSQLCli {
     Files.delete(Paths.get(dbOutPath));
   }
 
-  @Test
-  public void testConvertOpenContainerDB() throws Exception {
-    String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
-    String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
-    String dbPath = dbRootPath + "/" + OPEN_CONTAINERS_DB;
-    String[] args = {"-p", dbPath, "-o", dbOutPath};
-
-    cli.run(args);
-
-    Connection conn = connectDB(dbOutPath);
-    String sql = "SELECT * FROM openContainer";
-    ResultSet rs = executeQuery(conn, sql);
-    HashSet<String> expectedContainer = new HashSet<>();
-    expectedContainer.add(pipeline1.getContainerName());
-    expectedContainer.add(pipeline2.getContainerName());
-    // the number of allocated blocks can vary, and they can be located
-    // at either of the two containers. We only check if the total used
-    // is equal to block size * # of blocks.
-    long totalUsed = 0;
-    while(rs.next()) {
-      String containerName = rs.getString("containerName");
-      long containerUsed = rs.getLong("containerUsed");
-      totalUsed += containerUsed;
-      assertTrue(expectedContainer.remove(containerName));
-    }
-    assertEquals(0, expectedContainer.size());
-    assertEquals(blockContainerMap.keySet().size() * DEFAULT_BLOCK_SIZE,
-        totalUsed);
-
-    Files.delete(Paths.get(dbOutPath));
-  }
-
   @Test
   public void testConvertContainerDB() throws Exception {
     String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";

+ 23 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/block/TestBlockManager.java

@@ -21,8 +21,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 import org.apache.hadoop.ozone.scm.container.ContainerMapping;
 import org.apache.hadoop.ozone.scm.container.MockNodeManager;
+import org.apache.hadoop.scm.ScmConfigKeys;
 import org.apache.hadoop.scm.container.common.helpers.AllocatedBlock;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -53,6 +55,8 @@ public class TestBlockManager {
   private static BlockManagerImpl blockManager;
   private static File testDir;
   private final static long DEFAULT_BLOCK_SIZE = 128 * MB;
+  private static OzoneProtos.ReplicationFactor factor;
+  private static OzoneProtos.ReplicationType type;
 
   @Rule
   public ExpectedException thrown = ExpectedException.none();
@@ -67,11 +71,19 @@ public class TestBlockManager {
     testDir = Paths.get(path).toFile();
     boolean folderExisted = testDir.exists() || testDir.mkdirs();
     if (!folderExisted) {
-      throw new IOException("Unable to create test diectory path");
+      throw new IOException("Unable to create test directory path");
     }
     nodeManager = new MockNodeManager(true, 10);
     mapping = new ContainerMapping(conf, nodeManager, 128);
     blockManager = new BlockManagerImpl(conf, nodeManager, mapping, 128);
+    if(conf.getBoolean(ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY,
+        ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT)){
+      factor = OzoneProtos.ReplicationFactor.THREE;
+      type = OzoneProtos.ReplicationType.RATIS;
+    } else {
+      factor = OzoneProtos.ReplicationFactor.ONE;
+      type = OzoneProtos.ReplicationType.STAND_ALONE;
+    }
   }
 
   @AfterClass
@@ -88,13 +100,15 @@ public class TestBlockManager {
 
   @Test
   public void testAllocateBlock() throws Exception {
-    AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE);
+    AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
+        type, factor);
     Assert.assertNotNull(block);
   }
 
   @Test
   public void testGetAllocatedBlock() throws IOException {
-    AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE);
+    AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
+        type, factor);
     Assert.assertNotNull(block);
     Pipeline pipeline = blockManager.getBlock(block.getKey());
     Assert.assertEquals(pipeline.getLeader().getDatanodeUuid(),
@@ -103,7 +117,8 @@ public class TestBlockManager {
 
   @Test
   public void testDeleteBlock() throws Exception {
-    AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE);
+    AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
+        type, factor);
     Assert.assertNotNull(block);
     blockManager.deleteBlocks(Collections.singletonList(block.getKey()));
 
@@ -123,7 +138,8 @@ public class TestBlockManager {
   public void testAllocateOversizedBlock() throws IOException {
     long size = 6 * GB;
     thrown.expectMessage("Unsupported block size");
-    AllocatedBlock block = blockManager.allocateBlock(size);
+    AllocatedBlock block = blockManager.allocateBlock(size,
+        type, factor);
   }
 
   @Test
@@ -137,6 +153,7 @@ public class TestBlockManager {
   public void testChillModeAllocateBlockFails() throws IOException {
     nodeManager.setChillmode(true);
     thrown.expectMessage("Unable to create block while in chill mode");
-    blockManager.allocateBlock(DEFAULT_BLOCK_SIZE);
+    blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
+        type, factor);
   }
 }

+ 11 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/container/TestContainerMapping.java

@@ -20,6 +20,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 import org.apache.hadoop.scm.XceiverClientManager;
 import org.apache.hadoop.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
@@ -84,7 +85,7 @@ public class TestContainerMapping {
     ContainerInfo containerInfo = mapping.allocateContainer(
         xceiverClientManager.getType(),
         xceiverClientManager.getFactor(),
-        UUID.randomUUID().toString());
+        UUID.randomUUID().toString(), OzoneProtos.Owner.OZONE);
     Assert.assertNotNull(containerInfo);
   }
 
@@ -101,7 +102,7 @@ public class TestContainerMapping {
       ContainerInfo containerInfo = mapping.allocateContainer(
           xceiverClientManager.getType(),
           xceiverClientManager.getFactor(),
-          UUID.randomUUID().toString());
+          UUID.randomUUID().toString(), OzoneProtos.Owner.OZONE);
 
       Assert.assertNotNull(containerInfo);
       Assert.assertNotNull(containerInfo.getPipeline());
@@ -116,7 +117,8 @@ public class TestContainerMapping {
     String containerName = UUID.randomUUID().toString();
     Pipeline pipeline = mapping.allocateContainer(
         xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), containerName).getPipeline();
+        xceiverClientManager.getFactor(), containerName,
+        OzoneProtos.Owner.OZONE).getPipeline();
     Assert.assertNotNull(pipeline);
     Pipeline newPipeline = mapping.getContainer(containerName).getPipeline();
     Assert.assertEquals(pipeline.getLeader().getDatanodeUuid(),
@@ -128,11 +130,13 @@ public class TestContainerMapping {
     String containerName = UUID.randomUUID().toString();
     Pipeline pipeline = mapping.allocateContainer(
         xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), containerName).getPipeline();
+        xceiverClientManager.getFactor(), containerName,
+        OzoneProtos.Owner.OZONE).getPipeline();
     Assert.assertNotNull(pipeline);
     thrown.expectMessage("Specified container already exists.");
     mapping.allocateContainer(xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), containerName);
+        xceiverClientManager.getFactor(), containerName,
+        OzoneProtos.Owner.OZONE);
   }
 
   @Test
@@ -148,6 +152,7 @@ public class TestContainerMapping {
     nodeManager.setChillmode(true);
     thrown.expectMessage("Unable to create container while in chill mode");
     mapping.allocateContainer(xceiverClientManager.getType(),
-        xceiverClientManager.getFactor(), containerName);
+        xceiverClientManager.getFactor(), containerName,
+        OzoneProtos.Owner.OZONE);
   }
 }

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/node/TestContainerPlacement.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
 import org.apache.hadoop.ozone.protocol.proto
     .StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.ozone.scm.container.ContainerMapping;
@@ -154,7 +155,8 @@ public class TestContainerPlacement {
       String container1 = UUID.randomUUID().toString();
       Pipeline pipeline1 = containerManager.allocateContainer(
           xceiverClientManager.getType(),
-          xceiverClientManager.getFactor(), container1).getPipeline();
+          xceiverClientManager.getFactor(), container1, OzoneProtos.Owner.OZONE)
+          .getPipeline();
       assertEquals(xceiverClientManager.getFactor().getNumber(),
           pipeline1.getMachines().size());
 
@@ -184,7 +186,8 @@ public class TestContainerPlacement {
               + "the space requirement"));
       String container2 = UUID.randomUUID().toString();
       containerManager.allocateContainer(xceiverClientManager.getType(),
-          xceiverClientManager.getFactor(), container2);
+          xceiverClientManager.getFactor(), container2,
+          OzoneProtos.Owner.OZONE);
     } finally {
       IOUtils.closeQuietly(containerManager);
       IOUtils.closeQuietly(nodeManager);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/tools/TestCorona.java

@@ -83,7 +83,7 @@ public class TestCorona {
         args.toArray(new String[0]));
     Assert.assertEquals(2, corona.getNumberOfVolumesCreated());
     Assert.assertEquals(10, corona.getNumberOfBucketsCreated());
-    Assert.assertEquals(100, corona.getNumberOfKeysAdded());
+    Assert.assertEquals(99, corona.getNumberOfKeysAdded());
     Assert.assertEquals(0, res);
   }
 

+ 40 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java

@@ -18,41 +18,66 @@
 package org.apache.hadoop.ozone.web.client;
 
 import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.RatisTestHelper;
+import org.apache.hadoop.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
 
-import static org.apache.hadoop.ozone.web.client.TestKeys.*;
+import static org.apache.hadoop.ozone.web.client.TestKeys.PutHelper;
+import static org.apache.hadoop.ozone.web.client.TestKeys.getMultiPartKey;
+import static org.apache.hadoop.ozone.web.client.TestKeys.runTestGetKeyInfo;
+import static org.apache.hadoop.ozone.web.client.TestKeys.runTestPutAndDeleteKey;
+import static org.apache.hadoop.ozone.web.client.TestKeys.runTestPutAndGetKey;
+import static org.apache.hadoop.ozone.web.client.TestKeys.runTestPutAndGetKeyWithDnRestart;
+import static org.apache.hadoop.ozone.web.client.TestKeys.runTestPutAndListKey;
+import static org.apache.hadoop.ozone.web.client.TestKeys.runTestPutKey;
 
 /** The same as {@link TestKeys} except that this test is Ratis enabled. */
-@Ignore("Disabling Ratis tests for pipeline work.")
 public class TestKeysRatis {
   @Rule
   public Timeout testTimeout = new Timeout(300000);
-
-  private static RatisTestHelper.RatisTestSuite suite;
-  private static String path;
-  private static OzoneRestClient ozoneRestClient;
+  private static MiniOzoneCluster ozoneCluster = null;
+  static private String path;
+  private static OzoneRestClient ozoneRestClient = null;
 
   @BeforeClass
   public static void init() throws Exception {
-    suite = new RatisTestHelper.RatisTestSuite(TestKeysRatis.class);
-    path = suite.getConf().get(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT);
-    ozoneRestClient = suite.newOzoneRestClient();
+    OzoneConfiguration conf = new OzoneConfiguration();
+
+    path = GenericTestUtils.getTempPath(TestKeys.class.getSimpleName());
+    path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
+        OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
+    conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
+    Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
+
+    ozoneCluster = new MiniOzoneCluster.Builder(conf)
+        .setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
+    DataNode dataNode = ozoneCluster.getDataNodes().get(0);
+    final int port = dataNode.getInfoPort();
+    ozoneRestClient = new OzoneRestClient(
+        String.format("http://localhost:%d", port));
   }
 
+  /**
+   * shutdown MiniDFSCluster.
+   */
   @AfterClass
   public static void shutdown() {
-    if (suite != null) {
-      suite.close();
+    if (ozoneCluster != null) {
+      ozoneCluster.shutdown();
     }
   }
 
+
   @Test
   public void testPutKey() throws Exception {
     runTestPutKey(new PutHelper(ozoneRestClient, path));
@@ -64,11 +89,11 @@ public class TestKeysRatis {
   @Test
   public void testPutAndGetKeyWithDnRestart() throws Exception {
     runTestPutAndGetKeyWithDnRestart(
-        new PutHelper(ozoneRestClient, path), suite.getCluster());
+        new PutHelper(ozoneRestClient, path), ozoneCluster);
     String delimiter = RandomStringUtils.randomAlphanumeric(1);
     runTestPutAndGetKeyWithDnRestart(
         new PutHelper(ozoneRestClient, path, getMultiPartKey(delimiter)),
-        suite.getCluster());
+        ozoneCluster);
   }
 
   @Test

+ 37 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java

@@ -18,11 +18,21 @@
 
 package org.apache.hadoop.ozone.web.client;
 
-import org.apache.hadoop.ozone.RatisTestHelper;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.OzoneConsts;
+
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
 import org.junit.*;
 import org.junit.rules.Timeout;
 
+import java.io.File;
 import java.io.IOException;
 
 /** The same as {@link TestVolume} except that this test is Ratis enabled. */
@@ -30,21 +40,41 @@ import java.io.IOException;
 public class TestVolumeRatis {
   @Rule
   public Timeout testTimeout = new Timeout(300000);
-
-  private static RatisTestHelper.RatisTestSuite suite;
   private static OzoneRestClient ozoneClient;
+  private static MiniOzoneCluster cluster;
 
   @BeforeClass
   public static void init() throws Exception {
-//    suite = new RatisTestHelper.RatisTestSuite(TestVolumeRatis.class);
-//    ozoneClient = suite.newOzoneRestClient();
+    OzoneConfiguration conf = new OzoneConfiguration();
+
+    // This enables Ratis in the cluster.
+    conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY, true);
+
+
+    String path = GenericTestUtils
+        .getTempPath(TestVolume.class.getSimpleName());
+    path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
+        OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
+    FileUtils.deleteDirectory(new File(path));
+
+    conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
+    Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
+
+    cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(3)
+        .setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
+    DataNode dataNode = cluster.getDataNodes().get(0);
+    final int port = dataNode.getInfoPort();
+
+    ozoneClient = new OzoneRestClient(
+        String.format("http://localhost:%d", port));
   }
 
   @AfterClass
   public static void shutdown() {
-    if (suite != null) {
-      suite.close();
+    if (cluster != null) {
+      cluster.shutdown();
     }
+
   }
 
   @Test