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

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

Repository: hadoop
Updated Branches:
  refs/heads/trunk 2403231c8 -> 9bd5bef29


HDDS-90: Create ContainerData, Container classes. Contributed by Bharat Viswanadham


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

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

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd19b45/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkLayOutVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkLayOutVersion.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkLayOutVersion.java
new file mode 100644
index 0000000..fff68de6
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkLayOutVersion.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.common.impl;
+
+
+/**
+ * Defines layout versions for the Chunks.
+ */
+
+public final class ChunkLayOutVersion {
+
+  private final static ChunkLayOutVersion[] CHUNK_LAYOUT_VERSION_INFOS =
+      {new ChunkLayOutVersion(1, "Data without checksums.")};
+
+  private int version;
+  private String description;
+
+
+  /**
+   * Never created outside this class.
+   *
+   * @param description -- description
+   * @param version     -- version number
+   */
+  private ChunkLayOutVersion(int version, String description) {
+    this.version = version;
+    this.description = description;
+  }
+
+  /**
+   * Returns all versions.
+   *
+   * @return Version info array.
+   */
+  public static ChunkLayOutVersion[] getAllVersions() {
+    return CHUNK_LAYOUT_VERSION_INFOS.clone();
+  }
+
+  /**
+   * Returns the latest version.
+   *
+   * @return versionInfo
+   */
+  public static ChunkLayOutVersion getLatestVersion() {
+    return CHUNK_LAYOUT_VERSION_INFOS[CHUNK_LAYOUT_VERSION_INFOS.length - 1];
+  }
+
+  /**
+   * Return version.
+   *
+   * @return int
+   */
+  public int getVersion() {
+    return version;
+  }
+
+  /**
+   * Returns description.
+   * @return String
+   */
+  public String getDescription() {
+    return description;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd19b45/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
new file mode 100644
index 0000000..a4b2130
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
@@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.common.impl;
+
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
+    ContainerType;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
+    ContainerLifeCycleState;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * ContainerData is the in-memory representation of container metadata and is
+ * represented on disk by the .container file.
+ */
+public class ContainerData {
+
+  //Type of the container.
+  // For now, we support only KeyValueContainer.
+  private final ContainerType containerType;
+
+  // Unique identifier for the container
+  private final long containerId;
+
+  // Layout version of the container data
+  private final ChunkLayOutVersion layOutVersion;
+
+  // Metadata of the container will be a key value pair.
+  // This can hold information like volume name, owner etc.,
+  private final Map<String, String> metadata;
+
+  // State of the Container
+  private ContainerLifeCycleState state;
+
+  /** parameters for read/write statistics on the container. **/
+  private final AtomicLong readBytes;
+  private final AtomicLong writeBytes;
+  private final AtomicLong readCount;
+  private final AtomicLong writeCount;
+
+
+  /**
+   * Creates a ContainerData Object, which holds metadata of the container.
+   * @param type - ContainerType
+   * @param containerId - ContainerId
+   */
+  public ContainerData(ContainerType type, long containerId) {
+    this.containerType = type;
+    this.containerId = containerId;
+    this.layOutVersion = ChunkLayOutVersion.getLatestVersion();
+    this.metadata = new TreeMap<>();
+    this.state = ContainerLifeCycleState.OPEN;
+    this.readCount = new AtomicLong(0L);
+    this.readBytes =  new AtomicLong(0L);
+    this.writeCount =  new AtomicLong(0L);
+    this.writeBytes =  new AtomicLong(0L);
+  }
+
+  /**
+   * Returns the containerId.
+   */
+  public long getContainerId() {
+    return containerId;
+  }
+
+  /**
+   * Returns the type of the container.
+   * @return ContainerType
+   */
+  public ContainerType getContainerType() {
+    return containerType;
+  }
+
+
+  /**
+   * Returns the state of the container.
+   * @return ContainerLifeCycleState
+   */
+  public synchronized ContainerLifeCycleState getState() {
+    return state;
+  }
+
+  /**
+   * Set the state of the container.
+   * @param state
+   */
+  public synchronized void setState(ContainerLifeCycleState state) {
+    this.state = state;
+  }
+
+  /**
+   * Returns the layOutVersion of the actual container data format.
+   * @return layOutVersion
+   */
+  public ChunkLayOutVersion getLayOutVersion() {
+    return layOutVersion;
+  }
+
+  /**
+   * Adds metadata.
+   */
+  public void addMetadata(String key, String value) throws IOException {
+    synchronized (this.metadata) {
+      if (this.metadata.containsKey(key)) {
+        throw new IOException("This key already exists. Key " + key);
+      }
+      metadata.put(key, value);
+    }
+  }
+
+  /**
+   * Retuns metadata of the container.
+   * @return metadata
+   */
+  public Map<String, String> getMetadata() {
+    synchronized (this.metadata) {
+      return Collections.unmodifiableMap(this.metadata);
+    }
+  }
+
+  /**
+   * checks if the container is open.
+   * @return - boolean
+   */
+  public synchronized  boolean isOpen() {
+    return ContainerLifeCycleState.OPEN == state;
+  }
+
+  /**
+   * checks if the container is invalid.
+   * @return - boolean
+   */
+  public synchronized boolean isValid() {
+    return !(ContainerLifeCycleState.INVALID == state);
+  }
+
+  /**
+   * checks if the container is closed.
+   * @return - boolean
+   */
+  public synchronized  boolean isClosed() {
+    return ContainerLifeCycleState.CLOSED == state;
+  }
+
+  /**
+   * Marks this container as closed.
+   */
+  public synchronized void closeContainer() {
+    // TODO: closed or closing here
+    setState(ContainerLifeCycleState.CLOSED);
+  }
+
+  /**
+   * Get the number of bytes read from the container.
+   * @return the number of bytes read from the container.
+   */
+  public long getReadBytes() {
+    return readBytes.get();
+  }
+
+  /**
+   * Increase the number of bytes read from the container.
+   * @param bytes number of bytes read.
+   */
+  public void incrReadBytes(long bytes) {
+    this.readBytes.addAndGet(bytes);
+  }
+
+  /**
+   * Get the number of times the container is read.
+   * @return the number of times the container is read.
+   */
+  public long getReadCount() {
+    return readCount.get();
+  }
+
+  /**
+   * Increase the number of container read count by 1.
+   */
+  public void incrReadCount() {
+    this.readCount.incrementAndGet();
+  }
+
+  /**
+   * Get the number of bytes write into the container.
+   * @return the number of bytes write into the container.
+   */
+  public long getWriteBytes() {
+    return writeBytes.get();
+  }
+
+  /**
+   * Increase the number of bytes write into the container.
+   * @param bytes the number of bytes write into the container.
+   */
+  public void incrWriteBytes(long bytes) {
+    this.writeBytes.addAndGet(bytes);
+  }
+
+  /**
+   * Get the number of writes into the container.
+   * @return the number of writes into the container.
+   */
+  public long getWriteCount() {
+    return writeCount.get();
+  }
+
+  /**
+   * Increase the number of writes into the container by 1.
+   */
+  public void incrWriteCount() {
+    this.writeCount.incrementAndGet();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd19b45/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java
new file mode 100644
index 0000000..956840b
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.impl;
+
+
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+
+
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.security.NoSuchAlgorithmException;
+
+
+/**
+ * Class to perform KeyValue Container operations.
+ */
+public class KeyValueContainer implements Container {
+
+  static final Logger LOG =
+      LoggerFactory.getLogger(Container.class);
+
+  private KeyValueContainerData containerData;
+
+  public KeyValueContainer(KeyValueContainerData containerData) {
+    this.containerData = containerData;
+  }
+
+  @Override
+  public void create(ContainerData cData) throws StorageContainerException {
+
+  }
+
+  @Override
+  public void delete(boolean forceDelete)
+      throws StorageContainerException {
+
+  }
+
+  @Override
+  public void update(boolean forceUpdate)
+      throws StorageContainerException {
+
+  }
+
+  @Override
+  public ContainerData getContainerData() throws StorageContainerException {
+    return null;
+  }
+
+  @Override
+  public void close() throws StorageContainerException,
+      NoSuchAlgorithmException {
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd19b45/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
new file mode 100644
index 0000000..37eaa49
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.impl;
+
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+
+
+import java.io.IOException;
+
+/**
+ * This class represents the KeyValueContainer metadata, which is the
+ * in-memory representation of container metadata and is represented on disk
+ * by the .container file.
+ */
+public class KeyValueContainerData extends ContainerData {
+
+  // Path to Level DB/RocksDB Store.
+  private String dbPath;
+
+  // Path to Physical file system where container and checksum are stored.
+  private String containerFilePath;
+
+  //Type of DB used to store key to chunks mapping
+  private String containerDBType;
+
+  //Number of pending deletion blocks in container.
+  private int numPendingDeletionBlocks;
+
+  /**
+   * Constructs KeyValueContainerData object.
+   * @param type - containerType
+   * @param id - ContainerId
+   */
+  public KeyValueContainerData(ContainerProtos.ContainerType type, long id) {
+    super(type, id);
+    this.numPendingDeletionBlocks = 0;
+  }
+
+  /**
+   * Returns path.
+   *
+   * @return - path
+   */
+  public String getDBPath() {
+    return dbPath;
+  }
+
+  /**
+   * Sets path.
+   *
+   * @param path - String.
+   */
+  public void setDBPath(String path) {
+    this.dbPath = path;
+  }
+
+  /**
+   * Get container file path.
+   * @return - Physical path where container file and checksum is stored.
+   */
+  public String getContainerPath() {
+    return containerFilePath;
+  }
+
+  /**
+   * Set container Path.
+   * @param containerPath - File path.
+   */
+  public void setContainerPath(String containerPath) {
+    this.containerFilePath = containerPath;
+  }
+
+  /**
+   * Returns the DBType used for the container.
+   * @return containerDBType
+   */
+  public String getContainerDBType() {
+    return containerDBType;
+  }
+
+  /**
+   * Sets the DBType used for the container.
+   * @param containerDBType
+   */
+  public void setContainerDBType(String containerDBType) {
+    this.containerDBType = containerDBType;
+  }
+
+  /**
+   * Returns the number of pending deletion blocks in container.
+   * @return numPendingDeletionBlocks
+   */
+  public int getNumPendingDeletionBlocks() {
+    return numPendingDeletionBlocks;
+  }
+
+
+  /**
+   * Increase the count of pending deletion blocks.
+   *
+   * @param numBlocks increment number
+   */
+  public void incrPendingDeletionBlocks(int numBlocks) {
+    this.numPendingDeletionBlocks += numBlocks;
+  }
+
+  /**
+   * Decrease the count of pending deletion blocks.
+   *
+   * @param numBlocks decrement number
+   */
+  public void decrPendingDeletionBlocks(int numBlocks) {
+    this.numPendingDeletionBlocks -= numBlocks;
+  }
+
+
+  /**
+   * Constructs a KeyValueContainerData object from ProtoBuf classes.
+   *
+   * @param protoData - ProtoBuf Message
+   * @throws IOException
+   */
+  public static KeyValueContainerData getFromProtoBuf(
+      ContainerProtos.CreateContainerData protoData) throws IOException {
+
+    long containerID;
+    ContainerProtos.ContainerType containerType;
+
+    containerID = protoData.getContainerId();
+    containerType = protoData.getContainerType();
+
+    KeyValueContainerData keyValueContainerData = new KeyValueContainerData(
+        containerType, containerID);
+
+    for (int x = 0; x < protoData.getMetadataCount(); x++) {
+      keyValueContainerData.addMetadata(protoData.getMetadata(x).getKey(),
+          protoData.getMetadata(x).getValue());
+    }
+
+    return keyValueContainerData;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd19b45/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
new file mode 100644
index 0000000..a680e6a
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.interfaces;
+
+
+import org.apache.hadoop.hdds.scm.container.common.helpers.
+    StorageContainerException;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+
+import java.security.NoSuchAlgorithmException;
+
+/**
+ * Interface for Container Operations.
+ */
+public interface Container {
+
+  /**
+   * Creates a container.
+   *
+   * @throws StorageContainerException
+   */
+  void create(ContainerData containerData) throws StorageContainerException;
+
+  /**
+   * Deletes the container.
+   *
+   * @param forceDelete   - whether this container should be deleted forcibly.
+   * @throws StorageContainerException
+   */
+  void delete(boolean forceDelete) throws StorageContainerException;
+
+  /**
+   * Update the container.
+   *
+   * @param forceUpdate if true, update container forcibly.
+   * @throws StorageContainerException
+   */
+  void update(boolean forceUpdate)
+      throws StorageContainerException;
+
+  /**
+   * Get metadata about the container.
+   *
+   * @return ContainerData - Container Data.
+   * @throws StorageContainerException
+   */
+  ContainerData getContainerData() throws StorageContainerException;
+
+  /**
+   * Closes a open container, if it is already closed or does not exist a
+   * StorageContainerException is thrown.
+   *
+   * @throws StorageContainerException
+   */
+  void close() throws StorageContainerException,
+      NoSuchAlgorithmException;
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd19b45/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestChunkLayOutVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestChunkLayOutVersion.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestChunkLayOutVersion.java
new file mode 100644
index 0000000..a4e0028
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestChunkLayOutVersion.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common;
+
+import org.apache.hadoop.ozone.container.common.impl.ChunkLayOutVersion;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * This class tests ChunkLayOutVersion.
+ */
+public class TestChunkLayOutVersion {
+
+  @Test
+  public void testChunkLayOutVersion() {
+
+    // Check Latest Version and description
+    Assert.assertEquals(1, ChunkLayOutVersion.getLatestVersion().getVersion());
+    Assert.assertEquals("Data without checksums.", ChunkLayOutVersion
+        .getLatestVersion().getDescription());
+
+    Assert.assertEquals(1, ChunkLayOutVersion.getAllVersions().length);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd19b45/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
new file mode 100644
index 0000000..1541921
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common;
+
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This class is used to test the KeyValueContainerData.
+ */
+public class TestKeyValueContainerData {
+
+  @Test
+  public void testGetFromProtoBuf() throws IOException {
+
+    long containerId = 1L;
+    ContainerProtos.ContainerType containerType = ContainerProtos
+        .ContainerType.KeyValueContainer;
+    String path = "/tmp";
+    String containerDBType = "RocksDB";
+    int layOutVersion = 1;
+    ContainerProtos.ContainerLifeCycleState state = ContainerProtos
+        .ContainerLifeCycleState.OPEN;
+
+    ContainerProtos.KeyValue.Builder keyValBuilder =
+        ContainerProtos.KeyValue.newBuilder();
+    ContainerProtos.CreateContainerData containerData = ContainerProtos
+        .CreateContainerData.newBuilder()
+        .setContainerType(containerType)
+        .setContainerId(containerId)
+        .addMetadata(0, keyValBuilder.setKey("VOLUME").setValue("ozone")
+            .build())
+        .addMetadata(1, keyValBuilder.setKey("OWNER").setValue("hdfs")
+            .build()).build();
+
+    KeyValueContainerData kvData = KeyValueContainerData.getFromProtoBuf(
+        containerData);
+
+    assertEquals(containerType, kvData.getContainerType());
+    assertEquals(containerId, kvData.getContainerId());
+    assertEquals(layOutVersion, kvData.getLayOutVersion().getVersion());
+    assertEquals(state, kvData.getState());
+    assertEquals(2, kvData.getMetadata().size());
+    assertEquals("ozone", kvData.getMetadata().get("VOLUME"));
+    assertEquals("hdfs", kvData.getMetadata().get("OWNER"));
+
+  }
+
+  @Test
+  public void testKeyValueData() {
+    long containerId = 1L;
+    ContainerProtos.ContainerType containerType = ContainerProtos
+        .ContainerType.KeyValueContainer;
+    String path = "/tmp";
+    String containerDBType = "RocksDB";
+    int layOutVersion = 1;
+    ContainerProtos.ContainerLifeCycleState state = ContainerProtos
+        .ContainerLifeCycleState.CLOSED;
+    AtomicLong val = new AtomicLong(0);
+    AtomicLong updatedVal = new AtomicLong(100);
+
+    KeyValueContainerData kvData = new KeyValueContainerData(containerType,
+        containerId);
+
+    assertEquals(containerType, kvData.getContainerType());
+    assertEquals(containerId, kvData.getContainerId());
+    assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN, kvData
+        .getState());
+    assertEquals(0, kvData.getMetadata().size());
+    assertEquals(0, kvData.getNumPendingDeletionBlocks());
+    assertEquals(val.get(), kvData.getReadBytes());
+    assertEquals(val.get(), kvData.getWriteBytes());
+    assertEquals(val.get(), kvData.getReadCount());
+    assertEquals(val.get(), kvData.getWriteCount());
+
+    kvData.setState(state);
+    kvData.setContainerDBType(containerDBType);
+    kvData.setContainerPath(path);
+    kvData.setDBPath(path);
+    kvData.incrReadBytes(10);
+    kvData.incrWriteBytes(10);
+    kvData.incrReadCount();
+    kvData.incrWriteCount();
+
+    assertEquals(state, kvData.getState());
+    assertEquals(containerDBType, kvData.getContainerDBType());
+    assertEquals(path, kvData.getContainerPath());
+    assertEquals(path, kvData.getDBPath());
+
+    assertEquals(10, kvData.getReadBytes());
+    assertEquals(10, kvData.getWriteBytes());
+    assertEquals(1, kvData.getReadCount());
+    assertEquals(1, kvData.getWriteCount());
+
+  }
+
+}


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


[17/37] hadoop git commit: HDDS-173. Refactor Dispatcher and implement Handler for new ContainerIO design.

Posted by bh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyManagerImpl.java
deleted file mode 100644
index 87565ce..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyManagerImpl.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.keyvalue;
-
-import com.google.common.base.Preconditions;
-import com.google.common.primitives.Longs;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
-import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
-import org.apache.hadoop.ozone.container.common.helpers.KeyData;
-import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
-import org.apache.hadoop.ozone.container.common.interfaces.Container;
-import org.apache.hadoop.ozone.container.keyvalue.interfaces.KeyManager;
-import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
-import org.apache.hadoop.utils.MetadataStore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_KEY;
-
-/**
- * This class is for performing key related operations on the KeyValue
- * Container.
- */
-public class KeyManagerImpl implements KeyManager {
-
-  static final Logger LOG = LoggerFactory.getLogger(KeyManagerImpl.class);
-
-  private Configuration config;
-
-  /**
-   * Constructs a key Manager.
-   *
-   * @param conf - Ozone configuration
-   */
-  public KeyManagerImpl(Configuration conf) {
-    Preconditions.checkNotNull(conf, "Config cannot be null");
-    this.config = conf;
-  }
-
-  /**
-   * Puts or overwrites a key.
-   *
-   * @param container - Container for which key need to be added.
-   * @param data     - Key Data.
-   * @throws IOException
-   */
-  public void putKey(Container container, KeyData data) throws IOException {
-    Preconditions.checkNotNull(data, "KeyData cannot be null for put " +
-        "operation.");
-    Preconditions.checkState(data.getContainerID() >= 0, "Container Id " +
-        "cannot be negative");
-    // We are not locking the key manager since LevelDb serializes all actions
-    // against a single DB. We rely on DB level locking to avoid conflicts.
-    MetadataStore db = KeyUtils.getDB((KeyValueContainerData) container
-        .getContainerData(), config);
-
-    // This is a post condition that acts as a hint to the user.
-    // Should never fail.
-    Preconditions.checkNotNull(db, "DB cannot be null here");
-    db.put(Longs.toByteArray(data.getLocalID()), data.getProtoBufMessage()
-        .toByteArray());
-  }
-
-  /**
-   * Gets an existing key.
-   *
-   * @param container - Container from which key need to be get.
-   * @param data - Key Data.
-   * @return Key Data.
-   * @throws IOException
-   */
-  public KeyData getKey(Container container, KeyData data) throws IOException {
-    Preconditions.checkNotNull(data, "Key data cannot be null");
-    Preconditions.checkNotNull(data.getContainerID(), "Container name cannot" +
-        " be null");
-    KeyValueContainerData containerData = (KeyValueContainerData) container
-        .getContainerData();
-    MetadataStore db = KeyUtils.getDB(containerData, config);
-    // This is a post condition that acts as a hint to the user.
-    // Should never fail.
-    Preconditions.checkNotNull(db, "DB cannot be null here");
-    byte[] kData = db.get(Longs.toByteArray(data.getLocalID()));
-    if (kData == null) {
-      throw new StorageContainerException("Unable to find the key.",
-          NO_SUCH_KEY);
-    }
-    ContainerProtos.KeyData keyData = ContainerProtos.KeyData.parseFrom(kData);
-    return KeyData.getFromProtoBuf(keyData);
-  }
-
-  /**
-   * Deletes an existing Key.
-   *
-   * @param container - Container from which key need to be deleted.
-   * @param blockID - ID of the block.
-   * @throws StorageContainerException
-   */
-  public void deleteKey(Container container, BlockID blockID) throws
-      IOException {
-    Preconditions.checkNotNull(blockID, "block ID cannot be null.");
-    Preconditions.checkState(blockID.getContainerID() >= 0,
-        "Container ID cannot be negative.");
-    Preconditions.checkState(blockID.getLocalID() >= 0,
-        "Local ID cannot be negative.");
-
-    KeyValueContainerData cData = (KeyValueContainerData) container
-        .getContainerData();
-    MetadataStore db = KeyUtils.getDB(cData, config);
-    // This is a post condition that acts as a hint to the user.
-    // Should never fail.
-    Preconditions.checkNotNull(db, "DB cannot be null here");
-    // Note : There is a race condition here, since get and delete
-    // are not atomic. Leaving it here since the impact is refusing
-    // to delete a key which might have just gotten inserted after
-    // the get check.
-    byte[] kKey = Longs.toByteArray(blockID.getLocalID());
-    byte[] kData = db.get(kKey);
-    if (kData == null) {
-      throw new StorageContainerException("Unable to find the key.",
-          NO_SUCH_KEY);
-    }
-    db.delete(kKey);
-  }
-
-  /**
-   * List keys in a container.
-   *
-   * @param container - Container from which keys need to be listed.
-   * @param startLocalID  - Key to start from, 0 to begin.
-   * @param count    - Number of keys to return.
-   * @return List of Keys that match the criteria.
-   */
-  public List<KeyData> listKey(Container container, long startLocalID, int
-      count) throws IOException {
-    Preconditions.checkNotNull(container, "container cannot be null");
-    Preconditions.checkState(startLocalID >= 0, "startLocal ID cannot be " +
-        "negative");
-    Preconditions.checkArgument(count > 0,
-        "Count must be a positive number.");
-    container.readLock();
-    List<KeyData> result = null;
-    KeyValueContainerData cData = (KeyValueContainerData) container
-        .getContainerData();
-    MetadataStore db = KeyUtils.getDB(cData, config);
-    result = new ArrayList<>();
-    byte[] startKeyInBytes = Longs.toByteArray(startLocalID);
-    List<Map.Entry<byte[], byte[]>> range = db.getSequentialRangeKVs(
-        startKeyInBytes, count, null);
-    for (Map.Entry<byte[], byte[]> entry : range) {
-      KeyData value = KeyUtils.getKeyData(entry.getValue());
-      KeyData data = new KeyData(value.getBlockID());
-      result.add(data);
-    }
-    return result;
-  }
-
-  /**
-   * Shutdown KeyValueContainerManager.
-   */
-  public void shutdown() {
-    KeyUtils.shutdownCache(ContainerCache.getInstance(config));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index 740967b..a1cbb4e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -18,29 +18,29 @@
 
 package org.apache.hadoop.ozone.container.keyvalue;
 
-
 import com.google.common.base.Preconditions;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerLifeCycleState;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
-import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
-
-
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .StorageContainerException;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.container.common.impl.ContainerData;
-import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
-import org.apache.hadoop.ozone.container.common.impl.KeyValueYaml;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
+import org.apache.hadoop.ozone.container.keyvalue.helpers
+    .KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.utils.MetadataStore;
 import org.slf4j.Logger;
@@ -116,9 +116,9 @@ public class KeyValueContainer implements Container {
     Preconditions.checkNotNull(scmId, "scmId cannot be null");
 
     File containerMetaDataPath = null;
+    //acquiring volumeset lock and container lock
+    volumeSet.acquireLock();
     try {
-      //acquiring volumeset lock and container lock
-      volumeSet.acquireLock();
       HddsVolume containerVolume = volumeChoosingPolicy.chooseVolume(volumeSet
           .getVolumesList(), containerMaxSize);
       String containerBasePath = containerVolume.getHddsRootDir().toString();
@@ -404,10 +404,19 @@ public class KeyValueContainer implements Container {
   }
 
   @Override
-  public ContainerData getContainerData()  {
+  public KeyValueContainerData getContainerData()  {
     return containerData;
   }
 
+  @Override
+  public ContainerLifeCycleState getContainerState() {
+    return containerData.getState();
+  }
+
+  @Override
+  public ContainerProtos.ContainerType getContainerType() {
+    return ContainerProtos.ContainerType.KeyValueContainer;
+  }
 
   @Override
   public void update(Map<String, String> metadata, boolean forceUpdate)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
new file mode 100644
index 0000000..8da4084
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue;
+
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * This class represents the KeyValueContainer metadata, which is the
+ * in-memory representation of container metadata and is represented on disk
+ * by the .container file.
+ */
+public class KeyValueContainerData extends ContainerData {
+
+  // Path to Container metadata Level DB/RocksDB Store and .container file.
+  private String metadataPath;
+
+  // Path to Physical file system where chunks are stored.
+  private String chunksPath;
+
+  //Type of DB used to store key to chunks mapping
+  private String containerDBType;
+
+  //Number of pending deletion blocks in container.
+  private int numPendingDeletionBlocks;
+
+  private File dbFile = null;
+
+  /**
+   * Constructs KeyValueContainerData object.
+   * @param type - containerType
+   * @param id - ContainerId
+   */
+  public KeyValueContainerData(ContainerProtos.ContainerType type, long id) {
+    super(type, id);
+    this.numPendingDeletionBlocks = 0;
+  }
+
+  /**
+   * Constructs KeyValueContainerData object.
+   * @param type - containerType
+   * @param id - ContainerId
+   * @param layOutVersion
+   */
+  public KeyValueContainerData(ContainerProtos.ContainerType type, long id,
+                               int layOutVersion) {
+    super(type, id, layOutVersion);
+    this.numPendingDeletionBlocks = 0;
+  }
+
+
+  /**
+   * Sets Container dbFile. This should be called only during creation of
+   * KeyValue container.
+   * @param containerDbFile
+   */
+  public void setDbFile(File containerDbFile) {
+    dbFile = containerDbFile;
+  }
+
+  /**
+   * Returns container DB file.
+   * @return dbFile
+   */
+  public File getDbFile() {
+    return dbFile;
+  }
+  /**
+   * Returns container metadata path.
+   *
+   * @return - path
+   */
+  public String getMetadataPath() {
+    return metadataPath;
+  }
+
+  /**
+   * Sets container metadata path.
+   *
+   * @param path - String.
+   */
+  public void setMetadataPath(String path) {
+    this.metadataPath = path;
+  }
+
+  /**
+   * Get chunks path.
+   * @return - Physical path where container file and checksum is stored.
+   */
+  public String getChunksPath() {
+    return chunksPath;
+  }
+
+  /**
+   * Set chunks Path.
+   * @param chunkPath - File path.
+   */
+  public void setChunksPath(String chunkPath) {
+    this.chunksPath = chunkPath;
+  }
+
+  /**
+   * Returns the DBType used for the container.
+   * @return containerDBType
+   */
+  public String getContainerDBType() {
+    return containerDBType;
+  }
+
+  /**
+   * Sets the DBType used for the container.
+   * @param containerDBType
+   */
+  public void setContainerDBType(String containerDBType) {
+    this.containerDBType = containerDBType;
+  }
+
+  /**
+   * Returns the number of pending deletion blocks in container.
+   * @return numPendingDeletionBlocks
+   */
+  public int getNumPendingDeletionBlocks() {
+    return numPendingDeletionBlocks;
+  }
+
+
+  /**
+   * Increase the count of pending deletion blocks.
+   *
+   * @param numBlocks increment number
+   */
+  public void incrPendingDeletionBlocks(int numBlocks) {
+    this.numPendingDeletionBlocks += numBlocks;
+  }
+
+  /**
+   * Decrease the count of pending deletion blocks.
+   *
+   * @param numBlocks decrement number
+   */
+  public void decrPendingDeletionBlocks(int numBlocks) {
+    this.numPendingDeletionBlocks -= numBlocks;
+  }
+
+  /**
+   * Returns a ProtoBuf Message from ContainerData.
+   *
+   * @return Protocol Buffer Message
+   */
+  public ContainerProtos.ContainerData getProtoBufMessage() {
+    ContainerProtos.ContainerData.Builder builder = ContainerProtos
+        .ContainerData.newBuilder();
+    builder.setContainerID(this.getContainerId());
+    builder.setDbPath(this.getDbFile().getPath());
+    builder.setContainerPath(this.getMetadataPath());
+    builder.setState(this.getState());
+
+    for (Map.Entry<String, String> entry : getMetadata().entrySet()) {
+      ContainerProtos.KeyValue.Builder keyValBuilder =
+          ContainerProtos.KeyValue.newBuilder();
+      builder.addMetadata(keyValBuilder.setKey(entry.getKey())
+          .setValue(entry.getValue()).build());
+    }
+
+    if (this.getBytesUsed() >= 0) {
+      builder.setBytesUsed(this.getBytesUsed());
+    }
+
+    if(this.getContainerType() != null) {
+      builder.setContainerType(ContainerProtos.ContainerType.KeyValueContainer);
+    }
+
+    if(this.getContainerDBType() != null) {
+      builder.setContainerDBType(containerDBType);
+    }
+
+    return builder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerLocationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerLocationUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerLocationUtil.java
deleted file mode 100644
index 2c15c94..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerLocationUtil.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.hadoop.ozone.container.keyvalue;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.common.Storage;
-
-import java.io.File;
-
-/**
- * Class which provides utility methods for container locations.
- */
-public final class KeyValueContainerLocationUtil {
-
-  /* Never constructed. */
-  private KeyValueContainerLocationUtil() {
-
-  }
-  /**
-   * Returns Container Metadata Location.
-   * @param baseDir
-   * @param scmId
-   * @param containerId
-   * @return containerMetadata Path
-   */
-  public static File getContainerMetaDataPath(String baseDir, String scmId,
-                                              long containerId) {
-    String containerMetaDataPath = getBaseContainerLocation(baseDir, scmId,
-        containerId);
-    containerMetaDataPath = containerMetaDataPath + File.separator +
-        OzoneConsts.CONTAINER_META_PATH;
-    return new File(containerMetaDataPath);
-  }
-
-
-  /**
-   * Returns Container Chunks Location.
-   * @param baseDir
-   * @param scmId
-   * @param containerId
-   * @return chunksPath
-   */
-  public static File getChunksLocationPath(String baseDir, String scmId,
-                                           long containerId) {
-    String chunksPath = getBaseContainerLocation(baseDir, scmId, containerId)
-        + File.separator + OzoneConsts.STORAGE_DIR_CHUNKS;
-    return new File(chunksPath);
-  }
-
-  /**
-   * Returns base directory for specified container.
-   * @param baseDir
-   * @param scmId
-   * @param containerId
-   * @return base directory for container.
-   */
-  private static String getBaseContainerLocation(String baseDir, String scmId,
-                                        long containerId) {
-    Preconditions.checkNotNull(baseDir, "Base Directory cannot be null");
-    Preconditions.checkNotNull(scmId, "scmUuid cannot be null");
-    Preconditions.checkState(containerId >= 0,
-        "Container Id cannot be negative.");
-
-    String containerSubDirectory = getContainerSubDirectory(containerId);
-
-    String containerMetaDataPath = baseDir  + File.separator + scmId +
-        File.separator + Storage.STORAGE_DIR_CURRENT + File.separator +
-        containerSubDirectory + File.separator + containerId;
-
-    return containerMetaDataPath;
-  }
-
-  /**
-   * Returns subdirectory, where this container needs to be placed.
-   * @param containerId
-   * @return container sub directory
-   */
-  private static String getContainerSubDirectory(long containerId){
-    int directory = (int) ((containerId >> 9) & 0xFF);
-    return Storage.CONTAINER_DIR + directory;
-  }
-
-  /**
-   * Returns containerFile.
-   * @param containerMetaDataPath
-   * @param containerName
-   * @return .container File name
-   */
-  public static File getContainerFile(File containerMetaDataPath, String
-      containerName) {
-    Preconditions.checkNotNull(containerMetaDataPath);
-    Preconditions.checkNotNull(containerName);
-    return new File(containerMetaDataPath, containerName +
-        OzoneConsts.CONTAINER_EXTENSION);
-  }
-
-  /**
-   * Return containerDB File.
-   * @param containerMetaDataPath
-   * @param containerName
-   * @return containerDB File name
-   */
-  public static File getContainerDBFile(File containerMetaDataPath, String
-      containerName) {
-    Preconditions.checkNotNull(containerMetaDataPath);
-    Preconditions.checkNotNull(containerName);
-    return new File(containerMetaDataPath, containerName + OzoneConsts
-        .DN_CONTAINER_DB);
-  }
-
-  /**
-   * Returns container checksum file.
-   * @param containerMetaDataPath
-   * @param containerName
-   * @return container checksum file
-   */
-  public static File getContainerCheckSumFile(File containerMetaDataPath,
-                                              String containerName) {
-    Preconditions.checkNotNull(containerMetaDataPath);
-    Preconditions.checkNotNull(containerName);
-    return new File(containerMetaDataPath, containerName + OzoneConsts
-        .CONTAINER_FILE_CHECKSUM_EXTENSION);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerUtil.java
deleted file mode 100644
index 55e2ab0..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerUtil.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.hadoop.ozone.container.keyvalue;
-
-import com.google.common.base.Preconditions;
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
-
-import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
-import org.apache.hadoop.utils.MetadataStore;
-import org.apache.hadoop.utils.MetadataStoreBuilder;
-import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-
-/**
- * Class which defines utility methods for KeyValueContainer.
- */
-
-public final class KeyValueContainerUtil {
-
-  /* Never constructed. */
-  private KeyValueContainerUtil() {
-
-  }
-
-  private static final Logger LOG = LoggerFactory.getLogger(
-      KeyValueContainerUtil.class);
-
-
-  public static void verifyIsNewContainer(File containerFile) throws
-      FileAlreadyExistsException {
-    Preconditions.checkNotNull(containerFile, "containerFile Should not be " +
-        "null");
-    if (containerFile.getParentFile().exists()) {
-      LOG.error("container already exists on disk. File: {}", containerFile
-          .toPath());
-      throw new FileAlreadyExistsException("container already exists on " +
-            "disk.");
-    }
-  }
-
-  /**
-   * creates metadata path, chunks path and  metadata DB for the specified
-   * container.
-   *
-   * @param containerMetaDataPath
-   * @throws IOException
-   */
-  public static void createContainerMetaData(File containerMetaDataPath, File
-      chunksPath, File dbFile, String containerName, Configuration conf) throws
-      IOException {
-    Preconditions.checkNotNull(containerMetaDataPath);
-    Preconditions.checkNotNull(containerName);
-    Preconditions.checkNotNull(conf);
-
-    if (!containerMetaDataPath.mkdirs()) {
-      LOG.error("Unable to create directory for metadata storage. Path: {}",
-          containerMetaDataPath);
-      throw new IOException("Unable to create directory for metadata storage." +
-          " Path: " + containerMetaDataPath);
-    }
-    MetadataStore store = MetadataStoreBuilder.newBuilder().setConf(conf)
-        .setCreateIfMissing(true).setDbFile(dbFile).build();
-
-    // we close since the SCM pre-creates containers.
-    // we will open and put Db handle into a cache when keys are being created
-    // in a container.
-
-    store.close();
-
-    if (!chunksPath.mkdirs()) {
-      LOG.error("Unable to create chunks directory Container {}",
-          chunksPath);
-      //clean up container metadata path and metadata db
-      FileUtils.deleteDirectory(containerMetaDataPath);
-      FileUtils.deleteDirectory(containerMetaDataPath.getParentFile());
-      throw new IOException("Unable to create directory for data storage." +
-          " Path: " + chunksPath);
-    }
-  }
-
-  /**
-   * remove Container if it is empty.
-   * <p/>
-   * There are three things we need to delete.
-   * <p/>
-   * 1. Container file and metadata file. 2. The Level DB file 3. The path that
-   * we created on the data location.
-   *
-   * @param containerData - Data of the container to remove.
-   * @param conf - configuration of the cluster.
-   * @param forceDelete - whether this container should be deleted forcibly.
-   * @throws IOException
-   */
-  public static void removeContainer(KeyValueContainerData containerData,
-                                     Configuration conf, boolean forceDelete)
-      throws IOException {
-    Preconditions.checkNotNull(containerData);
-    File containerMetaDataPath = new File(containerData
-        .getMetadataPath());
-    File chunksPath = new File(containerData.getChunksPath());
-
-    MetadataStore db = KeyUtils.getDB(containerData, conf);
-
-    // If the container is not empty and cannot be deleted forcibly,
-    // then throw a SCE to stop deleting.
-    if(!forceDelete && !db.isEmpty()) {
-      throw new StorageContainerException(
-          "Container cannot be deleted because it is not empty.",
-          ContainerProtos.Result.ERROR_CONTAINER_NOT_EMPTY);
-    }
-
-    // Close the DB connection and remove the DB handler from cache
-    KeyUtils.removeDB(containerData, conf);
-
-    // Delete the Container MetaData path.
-    FileUtils.deleteDirectory(containerMetaDataPath);
-
-    //Delete the Container Chunks Path.
-    FileUtils.deleteDirectory(chunksPath);
-
-    //Delete Container directory
-    FileUtils.deleteDirectory(containerMetaDataPath.getParentFile());
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
new file mode 100644
index 0000000..d9ee7fd
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -0,0 +1,643 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+import com.sun.jersey.spi.resource.Singleton;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandResponseProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerType;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .CreateContainerRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .GetSmallFileRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .KeyValue;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .PutSmallFileRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Type;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .StorageContainerException;
+import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.SmallFileUtils;
+import org.apache.hadoop.ozone.container.common.helpers.KeyData;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.common.interfaces.Handler;
+import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.volume
+    .RoundRobinVolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils;
+import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerImpl;
+import org.apache.hadoop.ozone.container.keyvalue.impl.KeyManagerImpl;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
+import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
+import org.apache.hadoop.ozone.container.keyvalue.interfaces.KeyManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.*;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Stage;
+
+/**
+ * Handler for KeyValue Container type.
+ */
+@Singleton
+public class KeyValueHandler extends Handler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      KeyValueHandler.class);
+
+  private static volatile KeyValueHandler INSTANCE = null; // Singleton class
+
+  private final ContainerType containerType;
+  private final KeyManager keyManager;
+  private final ChunkManager chunkManager;
+  private VolumeChoosingPolicy volumeChoosingPolicy;
+
+  // TODO : Add metrics and populate it.
+
+  public static KeyValueHandler getInstance(Configuration config,
+      ContainerSet contSet, VolumeSet volSet, String scmID) {
+    if (INSTANCE == null) {
+      INSTANCE = new KeyValueHandler(config, contSet, volSet, scmID);
+    }
+    return INSTANCE;
+  }
+
+  private KeyValueHandler(Configuration config, ContainerSet contSet,
+      VolumeSet volSet, String scmID) {
+    super(config, contSet, volSet, scmID);
+    containerType = ContainerType.KeyValueContainer;
+    keyManager = new KeyManagerImpl(config);
+    chunkManager = new ChunkManagerImpl();
+    // TODO: Add supoort for different volumeChoosingPolicies.
+    volumeChoosingPolicy = new RoundRobinVolumeChoosingPolicy();
+  }
+
+  @Override
+  public ContainerCommandResponseProto handle(
+      ContainerCommandRequestProto request, Container container) {
+
+    Type cmdType = request.getCmdType();
+    KeyValueContainer kvContainer = (KeyValueContainer) container;
+    switch(cmdType) {
+    case CreateContainer:
+      return handleCreateContainer(request, kvContainer);
+    case ReadContainer:
+      return handleReadContainer(request, kvContainer);
+    case UpdateContainer:
+      return handleUpdateContainer(request, kvContainer);
+    case DeleteContainer:
+      return handleDeleteContainer(request, kvContainer);
+    case ListContainer:
+      return handleUnsupportedOp(request);
+    case CloseContainer:
+      return handleCloseContainer(request, kvContainer);
+    case PutKey:
+      return handlePutKey(request, kvContainer);
+    case GetKey:
+      return handleGetKey(request, kvContainer);
+    case DeleteKey:
+      return handleDeleteKey(request, kvContainer);
+    case ListKey:
+      return handleUnsupportedOp(request);
+    case ReadChunk:
+      return handleReadChunk(request, kvContainer);
+    case DeleteChunk:
+      return handleDeleteChunk(request, kvContainer);
+    case WriteChunk:
+      return handleWriteChunk(request, kvContainer);
+    case ListChunk:
+      return handleUnsupportedOp(request);
+    case CompactChunk:
+      return handleUnsupportedOp(request);
+    case PutSmallFile:
+      return handlePutSmallFile(request, kvContainer);
+    case GetSmallFile:
+      return handleGetSmallFile(request, kvContainer);
+    }
+
+    return null;
+  }
+
+  /**
+   * Handles Create Container Request. If successful, adds the container to
+   * ContainerSet.
+   */
+  ContainerCommandResponseProto handleCreateContainer(
+      ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+    if (!request.hasCreateContainer()) {
+      LOG.debug("Malformed Create Container request. trace ID: {}",
+          request.getTraceID());
+      return ContainerUtils.malformedRequest(request);
+    }
+    // Create Container request should be passed a null container as the
+    // container would be created here.
+    Preconditions.checkArgument(kvContainer == null);
+
+    CreateContainerRequestProto createContainerReq =
+        request.getCreateContainer();
+    long containerID = createContainerReq.getContainerID();
+    if (createContainerReq.hasContainerType()) {
+      Preconditions.checkArgument(createContainerReq.getContainerType()
+          .equals(ContainerType.KeyValueContainer));
+    }
+
+    KeyValueContainerData newContainerData = new KeyValueContainerData(
+        containerType, containerID);
+    // TODO: Add support to add metadataList to ContainerData. Add metadata
+    // to container during creation.
+    KeyValueContainer newContainer = new KeyValueContainer(
+        newContainerData, conf);
+
+    try {
+      newContainer.create(volumeSet, volumeChoosingPolicy, scmID);
+      containerSet.addContainer(newContainer);
+    } catch (StorageContainerException ex) {
+      return ContainerUtils.logAndReturnError(LOG, ex, request);
+    }
+
+    return ContainerUtils.getSuccessResponse(request);
+  }
+
+  /**
+   * Handles Read Container Request. Returns the ContainerData as response.
+   */
+  ContainerCommandResponseProto handleReadContainer(
+      ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+    if (!request.hasReadContainer()) {
+      LOG.debug("Malformed Read Container request. trace ID: {}",
+          request.getTraceID());
+      return ContainerUtils.malformedRequest(request);
+    }
+
+    KeyValueContainerData containerData = kvContainer.getContainerData();
+    return KeyValueContainerUtil.getReadContainerResponse(
+        request, containerData);
+  }
+
+
+  /**
+   * Handles Update Container Request. If successful, the container metadata
+   * is updated.
+   */
+  ContainerCommandResponseProto handleUpdateContainer(
+      ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+
+    if (!request.hasUpdateContainer()) {
+      LOG.debug("Malformed Update Container request. trace ID: {}",
+          request.getTraceID());
+      return ContainerUtils.malformedRequest(request);
+    }
+
+    boolean forceUpdate = request.getUpdateContainer().getForceUpdate();
+    List<KeyValue> keyValueList =
+        request.getUpdateContainer().getMetadataList();
+    Map<String, String> metadata = new HashMap<>();
+    for (KeyValue keyValue : keyValueList) {
+      metadata.put(keyValue.getKey(), keyValue.getValue());
+    }
+
+    try {
+      if (!metadata.isEmpty()) {
+        kvContainer.update(metadata, forceUpdate);
+      }
+    } catch (StorageContainerException ex) {
+      return ContainerUtils.logAndReturnError(LOG, ex, request);
+    }
+    return ContainerUtils.getSuccessResponse(request);
+  }
+
+  /**
+   * Handles Delete Container Request.
+   * Open containers cannot be deleted.
+   * Holds writeLock on ContainerSet till the container is removed from
+   * containerMap. On disk deletion of container files will happen
+   * asynchornously without the lock.
+   */
+  ContainerCommandResponseProto handleDeleteContainer(
+      ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+
+    if (!request.hasDeleteContainer()) {
+      LOG.debug("Malformed Delete container request. trace ID: {}",
+          request.getTraceID());
+      return ContainerUtils.malformedRequest(request);
+    }
+
+    boolean forceDelete = request.getDeleteContainer().getForceDelete();
+    kvContainer.writeLock();
+
+    try {
+      // Check if container is open
+      if (kvContainer.getContainerData().isOpen()) {
+        kvContainer.writeUnlock();
+        throw new StorageContainerException(
+            "Deletion of Open Container is not allowed.",
+            DELETE_ON_OPEN_CONTAINER);
+      } else {
+        containerSet.removeContainer(
+            kvContainer.getContainerData().getContainerId());
+        // Release the lock first.
+        // Avoid holding write locks for disk operations
+        kvContainer.writeUnlock();
+
+        kvContainer.delete(forceDelete);
+      }
+    } catch (StorageContainerException ex) {
+      return ContainerUtils.logAndReturnError(LOG, ex, request);
+    } finally {
+      if (kvContainer.hasWriteLock()) {
+        kvContainer.writeUnlock();
+      }
+    }
+    return ContainerUtils.getSuccessResponse(request);
+  }
+
+  /**
+   * Handles Close Container Request. An open container is closed.
+   */
+  ContainerCommandResponseProto handleCloseContainer(
+      ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+
+    if (!request.hasCloseContainer()) {
+      LOG.debug("Malformed Update Container request. trace ID: {}",
+          request.getTraceID());
+      return ContainerUtils.malformedRequest(request);
+    }
+
+    try {
+      checkContainerOpen(kvContainer);
+
+      kvContainer.close();
+    } catch (StorageContainerException ex) {
+      return ContainerUtils.logAndReturnError(LOG, ex, request);
+    }
+
+    return ContainerUtils.getSuccessResponse(request);
+  }
+
+  /**
+   * Handle Put Key operation. Calls KeyManager to process the request.
+   */
+  ContainerCommandResponseProto handlePutKey(
+      ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+
+    if (!request.hasPutKey()) {
+      LOG.debug("Malformed Put Key request. trace ID: {}",
+          request.getTraceID());
+      return ContainerUtils.malformedRequest(request);
+    }
+
+    try {
+      checkContainerOpen(kvContainer);
+
+      KeyData keyData = KeyData.getFromProtoBuf(
+          request.getPutKey().getKeyData());
+      Preconditions.checkNotNull(keyData);
+
+      keyManager.putKey(kvContainer, keyData);
+    } catch (StorageContainerException ex) {
+      return ContainerUtils.logAndReturnError(LOG, ex, request);
+    } catch (IOException ex) {
+      return ContainerUtils.logAndReturnError(LOG,
+          new StorageContainerException("Put Key failed", ex, IO_EXCEPTION),
+          request);
+    }
+
+    return KeyUtils.getKeyResponseSuccess(request);
+  }
+
+  /**
+   * Handle Get Key operation. Calls KeyManager to process the request.
+   */
+  ContainerCommandResponseProto handleGetKey(
+      ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+
+    if (!request.hasGetKey()) {
+      LOG.debug("Malformed Get Key request. trace ID: {}",
+          request.getTraceID());
+      return ContainerUtils.malformedRequest(request);
+    }
+
+    KeyData responseData;
+    try {
+      BlockID blockID = BlockID.getFromProtobuf(
+          request.getGetKey().getBlockID());
+      responseData = keyManager.getKey(kvContainer, blockID);
+
+    } catch (StorageContainerException ex) {
+      return ContainerUtils.logAndReturnError(LOG, ex, request);
+    } catch (IOException ex) {
+      return ContainerUtils.logAndReturnError(LOG,
+          new StorageContainerException("Get Key failed", ex, IO_EXCEPTION),
+          request);
+    }
+
+    return KeyUtils.getKeyDataResponse(request, responseData);
+  }
+
+  /**
+   * Handle Delete Key operation. Calls KeyManager to process the request.
+   */
+  ContainerCommandResponseProto handleDeleteKey(
+      ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+
+    if (!request.hasDeleteKey()) {
+      LOG.debug("Malformed Delete Key request. trace ID: {}",
+          request.getTraceID());
+      return ContainerUtils.malformedRequest(request);
+    }
+
+    try {
+      checkContainerOpen(kvContainer);
+
+      BlockID blockID = BlockID.getFromProtobuf(
+          request.getDeleteKey().getBlockID());
+
+      keyManager.deleteKey(kvContainer, blockID);
+    } catch (StorageContainerException ex) {
+      return ContainerUtils.logAndReturnError(LOG, ex, request);
+    } catch (IOException ex) {
+      return ContainerUtils.logAndReturnError(LOG,
+          new StorageContainerException("Delete Key failed", ex, IO_EXCEPTION),
+          request);
+    }
+
+    return KeyUtils.getKeyResponseSuccess(request);
+  }
+
+  /**
+   * Handle Read Chunk operation. Calls ChunkManager to process the request.
+   */
+  ContainerCommandResponseProto handleReadChunk(
+      ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+
+    if (!request.hasReadChunk()) {
+      LOG.debug("Malformed Read Chunk request. trace ID: {}",
+          request.getTraceID());
+      return ContainerUtils.malformedRequest(request);
+    }
+
+    ChunkInfo chunkInfo;
+    byte[] data;
+    try {
+      BlockID blockID = BlockID.getFromProtobuf(
+          request.getReadChunk().getBlockID());
+      chunkInfo = ChunkInfo.getFromProtoBuf(request.getReadChunk()
+          .getChunkData());
+      Preconditions.checkNotNull(chunkInfo);
+
+      data = chunkManager.readChunk(kvContainer, blockID, chunkInfo);
+    } catch (StorageContainerException ex) {
+      return ContainerUtils.logAndReturnError(LOG, ex, request);
+    } catch (IOException ex) {
+      return ContainerUtils.logAndReturnError(LOG,
+          new StorageContainerException("Read Chunk failed", ex, IO_EXCEPTION),
+          request);
+    }
+
+    return ChunkUtils.getReadChunkResponse(request, data, chunkInfo);
+  }
+
+  /**
+   * Handle Delete Chunk operation. Calls ChunkManager to process the request.
+   */
+  ContainerCommandResponseProto handleDeleteChunk(
+      ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+
+    if (!request.hasDeleteChunk()) {
+      LOG.debug("Malformed Delete Chunk request. trace ID: {}",
+          request.getTraceID());
+      return ContainerUtils.malformedRequest(request);
+    }
+
+    try {
+      checkContainerOpen(kvContainer);
+
+      BlockID blockID = BlockID.getFromProtobuf(
+          request.getDeleteChunk().getBlockID());
+      ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(request.getDeleteChunk()
+          .getChunkData());
+      Preconditions.checkNotNull(chunkInfo);
+
+      chunkManager.deleteChunk(kvContainer, blockID, chunkInfo);
+    } catch (StorageContainerException ex) {
+      return ContainerUtils.logAndReturnError(LOG, ex, request);
+    } catch (IOException ex) {
+      return ContainerUtils.logAndReturnError(LOG,
+          new StorageContainerException("Delete Chunk failed", ex,
+              IO_EXCEPTION), request);
+    }
+
+    return ChunkUtils.getChunkResponseSuccess(request);
+  }
+
+  /**
+   * Handle Write Chunk operation. Calls ChunkManager to process the request.
+   */
+  ContainerCommandResponseProto handleWriteChunk(
+      ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+
+    if (!request.hasWriteChunk()) {
+      LOG.debug("Malformed Write Chunk request. trace ID: {}",
+          request.getTraceID());
+      return ContainerUtils.malformedRequest(request);
+    }
+
+    try {
+      checkContainerOpen(kvContainer);
+
+      BlockID blockID = BlockID.getFromProtobuf(
+          request.getWriteChunk().getBlockID());
+      ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(request.getWriteChunk()
+          .getChunkData());
+      Preconditions.checkNotNull(chunkInfo);
+
+      byte[] data = null;
+      if (request.getWriteChunk().getStage() == Stage.WRITE_DATA ||
+          request.getWriteChunk().getStage() == Stage.COMBINED) {
+        data = request.getWriteChunk().getData().toByteArray();
+      }
+
+      chunkManager.writeChunk(kvContainer, blockID, chunkInfo, data,
+          request.getWriteChunk().getStage());
+    } catch (StorageContainerException ex) {
+      return ContainerUtils.logAndReturnError(LOG, ex, request);
+    } catch (IOException ex) {
+      return ContainerUtils.logAndReturnError(LOG,
+          new StorageContainerException("Write Chunk failed", ex, IO_EXCEPTION),
+          request);
+    }
+
+    return ChunkUtils.getChunkResponseSuccess(request);
+  }
+
+  /**
+   * Handle Put Small File operation. Writes the chunk and associated key
+   * using a single RPC. Calls KeyManager and ChunkManager to process the
+   * request.
+   */
+  ContainerCommandResponseProto handlePutSmallFile(
+      ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+
+    if (!request.hasPutSmallFile()) {
+      LOG.debug("Malformed Put Small File request. trace ID: {}",
+          request.getTraceID());
+      return ContainerUtils.malformedRequest(request);
+    }
+    PutSmallFileRequestProto putSmallFileReq =
+        request.getPutSmallFile();
+
+    try {
+      checkContainerOpen(kvContainer);
+
+      BlockID blockID = BlockID.getFromProtobuf(
+        putSmallFileReq.getKey().getKeyData().getBlockID());
+      KeyData keyData = KeyData.getFromProtoBuf(
+          putSmallFileReq.getKey().getKeyData());
+      Preconditions.checkNotNull(keyData);
+
+      ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(
+          putSmallFileReq.getChunkInfo());
+      Preconditions.checkNotNull(chunkInfo);
+
+      byte[] data = putSmallFileReq.getData().toByteArray();
+      chunkManager.writeChunk(
+          kvContainer, blockID, chunkInfo, data, Stage.COMBINED);
+
+      List<ContainerProtos.ChunkInfo> chunks = new LinkedList<>();
+      chunks.add(chunkInfo.getProtoBufMessage());
+      keyData.setChunks(chunks);
+      keyManager.putKey(kvContainer, keyData);
+
+    } catch (StorageContainerException ex) {
+      return ContainerUtils.logAndReturnError(LOG, ex, request);
+    } catch (IOException ex) {
+      return ContainerUtils.logAndReturnError(LOG,
+          new StorageContainerException("Read Chunk failed", ex,
+              PUT_SMALL_FILE_ERROR), request);
+    }
+
+    return SmallFileUtils.getPutFileResponseSuccess(request);
+  }
+
+  /**
+   * Handle Get Small File operation. Gets a data stream using a key. This
+   * helps in reducing the RPC overhead for small files. Calls KeyManager and
+   * ChunkManager to process the request.
+   */
+  ContainerCommandResponseProto handleGetSmallFile(
+      ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+
+    if (!request.hasGetSmallFile()) {
+      LOG.debug("Malformed Get Small File request. trace ID: {}",
+          request.getTraceID());
+      return ContainerUtils.malformedRequest(request);
+    }
+
+    GetSmallFileRequestProto getSmallFileReq = request.getGetSmallFile();
+
+    try {
+      BlockID blockID = BlockID.getFromProtobuf(
+        getSmallFileReq.getKey().getBlockID());
+      KeyData responseData = keyManager.getKey(kvContainer, blockID);
+
+      ContainerProtos.ChunkInfo chunkInfo = null;
+      ByteString dataBuf = ByteString.EMPTY;
+      for (ContainerProtos.ChunkInfo chunk : responseData.getChunks()) {
+        byte[] data = chunkManager.readChunk(kvContainer, blockID,
+            ChunkInfo.getFromProtoBuf(chunk));
+        ByteString current = ByteString.copyFrom(data);
+        dataBuf = dataBuf.concat(current);
+        chunkInfo = chunk;
+      }
+
+      return SmallFileUtils.getGetSmallFileResponseSuccess(request, dataBuf
+          .toByteArray(), ChunkInfo.getFromProtoBuf(chunkInfo));
+    } catch (StorageContainerException e) {
+      return ContainerUtils.logAndReturnError(LOG, e, request);
+    } catch (IOException ex) {
+      return ContainerUtils.logAndReturnError(LOG,
+          new StorageContainerException("Write Chunk failed", ex,
+              GET_SMALL_FILE_ERROR), request);
+    }
+  }
+
+  /**
+   * Handle unsupported operation.
+   */
+  ContainerCommandResponseProto handleUnsupportedOp(
+      ContainerCommandRequestProto request) {
+    // TODO : remove all unsupported operations or handle them.
+    return ContainerUtils.unsupportedRequest(request);
+  }
+
+  /**
+   * Check if container is open. Throw exception otherwise.
+   * @param kvContainer
+   * @throws StorageContainerException
+   */
+  private void checkContainerOpen(KeyValueContainer kvContainer)
+      throws StorageContainerException {
+
+    ContainerProtos.ContainerLifeCycleState containerState =
+        kvContainer.getContainerState();
+
+    if (containerState == ContainerProtos.ContainerLifeCycleState.OPEN) {
+      return;
+    } else {
+      String msg = "Requested operation not allowed as ContainerState is " +
+          containerState;
+      ContainerProtos.Result result = null;
+      switch (containerState) {
+      case CLOSING:
+      case CLOSED:
+        result = CLOSED_CONTAINER_IO;
+        break;
+      case INVALID:
+        result = INVALID_CONTAINER_STATE;
+        break;
+      default:
+        result = CONTAINER_INTERNAL_ERROR;
+      }
+
+      throw new StorageContainerException(msg, result);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueYaml.java
new file mode 100644
index 0000000..64f7152
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueYaml.java
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.yaml.snakeyaml.Yaml;
+
+import java.beans.IntrospectionException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Writer;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.OutputStreamWriter;
+import java.io.File;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.Map;
+
+import org.yaml.snakeyaml.constructor.AbstractConstruct;
+import org.yaml.snakeyaml.constructor.Constructor;
+import org.yaml.snakeyaml.introspector.BeanAccess;
+import org.yaml.snakeyaml.introspector.Property;
+import org.yaml.snakeyaml.introspector.PropertyUtils;
+import org.yaml.snakeyaml.nodes.MappingNode;
+import org.yaml.snakeyaml.nodes.Node;
+import org.yaml.snakeyaml.nodes.ScalarNode;
+import org.yaml.snakeyaml.nodes.Tag;
+import org.yaml.snakeyaml.representer.Representer;
+
+/**
+ * Class for creating and reading .container files.
+ */
+
+public final class KeyValueYaml {
+
+  private KeyValueYaml() {
+
+  }
+  /**
+   * Creates a .container file in yaml format.
+   *
+   * @param containerFile
+   * @param containerData
+   * @throws IOException
+   */
+  public static void createContainerFile(File containerFile, ContainerData
+      containerData) throws IOException {
+
+    Preconditions.checkNotNull(containerFile, "yamlFile cannot be null");
+    Preconditions.checkNotNull(containerData, "containerData cannot be null");
+
+    PropertyUtils propertyUtils = new PropertyUtils();
+    propertyUtils.setBeanAccess(BeanAccess.FIELD);
+    propertyUtils.setAllowReadOnlyProperties(true);
+
+    Representer representer = new KeyValueContainerDataRepresenter();
+    representer.setPropertyUtils(propertyUtils);
+    representer.addClassTag(
+        KeyValueContainerData.class, new Tag("KeyValueContainerData"));
+
+    Constructor keyValueDataConstructor = new KeyValueDataConstructor();
+
+    Yaml yaml = new Yaml(keyValueDataConstructor, representer);
+
+    Writer writer = new OutputStreamWriter(new FileOutputStream(containerFile),
+        "UTF-8");
+    yaml.dump(containerData, writer);
+    writer.close();
+  }
+
+  /**
+   * Read the yaml file, and return containerData.
+   *
+   * @param containerFile
+   * @throws IOException
+   */
+  public static KeyValueContainerData readContainerFile(File containerFile)
+      throws IOException {
+    Preconditions.checkNotNull(containerFile, "containerFile cannot be null");
+
+    InputStream input = null;
+    KeyValueContainerData keyValueContainerData;
+    try {
+      PropertyUtils propertyUtils = new PropertyUtils();
+      propertyUtils.setBeanAccess(BeanAccess.FIELD);
+      propertyUtils.setAllowReadOnlyProperties(true);
+
+      Representer representer = new KeyValueContainerDataRepresenter();
+      representer.setPropertyUtils(propertyUtils);
+      representer.addClassTag(
+          KeyValueContainerData.class, new Tag("KeyValueContainerData"));
+
+      Constructor keyValueDataConstructor = new KeyValueDataConstructor();
+
+      Yaml yaml = new Yaml(keyValueDataConstructor, representer);
+      yaml.setBeanAccess(BeanAccess.FIELD);
+
+      input = new FileInputStream(containerFile);
+      keyValueContainerData = (KeyValueContainerData)
+          yaml.load(input);
+    } finally {
+      if (input!= null) {
+        input.close();
+      }
+    }
+    return keyValueContainerData;
+  }
+
+  /**
+   * Representer class to define which fields need to be stored in yaml file.
+   */
+  private static class KeyValueContainerDataRepresenter extends Representer {
+    @Override
+    protected Set<Property> getProperties(Class<? extends Object> type)
+        throws IntrospectionException {
+      Set<Property> set = super.getProperties(type);
+      Set<Property> filtered = new TreeSet<Property>();
+      if (type.equals(KeyValueContainerData.class)) {
+        // filter properties
+        for (Property prop : set) {
+          String name = prop.getName();
+          // When a new field needs to be added, it needs to be added here.
+          if (name.equals("containerType") || name.equals("containerId") ||
+              name.equals("layOutVersion") || name.equals("state") ||
+              name.equals("metadata") || name.equals("metadataPath") ||
+              name.equals("chunksPath") || name.equals(
+                  "containerDBType")) {
+            filtered.add(prop);
+          }
+        }
+      }
+      return filtered;
+    }
+  }
+
+  /**
+   * Constructor class for KeyValueData, which will be used by Yaml.
+   */
+  private static class KeyValueDataConstructor extends Constructor {
+    KeyValueDataConstructor() {
+      //Adding our own specific constructors for tags.
+      this.yamlConstructors.put(new Tag("KeyValueContainerData"),
+          new ConstructKeyValueContainerData());
+      this.yamlConstructors.put(Tag.INT, new ConstructLong());
+    }
+
+    private class ConstructKeyValueContainerData extends AbstractConstruct {
+      public Object construct(Node node) {
+        MappingNode mnode = (MappingNode) node;
+        Map<Object, Object> nodes = constructMapping(mnode);
+        String type = (String) nodes.get("containerType");
+
+        ContainerProtos.ContainerType containerType = ContainerProtos
+            .ContainerType.KeyValueContainer;
+        if (type.equals("KeyValueContainer")) {
+          containerType = ContainerProtos.ContainerType.KeyValueContainer;
+        }
+
+        //Needed this, as TAG.INT type is by default converted to Long.
+        long layOutVersion = (long) nodes.get("layOutVersion");
+        int lv = (int) layOutVersion;
+
+        //When a new field is added, it needs to be added here.
+        KeyValueContainerData kvData = new KeyValueContainerData(containerType,
+            (long) nodes.get("containerId"), lv);
+        kvData.setContainerDBType((String)nodes.get("containerDBType"));
+        kvData.setMetadataPath((String) nodes.get(
+            "metadataPath"));
+        kvData.setChunksPath((String) nodes.get("chunksPath"));
+        Map<String, String> meta = (Map) nodes.get("metadata");
+        meta.forEach((key, val) -> {
+          try {
+            kvData.addMetadata(key, val);
+          } catch (IOException e) {
+            throw new IllegalStateException("Unexpected " +
+                "Key Value Pair " + "(" + key + "," + val +")in the metadata " +
+                "for containerId " + (long) nodes.get("containerId"));
+          }
+        });
+        String state = (String) nodes.get("state");
+        switch (state) {
+        case "OPEN":
+          kvData.setState(ContainerProtos.ContainerLifeCycleState.OPEN);
+          break;
+        case "CLOSING":
+          kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSING);
+          break;
+        case "CLOSED":
+          kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED);
+          break;
+        default:
+          throw new IllegalStateException("Unexpected " +
+              "ContainerLifeCycleState " + state + " for the containerId " +
+              (long) nodes.get("containerId"));
+        }
+        return kvData;
+      }
+    }
+
+    //Below code is taken from snake yaml, as snakeyaml tries to fit the
+    // number if it fits in integer, otherwise returns long. So, slightly
+    // modified the code to return long in all cases.
+    private class ConstructLong extends AbstractConstruct {
+      public Object construct(Node node) {
+        String value = constructScalar((ScalarNode) node).toString()
+            .replaceAll("_", "");
+        int sign = +1;
+        char first = value.charAt(0);
+        if (first == '-') {
+          sign = -1;
+          value = value.substring(1);
+        } else if (first == '+') {
+          value = value.substring(1);
+        }
+        int base = 10;
+        if ("0".equals(value)) {
+          return Long.valueOf(0);
+        } else if (value.startsWith("0b")) {
+          value = value.substring(2);
+          base = 2;
+        } else if (value.startsWith("0x")) {
+          value = value.substring(2);
+          base = 16;
+        } else if (value.startsWith("0")) {
+          value = value.substring(1);
+          base = 8;
+        } else if (value.indexOf(':') != -1) {
+          String[] digits = value.split(":");
+          int bes = 1;
+          int val = 0;
+          for (int i = 0, j = digits.length; i < j; i++) {
+            val += (Long.parseLong(digits[(j - i) - 1]) * bes);
+            bes *= 60;
+          }
+          return createNumber(sign, String.valueOf(val), 10);
+        } else {
+          return createNumber(sign, value, 10);
+        }
+        return createNumber(sign, value, base);
+      }
+    }
+
+    private Number createNumber(int sign, String number, int radix) {
+      Number result;
+      if (sign < 0) {
+        number = "-" + number;
+      }
+      result = Long.valueOf(number, radix);
+      return result;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
index c837ccc..872d84d 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
@@ -21,12 +21,21 @@ package org.apache.hadoop.ozone.container.keyvalue.helpers;
 import com.google.common.base.Preconditions;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandResponseProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ReadChunkResponseProto;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .StorageContainerException;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
-import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.ratis.shaded.com.google.protobuf.ByteString;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -292,4 +301,41 @@ public final class ChunkUtils {
         (Boolean.valueOf(overWrite));
   }
 
+  /**
+   * Returns a CreateContainer Response. This call is used by create and delete
+   * containers which have null success responses.
+   *
+   * @param msg Request
+   * @return Response.
+   */
+  public static ContainerCommandResponseProto getChunkResponseSuccess(
+      ContainerCommandRequestProto msg) {
+    return ContainerUtils.getSuccessResponse(msg);
+  }
+
+  /**
+   * Gets a response to the read chunk calls.
+   *
+   * @param msg - Msg
+   * @param data - Data
+   * @param info - Info
+   * @return Response.
+   */
+  public static ContainerCommandResponseProto getReadChunkResponse(
+      ContainerCommandRequestProto msg, byte[] data, ChunkInfo info) {
+    Preconditions.checkNotNull(msg);
+    Preconditions.checkNotNull("Chunk data is null", data);
+    Preconditions.checkNotNull("Chunk Info is null", info);
+
+    ReadChunkResponseProto.Builder response =
+        ReadChunkResponseProto.newBuilder();
+    response.setChunkData(info.getProtoBufMessage());
+    response.setData(ByteString.copyFrom(data));
+    response.setBlockID(msg.getReadChunk().getBlockID());
+
+    ContainerCommandResponseProto.Builder builder =
+        ContainerUtils.getSuccessResponseBuilder(msg);
+    builder.setReadChunk(response);
+    return builder.build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
index d45f598..714f445 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
@@ -21,9 +21,17 @@ package org.apache.hadoop.ozone.container.keyvalue.helpers;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandResponseProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .GetKeyResponseProto;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .StorageContainerException;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
-import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
 import org.apache.hadoop.utils.MetadataStore;
 
@@ -112,4 +120,27 @@ public final class KeyUtils {
           " bytes array.", NO_SUCH_KEY);
     }
   }
+
+  /**
+   * Returns successful keyResponse.
+   * @param msg - Request.
+   * @return Response.
+   */
+  public static ContainerCommandResponseProto getKeyResponseSuccess(
+      ContainerCommandRequestProto msg) {
+    return ContainerUtils.getSuccessResponse(msg);
+  }
+
+
+  public static ContainerCommandResponseProto getKeyDataResponse(
+      ContainerCommandRequestProto msg, KeyData data) {
+    GetKeyResponseProto.Builder getKey = ContainerProtos
+        .GetKeyResponseProto
+        .newBuilder();
+    getKey.setKeyData(data.getProtoBufMessage());
+    ContainerProtos.ContainerCommandResponseProto.Builder builder =
+        ContainerUtils.getSuccessResponseBuilder(msg);
+    builder.setGetKey(getKey);
+    return  builder.build();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
new file mode 100644
index 0000000..4710c51
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.keyvalue.helpers;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.common.Storage;
+
+import java.io.File;
+
+/**
+ * Class which provides utility methods for container locations.
+ */
+public final class KeyValueContainerLocationUtil {
+
+  /* Never constructed. */
+  private KeyValueContainerLocationUtil() {
+
+  }
+  /**
+   * Returns Container Metadata Location.
+   * @param baseDir
+   * @param scmId
+   * @param containerId
+   * @return containerMetadata Path
+   */
+  public static File getContainerMetaDataPath(String baseDir, String scmId,
+                                              long containerId) {
+    String containerMetaDataPath = getBaseContainerLocation(baseDir, scmId,
+        containerId);
+    containerMetaDataPath = containerMetaDataPath + File.separator +
+        OzoneConsts.CONTAINER_META_PATH;
+    return new File(containerMetaDataPath);
+  }
+
+
+  /**
+   * Returns Container Chunks Location.
+   * @param baseDir
+   * @param scmId
+   * @param containerId
+   * @return chunksPath
+   */
+  public static File getChunksLocationPath(String baseDir, String scmId,
+                                           long containerId) {
+    String chunksPath = getBaseContainerLocation(baseDir, scmId, containerId)
+        + File.separator + OzoneConsts.STORAGE_DIR_CHUNKS;
+    return new File(chunksPath);
+  }
+
+  /**
+   * Returns base directory for specified container.
+   * @param baseDir
+   * @param scmId
+   * @param containerId
+   * @return base directory for container.
+   */
+  private static String getBaseContainerLocation(String baseDir, String scmId,
+                                        long containerId) {
+    Preconditions.checkNotNull(baseDir, "Base Directory cannot be null");
+    Preconditions.checkNotNull(scmId, "scmUuid cannot be null");
+    Preconditions.checkState(containerId >= 0,
+        "Container Id cannot be negative.");
+
+    String containerSubDirectory = getContainerSubDirectory(containerId);
+
+    String containerMetaDataPath = baseDir  + File.separator + scmId +
+        File.separator + Storage.STORAGE_DIR_CURRENT + File.separator +
+        containerSubDirectory + File.separator + containerId;
+
+    return containerMetaDataPath;
+  }
+
+  /**
+   * Returns subdirectory, where this container needs to be placed.
+   * @param containerId
+   * @return container sub directory
+   */
+  private static String getContainerSubDirectory(long containerId){
+    int directory = (int) ((containerId >> 9) & 0xFF);
+    return Storage.CONTAINER_DIR + directory;
+  }
+
+  /**
+   * Returns containerFile.
+   * @param containerMetaDataPath
+   * @param containerName
+   * @return .container File name
+   */
+  public static File getContainerFile(File containerMetaDataPath, String
+      containerName) {
+    Preconditions.checkNotNull(containerMetaDataPath);
+    Preconditions.checkNotNull(containerName);
+    return new File(containerMetaDataPath, containerName +
+        OzoneConsts.CONTAINER_EXTENSION);
+  }
+
+  /**
+   * Return containerDB File.
+   * @param containerMetaDataPath
+   * @param containerName
+   * @return containerDB File name
+   */
+  public static File getContainerDBFile(File containerMetaDataPath, String
+      containerName) {
+    Preconditions.checkNotNull(containerMetaDataPath);
+    Preconditions.checkNotNull(containerName);
+    return new File(containerMetaDataPath, containerName + OzoneConsts
+        .DN_CONTAINER_DB);
+  }
+
+  /**
+   * Returns container checksum file.
+   * @param containerMetaDataPath
+   * @param containerName
+   * @return container checksum file
+   */
+  public static File getContainerCheckSumFile(File containerMetaDataPath,
+                                              String containerName) {
+    Preconditions.checkNotNull(containerMetaDataPath);
+    Preconditions.checkNotNull(containerName);
+    return new File(containerMetaDataPath, containerName + OzoneConsts
+        .CONTAINER_FILE_CHECKSUM_EXTENSION);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
new file mode 100644
index 0000000..b868f1d
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.keyvalue.helpers;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandResponseProto;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .StorageContainerException;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.utils.MetadataStore;
+import org.apache.hadoop.utils.MetadataStoreBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Class which defines utility methods for KeyValueContainer.
+ */
+
+public final class KeyValueContainerUtil {
+
+  /* Never constructed. */
+  private KeyValueContainerUtil() {
+
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      KeyValueContainerUtil.class);
+
+
+  public static void verifyIsNewContainer(File containerFile) throws
+      FileAlreadyExistsException {
+    Preconditions.checkNotNull(containerFile, "containerFile Should not be " +
+        "null");
+    if (containerFile.getParentFile().exists()) {
+      LOG.error("container already exists on disk. File: {}", containerFile
+          .toPath());
+      throw new FileAlreadyExistsException("container already exists on " +
+            "disk.");
+    }
+  }
+
+  /**
+   * creates metadata path, chunks path and  metadata DB for the specified
+   * container.
+   *
+   * @param containerMetaDataPath
+   * @throws IOException
+   */
+  public static void createContainerMetaData(File containerMetaDataPath, File
+      chunksPath, File dbFile, String containerName, Configuration conf) throws
+      IOException {
+    Preconditions.checkNotNull(containerMetaDataPath);
+    Preconditions.checkNotNull(containerName);
+    Preconditions.checkNotNull(conf);
+
+    if (!containerMetaDataPath.mkdirs()) {
+      LOG.error("Unable to create directory for metadata storage. Path: {}",
+          containerMetaDataPath);
+      throw new IOException("Unable to create directory for metadata storage." +
+          " Path: " + containerMetaDataPath);
+    }
+    MetadataStore store = MetadataStoreBuilder.newBuilder().setConf(conf)
+        .setCreateIfMissing(true).setDbFile(dbFile).build();
+
+    // we close since the SCM pre-creates containers.
+    // we will open and put Db handle into a cache when keys are being created
+    // in a container.
+
+    store.close();
+
+    if (!chunksPath.mkdirs()) {
+      LOG.error("Unable to create chunks directory Container {}",
+          chunksPath);
+      //clean up container metadata path and metadata db
+      FileUtils.deleteDirectory(containerMetaDataPath);
+      FileUtils.deleteDirectory(containerMetaDataPath.getParentFile());
+      throw new IOException("Unable to create directory for data storage." +
+          " Path: " + chunksPath);
+    }
+  }
+
+  /**
+   * remove Container if it is empty.
+   * <p/>
+   * There are three things we need to delete.
+   * <p/>
+   * 1. Container file and metadata file. 2. The Level DB file 3. The path that
+   * we created on the data location.
+   *
+   * @param containerData - Data of the container to remove.
+   * @param conf - configuration of the cluster.
+   * @param forceDelete - whether this container should be deleted forcibly.
+   * @throws IOException
+   */
+  public static void removeContainer(KeyValueContainerData containerData,
+                                     Configuration conf, boolean forceDelete)
+      throws IOException {
+    Preconditions.checkNotNull(containerData);
+    File containerMetaDataPath = new File(containerData
+        .getMetadataPath());
+    File chunksPath = new File(containerData.getChunksPath());
+
+    MetadataStore db = KeyUtils.getDB(containerData, conf);
+
+    // If the container is not empty and cannot be deleted forcibly,
+    // then throw a SCE to stop deleting.
+    if(!forceDelete && !db.isEmpty()) {
+      throw new StorageContainerException(
+          "Container cannot be deleted because it is not empty.",
+          ContainerProtos.Result.ERROR_CONTAINER_NOT_EMPTY);
+    }
+
+    // Close the DB connection and remove the DB handler from cache
+    KeyUtils.removeDB(containerData, conf);
+
+    // Delete the Container MetaData path.
+    FileUtils.deleteDirectory(containerMetaDataPath);
+
+    //Delete the Container Chunks Path.
+    FileUtils.deleteDirectory(chunksPath);
+
+    //Delete Container directory
+    FileUtils.deleteDirectory(containerMetaDataPath.getParentFile());
+  }
+
+  /**
+   * Returns a ReadContainer Response.
+   *
+   * @param request Request
+   * @param containerData - data
+   * @return Response.
+   */
+  public static ContainerCommandResponseProto getReadContainerResponse(
+      ContainerCommandRequestProto request,
+      KeyValueContainerData containerData) {
+    Preconditions.checkNotNull(containerData);
+
+    ContainerProtos.ReadContainerResponseProto.Builder response =
+        ContainerProtos.ReadContainerResponseProto.newBuilder();
+    response.setContainerData(containerData.getProtoBufMessage());
+
+    ContainerCommandResponseProto.Builder builder =
+        ContainerUtils.getSuccessResponseBuilder(request);
+    builder.setReadContainer(response);
+    return builder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/SmallFileUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/SmallFileUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/SmallFileUtils.java
new file mode 100644
index 0000000..df60c60
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/SmallFileUtils.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.keyvalue.helpers;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandResponseProto;
+
+/**
+ * File Utils are helper routines used by putSmallFile and getSmallFile
+ * RPCs.
+ */
+public final class SmallFileUtils {
+  /**
+   * Never Constructed.
+   */
+  private SmallFileUtils() {
+  }
+
+  /**
+   * Gets a response for the putSmallFile RPC.
+   * @param msg - ContainerCommandRequestProto
+   * @return - ContainerCommandResponseProto
+   */
+  public static ContainerCommandResponseProto getPutFileResponseSuccess(
+      ContainerCommandRequestProto msg) {
+    ContainerProtos.PutSmallFileResponseProto.Builder getResponse =
+        ContainerProtos.PutSmallFileResponseProto.newBuilder();
+    ContainerCommandResponseProto.Builder builder =
+        ContainerUtils.getSuccessResponseBuilder(msg);
+    builder.setCmdType(ContainerProtos.Type.PutSmallFile);
+    builder.setPutSmallFile(getResponse);
+    return  builder.build();
+  }
+
+  /**
+   * Gets a response to the read small file call.
+   * @param msg - Msg
+   * @param data  - Data
+   * @param info  - Info
+   * @return    Response.
+   */
+  public static ContainerCommandResponseProto getGetSmallFileResponseSuccess(
+      ContainerCommandRequestProto msg, byte[] data, ChunkInfo info) {
+    Preconditions.checkNotNull(msg);
+
+    ContainerProtos.ReadChunkResponseProto.Builder readChunkresponse =
+        ContainerProtos.ReadChunkResponseProto.newBuilder();
+    readChunkresponse.setChunkData(info.getProtoBufMessage());
+    readChunkresponse.setData(ByteString.copyFrom(data));
+    readChunkresponse.setBlockID(msg.getGetSmallFile().getKey().getBlockID());
+
+    ContainerProtos.GetSmallFileResponseProto.Builder getSmallFile =
+        ContainerProtos.GetSmallFileResponseProto.newBuilder();
+    getSmallFile.setData(readChunkresponse.build());
+    ContainerCommandResponseProto.Builder builder =
+        ContainerUtils.getSuccessResponseBuilder(msg);
+    builder.setCmdType(ContainerProtos.Type.GetSmallFile);
+    builder.setGetSmallFile(getSmallFile);
+    return builder.build();
+  }
+
+}


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


[05/37] hadoop git commit: Create Version File in Datanode. Contributed by Bharat Viswanadham.

Posted by bh...@apache.org.
Create Version File in Datanode. Contributed by Bharat Viswanadham.


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

Branch: refs/heads/trunk
Commit: f26d3466d79125123cba00ab81481655d7bfe3c1
Parents: 79b2981
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Mon Jun 4 15:28:19 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Mon Jun 4 15:28:19 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/ozone/OzoneConsts.java    |   2 +
 .../org/apache/hadoop/ozone/common/Storage.java |   6 +-
 .../container/common/DataNodeLayoutVersion.java |  80 +++++++++
 .../common/helpers/DatanodeVersionFile.java     | 172 +++++++++++++++++++
 .../states/datanode/RunningDatanodeState.java   |   3 +-
 .../states/endpoint/VersionEndpointTask.java    |  71 +++++++-
 .../container/ozoneimpl/OzoneContainer.java     |   8 +-
 .../hadoop/ozone/protocol/VersionResponse.java  |   4 +
 .../ozone/container/common/ScmTestMock.java     |  24 +++
 .../common/TestDatanodeLayOutVersion.java       |  38 ++++
 .../common/TestDatanodeStateMachine.java        |   3 +-
 .../common/helpers/TestDatanodeVersionFile.java | 120 +++++++++++++
 .../hadoop/hdds/scm/node/SCMNodeManager.java    |   2 +
 .../ozone/container/common/TestEndPoint.java    | 169 +++++++++++++++++-
 14 files changed, 688 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f26d3466/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index 451a08f..ce1a733 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -29,6 +29,8 @@ public final class OzoneConsts {
 
   public static final String STORAGE_DIR = "scm";
   public static final String SCM_ID = "scmUuid";
+  public static final String LAYOUTVERSION = "layOutVersion";
+  public static final String CTIME = "ctime";
 
   public static final String OZONE_SIMPLE_ROOT_USER = "root";
   public static final String OZONE_SIMPLE_HDFS_USER = "hdfs";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f26d3466/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
index fb30d92..35ddc71 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
@@ -45,8 +45,10 @@ import java.util.Properties;
 public abstract class Storage {
   private static final Logger LOG = LoggerFactory.getLogger(Storage.class);
 
-  protected static final String STORAGE_DIR_CURRENT = "current";
-  protected static final String STORAGE_FILE_VERSION = "VERSION";
+  public static final String STORAGE_DIR_CURRENT = "current";
+  public static final String STORAGE_FILE_VERSION = "VERSION";
+  public static final String STORAGE_DIR_HDDS = "hdds";
+
 
   private final NodeType nodeType;
   private final File root;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f26d3466/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/DataNodeLayoutVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/DataNodeLayoutVersion.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/DataNodeLayoutVersion.java
new file mode 100644
index 0000000..2d58c39
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/DataNodeLayoutVersion.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.common;
+
+/**
+ * Datanode layout version which describes information about the layout version
+ * on the datanode.
+ */
+public final class DataNodeLayoutVersion {
+
+  // We will just be normal and use positive counting numbers for versions.
+  private final static DataNodeLayoutVersion[] VERSION_INFOS =
+      {new DataNodeLayoutVersion(1, "HDDS Datanode LayOut Version 1")};
+
+  private final String description;
+  private final int version;
+
+  /**
+   * Never created outside this class.
+   *
+   * @param description -- description
+   * @param version     -- version number
+   */
+  private DataNodeLayoutVersion(int version, String description) {
+    this.description = description;
+    this.version = version;
+  }
+
+  /**
+   * Returns all versions.
+   *
+   * @return Version info array.
+   */
+  public static DataNodeLayoutVersion[] getAllVersions() {
+    return VERSION_INFOS.clone();
+  }
+
+  /**
+   * Returns the latest version.
+   *
+   * @return versionInfo
+   */
+  public static DataNodeLayoutVersion getLatestVersion() {
+    return VERSION_INFOS[VERSION_INFOS.length - 1];
+  }
+
+  /**
+   * Return description.
+   *
+   * @return String
+   */
+  public String getDescription() {
+    return description;
+  }
+
+  /**
+   * Return the version.
+   *
+   * @return int.
+   */
+  public int getVersion() {
+    return version;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f26d3466/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeVersionFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeVersionFile.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeVersionFile.java
new file mode 100644
index 0000000..0010d7e
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeVersionFile.java
@@ -0,0 +1,172 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.helpers;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
+import org.apache.hadoop.ozone.common.Storage;
+import org.apache.hadoop.ozone.container.common.DataNodeLayoutVersion;
+
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.util.Properties;
+
+/**
+ * This is a utility class which helps to create the version file on datanode
+ * and also validate the content of the version file.
+ */
+public class DatanodeVersionFile {
+
+  private final String scmUuid;
+  private final long cTime;
+  private final int layOutVersion;
+
+  public DatanodeVersionFile(String scmUuid, long cTime, int layOutVersion) {
+    this.scmUuid = scmUuid;
+    this.cTime = cTime;
+    this.layOutVersion = layOutVersion;
+  }
+
+  private Properties createProperties() {
+    Properties properties = new Properties();
+    properties.setProperty(OzoneConsts.SCM_ID, scmUuid);
+    properties.setProperty(OzoneConsts.CTIME, String.valueOf(cTime));
+    properties.setProperty(OzoneConsts.LAYOUTVERSION, String.valueOf(
+        layOutVersion));
+    return properties;
+  }
+
+  /**
+   * Creates a version File in specified path.
+   * @param path
+   * @throws IOException
+   */
+  public void createVersionFile(File path) throws
+      IOException {
+    try (RandomAccessFile file = new RandomAccessFile(path, "rws");
+         FileOutputStream out = new FileOutputStream(file.getFD())) {
+      file.getChannel().truncate(0);
+      Properties properties = createProperties();
+      /*
+       * If server is interrupted before this line,
+       * the version file will remain unchanged.
+       */
+      properties.store(out, null);
+      /*
+       * Now the new fields are flushed to the head of the file, but file
+       * length can still be larger then required and therefore the file can
+       * contain whole or corrupted fields from its old contents in the end.
+       * If server is interrupted here and restarted later these extra fields
+       * either should not effect server behavior or should be handled
+       * by the server correctly.
+       */
+      file.getChannel().truncate(file.getChannel().size());
+    }
+  }
+
+
+  /**
+   * Creates a property object from the specified file content.
+   * @param  versionFile
+   * @return Properties
+   * @throws IOException
+   */
+  public static Properties readFrom(File versionFile) throws IOException {
+    try (RandomAccessFile file = new RandomAccessFile(versionFile, "rws");
+         FileInputStream in = new FileInputStream(file.getFD())) {
+      Properties props = new Properties();
+      props.load(in);
+      return props;
+    }
+  }
+
+  /**
+   * Verifies scmUuid is valid or not.
+   * @param scmIdVersionFile
+   * @param scmId
+   * @throws InconsistentStorageStateException
+   */
+  @VisibleForTesting
+  public static void verifyScmUuid(String scmIdVersionFile, String scmId) throws
+      InconsistentStorageStateException {
+    Preconditions.checkState(StringUtils.isNotBlank(scmIdVersionFile),
+        "Invalid scmUuid from Version File.");
+    Preconditions.checkState(StringUtils.isNotBlank(scmId),
+        "Invalid scmUuid from SCM version request response");
+    if(!scmIdVersionFile.equals(scmId)) {
+      throw new InconsistentStorageStateException("MisMatch of ScmUuid " +
+          "scmUuid from version File is: " + scmIdVersionFile + "SCM " +
+          "version response scmUuid is" + scmId);
+    }
+  }
+
+  /**
+   * Verifies creationTime is valid or not.
+   * @param creationTime
+   */
+  @VisibleForTesting
+  public static void verifyCreationTime(String creationTime) {
+    Preconditions.checkState(!StringUtils.isBlank(creationTime),
+        "Invalid creation Time.");
+  }
+
+  /**
+   * Verifies layOutVersion is valid or not.
+   * @param lv
+   * @throws InconsistentStorageStateException
+   */
+  @VisibleForTesting
+  public static void verifyLayOutVersion(String lv) throws
+      InconsistentStorageStateException {
+    Preconditions.checkState(!StringUtils.isBlank(lv),
+        "Invalid layOutVersion.");
+    int version = Integer.parseInt(lv);
+    if(DataNodeLayoutVersion.getLatestVersion().getVersion() != version) {
+      throw new InconsistentStorageStateException("Incorrect layOutVersion");
+    }
+  }
+
+  /**
+   * Returns the versionFile path for the StorageLocation.
+   * @param location
+   * @param scmUuid
+   * @return versionFile - File
+   */
+  @VisibleForTesting
+  public static File getVersionFile(StorageLocation location, String scmUuid) {
+    if (location != null) {
+      String path = location.getUri().getPath();
+      File parentPath = new File(path + File.separator + Storage
+          .STORAGE_DIR_HDDS + File.separator +  scmUuid + File.separator +
+          Storage.STORAGE_DIR_CURRENT + File.separator);
+      File versionFile = new File(parentPath, Storage.STORAGE_FILE_VERSION);
+      return versionFile;
+    } else {
+      return null;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f26d3466/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
index 3e11d12..6e30ebc 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
@@ -95,7 +95,8 @@ public class RunningDatanodeState implements DatanodeState {
       getEndPointTask(EndpointStateMachine endpoint) {
     switch (endpoint.getState()) {
     case GETVERSION:
-      return new VersionEndpointTask(endpoint, conf);
+      return new VersionEndpointTask(endpoint, conf, context.getParent().
+          getContainer());
     case REGISTER:
       return  RegisterEndpointTask.newBuilder()
           .setConfig(conf)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f26d3466/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
index b048ee5..e03a438 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
@@ -16,14 +16,30 @@
  */
 package org.apache.hadoop.ozone.container.common.states.endpoint;
 
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
+import org.apache.hadoop.ozone.common.Storage;
+import org.apache.hadoop.ozone.container.common.DataNodeLayoutVersion;
+import org.apache.hadoop.ozone.container.common.helpers.DatanodeVersionFile;
 import org.apache.hadoop.ozone.container.common.statemachine
     .EndpointStateMachine;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.ozone.protocol.VersionResponse;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+
+import java.io.File;
 import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
 import java.util.concurrent.Callable;
 
 /**
@@ -33,11 +49,15 @@ public class VersionEndpointTask implements
     Callable<EndpointStateMachine.EndPointStates> {
   private final EndpointStateMachine rpcEndPoint;
   private final Configuration configuration;
+  private final OzoneContainer datanodeContainerManager;
+  static final Logger LOG =
+      LoggerFactory.getLogger(VersionEndpointTask.class);
 
   public VersionEndpointTask(EndpointStateMachine rpcEndPoint,
-      Configuration conf) {
+                             Configuration conf, OzoneContainer container) {
     this.rpcEndPoint = rpcEndPoint;
     this.configuration = conf;
+    this.datanodeContainerManager = container;
   }
 
   /**
@@ -49,15 +69,56 @@ public class VersionEndpointTask implements
   @Override
   public EndpointStateMachine.EndPointStates call() throws Exception {
     rpcEndPoint.lock();
-    try{
+    try {
       SCMVersionResponseProto versionResponse =
           rpcEndPoint.getEndPoint().getVersion(null);
-      rpcEndPoint.setVersion(VersionResponse.getFromProtobuf(versionResponse));
+      VersionResponse response = VersionResponse.getFromProtobuf(
+          versionResponse);
+      String scmUuid = response.getValue(OzoneConsts.SCM_ID);
+      Preconditions.checkState(!StringUtils.isBlank(scmUuid),
+          "Invalid SCM UuiD in the response.");
+
+      rpcEndPoint.setVersion(response);
+      LOG.debug("scmUuid is {}", scmUuid);
+
+      List<StorageLocation> locations = datanodeContainerManager.getLocations();
 
-      EndpointStateMachine.EndPointStates nextState =
-          rpcEndPoint.getState().getNextState();
+      for (StorageLocation location : locations) {
+        String path = location.getUri().getPath();
+        File parentPath = new File(path + File.separator + Storage
+            .STORAGE_DIR_HDDS + File.separator + scmUuid + File.separator +
+            Storage.STORAGE_DIR_CURRENT);
+        File versionFile = DatanodeVersionFile.getVersionFile(location,
+            scmUuid);
+        if (!parentPath.exists() && !parentPath.mkdirs()) {
+          LOG.error("Directory doesn't exist and cannot be created. Path: {}",
+              parentPath.toString());
+          rpcEndPoint.setState(EndpointStateMachine.EndPointStates.SHUTDOWN);
+          throw new IllegalArgumentException("Directory doesn't exist and " +
+              "cannot be created. " + parentPath.toString());
+        } else {
+          if (versionFile.exists()) {
+            Properties properties = DatanodeVersionFile.readFrom(versionFile);
+            DatanodeVersionFile.verifyScmUuid(properties.getProperty(
+                OzoneConsts.SCM_ID), scmUuid);
+            DatanodeVersionFile.verifyCreationTime(properties.getProperty(
+                OzoneConsts.CTIME));
+            DatanodeVersionFile.verifyLayOutVersion(properties.getProperty(
+                OzoneConsts.LAYOUTVERSION));
+          } else {
+            DatanodeVersionFile dnVersionFile = new DatanodeVersionFile(scmUuid,
+                Time.now(), DataNodeLayoutVersion.getLatestVersion()
+                .getVersion());
+            dnVersionFile.createVersionFile(versionFile);
+          }
+        }
+      }
+      EndpointStateMachine.EndPointStates nextState = rpcEndPoint.getState().
+          getNextState();
       rpcEndPoint.setState(nextState);
       rpcEndPoint.zeroMissedCount();
+    } catch (InconsistentStorageStateException ex) {
+      throw ex;
     } catch (IOException ex) {
       rpcEndPoint.logIfNeeded(ex);
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f26d3466/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
index b357fef..69bdf32 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
@@ -51,6 +51,7 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.nio.file.Paths;
+import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
@@ -82,6 +83,7 @@ public class OzoneContainer {
   private final ChunkManager chunkManager;
   private final KeyManager keyManager;
   private final BlockDeletingService blockDeletingService;
+  private final List<StorageLocation> locations;
 
   /**
    * Creates a network endpoint and enables Ozone container.
@@ -93,7 +95,7 @@ public class OzoneContainer {
       DatanodeDetails datanodeDetails, Configuration ozoneConfig)
       throws IOException {
     this.ozoneConfig = ozoneConfig;
-    List<StorageLocation> locations = new LinkedList<>();
+    locations = new LinkedList<>();
     String[] paths = ozoneConfig.getStrings(
         OzoneConfigKeys.OZONE_METADATA_DIRS);
     if (paths != null && paths.length > 0) {
@@ -137,6 +139,10 @@ public class OzoneContainer {
     };
   }
 
+  public List<StorageLocation> getLocations() {
+    return Collections.unmodifiableList(this.locations);
+  }
+
   /**
    * Starts serving requests to ozone container.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f26d3466/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/VersionResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/VersionResponse.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/VersionResponse.java
index 83acf5b..c26fbfa 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/VersionResponse.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/VersionResponse.java
@@ -88,6 +88,10 @@ public class VersionResponse {
     values.put(key, value);
   }
 
+  public String getValue(String key) {
+    return this.values.get(key);
+  }
+
   /**
    * Return a protobuf message.
    * @return SCMVersionResponseProto.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f26d3466/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
index 0ee6321..cd940b5 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
@@ -57,6 +57,28 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
   private Map<DatanodeDetails, Map<String, ContainerInfo>> nodeContainers =
       new HashMap();
   private Map<DatanodeDetails, NodeReportProto> nodeReports = new HashMap<>();
+  private UUID scmUuid;
+
+  public ScmTestMock() {
+    scmUuid = UUID.randomUUID();
+  }
+
+  /**
+   * Return scmUuid.
+   * @return UUID
+   */
+  public UUID getScmUuid() {
+    return scmUuid;
+  }
+
+  /**
+   * set scmUuid.
+   * @param id
+   */
+  public void setSCMUuid(UUID id) {
+    this.scmUuid = id;
+  }
+
   /**
    * Returns the number of heartbeats made to this class.
    *
@@ -147,9 +169,11 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
     rpcCount.incrementAndGet();
     sleepIfNeeded();
     VersionInfo versionInfo = VersionInfo.getLatestVersion();
+
     return VersionResponse.newBuilder()
         .setVersion(versionInfo.getVersion())
         .addValue(VersionInfo.DESCRIPTION_KEY, versionInfo.getDescription())
+        .addValue("scmUuid", scmUuid.toString())
         .build().getProtobufMessage();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f26d3466/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeLayOutVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeLayOutVersion.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeLayOutVersion.java
new file mode 100644
index 0000000..5cabef2
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeLayOutVersion.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.container.common;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * This class tests DatanodeLayOutVersion.
+ */
+public class TestDatanodeLayOutVersion {
+
+  @Test
+  public void testDatanodeLayOutVersion() {
+    // Check Latest Version and description
+    Assert.assertEquals(1, DataNodeLayoutVersion.getLatestVersion()
+        .getVersion());
+    Assert.assertEquals("HDDS Datanode LayOut Version 1", DataNodeLayoutVersion
+        .getLatestVersion().getDescription());
+    Assert.assertEquals(DataNodeLayoutVersion.getAllVersions().length,
+        DataNodeLayoutVersion.getAllVersions().length);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f26d3466/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
index ece7545..79841bd 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.container.common;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ipc.RPC;
@@ -145,7 +146,7 @@ public class TestDatanodeStateMachine {
     } catch (Exception e) {
       //ignore all execption from the shutdown
     } finally {
-      testRoot.delete();
+      FileUtil.fullyDelete(testRoot);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f26d3466/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java
new file mode 100644
index 0000000..e798a7d
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java
@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.helpers;
+
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
+import org.apache.hadoop.ozone.container.common.DataNodeLayoutVersion;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Time;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Properties;
+import java.util.UUID;
+
+import static org.junit.Assert.*;
+
+/**
+ * This class tests DatanodeVersionFile.
+ */
+public class TestDatanodeVersionFile {
+
+  @Rule
+  public TemporaryFolder folder= new TemporaryFolder();
+
+  @Test
+  public void testCreateAndReadVersionFile() throws IOException{
+    File versionFile = folder.newFile("Version");
+    String uuid = UUID.randomUUID().toString();
+    long now = Time.now();
+    int lv = DataNodeLayoutVersion.getLatestVersion().getVersion();
+
+    DatanodeVersionFile dnVersionFile = new DatanodeVersionFile(uuid, now, lv);
+
+    dnVersionFile.createVersionFile(versionFile);
+
+    //Check VersionFile exists
+    assertTrue(versionFile.exists());
+
+    Properties properties = dnVersionFile.readFrom(versionFile);
+
+    assertEquals(uuid, properties.getProperty(OzoneConsts.SCM_ID));
+    assertEquals(String.valueOf(now), properties.get(OzoneConsts.CTIME));
+    assertEquals(String.valueOf(lv), properties.get(OzoneConsts.LAYOUTVERSION));
+
+    DatanodeVersionFile.verifyCreationTime(String.valueOf(properties.get(
+        OzoneConsts.CTIME)));
+    DatanodeVersionFile.verifyLayOutVersion(String.valueOf(properties
+        .getProperty(OzoneConsts.LAYOUTVERSION)));
+    DatanodeVersionFile.verifyScmUuid(uuid, String.valueOf(properties
+        .getProperty(OzoneConsts.SCM_ID)));
+
+
+  }
+
+  @Test
+  public void testVerifyUuid() throws IOException{
+    String uuid = UUID.randomUUID().toString();
+    try {
+      DatanodeVersionFile.verifyScmUuid(uuid, uuid);
+      DatanodeVersionFile.verifyScmUuid(uuid, UUID.randomUUID().toString());
+      fail("Test failure in testVerifyUuid");
+    } catch (InconsistentStorageStateException ex) {
+      GenericTestUtils.assertExceptionContains("MisMatch of ScmUuid", ex);
+    }
+  }
+
+  @Test
+  public void testVerifyCTime() throws IOException{
+    try {
+      DatanodeVersionFile.verifyCreationTime(String.valueOf(Time.now()));
+      DatanodeVersionFile.verifyCreationTime(null);
+      fail("Test failure in testVerifyCTime");
+    } catch (IllegalStateException ex) {
+      GenericTestUtils.assertExceptionContains("Invalid creation Time.", ex);
+    }
+  }
+
+  @Test
+  public void testVerifyLayOut() throws IOException{
+    String lv = String.valueOf(DataNodeLayoutVersion.getLatestVersion()
+        .getVersion());
+    try {
+      DatanodeVersionFile.verifyLayOutVersion(lv);
+      DatanodeVersionFile.verifyLayOutVersion(null);
+      fail("Test failure in testVerifyLayOut");
+    } catch (IllegalStateException ex) {
+      GenericTestUtils.assertExceptionContains("Invalid layOutVersion.", ex);
+    }
+  }
+
+  @Test
+  public void testGetVersionFile() throws IOException {
+    StorageLocation location = StorageLocation.parse("/tmp/disk1");
+    String scmId = UUID.randomUUID().toString();
+    assertEquals(new File("/tmp/disk1/hdds/" + scmId + "/current/VERSION"),
+        DatanodeVersionFile.getVersionFile(location, scmId));
+    assertEquals(null, DatanodeVersionFile.getVersionFile(null, scmId));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f26d3466/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
index b339fb7..016e1e1 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.protocol.StorageContainerNodeProtocol;
 import org.apache.hadoop.ozone.protocol.VersionResponse;
 import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
@@ -703,6 +704,7 @@ public class SCMNodeManager
   public VersionResponse getVersion(SCMVersionRequestProto versionRequest) {
     return VersionResponse.newBuilder()
         .setVersion(this.version.getVersion())
+        .addValue(OzoneConsts.SCM_ID, scmManager.getScmStorage().getScmId())
         .build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f26d3466/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
index 34779da..9d2e61a 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.container.common;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.scm.TestUtils;
@@ -33,6 +34,7 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdds.protocol.proto
@@ -41,6 +43,7 @@ import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerReport;
+import org.apache.hadoop.ozone.container.common.helpers.DatanodeVersionFile;
 import org.apache.hadoop.ozone.container.common.statemachine
     .DatanodeStateMachine;
 import org.apache.hadoop.ozone.container.common.statemachine
@@ -53,17 +56,22 @@ import org.apache.hadoop.ozone.container.common.states.endpoint
 import org.apache.hadoop.ozone.container.common.states.endpoint
     .VersionEndpointTask;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.Time;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
+
+import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 
 import java.io.File;
 import java.net.InetSocketAddress;
+import java.util.ArrayList;
 import java.util.List;
+import java.util.Properties;
 import java.util.UUID;
 
 import static org.apache.hadoop.hdds.scm.TestUtils.getDatanodeDetails;
@@ -116,6 +124,11 @@ public class TestEndPoint {
           responseProto.getKeys(0).getKey());
       Assert.assertEquals(VersionInfo.getLatestVersion().getDescription(),
           responseProto.getKeys(0).getValue());
+      Assert.assertEquals("scmUuid", responseProto.getKeys(
+          1).getKey());
+      Assert.assertEquals(scmServerImpl.getScmUuid().toString(),
+          responseProto.getKeys(1).getValue());
+
     }
   }
 
@@ -126,11 +139,20 @@ public class TestEndPoint {
    */
   public void testGetVersionTask() throws Exception {
     Configuration conf = SCMTestUtils.getConf();
+    String path = new FileSystemTestHelper().getTestRootDir();
+    conf.set(DFS_DATANODE_DATA_DIR_KEY, path);
     try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
         serverAddress, 1000)) {
       rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
+      OzoneContainer ozoneContainer = mock(OzoneContainer.class);
+      List<StorageLocation> pathList = new ArrayList<>();
+      for (String dir : conf.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
+        StorageLocation location = StorageLocation.parse(dir);
+        pathList.add(location);
+      }
+      when(ozoneContainer.getLocations()).thenReturn(pathList);
       VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
-          conf);
+          conf, ozoneContainer);
       EndpointStateMachine.EndPointStates newState = versionTask.call();
 
       // if version call worked the endpoint should automatically move to the
@@ -140,10 +162,132 @@ public class TestEndPoint {
 
       // Now rpcEndpoint should remember the version it got from SCM
       Assert.assertNotNull(rpcEndPoint.getVersion());
+      FileUtil.fullyDelete(new File(path));
     }
   }
 
   @Test
+  public void testVersionCheckFail() throws Exception {
+    Configuration conf = SCMTestUtils.getConf();
+    String path = new FileSystemTestHelper().getTestRootDir();
+    conf.set(DFS_DATANODE_DATA_DIR_KEY, path);
+    try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
+        serverAddress, 1000)) {
+      rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
+      OzoneContainer ozoneContainer = mock(OzoneContainer.class);
+      List<StorageLocation> pathList = new ArrayList<>();
+      for (String dir : conf.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
+        StorageLocation location = StorageLocation.parse(dir);
+        pathList.add(location);
+      }
+      when(ozoneContainer.getLocations()).thenReturn(pathList);
+      VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
+          conf, ozoneContainer);
+      EndpointStateMachine.EndPointStates newState = versionTask.call();
+
+      // if version call worked the endpoint should automatically move to the
+      // next state.
+      Assert.assertEquals(EndpointStateMachine.EndPointStates.REGISTER,
+          newState);
+
+      // Now rpcEndpoint should remember the version it got from SCM
+      Assert.assertNotNull(rpcEndPoint.getVersion());
+
+      // Now call again version task with an incorrect layout version.
+      // This will fail with Incorrect layOutVersion error.
+      DatanodeVersionFile datanodeVersionFile = new DatanodeVersionFile(
+          scmServerImpl.getScmUuid().toString(), Time.now(), 2);
+      datanodeVersionFile.createVersionFile(DatanodeVersionFile
+          .getVersionFile(pathList.get(0), scmServerImpl.getScmUuid()
+              .toString()));
+      rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
+      versionTask.call();
+      fail("Test fail");
+    } catch(Throwable t) {
+      GenericTestUtils.assertExceptionContains("Incorrect layOutVersion", t);
+      FileUtil.fullyDelete(new File(path));
+    }
+  }
+
+
+  @Test
+  public void testVersionCheckSuccess() throws Exception {
+    Configuration conf = SCMTestUtils.getConf();
+    String path = new FileSystemTestHelper().getTestRootDir();
+    conf.set(DFS_DATANODE_DATA_DIR_KEY, path);
+    try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
+        serverAddress, 1000)) {
+      rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
+      OzoneContainer ozoneContainer = mock(OzoneContainer.class);
+      List<StorageLocation> pathList = new ArrayList<>();
+      for (String dir : conf.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
+        StorageLocation location = StorageLocation.parse(dir);
+        pathList.add(location);
+      }
+      when(ozoneContainer.getLocations()).thenReturn(pathList);
+      VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
+          conf, ozoneContainer);
+      EndpointStateMachine.EndPointStates newState = versionTask.call();
+
+      // if version call worked the endpoint should automatically move to the
+      // next state.
+      Assert.assertEquals(EndpointStateMachine.EndPointStates.REGISTER,
+          newState);
+
+      // Now rpcEndpoint should remember the version it got from SCM
+      Assert.assertNotNull(rpcEndPoint.getVersion());
+
+      // Now call again Version Task, this time version check should succeed.
+      rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
+      newState = versionTask.call();
+      Assert.assertEquals(EndpointStateMachine.EndPointStates.REGISTER,
+          newState);
+      FileUtil.fullyDelete(new File(path));
+    }
+  }
+
+  @Test
+  public void testVersionCheckFile() throws Exception {
+    Configuration conf = SCMTestUtils.getConf();
+    FileUtil.fullyDelete(new File("/tmp/hadoop"));
+    conf.set(DFS_DATANODE_DATA_DIR_KEY, "/tmp/hadoop");
+    try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
+        serverAddress, 1000)) {
+      rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
+      OzoneContainer ozoneContainer = mock(OzoneContainer.class);
+      List<StorageLocation> pathList = new ArrayList<>();
+      String dir = conf.get(DFS_DATANODE_DATA_DIR_KEY);
+      StorageLocation location = StorageLocation.parse(dir);
+      pathList.add(location);
+
+      when(ozoneContainer.getLocations()).thenReturn(pathList);
+      VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
+          conf, ozoneContainer);
+      EndpointStateMachine.EndPointStates newState = versionTask.call();
+
+      // if version call worked the endpoint should automatically move to the
+      // next state.
+      Assert.assertEquals(EndpointStateMachine.EndPointStates.REGISTER,
+          newState);
+
+      // Now rpcEndpoint should remember the version it got from SCM
+      Assert.assertNotNull(rpcEndPoint.getVersion());
+
+      // Check Version File created or not and content is expected or not.
+      File versionFile = DatanodeVersionFile.getVersionFile(pathList.get(0),
+          scmServerImpl.getScmUuid().toString());
+      Assert.assertTrue(versionFile.exists());
+
+      Properties props = DatanodeVersionFile.readFrom(versionFile);
+      DatanodeVersionFile.verifyCreationTime(props.getProperty(OzoneConsts
+          .CTIME));
+      DatanodeVersionFile.verifyScmUuid(scmServerImpl.getScmUuid().toString(),
+          props.getProperty(OzoneConsts.SCM_ID));
+      DatanodeVersionFile.verifyLayOutVersion(props.getProperty(OzoneConsts
+          .LAYOUTVERSION));
+    }
+  }
+  @Test
   /**
    * This test makes a call to end point where there is no SCM server. We
    * expect that versionTask should be able to handle it.
@@ -152,11 +296,20 @@ public class TestEndPoint {
     Configuration conf = SCMTestUtils.getConf();
     InetSocketAddress nonExistentServerAddress = SCMTestUtils
         .getReuseableAddress();
+    FileUtil.fullyDelete(new File("/tmp/hadoop"));
+    conf.set(DFS_DATANODE_DATA_DIR_KEY, "/tmp/hadoop");
     try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
         nonExistentServerAddress, 1000)) {
       rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
+      OzoneContainer ozoneContainer = mock(OzoneContainer.class);
+      List<StorageLocation> pathList = new ArrayList<>();
+      for (String dir : conf.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
+        StorageLocation location = StorageLocation.parse(dir);
+        pathList.add(location);
+      }
+      when(ozoneContainer.getLocations()).thenReturn(pathList);
       VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
-          conf);
+          conf, ozoneContainer);
       EndpointStateMachine.EndPointStates newState = versionTask.call();
 
       // This version call did NOT work, so endpoint should remain in the same
@@ -176,12 +329,20 @@ public class TestEndPoint {
     final long rpcTimeout = 1000;
     final long tolerance = 100;
     Configuration conf = SCMTestUtils.getConf();
-
+    FileUtil.fullyDelete(new File("/tmp/hadoop"));
+    conf.set(DFS_DATANODE_DATA_DIR_KEY, "/tmp/hadoop");
     try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
         serverAddress, (int) rpcTimeout)) {
       rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
+      OzoneContainer ozoneContainer = mock(OzoneContainer.class);
+      List<StorageLocation> pathList = new ArrayList<>();
+      for (String dir : conf.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
+        StorageLocation location = StorageLocation.parse(dir);
+        pathList.add(location);
+      }
+      when(ozoneContainer.getLocations()).thenReturn(pathList);
       VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
-          conf);
+          conf, ozoneContainer);
 
       scmServerImpl.setRpcResponseDelay(1500);
       long start = Time.monotonicNow();


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


[18/37] hadoop git commit: HDDS-173. Refactor Dispatcher and implement Handler for new ContainerIO design.

Posted by bh...@apache.org.
HDDS-173. Refactor Dispatcher and implement Handler for new ContainerIO design.


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

Branch: refs/heads/trunk
Commit: 13579f9296ddb087ff5fd24061f3aaf17741e6c9
Parents: ca192cb
Author: Hanisha Koneru <ha...@apache.org>
Authored: Thu Jun 28 10:13:30 2018 -0700
Committer: Hanisha Koneru <ha...@apache.org>
Committed: Thu Jun 28 10:13:30 2018 -0700

----------------------------------------------------------------------
 .../scm/storage/ContainerProtocolCalls.java     |  24 +-
 .../apache/hadoop/ozone/OzoneConfigKeys.java    |  12 +
 .../main/proto/DatanodeContainerProtocol.proto  |  20 +-
 .../common/src/main/resources/ozone-default.xml |  33 +
 .../container/common/helpers/ChunkUtils.java    |   5 +-
 .../common/helpers/ContainerUtils.java          | 169 ++---
 .../container/common/helpers/FileUtils.java     |  82 ---
 .../container/common/helpers/KeyUtils.java      |   5 +-
 .../container/common/impl/ContainerSet.java     |   7 +-
 .../ozone/container/common/impl/Dispatcher.java |  49 +-
 .../container/common/impl/HddsDispatcher.java   | 180 ++++++
 .../common/impl/KeyValueContainerData.java      | 191 ------
 .../container/common/impl/KeyValueYaml.java     | 275 --------
 .../container/common/interfaces/Container.java  |  15 +
 .../container/common/interfaces/Handler.java    |  71 ++
 .../server/ratis/ContainerStateMachine.java     |   5 +-
 .../container/common/volume/VolumeSet.java      |  11 +-
 .../container/keyvalue/ChunkManagerImpl.java    | 240 -------
 .../container/keyvalue/KeyManagerImpl.java      | 188 ------
 .../container/keyvalue/KeyValueContainer.java   |  29 +-
 .../keyvalue/KeyValueContainerData.java         | 200 ++++++
 .../keyvalue/KeyValueContainerLocationUtil.java | 140 ----
 .../keyvalue/KeyValueContainerUtil.java         | 148 -----
 .../container/keyvalue/KeyValueHandler.java     | 643 +++++++++++++++++++
 .../ozone/container/keyvalue/KeyValueYaml.java  | 272 ++++++++
 .../container/keyvalue/helpers/ChunkUtils.java  |  50 +-
 .../container/keyvalue/helpers/KeyUtils.java    |  35 +-
 .../helpers/KeyValueContainerLocationUtil.java  | 140 ++++
 .../keyvalue/helpers/KeyValueContainerUtil.java | 173 +++++
 .../keyvalue/helpers/SmallFileUtils.java        |  84 +++
 .../keyvalue/impl/ChunkManagerImpl.java         | 240 +++++++
 .../container/keyvalue/impl/KeyManagerImpl.java | 192 ++++++
 .../keyvalue/interfaces/KeyManager.java         |   4 +-
 .../common/TestKeyValueContainerData.java       |  36 +-
 .../container/common/impl/TestContainerSet.java |  12 +-
 .../container/common/impl/TestKeyValueYaml.java |   2 +
 .../common/interfaces/TestHandler.java          |  91 +++
 .../keyvalue/TestChunkManagerImpl.java          |   3 +-
 .../container/keyvalue/TestKeyManagerImpl.java  |   7 +-
 .../keyvalue/TestKeyValueContainer.java         |  10 +-
 .../container/keyvalue/TestKeyValueHandler.java | 246 +++++++
 .../ozone/client/io/ChunkGroupInputStream.java  |   6 +-
 .../client/io/OzoneContainerTranslation.java    |  50 --
 .../ozone/container/ContainerTestHelper.java    |  25 +-
 .../container/ozoneimpl/TestOzoneContainer.java |   6 +-
 .../genesis/BenchMarkDatanodeDispatcher.java    |  12 +-
 46 files changed, 2881 insertions(+), 1557 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
index d3af083..f4f14ef 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
@@ -29,6 +29,8 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandResponseProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .DatanodeBlockID;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .GetKeyRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .GetKeyResponseProto;
@@ -73,16 +75,16 @@ public final class ContainerProtocolCalls  {
    * Calls the container protocol to get a container key.
    *
    * @param xceiverClient client to perform call
-   * @param containerKeyData key data to identify container
+   * @param datanodeBlockID blockID to identify container
    * @param traceID container protocol call args
    * @return container protocol get key response
    * @throws IOException if there is an I/O error while performing the call
    */
   public static GetKeyResponseProto getKey(XceiverClientSpi xceiverClient,
-      KeyData containerKeyData, String traceID) throws IOException {
+      DatanodeBlockID datanodeBlockID, String traceID) throws IOException {
     GetKeyRequestProto.Builder readKeyRequest = GetKeyRequestProto
         .newBuilder()
-        .setKeyData(containerKeyData);
+        .setBlockID(datanodeBlockID);
     String id = xceiverClient.getPipeline().getLeader().getUuidString();
     ContainerCommandRequestProto request = ContainerCommandRequestProto
         .newBuilder()
@@ -240,18 +242,15 @@ public final class ContainerProtocolCalls  {
     ContainerProtos.CreateContainerRequestProto.Builder createRequest =
         ContainerProtos.CreateContainerRequestProto
             .newBuilder();
-    ContainerProtos.ContainerData.Builder containerData = ContainerProtos
-        .ContainerData.newBuilder();
-    containerData.setContainerID(containerID);
-    containerData.setContainerType(ContainerProtos.ContainerType
+    createRequest.setContainerID(containerID);
+    createRequest.setContainerType(ContainerProtos.ContainerType
         .KeyValueContainer);
-    createRequest.setContainerData(containerData.build());
 
     String id = client.getPipeline().getLeader().getUuidString();
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
     request.setCmdType(ContainerProtos.Type.CreateContainer);
-    request.setCreateContainer(createRequest);
+    request.setCreateContainer(createRequest.build());
     request.setDatanodeUuid(id);
     request.setTraceID(traceID);
     ContainerCommandResponseProto response = client.sendCommand(
@@ -348,14 +347,9 @@ public final class ContainerProtocolCalls  {
    */
   public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client,
       BlockID blockID, String traceID) throws IOException {
-    KeyData containerKeyData = KeyData
-        .newBuilder()
-        .setBlockID(blockID.getDatanodeBlockIDProtobuf())
-        .build();
-
     GetKeyRequestProto.Builder getKey = GetKeyRequestProto
         .newBuilder()
-        .setKeyData(containerKeyData);
+        .setBlockID(blockID.getDatanodeBlockIDProtobuf());
     ContainerProtos.GetSmallFileRequestProto getSmallFileRequest =
         GetSmallFileRequestProto
             .newBuilder().setKey(getKey)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
index 856d088..fc10fbb 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
@@ -256,6 +256,18 @@ public final class OzoneConfigKeys {
       "hdds.datanode.storage.utilization.critical.threshold";
   public static final double
       HDDS_DATANODE_STORAGE_UTILIZATION_CRITICAL_THRESHOLD_DEFAULT = 0.75;
+
+  public static final String
+      HDDS_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY =
+      "hdds.write.lock.reporting.threshold.ms";
+  public static final long
+      HDDS_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT = 5000L;
+  public static final String
+      HDDS_LOCK_SUPPRESS_WARNING_INTERVAL_MS_KEY =
+      "hdds.lock.suppress.warning.interval.ms";
+  public static final long
+      HDDS_LOCK_SUPPRESS_WARNING_INTERVAL_MS_DEAFULT = 10000L;
+
   /**
    * There is no need to instantiate this class.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
index 88645be..d29e479 100644
--- a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
+++ b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
@@ -192,6 +192,7 @@ message ContainerCommandRequestProto {
   optional   PutSmallFileRequestProto putSmallFile = 16;
   optional   GetSmallFileRequestProto getSmallFile = 17;
   optional   CloseContainerRequestProto closeContainer = 18;
+
   required   string datanodeUuid = 19;
 }
 
@@ -237,14 +238,6 @@ message ContainerData {
   optional string containerDBType = 11;
 }
 
-// This is used for create Container Request.
-message CreateContainerData {
-  required int64 containerId = 1;
-  repeated KeyValue metadata = 2;
-  optional ContainerType containerType = 3 [default = KeyValueContainer];
-}
-
-
 enum ContainerType {
   KeyValueContainer = 1;
 }
@@ -252,7 +245,9 @@ enum ContainerType {
 
 // Container Messages.
 message  CreateContainerRequestProto {
-  required ContainerData containerData = 1;
+  required int64 containerID = 1;
+  repeated KeyValue metadata = 2;
+  optional ContainerType containerType = 3 [default = KeyValueContainer];
 }
 
 message  CreateContainerResponseProto {
@@ -267,8 +262,9 @@ message  ReadContainerResponseProto {
 }
 
 message  UpdateContainerRequestProto {
-  required ContainerData containerData = 1;
-  optional bool forceUpdate = 2 [default = false];
+  required int64 containerID = 1;
+  repeated KeyValue metadata = 2;
+  optional bool forceUpdate = 3 [default = false];
 }
 
 message  UpdateContainerResponseProto {
@@ -316,7 +312,7 @@ message  PutKeyResponseProto {
 }
 
 message  GetKeyRequestProto  {
-  required KeyData keyData = 1;
+  required DatanodeBlockID blockID = 1;
 }
 
 message  GetKeyResponseProto  {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/common/src/main/resources/ozone-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 7a91610..a3e4776 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -63,6 +63,18 @@
     </description>
   </property>
   <property>
+    <name>hdds.datanode.dir</name>
+    <value/>
+    <tag>OZONE, CONTAINER, STORAGE, MANAGEMENT</tag>
+    <description>Determines where on the local filesystem HDDS data will be
+      stored. Defaults to dfs.datanode.data.dir if not specified.
+      The directories should be tagged with corresponding storage types
+      ([SSD]/[DISK]/[ARCHIVE]/[RAM_DISK]) for storage policies. The default
+      storage type will be DISK if the directory does not have a storage type
+      tagged explicitly.
+    </description>
+  </property>
+  <property>
     <name>dfs.container.ratis.enabled</name>
     <value>false</value>
     <tag>OZONE, MANAGEMENT, PIPELINE, RATIS</tag>
@@ -1086,4 +1098,25 @@
     </description>
   </property>
 
+  <property>
+    <name>hdds.write.lock.reporting.threshold.ms</name>
+    <value>5000</value>
+    <tag>OZONE, DATANODE, MANAGEMENT</tag>
+    <description>
+      When a write lock is held for a long time, this will be logged as the
+      lock is released. This sets how long the lock must be held for logging
+      to occur.
+    </description>
+  </property>
+
+  <property>
+    <name>hdds.lock.suppress.warning.interval.ms</name>
+    <value>10000</value>
+    <tag>OZONE, DATANODE, MANAGEMENT</tag>
+    <description>
+      Instrumentation reporting long critical sections will suppress
+      consecutive warnings within this interval.
+    </description>
+  </property>
+
 </configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java
index eba8594..e0bf213 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java
@@ -313,7 +313,7 @@ public final class ChunkUtils {
    */
   public static ContainerProtos.ContainerCommandResponseProto
       getChunkResponse(ContainerProtos.ContainerCommandRequestProto msg) {
-    return ContainerUtils.getContainerResponse(msg);
+    return ContainerUtils.getSuccessResponse(msg);
   }
 
   /**
@@ -336,8 +336,7 @@ public final class ChunkUtils {
     response.setBlockID(msg.getReadChunk().getBlockID());
 
     ContainerProtos.ContainerCommandResponseProto.Builder builder =
-        ContainerUtils.getContainerResponse(msg, ContainerProtos.Result
-            .SUCCESS, "");
+        ContainerUtils.getSuccessResponseBuilder(msg);
     builder.setReadChunk(response);
     return builder.build();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
index 9b52316..b975217 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
@@ -22,13 +22,20 @@ import com.google.common.base.Preconditions;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.hdds.scm.container.common.helpers
-    .StorageContainerException;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandResponseProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .StorageContainerException;
 import org.apache.hadoop.ozone.OzoneConsts;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
 import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.utils.MetadataStore;
 import org.apache.hadoop.utils.MetadataStoreBuilder;
 import org.slf4j.Logger;
@@ -42,12 +49,14 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 
 import static org.apache.commons.io.FilenameUtils.removeExtension;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result
-    .INVALID_ARGUMENT;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result
-    .UNABLE_TO_FIND_DATA_DIR;
-import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
-
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.CLOSED_CONTAINER_IO;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.INVALID_CONTAINER_STATE;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.SUCCESS;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.UNABLE_TO_FIND_DATA_DIR;
 
 /**
  * A set of helper functions to create proper responses.
@@ -59,28 +68,61 @@ public final class ContainerUtils {
   }
 
   /**
-   * Returns a CreateContainer Response. This call is used by create and delete
-   * containers which have null success responses.
-   *
-   * @param msg Request
-   * @return Response.
+   * Returns a Container Command Response Builder with the specified result
+   * and message.
+   * @param request requestProto message.
+   * @param result result of the command.
+   * @param message response message.
+   * @return ContainerCommand Response Builder.
    */
-  public static ContainerProtos.ContainerCommandResponseProto
-      getContainerResponse(ContainerProtos.ContainerCommandRequestProto msg) {
-    ContainerProtos.ContainerCommandResponseProto.Builder builder =
-        getContainerResponse(msg, ContainerProtos.Result.SUCCESS, "");
+  public static ContainerCommandResponseProto.Builder
+  getContainerCommandResponse(
+      ContainerCommandRequestProto request, Result result, String message) {
+    return
+        ContainerCommandResponseProto.newBuilder()
+            .setCmdType(request.getCmdType())
+            .setTraceID(request.getTraceID())
+            .setResult(result)
+            .setMessage(message);
+  }
+
+  /**
+   * Returns a Container Command Response Builder. This call is used to build
+   * success responses. Calling function can add other fields to the response
+   * as required.
+   * @param request requestProto message.
+   * @return ContainerCommand Response Builder with result as SUCCESS.
+   */
+  public static ContainerCommandResponseProto.Builder getSuccessResponseBuilder(
+      ContainerCommandRequestProto request) {
+    return
+        ContainerCommandResponseProto.newBuilder()
+            .setCmdType(request.getCmdType())
+            .setTraceID(request.getTraceID())
+            .setResult(Result.SUCCESS);
+  }
+
+  /**
+   * Returns a Container Command Response. This call is used for creating null
+   * success responses.
+   * @param request requestProto message.
+   * @return ContainerCommand Response with result as SUCCESS.
+   */
+  public static ContainerCommandResponseProto getSuccessResponse(
+      ContainerCommandRequestProto request) {
+    ContainerCommandResponseProto.Builder builder =
+        getContainerCommandResponse(request, Result.SUCCESS, "");
     return builder.build();
   }
 
   /**
    * Returns a ReadContainer Response.
-   *
-   * @param msg Request
-   * @param containerData - data
-   * @return Response.
+   * @param msg requestProto message.
+   * @param containerData container data to be returned.
+   * @return ReadContainer Response
    */
   public static ContainerProtos.ContainerCommandResponseProto
-      getReadContainerResponse(ContainerProtos.ContainerCommandRequestProto msg,
+    getReadContainerResponse(ContainerProtos.ContainerCommandRequestProto msg,
       ContainerData containerData) {
     Preconditions.checkNotNull(containerData);
 
@@ -89,7 +131,7 @@ public final class ContainerUtils {
     response.setContainerData(containerData.getProtoBufMessage());
 
     ContainerProtos.ContainerCommandResponseProto.Builder builder =
-        getContainerResponse(msg, ContainerProtos.Result.SUCCESS, "");
+        getSuccessResponseBuilder(msg);
     builder.setReadContainer(response);
     return builder.build();
   }
@@ -98,37 +140,25 @@ public final class ContainerUtils {
    * We found a command type but no associated payload for the command. Hence
    * return malformed Command as response.
    *
-   * @param msg - Protobuf message.
-   * @param result - result
-   * @param message - Error message.
+   * @param request - Protobuf message.
    * @return ContainerCommandResponseProto - MALFORMED_REQUEST.
    */
-  public static ContainerProtos.ContainerCommandResponseProto.Builder
-      getContainerResponse(ContainerProtos.ContainerCommandRequestProto msg,
-      ContainerProtos.Result result, String message) {
-    return
-        ContainerProtos.ContainerCommandResponseProto.newBuilder()
-            .setCmdType(msg.getCmdType())
-            .setTraceID(msg.getTraceID())
-            .setResult(result)
-            .setMessage(message);
+  public static ContainerCommandResponseProto malformedRequest(
+      ContainerCommandRequestProto request) {
+    return getContainerCommandResponse(request, Result.MALFORMED_REQUEST,
+        "Cmd type does not match the payload.").build();
   }
 
   /**
-   * Logs the error and returns a response to the caller.
+   * We found a command type that is not supported yet.
    *
-   * @param log - Logger
-   * @param ex - Exception
-   * @param msg - Request Object
-   * @return Response
+   * @param request - Protobuf message.
+   * @return ContainerCommandResponseProto - UNSUPPORTED_REQUEST.
    */
-  public static ContainerProtos.ContainerCommandResponseProto logAndReturnError(
-      Logger log, StorageContainerException ex,
-      ContainerProtos.ContainerCommandRequestProto msg) {
-    log.info("Operation: {} : Trace ID: {} : Message: {} : Result: {}",
-        msg.getCmdType().name(), msg.getTraceID(),
-        ex.getMessage(), ex.getResult().getValueDescriptor().getName());
-    return getContainerResponse(msg, ex.getResult(), ex.getMessage()).build();
+  public static ContainerCommandResponseProto unsupportedRequest(
+      ContainerCommandRequestProto request) {
+    return getContainerCommandResponse(request, Result.UNSUPPORTED_REQUEST,
+        "Server does not support this command yet.").build();
   }
 
   /**
@@ -136,40 +166,17 @@ public final class ContainerUtils {
    *
    * @param log - Logger
    * @param ex - Exception
-   * @param msg - Request Object
+   * @param request - Request Object
    * @return Response
    */
-  public static ContainerProtos.ContainerCommandResponseProto logAndReturnError(
-      Logger log, RuntimeException ex,
-      ContainerProtos.ContainerCommandRequestProto msg) {
-    log.info("Operation: {} : Trace ID: {} : Message: {} ",
-        msg.getCmdType().name(), msg.getTraceID(), ex.getMessage());
-    return getContainerResponse(msg, INVALID_ARGUMENT, ex.getMessage()).build();
-  }
-
-  /**
-   * We found a command type but no associated payload for the command. Hence
-   * return malformed Command as response.
-   *
-   * @param msg - Protobuf message.
-   * @return ContainerCommandResponseProto - MALFORMED_REQUEST.
-   */
-  public static ContainerProtos.ContainerCommandResponseProto
-      malformedRequest(ContainerProtos.ContainerCommandRequestProto msg) {
-    return getContainerResponse(msg, ContainerProtos.Result.MALFORMED_REQUEST,
-        "Cmd type does not match the payload.").build();
-  }
-
-  /**
-   * We found a command type that is not supported yet.
-   *
-   * @param msg - Protobuf message.
-   * @return ContainerCommandResponseProto - MALFORMED_REQUEST.
-   */
-  public static ContainerProtos.ContainerCommandResponseProto
-      unsupportedRequest(ContainerProtos.ContainerCommandRequestProto msg) {
-    return getContainerResponse(msg, ContainerProtos.Result.UNSUPPORTED_REQUEST,
-        "Server does not support this command yet.").build();
+  public static ContainerCommandResponseProto logAndReturnError(
+      Logger log, StorageContainerException ex,
+      ContainerCommandRequestProto request) {
+    log.info("Operation: {} : Trace ID: {} : Message: {} : Result: {}",
+        request.getCmdType().name(), request.getTraceID(),
+        ex.getMessage(), ex.getResult().getValueDescriptor().getName());
+    return getContainerCommandResponse(request, ex.getResult(), ex.getMessage())
+        .build();
   }
 
   /**
@@ -191,7 +198,7 @@ public final class ContainerUtils {
   }
 
   /**
-   * Verifies that this in indeed a new container.
+   * Verifies that this is indeed a new container.
    *
    * @param containerFile - Container File to verify
    * @throws IOException
@@ -343,7 +350,7 @@ public final class ContainerUtils {
     if(!forceDelete && !db.isEmpty()) {
       throw new StorageContainerException(
           "Container cannot be deleted because it is not empty.",
-          ContainerProtos.Result.ERROR_CONTAINER_NOT_EMPTY);
+          Result.ERROR_CONTAINER_NOT_EMPTY);
     }
     // Close the DB connection and remove the DB handler from cache
     KeyUtils.removeDB(containerData, conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/FileUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/FileUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/FileUtils.java
deleted file mode 100644
index a2875be..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/FileUtils.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.hadoop.ozone.container.common.helpers;
-
-import com.google.common.base.Preconditions;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-
-/**
- * File Utils are helper routines used by putSmallFile and getSmallFile
- * RPCs.
- */
-public final class FileUtils {
-  /**
-   * Never Constructed.
-   */
-  private FileUtils() {
-  }
-
-  /**
-   * Gets a response for the putSmallFile RPC.
-   * @param msg - ContainerCommandRequestProto
-   * @return - ContainerCommandResponseProto
-   */
-  public static ContainerProtos.ContainerCommandResponseProto
-      getPutFileResponse(ContainerProtos.ContainerCommandRequestProto msg) {
-    ContainerProtos.PutSmallFileResponseProto.Builder getResponse =
-        ContainerProtos.PutSmallFileResponseProto.newBuilder();
-    ContainerProtos.ContainerCommandResponseProto.Builder builder =
-        ContainerUtils.getContainerResponse(msg, ContainerProtos.Result
-            .SUCCESS, "");
-    builder.setCmdType(ContainerProtos.Type.PutSmallFile);
-    builder.setPutSmallFile(getResponse);
-    return  builder.build();
-  }
-
-  /**
-   * Gets a response to the read small file call.
-   * @param msg - Msg
-   * @param data  - Data
-   * @param info  - Info
-   * @return    Response.
-   */
-  public static ContainerProtos.ContainerCommandResponseProto
-      getGetSmallFileResponse(ContainerProtos.ContainerCommandRequestProto msg,
-      byte[] data, ChunkInfo info) {
-    Preconditions.checkNotNull(msg);
-
-    ContainerProtos.ReadChunkResponseProto.Builder readChunkresponse =
-        ContainerProtos.ReadChunkResponseProto.newBuilder();
-    readChunkresponse.setChunkData(info.getProtoBufMessage());
-    readChunkresponse.setData(ByteString.copyFrom(data));
-    readChunkresponse.setBlockID(msg.getGetSmallFile().getKey().
-        getKeyData().getBlockID());
-
-    ContainerProtos.GetSmallFileResponseProto.Builder getSmallFile =
-        ContainerProtos.GetSmallFileResponseProto.newBuilder();
-    getSmallFile.setData(readChunkresponse.build());
-    ContainerProtos.ContainerCommandResponseProto.Builder builder =
-        ContainerUtils.getContainerResponse(msg, ContainerProtos.Result
-            .SUCCESS, "");
-    builder.setCmdType(ContainerProtos.Type.GetSmallFile);
-    builder.setGetSmallFile(getSmallFile);
-    return builder.build();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java
index d52bc18..a710864 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java
@@ -103,7 +103,7 @@ public final class KeyUtils {
    */
   public static ContainerProtos.ContainerCommandResponseProto
       getKeyResponse(ContainerProtos.ContainerCommandRequestProto msg) {
-    return ContainerUtils.getContainerResponse(msg);
+    return ContainerUtils.getSuccessResponse(msg);
   }
 
 
@@ -114,8 +114,7 @@ public final class KeyUtils {
         .GetKeyResponseProto.newBuilder();
     getKey.setKeyData(data.getProtoBufMessage());
     ContainerProtos.ContainerCommandResponseProto.Builder builder =
-        ContainerUtils.getContainerResponse(msg, ContainerProtos.Result
-            .SUCCESS, "");
+        ContainerUtils.getSuccessResponseBuilder(msg);
     builder.setGetKey(getKey);
     return  builder.build();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
index 79f038f..18a7839 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.container.common.impl;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerInfo;
@@ -68,8 +69,10 @@ public class ContainerSet {
           containerId);
       return true;
     } else {
-      LOG.debug("Container already exists with container Id {}", containerId);
-      return false;
+      LOG.warn("Container already exists with container Id {}", containerId);
+      throw new StorageContainerException("Container already exists with " +
+          "container Id " + containerId,
+          ContainerProtos.Result.CONTAINER_EXISTS);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
index 3b478cd..b5fb08d 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.ozone.container.common.helpers.ChunkUtils;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
-import org.apache.hadoop.ozone.container.common.helpers.FileUtils;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.SmallFileUtils;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
 import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
@@ -138,8 +138,6 @@ public class Dispatcher implements ContainerDispatcher {
     } catch (StorageContainerException e) {
       // This useful since the trace ID will allow us to correlate failures.
       return ContainerUtils.logAndReturnError(LOG, e, msg);
-    } catch (IllegalStateException | NullPointerException e) {
-      return ContainerUtils.logAndReturnError(LOG, e, msg);
     }
   }
 
@@ -186,13 +184,13 @@ public class Dispatcher implements ContainerDispatcher {
     } catch (IOException ex) {
       LOG.warn("Container operation failed. " +
               "Container: {} Operation: {}  trace ID: {} Error: {}",
-          msg.getCreateContainer().getContainerData().getContainerID(),
+          msg.getCreateContainer().getContainerID(),
           msg.getCmdType().name(),
           msg.getTraceID(),
           ex.toString(), ex);
 
       // TODO : Replace with finer error codes.
-      return ContainerUtils.getContainerResponse(msg,
+      return ContainerUtils.getContainerCommandResponse(msg,
           ContainerProtos.Result.CONTAINER_INTERNAL_ERROR,
           ex.toString()).build();
     }
@@ -230,13 +228,13 @@ public class Dispatcher implements ContainerDispatcher {
     } catch (IOException ex) {
       LOG.warn("Container operation failed. " +
               "Container: {} Operation: {}  trace ID: {} Error: {}",
-          msg.getCreateContainer().getContainerData().getContainerID(),
+          msg.getCreateContainer().getContainerID(),
           msg.getCmdType().name(),
           msg.getTraceID(),
           ex.toString(), ex);
 
       // TODO : Replace with finer error codes.
-      return ContainerUtils.getContainerResponse(msg,
+      return ContainerUtils.getContainerCommandResponse(msg,
           ContainerProtos.Result.CONTAINER_INTERNAL_ERROR,
           ex.toString()).build();
     }
@@ -273,13 +271,13 @@ public class Dispatcher implements ContainerDispatcher {
     } catch (IOException ex) {
       LOG.warn("Container operation failed. " +
               "Container: {} Operation: {}  trace ID: {} Error: {}",
-          msg.getCreateContainer().getContainerData().getContainerID(),
+          msg.getCreateContainer().getContainerID(),
           msg.getCmdType().name(),
           msg.getTraceID(),
           ex.toString(), ex);
 
       // TODO : Replace with finer error codes.
-      return ContainerUtils.getContainerResponse(msg,
+      return ContainerUtils.getContainerCommandResponse(msg,
           ContainerProtos.Result.CONTAINER_INTERNAL_ERROR,
           ex.toString()).build();
     }
@@ -318,15 +316,14 @@ public class Dispatcher implements ContainerDispatcher {
           msg.getTraceID());
       return ContainerUtils.malformedRequest(msg);
     }
-    long containerID = msg.getUpdateContainer()
-        .getContainerData().getContainerID();
+    long containerID = msg.getUpdateContainer().getContainerID();
 
-    ContainerData data = ContainerData.getFromProtBuf(
-        msg.getUpdateContainer().getContainerData(), conf);
+    ContainerData data = new ContainerData(msg.getUpdateContainer()
+        .getContainerID(), conf);
     boolean forceUpdate = msg.getUpdateContainer().getForceUpdate();
     this.containerManager.updateContainer(containerID,
         data, forceUpdate);
-    return ContainerUtils.getContainerResponse(msg);
+    return ContainerUtils.getSuccessResponse(msg);
   }
 
   /**
@@ -371,7 +368,7 @@ public class Dispatcher implements ContainerDispatcher {
     long containerID = msg.getDeleteContainer().getContainerID();
     boolean forceDelete = msg.getDeleteContainer().getForceDelete();
     this.containerManager.deleteContainer(containerID, forceDelete);
-    return ContainerUtils.getContainerResponse(msg);
+    return ContainerUtils.getSuccessResponse(msg);
   }
 
   /**
@@ -388,12 +385,11 @@ public class Dispatcher implements ContainerDispatcher {
           msg.getTraceID());
       return ContainerUtils.malformedRequest(msg);
     }
-    ContainerData cData = ContainerData.getFromProtBuf(
-        msg.getCreateContainer().getContainerData(), conf);
-    Preconditions.checkNotNull(cData, "Container data is null");
+    ContainerData cData = new ContainerData(
+        msg.getCreateContainer().getContainerID(), conf);
 
     this.containerManager.createContainer(cData);
-    return ContainerUtils.getContainerResponse(msg);
+    return ContainerUtils.getSuccessResponse(msg);
   }
 
   /**
@@ -417,7 +413,7 @@ public class Dispatcher implements ContainerDispatcher {
             "container.", CLOSED_CONTAINER_IO);
       }
       this.containerManager.closeContainer(containerID);
-      return ContainerUtils.getContainerResponse(msg);
+      return ContainerUtils.getSuccessResponse(msg);
     } catch (NoSuchAlgorithmException e) {
       throw new StorageContainerException("No such Algorithm", e,
           NO_SUCH_ALGORITHM);
@@ -561,7 +557,8 @@ public class Dispatcher implements ContainerDispatcher {
           msg.getTraceID());
       return ContainerUtils.malformedRequest(msg);
     }
-    KeyData keyData = KeyData.getFromProtoBuf(msg.getGetKey().getKeyData());
+    KeyData keyData = new KeyData(
+        BlockID.getFromProtobuf(msg.getGetKey().getBlockID()));
     Preconditions.checkNotNull(keyData);
     KeyData responseData =
         this.containerManager.getKeyManager().getKey(keyData);
@@ -634,7 +631,7 @@ public class Dispatcher implements ContainerDispatcher {
       chunks.add(chunkInfo.getProtoBufMessage());
       keyData.setChunks(chunks);
       this.containerManager.getKeyManager().putKey(keyData);
-      return FileUtils.getPutFileResponse(msg);
+      return SmallFileUtils.getPutFileResponseSuccess(msg);
     } catch (StorageContainerException e) {
       return ContainerUtils.logAndReturnError(LOG, e, msg);
     } catch (IOException e) {
@@ -661,8 +658,8 @@ public class Dispatcher implements ContainerDispatcher {
     }
     try {
       long bytes = 0;
-      KeyData keyData = KeyData.getFromProtoBuf(msg.getGetSmallFile()
-          .getKey().getKeyData());
+      KeyData keyData = new KeyData(BlockID.getFromProtobuf(
+          msg.getGetSmallFile().getKey().getBlockID()));
       KeyData data = this.containerManager.getKeyManager().getKey(keyData);
       ContainerProtos.ChunkInfo c = null;
       for (ContainerProtos.ChunkInfo chunk : data.getChunks()) {
@@ -675,8 +672,8 @@ public class Dispatcher implements ContainerDispatcher {
         c = chunk;
       }
       metrics.incContainerBytesStats(Type.GetSmallFile, bytes);
-      return FileUtils.getGetSmallFileResponse(msg, dataBuf.toByteArray(),
-          ChunkInfo.getFromProtoBuf(c));
+      return SmallFileUtils.getGetSmallFileResponseSuccess(
+          msg, dataBuf.toByteArray(), ChunkInfo.getFromProtoBuf(c));
     } catch (StorageContainerException e) {
       return ContainerUtils.logAndReturnError(LOG, e, msg);
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
new file mode 100644
index 0000000..e73b761
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.interfaces.Handler;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandResponseProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerType;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+/**
+ * Ozone Container dispatcher takes a call from the netty server and routes it
+ * to the right handler function.
+ */
+public class HddsDispatcher implements ContainerDispatcher {
+
+  static final Logger LOG = LoggerFactory.getLogger(HddsDispatcher.class);
+
+  private final Map<ContainerType, Handler> handlers;
+  private final Configuration conf;
+  private final ContainerSet containerSet;
+  private final VolumeSet volumeSet;
+  private final String scmID;
+
+  /**
+   * Constructs an OzoneContainer that receives calls from
+   * XceiverServerHandler.
+   */
+  public HddsDispatcher(Configuration config, ContainerSet contSet,
+      VolumeSet volumes, String scmId) {
+    // TODO: Pass ContainerSet, VolumeSet and scmID, intialize metrics
+    this.conf = config;
+    this.containerSet = contSet;
+    this.volumeSet = volumes;
+    this.scmID = scmId;
+    this.handlers = Maps.newHashMap();
+    for (ContainerType containerType : ContainerType.values()) {
+      handlers.put(containerType,
+          Handler.getHandlerForContainerType(
+              containerType, conf, containerSet, volumeSet, scmID));
+    }
+  }
+
+  @Override
+  public void init() {
+  }
+
+  @Override
+  public void shutdown() {
+  }
+
+  @Override
+  public ContainerCommandResponseProto dispatch(
+      ContainerCommandRequestProto msg) {
+    LOG.trace("Command {}, trace ID: {} ", msg.getCmdType().toString(),
+        msg.getTraceID());
+    Preconditions.checkNotNull(msg);
+
+    Container container = null;
+    ContainerType containerType = null;
+    try {
+      long containerID = getContainerID(msg);
+
+      if (msg.getCmdType() != ContainerProtos.Type.CreateContainer) {
+        container = getContainer(containerID);
+        containerType = getContainerType(container);
+      } else {
+        containerType = msg.getCreateContainer().getContainerType();
+      }
+    } catch (StorageContainerException ex) {
+      return ContainerUtils.logAndReturnError(LOG, ex, msg);
+    }
+
+    Handler handler = getHandlerForContainerType(containerType);
+    if (handler == null) {
+      StorageContainerException ex = new StorageContainerException("Invalid " +
+          "ContainerType " + containerType,
+          ContainerProtos.Result.CONTAINER_INTERNAL_ERROR);
+      return ContainerUtils.logAndReturnError(LOG, ex, msg);
+    }
+    return handler.handle(msg, container);
+  }
+
+  @VisibleForTesting
+  public Handler getHandlerForContainerType(ContainerType type) {
+    return handlers.get(type);
+  }
+
+  private long getContainerID(ContainerCommandRequestProto request)
+      throws StorageContainerException {
+    ContainerProtos.Type cmdType = request.getCmdType();
+
+    switch(cmdType) {
+    case CreateContainer:
+      return request.getCreateContainer().getContainerID();
+    case ReadContainer:
+      return request.getReadContainer().getContainerID();
+    case UpdateContainer:
+      return request.getUpdateContainer().getContainerID();
+    case DeleteContainer:
+      return request.getDeleteContainer().getContainerID();
+    case ListContainer:
+      return request.getListContainer().getStartContainerID();
+    case CloseContainer:
+      return request.getCloseContainer().getContainerID();
+    case PutKey:
+      return request.getPutKey().getKeyData().getBlockID().getContainerID();
+    case GetKey:
+      return request.getGetKey().getBlockID().getContainerID();
+    case DeleteKey:
+      return request.getDeleteKey().getBlockID().getContainerID();
+    case ListKey:
+      return request.getListKey().getContainerID();
+    case ReadChunk:
+      return request.getReadChunk().getBlockID().getContainerID();
+    case DeleteChunk:
+      return request.getDeleteChunk().getBlockID().getContainerID();
+    case WriteChunk:
+      return request.getWriteChunk().getBlockID().getContainerID();
+    case ListChunk:
+      return request.getListChunk().getBlockID().getContainerID();
+    case PutSmallFile:
+      return request.getPutSmallFile().getKey().getKeyData().getBlockID()
+          .getContainerID();
+    case GetSmallFile:
+      return request.getGetSmallFile().getKey().getBlockID().getContainerID();
+    }
+
+    throw new StorageContainerException(
+        ContainerProtos.Result.UNSUPPORTED_REQUEST);
+  }
+
+  @VisibleForTesting
+  public Container getContainer(long containerID)
+      throws StorageContainerException {
+    Container container = containerSet.getContainer(containerID);
+    if (container == null) {
+      throw new StorageContainerException(
+          "ContainerID " + containerID + " does not exist",
+          ContainerProtos.Result.CONTAINER_NOT_FOUND);
+    }
+    return container;
+  }
+
+  private ContainerType getContainerType(Container container) {
+    return container.getContainerType();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
deleted file mode 100644
index b74bab2..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
+++ /dev/null
@@ -1,191 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.impl;
-
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-
-
-import java.io.File;
-import java.io.IOException;
-
-/**
- * This class represents the KeyValueContainer metadata, which is the
- * in-memory representation of container metadata and is represented on disk
- * by the .container file.
- */
-public class KeyValueContainerData extends ContainerData {
-
-  // Path to Container metadata Level DB/RocksDB Store and .container file.
-  private String metadataPath;
-
-  // Path to Physical file system where chunks are stored.
-  private String chunksPath;
-
-  //Type of DB used to store key to chunks mapping
-  private String containerDBType;
-
-  //Number of pending deletion blocks in container.
-  private int numPendingDeletionBlocks;
-
-  private File dbFile = null;
-
-  /**
-   * Constructs KeyValueContainerData object.
-   * @param type - containerType
-   * @param id - ContainerId
-   */
-  public KeyValueContainerData(ContainerProtos.ContainerType type, long id) {
-    super(type, id);
-    this.numPendingDeletionBlocks = 0;
-  }
-
-  /**
-   * Constructs KeyValueContainerData object.
-   * @param type - containerType
-   * @param id - ContainerId
-   * @param layOutVersion
-   */
-  public KeyValueContainerData(ContainerProtos.ContainerType type, long id,
-                               int layOutVersion) {
-    super(type, id, layOutVersion);
-    this.numPendingDeletionBlocks = 0;
-  }
-
-
-  /**
-   * Sets Container dbFile. This should be called only during creation of
-   * KeyValue container.
-   * @param containerDbFile
-   */
-  public void setDbFile(File containerDbFile) {
-    dbFile = containerDbFile;
-  }
-
-  /**
-   * Returns container DB file.
-   * @return dbFile
-   */
-  public File getDbFile() {
-    return dbFile;
-  }
-  /**
-   * Returns container metadata path.
-   *
-   * @return - path
-   */
-  public String getMetadataPath() {
-    return metadataPath;
-  }
-
-  /**
-   * Sets container metadata path.
-   *
-   * @param path - String.
-   */
-  public void setMetadataPath(String path) {
-    this.metadataPath = path;
-  }
-
-  /**
-   * Get chunks path.
-   * @return - Physical path where container file and checksum is stored.
-   */
-  public String getChunksPath() {
-    return chunksPath;
-  }
-
-  /**
-   * Set chunks Path.
-   * @param chunkPath - File path.
-   */
-  public void setChunksPath(String chunkPath) {
-    this.chunksPath = chunkPath;
-  }
-
-  /**
-   * Returns the DBType used for the container.
-   * @return containerDBType
-   */
-  public String getContainerDBType() {
-    return containerDBType;
-  }
-
-  /**
-   * Sets the DBType used for the container.
-   * @param containerDBType
-   */
-  public void setContainerDBType(String containerDBType) {
-    this.containerDBType = containerDBType;
-  }
-
-  /**
-   * Returns the number of pending deletion blocks in container.
-   * @return numPendingDeletionBlocks
-   */
-  public int getNumPendingDeletionBlocks() {
-    return numPendingDeletionBlocks;
-  }
-
-
-  /**
-   * Increase the count of pending deletion blocks.
-   *
-   * @param numBlocks increment number
-   */
-  public void incrPendingDeletionBlocks(int numBlocks) {
-    this.numPendingDeletionBlocks += numBlocks;
-  }
-
-  /**
-   * Decrease the count of pending deletion blocks.
-   *
-   * @param numBlocks decrement number
-   */
-  public void decrPendingDeletionBlocks(int numBlocks) {
-    this.numPendingDeletionBlocks -= numBlocks;
-  }
-
-
-  /**
-   * Constructs a KeyValueContainerData object from ProtoBuf classes.
-   *
-   * @param protoData - ProtoBuf Message
-   * @throws IOException
-   */
-  public static KeyValueContainerData getFromProtoBuf(
-      ContainerProtos.CreateContainerData protoData) throws IOException {
-
-    long containerID;
-    ContainerProtos.ContainerType containerType;
-
-    containerID = protoData.getContainerId();
-    containerType = protoData.getContainerType();
-
-    KeyValueContainerData keyValueContainerData = new KeyValueContainerData(
-        containerType, containerID);
-
-    for (int x = 0; x < protoData.getMetadataCount(); x++) {
-      keyValueContainerData.addMetadata(protoData.getMetadata(x).getKey(),
-          protoData.getMetadata(x).getValue());
-    }
-
-    return keyValueContainerData;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueYaml.java
deleted file mode 100644
index d22092c..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueYaml.java
+++ /dev/null
@@ -1,275 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.impl;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.yaml.snakeyaml.Yaml;
-
-
-import java.beans.IntrospectionException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.Writer;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.OutputStreamWriter;
-
-import java.io.File;
-
-
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.Map;
-
-import org.yaml.snakeyaml.constructor.AbstractConstruct;
-import org.yaml.snakeyaml.constructor.Constructor;
-import org.yaml.snakeyaml.introspector.BeanAccess;
-import org.yaml.snakeyaml.introspector.Property;
-import org.yaml.snakeyaml.introspector.PropertyUtils;
-import org.yaml.snakeyaml.nodes.MappingNode;
-import org.yaml.snakeyaml.nodes.Node;
-import org.yaml.snakeyaml.nodes.ScalarNode;
-import org.yaml.snakeyaml.nodes.Tag;
-import org.yaml.snakeyaml.representer.Representer;
-
-/**
- * Class for creating and reading .container files.
- */
-
-public final class KeyValueYaml {
-
-  private KeyValueYaml() {
-
-  }
-  /**
-   * Creates a .container file in yaml format.
-   *
-   * @param containerFile
-   * @param containerData
-   * @throws IOException
-   */
-  public static void createContainerFile(File containerFile, ContainerData
-      containerData) throws IOException {
-
-    Preconditions.checkNotNull(containerFile, "yamlFile cannot be null");
-    Preconditions.checkNotNull(containerData, "containerData cannot be null");
-
-    PropertyUtils propertyUtils = new PropertyUtils();
-    propertyUtils.setBeanAccess(BeanAccess.FIELD);
-    propertyUtils.setAllowReadOnlyProperties(true);
-
-    Representer representer = new KeyValueContainerDataRepresenter();
-    representer.setPropertyUtils(propertyUtils);
-    representer.addClassTag(org.apache.hadoop.ozone.container.common.impl
-        .KeyValueContainerData.class, new Tag("KeyValueContainerData"));
-
-    Constructor keyValueDataConstructor = new KeyValueDataConstructor();
-
-    Yaml yaml = new Yaml(keyValueDataConstructor, representer);
-
-    Writer writer = new OutputStreamWriter(new FileOutputStream(containerFile),
-        "UTF-8");
-    yaml.dump(containerData, writer);
-    writer.close();
-  }
-
-  /**
-   * Read the yaml file, and return containerData.
-   *
-   * @param containerFile
-   * @throws IOException
-   */
-  public static KeyValueContainerData readContainerFile(File containerFile)
-      throws IOException {
-    Preconditions.checkNotNull(containerFile, "containerFile cannot be null");
-
-    InputStream input = null;
-    KeyValueContainerData keyValueContainerData;
-    try {
-      PropertyUtils propertyUtils = new PropertyUtils();
-      propertyUtils.setBeanAccess(BeanAccess.FIELD);
-      propertyUtils.setAllowReadOnlyProperties(true);
-
-      Representer representer = new KeyValueContainerDataRepresenter();
-      representer.setPropertyUtils(propertyUtils);
-      representer.addClassTag(org.apache.hadoop.ozone.container.common.impl
-          .KeyValueContainerData.class, new Tag("KeyValueContainerData"));
-
-      Constructor keyValueDataConstructor = new KeyValueDataConstructor();
-
-      Yaml yaml = new Yaml(keyValueDataConstructor, representer);
-      yaml.setBeanAccess(BeanAccess.FIELD);
-
-      input = new FileInputStream(containerFile);
-      keyValueContainerData = (KeyValueContainerData)
-          yaml.load(input);
-    } finally {
-      if (input!= null) {
-        input.close();
-      }
-    }
-    return keyValueContainerData;
-  }
-
-  /**
-   * Representer class to define which fields need to be stored in yaml file.
-   */
-  private static class KeyValueContainerDataRepresenter extends Representer {
-    @Override
-    protected Set<Property> getProperties(Class<? extends Object> type)
-        throws IntrospectionException {
-      Set<Property> set = super.getProperties(type);
-      Set<Property> filtered = new TreeSet<Property>();
-      if (type.equals(KeyValueContainerData.class)) {
-        // filter properties
-        for (Property prop : set) {
-          String name = prop.getName();
-          // When a new field needs to be added, it needs to be added here.
-          if (name.equals("containerType") || name.equals("containerId") ||
-              name.equals("layOutVersion") || name.equals("state") ||
-              name.equals("metadata") || name.equals("metadataPath") ||
-              name.equals("chunksPath") || name.equals(
-                  "containerDBType")) {
-            filtered.add(prop);
-          }
-        }
-      }
-      return filtered;
-    }
-  }
-
-  /**
-   * Constructor class for KeyValueData, which will be used by Yaml.
-   */
-  private static class KeyValueDataConstructor extends Constructor {
-    KeyValueDataConstructor() {
-      //Adding our own specific constructors for tags.
-      this.yamlConstructors.put(new Tag("KeyValueContainerData"),
-          new ConstructKeyValueContainerData());
-      this.yamlConstructors.put(Tag.INT, new ConstructLong());
-    }
-
-    private class ConstructKeyValueContainerData extends AbstractConstruct {
-      public Object construct(Node node) {
-        MappingNode mnode = (MappingNode) node;
-        Map<Object, Object> nodes = constructMapping(mnode);
-        String type = (String) nodes.get("containerType");
-
-        ContainerProtos.ContainerType containerType = ContainerProtos
-            .ContainerType.KeyValueContainer;
-        if (type.equals("KeyValueContainer")) {
-          containerType = ContainerProtos.ContainerType.KeyValueContainer;
-        }
-
-        //Needed this, as TAG.INT type is by default converted to Long.
-        long layOutVersion = (long) nodes.get("layOutVersion");
-        int lv = (int) layOutVersion;
-
-        //When a new field is added, it needs to be added here.
-        KeyValueContainerData kvData = new KeyValueContainerData(containerType,
-            (long) nodes.get("containerId"), lv);
-        kvData.setContainerDBType((String)nodes.get("containerDBType"));
-        kvData.setMetadataPath((String) nodes.get(
-            "metadataPath"));
-        kvData.setChunksPath((String) nodes.get("chunksPath"));
-        Map<String, String> meta = (Map) nodes.get("metadata");
-        meta.forEach((key, val) -> {
-          try {
-            kvData.addMetadata(key, val);
-          } catch (IOException e) {
-            throw new IllegalStateException("Unexpected " +
-                "Key Value Pair " + "(" + key + "," + val +")in the metadata " +
-                "for containerId " + (long) nodes.get("containerId"));
-          }
-        });
-        String state = (String) nodes.get("state");
-        switch (state) {
-        case "OPEN":
-          kvData.setState(ContainerProtos.ContainerLifeCycleState.OPEN);
-          break;
-        case "CLOSING":
-          kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSING);
-          break;
-        case "CLOSED":
-          kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED);
-          break;
-        default:
-          throw new IllegalStateException("Unexpected " +
-              "ContainerLifeCycleState " + state + " for the containerId " +
-              (long) nodes.get("containerId"));
-        }
-        return kvData;
-      }
-    }
-
-    //Below code is taken from snake yaml, as snakeyaml tries to fit the
-    // number if it fits in integer, otherwise returns long. So, slightly
-    // modified the code to return long in all cases.
-    private class ConstructLong extends AbstractConstruct {
-      public Object construct(Node node) {
-        String value = constructScalar((ScalarNode) node).toString()
-            .replaceAll("_", "");
-        int sign = +1;
-        char first = value.charAt(0);
-        if (first == '-') {
-          sign = -1;
-          value = value.substring(1);
-        } else if (first == '+') {
-          value = value.substring(1);
-        }
-        int base = 10;
-        if ("0".equals(value)) {
-          return Long.valueOf(0);
-        } else if (value.startsWith("0b")) {
-          value = value.substring(2);
-          base = 2;
-        } else if (value.startsWith("0x")) {
-          value = value.substring(2);
-          base = 16;
-        } else if (value.startsWith("0")) {
-          value = value.substring(1);
-          base = 8;
-        } else if (value.indexOf(':') != -1) {
-          String[] digits = value.split(":");
-          int bes = 1;
-          int val = 0;
-          for (int i = 0, j = digits.length; i < j; i++) {
-            val += (Long.parseLong(digits[(j - i) - 1]) * bes);
-            bes *= 60;
-          }
-          return createNumber(sign, String.valueOf(val), 10);
-        } else {
-          return createNumber(sign, value, 10);
-        }
-        return createNumber(sign, value, base);
-      }
-    }
-
-    private Number createNumber(int sign, String number, int radix) {
-      Number result;
-      if (sign < 0) {
-        number = "-" + number;
-      }
-      result = Long.valueOf(number, radix);
-      return result;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
index 3b7e332..a5559aa 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
@@ -19,6 +19,9 @@
 package org.apache.hadoop.ozone.container.common.interfaces;
 
 
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerLifeCycleState;
 import org.apache.hadoop.hdds.scm.container.common.helpers.
     StorageContainerException;
 
@@ -69,6 +72,14 @@ public interface Container extends RwLock {
   ContainerData getContainerData() throws StorageContainerException;
 
   /**
+   * Get the Container Lifecycle state.
+   *
+   * @return ContainerLifeCycleState - Container State.
+   * @throws StorageContainerException
+   */
+  ContainerLifeCycleState getContainerState();
+
+  /**
    * Closes a open container, if it is already closed or does not exist a
    * StorageContainerException is thrown.
    *
@@ -76,5 +87,9 @@ public interface Container extends RwLock {
    */
   void close() throws StorageContainerException;
 
+  /**
+   * Return the ContainerType for the container.
+   */
+  ContainerProtos.ContainerType getContainerType();
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
new file mode 100644
index 0000000..d08ad74
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.interfaces;
+
+import com.sun.jersey.spi.resource.Singleton;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandResponseProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerType;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
+
+import java.io.IOException;
+
+/**
+ * Dispatcher sends ContainerCommandRequests to Handler. Each Container Type
+ * should have an implementation for Handler.
+ */
+public class Handler {
+
+  protected final Configuration conf;
+  protected final ContainerSet containerSet;
+  protected final VolumeSet volumeSet;
+  protected final String scmID;
+
+  protected Handler(Configuration config, ContainerSet contSet,
+      VolumeSet volumeSet, String scmID) {
+    conf = config;
+    containerSet = contSet;
+    this.volumeSet = volumeSet;
+    this.scmID = scmID;
+  }
+
+  public static Handler getHandlerForContainerType(ContainerType containerType,
+      Configuration config, ContainerSet contSet, VolumeSet volumeSet,
+      String scmID) {
+    switch (containerType) {
+    case KeyValueContainer:
+      return KeyValueHandler.getInstance(config, contSet, volumeSet, scmID);
+    default:
+      throw new IllegalArgumentException("Handler for ContainerType: " +
+        containerType + "doesn't exist.");
+    }
+  }
+
+  public ContainerCommandResponseProto handle(
+      ContainerCommandRequestProto msg, Container container) {
+    return null;
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java
index 61aca79..9e052b0 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java
@@ -25,9 +25,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
 import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume.VolumeState;
@@ -101,10 +101,13 @@ public class VolumeSet {
         new InstrumentedLock(getClass().getName(), LOG,
             new ReentrantLock(true),
             conf.getTimeDuration(
-                DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
-                DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_DEFAULT,
+                OzoneConfigKeys.HDDS_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY,
+                OzoneConfigKeys.HDDS_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT,
                 TimeUnit.MILLISECONDS),
-            300));
+            conf.getTimeDuration(
+                OzoneConfigKeys.HDDS_LOCK_SUPPRESS_WARNING_INTERVAL_MS_KEY,
+                OzoneConfigKeys.HDDS_LOCK_SUPPRESS_WARNING_INTERVAL_MS_DEAFULT,
+                TimeUnit.MILLISECONDS)));
 
     initializeVolumeSet();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ChunkManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ChunkManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ChunkManagerImpl.java
deleted file mode 100644
index 6ee0fd3..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ChunkManagerImpl.java
+++ /dev/null
@@ -1,240 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.keyvalue;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
-import org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils;
-import org.apache.hadoop.ozone.container.common.impl.ChunkLayOutVersion;
-import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
-import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
-import org.apache.hadoop.ozone.container.common.interfaces.Container;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.file.Files;
-import java.nio.file.StandardCopyOption;
-import java.security.NoSuchAlgorithmException;
-import java.util.concurrent.ExecutionException;
-
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.CONTAINER_INTERNAL_ERROR;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.NO_SUCH_ALGORITHM;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNSUPPORTED_REQUEST;
-
-/**
- * This class is for performing chunk related operations.
- */
-public class ChunkManagerImpl implements ChunkManager {
-  static final Logger LOG = LoggerFactory.getLogger(ChunkManagerImpl.class);
-
-  /**
-   * writes a given chunk.
-   *
-   * @param container - Container for the chunk
-   * @param blockID - ID of the block
-   * @param info - ChunkInfo
-   * @param data - data of the chunk
-   * @param stage - Stage of the Chunk operation
-   * @throws StorageContainerException
-   */
-  public void writeChunk(Container container, BlockID blockID, ChunkInfo info,
-                         byte[] data, ContainerProtos.Stage stage)
-      throws StorageContainerException {
-
-    try {
-
-      KeyValueContainerData containerData = (KeyValueContainerData) container
-          .getContainerData();
-
-      File chunkFile = ChunkUtils.validateChunk(containerData, info);
-      File tmpChunkFile = getTmpChunkFile(chunkFile, info);
-
-      LOG.debug("writing chunk:{} chunk stage:{} chunk file:{} tmp chunk file",
-          info.getChunkName(), stage, chunkFile, tmpChunkFile);
-
-      switch (stage) {
-      case WRITE_DATA:
-        // Initially writes to temporary chunk file.
-        ChunkUtils.writeData(tmpChunkFile, info, data);
-        break;
-      case COMMIT_DATA:
-        // commit the data, means move chunk data from temporary chunk file
-        // to actual chunk file.
-        long sizeDiff = tmpChunkFile.length() - chunkFile.length();
-        commitChunk(tmpChunkFile, chunkFile);
-        containerData.incrBytesUsed(sizeDiff);
-        containerData.incrWriteCount();
-        containerData.incrWriteBytes(sizeDiff);
-        break;
-      case COMBINED:
-        // directly write to the chunk file
-        ChunkUtils.writeData(chunkFile, info, data);
-        containerData.incrBytesUsed(info.getLen());
-        containerData.incrWriteCount();
-        containerData.incrWriteBytes(info.getLen());
-        break;
-      default:
-        throw new IOException("Can not identify write operation.");
-      }
-    } catch (StorageContainerException ex) {
-      throw ex;
-    } catch (NoSuchAlgorithmException ex) {
-      LOG.error("write data failed. error: {}", ex);
-      throw new StorageContainerException("Internal error: ", ex,
-          NO_SUCH_ALGORITHM);
-    } catch (ExecutionException  | IOException ex) {
-      LOG.error("write data failed. error: {}", ex);
-      throw new StorageContainerException("Internal error: ", ex,
-          CONTAINER_INTERNAL_ERROR);
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      LOG.error("write data failed. error: {}", e);
-      throw new StorageContainerException("Internal error: ", e,
-          CONTAINER_INTERNAL_ERROR);
-    }
-  }
-
-  /**
-   * reads the data defined by a chunk.
-   *
-   * @param container - Container for the chunk
-   * @param blockID - ID of the block.
-   * @param info - ChunkInfo.
-   * @return byte array
-   * @throws StorageContainerException
-   * TODO: Right now we do not support partial reads and writes of chunks.
-   * TODO: Explore if we need to do that for ozone.
-   */
-  public byte[] readChunk(Container container, BlockID blockID, ChunkInfo info)
-      throws StorageContainerException {
-    try {
-      KeyValueContainerData containerData = (KeyValueContainerData) container
-          .getContainerData();
-      ByteBuffer data;
-
-      // Checking here, which layout version the container is, and reading
-      // the chunk file in that format.
-      // In version1, we verify checksum if it is available and return data
-      // of the chunk file.
-      if (containerData.getLayOutVersion() == ChunkLayOutVersion
-          .getLatestVersion().getVersion()) {
-        File chunkFile = ChunkUtils.getChunkFile(containerData, info);
-        data = ChunkUtils.readData(chunkFile, info);
-        containerData.incrReadCount();
-        containerData.incrReadBytes(chunkFile.length());
-        return data.array();
-      }
-    } catch(NoSuchAlgorithmException ex) {
-      LOG.error("read data failed. error: {}", ex);
-      throw new StorageContainerException("Internal error: ",
-          ex, NO_SUCH_ALGORITHM);
-    } catch (ExecutionException ex) {
-      LOG.error("read data failed. error: {}", ex);
-      throw new StorageContainerException("Internal error: ",
-          ex, CONTAINER_INTERNAL_ERROR);
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      LOG.error("read data failed. error: {}", e);
-      throw new StorageContainerException("Internal error: ",
-          e, CONTAINER_INTERNAL_ERROR);
-    }
-    return null;
-  }
-
-  /**
-   * Deletes a given chunk.
-   *
-   * @param container - Container for the chunk
-   * @param blockID - ID of the block
-   * @param info - Chunk Info
-   * @throws StorageContainerException
-   */
-  public void deleteChunk(Container container, BlockID blockID, ChunkInfo info)
-      throws StorageContainerException {
-    Preconditions.checkNotNull(blockID, "Block ID cannot be null.");
-    KeyValueContainerData containerData = (KeyValueContainerData) container
-        .getContainerData();
-    // Checking here, which layout version the container is, and performing
-    // deleting chunk operation.
-    // In version1, we have only chunk file.
-    if (containerData.getLayOutVersion() == ChunkLayOutVersion
-        .getLatestVersion().getVersion()) {
-      File chunkFile = ChunkUtils.getChunkFile(containerData, info);
-      if ((info.getOffset() == 0) && (info.getLen() == chunkFile.length())) {
-        FileUtil.fullyDelete(chunkFile);
-        containerData.decrBytesUsed(chunkFile.length());
-      } else {
-        LOG.error("Not Supported Operation. Trying to delete a " +
-            "chunk that is in shared file. chunk info : " + info.toString());
-        throw new StorageContainerException("Not Supported Operation. " +
-            "Trying to delete a chunk that is in shared file. chunk info : "
-            + info.toString(), UNSUPPORTED_REQUEST);
-      }
-    }
-  }
-
-  /**
-   * Shutdown the chunkManager.
-   *
-   * In the chunkManager we haven't acquired any resources, so nothing to do
-   * here.
-   */
-
-  public void shutdown() {
-    //TODO: need to revisit this during integration of container IO.
-  }
-
-  /**
-   * Returns the temporary chunkFile path.
-   * @param chunkFile
-   * @param info
-   * @return temporary chunkFile path
-   * @throws StorageContainerException
-   */
-  private File getTmpChunkFile(File chunkFile, ChunkInfo info)
-      throws StorageContainerException {
-    return new File(chunkFile.getParent(),
-        chunkFile.getName() +
-            OzoneConsts.CONTAINER_CHUNK_NAME_DELIMITER +
-            OzoneConsts.CONTAINER_TEMPORARY_CHUNK_PREFIX);
-  }
-
-  /**
-   * Commit the chunk by renaming the temporary chunk file to chunk file.
-   * @param tmpChunkFile
-   * @param chunkFile
-   * @throws IOException
-   */
-  private void commitChunk(File tmpChunkFile, File chunkFile) throws
-      IOException {
-    Files.move(tmpChunkFile.toPath(), chunkFile.toPath(),
-        StandardCopyOption.REPLACE_EXISTING);
-  }
-
-}


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


[21/37] hadoop git commit: Merge remote-tracking branch 'apache-commit/trunk' into HDDS-48

Posted by bh...@apache.org.
Merge remote-tracking branch 'apache-commit/trunk' into HDDS-48


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

Branch: refs/heads/trunk
Commit: c10452516804eed793e575dedcd1ee7758ec1f4c
Parents: 52d1d96 d3fa83a
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Thu Jun 28 14:28:23 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Thu Jun 28 14:28:23 2018 -0700

----------------------------------------------------------------------
 NOTICE.txt                                      |   8 +
 dev-support/bin/ozone-dist-layout-stitching     |   4 +-
 .../hadoop-client-minicluster/pom.xml           |   8 -
 hadoop-common-project/hadoop-common/pom.xml     |   5 -
 .../org/apache/hadoop/conf/Configuration.java   |   8 +-
 .../hadoop/conf/ReconfigurationServlet.java     |  42 +-
 .../apache/hadoop/crypto/key/KeyProvider.java   |   4 +-
 .../org/apache/hadoop/crypto/key/KeyShell.java  |   2 +-
 .../crypto/key/kms/KMSClientProvider.java       |  16 +-
 .../java/org/apache/hadoop/fs/BBPartHandle.java |  58 ++
 .../org/apache/hadoop/fs/BBUploadHandle.java    |  57 ++
 .../fs/CommonConfigurationKeysPublic.java       |   4 +-
 .../hadoop/fs/FSDataOutputStreamBuilder.java    |  22 +
 .../java/org/apache/hadoop/fs/FileContext.java  |  66 ++
 .../hadoop/fs/FileSystemMultipartUploader.java  | 132 ++++
 .../main/java/org/apache/hadoop/fs/FsShell.java |   3 +-
 .../hadoop/fs/LocalFileSystemPathHandle.java    | 100 +++
 .../org/apache/hadoop/fs/MultipartUploader.java |  90 +++
 .../hadoop/fs/MultipartUploaderFactory.java     |  65 ++
 .../main/java/org/apache/hadoop/fs/Options.java |   3 +
 .../java/org/apache/hadoop/fs/PartHandle.java   |  45 ++
 .../main/java/org/apache/hadoop/fs/Path.java    |   2 +-
 .../apache/hadoop/fs/RawLocalFileSystem.java    |  61 +-
 .../UnsupportedMultipartUploaderException.java  |  41 ++
 .../java/org/apache/hadoop/fs/UploadHandle.java |  47 ++
 .../java/org/apache/hadoop/fs/shell/Count.java  |   2 +-
 .../apache/hadoop/io/ElasticByteBufferPool.java |   2 +-
 .../apache/hadoop/io/FastByteComparisons.java   |  44 +-
 .../apache/hadoop/io/erasurecode/ECSchema.java  |   4 +-
 .../io/erasurecode/ErasureCodeNative.java       |   2 +-
 .../nativeio/SharedFileDescriptorFactory.java   |   2 +-
 .../org/apache/hadoop/ipc/CallerContext.java    |   4 +-
 .../apache/hadoop/ipc/DecayRpcScheduler.java    |   4 +-
 .../org/apache/hadoop/ipc/FairCallQueue.java    |   4 +-
 .../hadoop/metrics2/MetricsJsonBuilder.java     |   2 +-
 .../hadoop/metrics2/impl/MetricsConfig.java     |  15 +-
 .../hadoop/metrics2/lib/MethodMetric.java       |   2 +-
 .../metrics2/lib/MutableMetricsFactory.java     |   2 +-
 .../hadoop/metrics2/lib/MutableQuantiles.java   |   2 +-
 .../metrics2/lib/MutableRollingAverages.java    |   2 +-
 .../apache/hadoop/metrics2/lib/MutableStat.java |   2 +-
 .../metrics2/sink/RollingFileSystemSink.java    |   2 +-
 .../org/apache/hadoop/net/TableMapping.java     |   2 +-
 .../apache/hadoop/net/unix/DomainSocket.java    |   2 +-
 .../hadoop/net/unix/DomainSocketWatcher.java    |   2 +-
 .../security/ShellBasedUnixGroupsMapping.java   |  12 +-
 .../hadoop/security/UserGroupInformation.java   |   4 +-
 .../hadoop/security/alias/CredentialShell.java  |   2 +-
 .../hadoop/security/http/CrossOriginFilter.java |   2 +-
 .../hadoop/security/token/DtFileOperations.java |   2 +-
 .../web/DelegationTokenAuthenticator.java       |   8 +-
 .../org/apache/hadoop/tools/TableListing.java   |   6 +-
 .../apache/hadoop/util/HttpExceptionUtils.java  |  12 +-
 .../apache/hadoop/util/JsonSerialization.java   |  24 +
 .../main/java/org/apache/hadoop/util/Shell.java |   2 +-
 .../org/apache/hadoop/util/StringUtils.java     |  64 +-
 .../org/apache/hadoop/util/SysInfoLinux.java    |  19 +-
 .../hadoop-common/src/main/proto/FSProtos.proto |   8 +
 ...rg.apache.hadoop.fs.MultipartUploaderFactory |  16 +
 .../apache/hadoop/conf/TestConfiguration.java   |   2 +-
 .../conf/TestConfigurationFieldsBase.java       |   2 +-
 .../crypto/random/TestOsSecureRandom.java       |   2 +-
 .../fs/AbstractSystemMultipartUploaderTest.java | 143 ++++
 .../org/apache/hadoop/fs/FSTestWrapper.java     |   2 +-
 .../fs/FileContextMainOperationsBaseTest.java   |  44 +-
 .../apache/hadoop/fs/FileContextTestHelper.java |   1 -
 .../hadoop/fs/TestDFCachingGetSpaceUsed.java    |   2 +-
 .../fs/TestFileSystemStorageStatistics.java     |  18 +-
 .../apache/hadoop/fs/TestLocalFileSystem.java   |  23 +-
 .../TestLocalFileSystemMultipartUploader.java   |  65 ++
 .../AbstractContractPathHandleTest.java         |   6 +
 .../TestRawlocalContractPathHandle.java         |  40 ++
 .../hadoop/fs/shell/TestCopyFromLocal.java      |  10 +-
 .../TestSharedFileDescriptorFactory.java        |   2 +-
 .../org/apache/hadoop/ipc/TestProtoBufRpc.java  |   2 +-
 .../java/org/apache/hadoop/ipc/TestSaslRPC.java |   2 +-
 .../org/apache/hadoop/net/TestNetUtils.java     |   2 +-
 .../hadoop/net/unix/TestDomainSocket.java       |   2 +-
 .../TestShellBasedUnixGroupsMapping.java        |  39 +-
 ...tionTokenAuthenticationHandlerWithMocks.java |   2 +-
 .../AbstractServiceLauncherTestBase.java        |   2 +-
 .../apache/hadoop/test/GenericTestUtils.java    |   4 +-
 .../hadoop/util/TestShutdownHookManager.java    |   2 +-
 .../apache/hadoop/util/TestSignalLogger.java    |   2 +-
 .../apache/hadoop/util/TestSysInfoLinux.java    |  60 ++
 .../src/test/resources/contract/rawlocal.xml    |   5 +
 .../test/scripts/process_with_sigterm_trap.sh   |  24 +
 .../crypto/key/kms/server/KMSJSONWriter.java    |   6 +-
 hadoop-dist/pom.xml                             |   5 +
 .../src/main/compose/ozone/docker-compose.yaml  |  12 -
 .../src/main/compose/ozone/docker-config        |   5 -
 .../main/compose/ozoneperf/docker-compose.yaml  |  13 -
 .../src/main/compose/ozoneperf/docker-config    |   5 -
 .../apache/hadoop/hdds/scm/ScmConfigKeys.java   |  26 -
 .../container/common/helpers/ContainerInfo.java |  32 +
 .../scm/container/common/helpers/Pipeline.java  | 142 ++--
 .../common/helpers/PipelineChannel.java         | 124 ----
 .../org/apache/hadoop/ozone/OzoneConsts.java    |   1 -
 .../apache/hadoop/utils/BackgroundService.java  |   2 +-
 hadoop-hdds/common/src/main/proto/hdds.proto    |   8 +-
 .../common/src/main/resources/ozone-default.xml |  47 --
 .../hadoop/ozone/HddsDatanodeService.java       |  13 +-
 .../statemachine/DatanodeStateMachine.java      |   3 +
 .../background/BlockDeletingService.java        |   8 +-
 .../ReplicateContainerCommandHandler.java       |  67 ++
 .../states/endpoint/HeartbeatEndpointTask.java  |  12 +
 .../container/common/utils/HddsVolumeUtil.java  |   2 +-
 .../container/replication/ReplicationQueue.java |  76 ++
 .../replication/ReplicationRequest.java         | 106 +++
 .../container/replication/package-info.java     |  23 +
 .../protocol/commands/CommandForDatanode.java   |  45 ++
 .../commands/ReplicateContainerCommand.java     |  94 +++
 .../StorageContainerDatanodeProtocol.proto      |  12 +-
 .../replication/TestReplicationQueue.java       | 134 ++++
 .../container/replication/package-info.java     |  23 +
 .../testutils/BlockDeletingServiceTestImpl.java |   4 +-
 .../hadoop/hdds/server/events/EventWatcher.java | 157 +++++
 .../server/events/IdentifiableEventPayload.java |  30 +
 .../hdds/server/events/EventHandlerStub.java    |  38 +
 .../hdds/server/events/TestEventWatcher.java    | 220 ++++++
 .../hdds/scm/container/ContainerMapping.java    |  10 +-
 .../scm/container/closer/ContainerCloser.java   |   6 +-
 .../replication/ContainerSupervisor.java        | 340 ---------
 .../container/replication/InProgressPool.java   | 255 -------
 .../scm/container/replication/PeriodicPool.java | 119 ----
 .../scm/container/replication/package-info.java |  23 -
 .../hadoop/hdds/scm/node/NodeManager.java       |   6 -
 .../hadoop/hdds/scm/node/NodePoolManager.java   |  71 --
 .../hadoop/hdds/scm/node/SCMNodeManager.java    |  41 +-
 .../hdds/scm/node/SCMNodePoolManager.java       | 269 --------
 .../hdds/scm/pipelines/PipelineManager.java     |  67 +-
 .../hdds/scm/pipelines/PipelineSelector.java    |  11 +-
 .../scm/pipelines/ratis/RatisManagerImpl.java   |  13 +-
 .../standalone/StandaloneManagerImpl.java       |   8 +-
 .../server/SCMDatanodeHeartbeatDispatcher.java  | 126 ++++
 .../scm/server/SCMDatanodeProtocolServer.java   |  25 +-
 .../scm/server/StorageContainerManager.java     |  10 +-
 .../SCMDatanodeContainerReportHandler.java      |  76 --
 .../report/SCMDatanodeHeartbeatDispatcher.java  | 189 -----
 .../report/SCMDatanodeNodeReportHandler.java    |  43 --
 .../server/report/SCMDatanodeReportHandler.java |  83 ---
 .../report/SCMDatanodeReportHandlerFactory.java |  82 ---
 .../hdds/scm/server/report/package-info.java    |  57 --
 .../hdds/scm/block/TestDeletedBlockLog.java     |   8 +-
 .../hdds/scm/container/MockNodeManager.java     |   6 -
 .../hadoop/hdds/scm/node/TestNodeManager.java   |  39 ++
 .../hdds/scm/node/TestSCMNodePoolManager.java   | 160 -----
 .../TestSCMDatanodeHeartbeatDispatcher.java     | 119 ++++
 .../TestSCMDatanodeContainerReportHandler.java  |  34 -
 .../TestSCMDatanodeHeartbeatDispatcher.java     | 138 ----
 .../TestSCMDatanodeNodeReportHandler.java       |  36 -
 .../TestSCMDatanodeReportHandlerFactory.java    |  51 --
 .../hdds/scm/server/report/package-info.java    |  21 -
 .../testutils/ReplicationNodeManagerMock.java   |   5 -
 .../ReplicationNodePoolManagerMock.java         | 133 ----
 .../main/java/org/apache/hadoop/fs/XAttr.java   |   4 +-
 .../hdfs/DFSMultipartUploaderFactory.java       |  40 ++
 .../org/apache/hadoop/hdfs/ExtendedBlockId.java |   4 +-
 .../hdfs/client/impl/BlockReaderFactory.java    |   2 +-
 .../AddErasureCodingPolicyResponse.java         |   4 +-
 .../hdfs/protocol/CacheDirectiveInfo.java       |   4 +-
 .../hadoop/hdfs/protocol/CachePoolInfo.java     |   4 +-
 .../hadoop/hdfs/protocol/EncryptionZone.java    |   4 +-
 .../hdfs/protocol/ErasureCodingPolicy.java      |   4 +-
 .../hdfs/protocol/ErasureCodingPolicyInfo.java  |   4 +-
 .../hadoop/hdfs/protocol/ExtendedBlock.java     |   2 +-
 .../hdfs/shortcircuit/DfsClientShmManager.java  |   2 +-
 .../hdfs/shortcircuit/ShortCircuitCache.java    |   2 +-
 .../hdfs/shortcircuit/ShortCircuitShm.java      |   4 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |   7 +-
 ...onfRefreshTokenBasedAccessTokenProvider.java |   8 +-
 .../CredentialBasedAccessTokenProvider.java     |   8 +-
 ...rg.apache.hadoop.fs.MultipartUploaderFactory |  16 +
 .../ha/TestRequestHedgingProxyProvider.java     |   2 +-
 hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml     |   5 -
 .../federation/router/ConnectionPoolId.java     |   2 +-
 .../router/RemoteLocationContext.java           |   2 +-
 .../store/driver/impl/StateStoreFileImpl.java   |   2 +-
 .../federation/store/records/MountTable.java    |   2 +-
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   5 -
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   4 +-
 .../hadoop/hdfs/protocol/CacheDirective.java    |   2 +-
 .../qjournal/server/GetJournalEditServlet.java  |   4 +-
 .../hadoop/hdfs/qjournal/server/Journal.java    |  12 +-
 .../hadoop/hdfs/server/balancer/Balancer.java   |   6 +-
 .../hadoop/hdfs/server/balancer/Dispatcher.java |  30 +-
 .../server/blockmanagement/BlockManager.java    |   2 +-
 .../hdfs/server/datanode/DirectoryScanner.java  |   4 +-
 .../datanode/fsdataset/impl/FsDatasetCache.java |   2 +-
 .../server/diskbalancer/command/Command.java    |   4 +-
 .../diskbalancer/command/PlanCommand.java       |   4 +-
 .../diskbalancer/command/ReportCommand.java     |   4 +-
 .../server/namenode/EncryptionZoneManager.java  |   4 +-
 .../server/namenode/FSDirErasureCodingOp.java   |   2 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   6 +-
 .../hdfs/server/namenode/FSNamesystemLock.java  |  17 +-
 .../server/namenode/INodeAttributeProvider.java |   2 +-
 .../server/namenode/startupprogress/Step.java   |   6 +-
 .../server/namenode/top/metrics/TopMetrics.java |   2 +-
 .../apache/hadoop/hdfs/tools/CacheAdmin.java    |   2 +-
 .../apache/hadoop/hdfs/util/EnumCounters.java   |   2 +-
 .../src/main/resources/hdfs-default.xml         |  10 +
 .../hadoop/fs/TestEnhancedByteBufferAccess.java |   4 +-
 .../org/apache/hadoop/fs/TestGlobPaths.java     |   2 +-
 .../hadoop/fs/TestHDFSMultipartUploader.java    |  76 ++
 .../TestWebHdfsFileContextMainOperations.java   |   2 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   4 +-
 .../org/apache/hadoop/hdfs/TestDFSShell.java    |   2 +-
 .../apache/hadoop/hdfs/TestDecommission.java    |   2 +-
 .../hadoop/hdfs/TestFsShellPermission.java      |   2 +-
 .../hadoop/hdfs/TestHDFSPolicyProvider.java     |   2 +-
 .../hadoop/hdfs/TestLeaseRecoveryStriped.java   |   2 +-
 .../hdfs/TestSecureEncryptionZoneWithKMS.java   |  34 +-
 .../sasl/SaslDataTransferTestCase.java          |   2 +-
 .../hdfs/server/balancer/TestBalancer.java      |  81 ++-
 .../server/datanode/SimulatedFSDataset.java     |   2 +-
 .../command/TestDiskBalancerCommand.java        |   2 +-
 .../hadoop/hdfs/server/mover/TestMover.java     |  46 ++
 .../hdfs/server/namenode/FSImageTestUtil.java   |   2 +-
 .../hdfs/server/namenode/NameNodeAdapter.java   |   2 +-
 .../hdfs/server/namenode/TestAuditLogger.java   |   2 +-
 .../server/namenode/TestCacheDirectives.java    |   2 +-
 .../namenode/TestEditLogJournalFailures.java    |   2 +-
 .../hdfs/server/namenode/TestEditLogRace.java   |   6 +-
 .../server/namenode/TestFSNamesystemLock.java   |  10 +-
 .../hdfs/server/namenode/TestStartup.java       |   9 +-
 .../namenode/metrics/TestNameNodeMetrics.java   |   4 +-
 .../shortcircuit/TestShortCircuitCache.java     |   2 +-
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  |   2 +-
 .../hadoop/hdfs/tools/TestDFSAdminWithHA.java   |  12 +
 .../org/apache/hadoop/tracing/TestTracing.java  |   2 +-
 .../v2/app/job/impl/TestTaskAttempt.java        |   4 +-
 .../apache/hadoop/mapreduce/JobSubmitter.java   |   8 +-
 .../lib/input/CombineFileInputFormat.java       |   4 +-
 .../mapred/uploader/FrameworkUploader.java      |   2 +-
 .../mapred/uploader/TestFrameworkUploader.java  |  13 +
 hadoop-maven-plugins/pom.xml                    |  38 +-
 .../maven/plugin/resourcegz/ResourceGzMojo.java |   2 +-
 .../dev-support/bin/robot-all.sh                |   2 +-
 .../dev-support/bin/robot-dnd-all.sh            |  63 ++
 .../acceptance-test/dev-support/bin/robot.sh    |   9 +-
 .../dev-support/docker/Dockerfile               |  21 +
 .../dev-support/docker/docker-compose.yaml      |  23 +
 hadoop-ozone/acceptance-test/pom.xml            |   1 +
 .../src/test/acceptance/basic/.env              |  17 +
 .../src/test/acceptance/basic/basic.robot       |  50 ++
 .../test/acceptance/basic/docker-compose.yaml   |  50 ++
 .../src/test/acceptance/basic/docker-config     |  33 +
 .../src/test/acceptance/basic/ozone-shell.robot |  85 +++
 .../src/test/acceptance/commonlib.robot         |  78 +++
 .../src/test/acceptance/ozonefs/.env            |  17 +
 .../test/acceptance/ozonefs/docker-compose.yaml |  59 ++
 .../src/test/acceptance/ozonefs/docker-config   |  34 +
 .../src/test/acceptance/ozonefs/ozonefs.robot   |  39 ++
 .../acceptance-test/src/test/compose/.env       |  17 -
 .../src/test/compose/docker-compose.yaml        |  62 --
 .../src/test/compose/docker-config              |  36 -
 .../robotframework/acceptance/ozone-shell.robot | 256 -------
 .../test/robotframework/acceptance/ozone.robot  | 104 ---
 hadoop-ozone/common/src/main/bin/ozone          |   9 +-
 .../hadoop/ozone/TestMiniOzoneCluster.java      |   8 +-
 .../ozone/container/ContainerTestHelper.java    |  19 +-
 .../common/TestBlockDeletingService.java        |  11 +-
 .../TestReplicateContainerHandler.java          |  71 ++
 ...TestGenerateOzoneRequiredConfigurations.java |   2 +-
 .../hadoop/ozone/scm/TestContainerSQLCli.java   |  31 -
 .../apache/hadoop/ozone/scm/TestSCMMetrics.java | 253 -------
 .../ozone-manager/src/main/webapps/ksm/ksm.js   |   4 +-
 hadoop-ozone/ozonefs/pom.xml                    | 211 ++++++
 .../org/apache/hadoop/fs/ozone/Constants.java   |  42 ++
 .../java/org/apache/hadoop/fs/ozone/OzFs.java   |  44 ++
 .../hadoop/fs/ozone/OzoneFSInputStream.java     |  79 +++
 .../hadoop/fs/ozone/OzoneFSOutputStream.java    |  59 ++
 .../apache/hadoop/fs/ozone/OzoneFileSystem.java | 689 +++++++++++++++++++
 .../apache/hadoop/fs/ozone/package-info.java    |  30 +
 .../hadoop/fs/ozone/TestOzoneFSInputStream.java | 157 +++++
 .../fs/ozone/TestOzoneFileInterfaces.java       | 231 +++++++
 .../contract/ITestOzoneContractCreate.java      |  48 ++
 .../contract/ITestOzoneContractDelete.java      |  48 ++
 .../contract/ITestOzoneContractDistCp.java      |  50 ++
 .../ITestOzoneContractGetFileStatus.java        |  61 ++
 .../ozone/contract/ITestOzoneContractMkdir.java |  48 ++
 .../ozone/contract/ITestOzoneContractOpen.java  |  47 ++
 .../contract/ITestOzoneContractRename.java      |  49 ++
 .../contract/ITestOzoneContractRootDir.java     |  51 ++
 .../ozone/contract/ITestOzoneContractSeek.java  |  47 ++
 .../hadoop/fs/ozone/contract/OzoneContract.java | 123 ++++
 .../src/test/resources/contract/ozone.xml       | 113 +++
 .../ozonefs/src/test/resources/log4j.properties |  23 +
 hadoop-ozone/pom.xml                            |   1 +
 .../genesis/BenchMarkContainerStateMap.java     |  11 +-
 .../genesis/BenchMarkDatanodeDispatcher.java    |   6 +-
 .../org/apache/hadoop/ozone/scm/cli/SQLCLI.java |  78 +--
 hadoop-project/pom.xml                          |  16 +-
 .../aliyun/oss/AliyunCredentialsProvider.java   |   2 +-
 .../fs/aliyun/oss/AliyunOSSFileSystem.java      |   2 +-
 .../fs/aliyun/oss/AliyunOSSFileSystemStore.java |   2 +-
 .../hadoop/fs/aliyun/oss/AliyunOSSUtils.java    |   2 +-
 .../fs/aliyun/oss/AliyunOSSTestUtils.java       |   2 +-
 hadoop-tools/hadoop-aws/pom.xml                 |  42 +-
 .../fs/s3a/AWSCredentialProviderList.java       |   2 +-
 .../fs/s3a/BasicAWSCredentialsProvider.java     |   2 +-
 .../org/apache/hadoop/fs/s3a/Constants.java     |  11 +
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |   4 +-
 .../apache/hadoop/fs/s3a/S3AInputStream.java    |   2 +-
 .../hadoop/fs/s3a/S3AMultipartUploader.java     | 150 ++++
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |   2 +-
 .../fs/s3a/SimpleAWSCredentialsProvider.java    |   2 +-
 .../fs/s3a/TemporaryAWSCredentialsProvider.java |   2 +-
 .../s3a/auth/AssumedRoleCredentialProvider.java |   2 +-
 .../s3a/commit/files/SinglePendingCommit.java   |   2 +-
 .../hadoop/fs/s3a/commit/files/SuccessData.java |   2 +-
 .../fs/s3a/commit/magic/MagicCommitTracker.java |   2 +-
 .../hadoop/fs/s3a/commit/staging/Paths.java     |   2 +-
 .../fs/s3a/s3guard/DynamoDBClientFactory.java   |   2 +-
 .../fs/s3a/s3guard/DynamoDBMetadataStore.java   |   3 +-
 .../fs/s3a/s3guard/LocalMetadataEntry.java      |  81 +++
 .../fs/s3a/s3guard/LocalMetadataStore.java      | 249 ++++---
 .../hadoop/fs/s3a/s3guard/S3GuardTool.java      |   6 +-
 .../hadoop/fs/s3native/S3xLoginHelper.java      |   4 +-
 ...rg.apache.hadoop.fs.MultipartUploaderFactory |  15 +
 .../org.apache.hadoop.fs.MultipartUploader      |  16 +
 .../hadoop/fs/s3a/AbstractS3ATestBase.java      |  21 +-
 .../hadoop/fs/s3a/ITestS3AConfiguration.java    |   4 +-
 .../ITestS3AEncryptionSSEKMSUserDefinedKey.java |   2 +-
 ...onSSEKMSUserDefinedKeyBlockOutputStream.java |   2 +-
 .../apache/hadoop/fs/s3a/S3ATestConstants.java  |   1 -
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java  |  36 +-
 .../fs/s3a/commit/AbstractCommitITest.java      |   2 +-
 .../fs/s3a/commit/staging/StagingTestBase.java  |  30 +-
 .../s3a/s3guard/DynamoDBLocalClientFactory.java | 160 -----
 .../fs/s3a/s3guard/MetadataStoreTestBase.java   |   6 +-
 .../s3a/s3guard/TestDynamoDBMetadataStore.java  | 589 ----------------
 .../fs/s3a/s3guard/TestLocalMetadataStore.java  |  33 +-
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java |   2 +-
 .../hadoop/fs/adl/TestAzureADTokenProvider.java |   2 +-
 .../fs/azure/AzureNativeFileSystemStore.java    |   2 +-
 .../hadoop/fs/azure/BlockBlobAppendStream.java  |   2 +-
 .../fs/azure/ClientThrottlingAnalyzer.java      |   2 +-
 .../hadoop/fs/azure/NativeAzureFileSystem.java  |   4 +-
 .../hadoop/fs/azure/PageBlobOutputStream.java   |   2 +-
 .../fs/azure/RemoteWasbAuthorizerImpl.java      |   2 +-
 .../fs/azure/SecureWasbRemoteCallHelper.java    |   2 +-
 .../hadoop/fs/azure/security/JsonUtils.java     |   4 +-
 .../fs/azure/ITestBlobDataValidation.java       |   2 +-
 .../azure/ITestNativeAzureFileSystemAppend.java |   2 +-
 .../hadoop/fs/azure/MockStorageInterface.java   |  12 +-
 ...estNativeAzureFileSystemBlockCompaction.java |   2 +-
 .../fs/azure/integration/AzureTestUtils.java    |   2 +-
 .../org/apache/hadoop/tools/DistCpOptions.java  |   2 +-
 .../org/apache/hadoop/tools/OptionsParser.java  |   2 +-
 .../apache/hadoop/tools/mapred/CopyMapper.java  |   2 +-
 .../mapred/gridmix/ClusterSummarizer.java       |   2 +-
 .../mapred/gridmix/ExecutionSummarizer.java     |   4 +-
 .../hadoop/mapred/gridmix/JobFactory.java       |   2 +-
 .../mapred/gridmix/RandomTextDataGenerator.java |   2 +-
 hadoop-tools/hadoop-ozone/pom.xml               | 174 -----
 .../org/apache/hadoop/fs/ozone/Constants.java   |  42 --
 .../java/org/apache/hadoop/fs/ozone/OzFs.java   |  44 --
 .../hadoop/fs/ozone/OzoneFSInputStream.java     |  79 ---
 .../hadoop/fs/ozone/OzoneFSOutputStream.java    |  59 --
 .../apache/hadoop/fs/ozone/OzoneFileSystem.java | 689 -------------------
 .../apache/hadoop/fs/ozone/package-info.java    |  30 -
 .../hadoop/fs/ozone/TestOzoneFSInputStream.java | 157 -----
 .../fs/ozone/TestOzoneFileInterfaces.java       | 231 -------
 .../contract/ITestOzoneContractCreate.java      |  48 --
 .../contract/ITestOzoneContractDelete.java      |  48 --
 .../contract/ITestOzoneContractDistCp.java      |  50 --
 .../ITestOzoneContractGetFileStatus.java        |  61 --
 .../ozone/contract/ITestOzoneContractMkdir.java |  48 --
 .../ozone/contract/ITestOzoneContractOpen.java  |  47 --
 .../contract/ITestOzoneContractRename.java      |  49 --
 .../contract/ITestOzoneContractRootDir.java     |  51 --
 .../ozone/contract/ITestOzoneContractSeek.java  |  47 --
 .../hadoop/fs/ozone/contract/OzoneContract.java | 123 ----
 .../src/test/resources/contract/ozone.xml       | 113 ---
 .../src/test/resources/log4j.properties         |  23 -
 .../translator/impl/BaseLogParser.java          |   1 -
 .../WordListAnonymizerUtility.java              |   2 +-
 .../util/MapReduceJobPropertiesParser.java      |   2 +-
 .../hadoop/streaming/TestUnconsumedInput.java   |   2 +-
 hadoop-tools/hadoop-tools-dist/pom.xml          |  15 -
 hadoop-tools/pom.xml                            |  11 -
 .../yarn/api/records/ResourceRequest.java       |  20 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |   2 +-
 .../exceptions/RestApiErrorMessages.java        |   2 +
 .../yarn/service/utils/ServiceApiUtil.java      |   5 +
 .../hadoop/yarn/service/TestServiceApiUtil.java |  18 +
 .../yarn/client/api/impl/AMRMClientImpl.java    |  11 +-
 .../api/impl/TestAMRMClientOnRMRestart.java     |   6 +-
 .../yarn/client/api/impl/TestYarnClient.java    | 337 ++-------
 .../client/api/impl/TestYarnClientImpl.java     | 254 +++++++
 .../hadoop-yarn/hadoop-yarn-common/pom.xml      |   2 +-
 .../org/apache/hadoop/yarn/webapp/WebApps.java  |  14 +-
 .../hadoop/yarn/webapp/view/JQueryUI.java       |   2 +-
 .../jquery/jquery-ui-1.12.1.custom.min.js       |  13 +
 .../static/jquery/jquery-ui-1.9.1.custom.min.js |   6 -
 .../registry/client/api/RegistryConstants.java  |   2 +-
 .../hadoop/registry/server/dns/RegistryDNS.java |  29 +-
 .../registry/server/dns/TestRegistryDNS.java    |  23 +-
 .../hadoop/yarn/server/AMRMClientRelayer.java   |   8 +-
 .../LocalityMulticastAMRMProxyPolicy.java       |  10 +-
 .../server/scheduler/ResourceRequestSet.java    |  14 +-
 .../hadoop/yarn/server/utils/BuilderUtils.java  |  12 -
 .../src/CMakeLists.txt                          |   4 +-
 .../server/nodemanager/ContainerExecutor.java   |   4 +-
 .../containermanager/container/Container.java   |   6 +
 .../container/ContainerImpl.java                |  11 +
 .../launcher/ContainerLaunch.java               |  46 +-
 .../launcher/ContainerRelaunch.java             |   1 -
 .../CGroupsMemoryResourceHandlerImpl.java       |  24 +
 .../linux/resources/MemoryResourceHandler.java  |  10 +
 .../resources/gpu/GpuResourceAllocator.java     |  68 +-
 .../resources/gpu/GpuResourceHandlerImpl.java   |   1 -
 .../localizer/ContainerLocalizer.java           |   7 +-
 .../localizer/ResourceLocalizationService.java  |   5 +-
 .../monitor/ContainersMonitorImpl.java          | 108 +--
 .../oom-listener/test/oom_listener_test_main.cc |  33 +-
 .../container/TestContainer.java                |   8 +-
 .../launcher/TestContainerLaunch.java           |   2 -
 .../TestCGroupsMemoryResourceHandlerImpl.java   |  43 ++
 .../nodemanager/webapp/MockContainer.java       |   3 +
 .../server/resourcemanager/ResourceManager.java |   2 +-
 .../scheduler/capacity/CSQueueUtils.java        |  18 +-
 .../scheduler/capacity/CapacityScheduler.java   |  18 +-
 .../fair/AllocationFileLoaderService.java       |   4 +-
 .../LocalityAppPlacementAllocator.java          |  10 +-
 .../webapp/dao/ClusterMetricsInfo.java          |   2 +-
 .../server/resourcemanager/Application.java     |  10 +-
 .../server/resourcemanager/TestAppManager.java  |  18 +-
 .../src/site/markdown/FairScheduler.md          |   2 +-
 .../site/markdown/yarn-service/RegistryDNS.md   |  10 +-
 .../markdown/yarn-service/ServiceDiscovery.md   |   6 +-
 .../hadoop-yarn-ui/src/main/webapp/.bowerrc     |   3 +-
 .../src/main/webapp/WEB-INF/web.xml             |   6 +-
 pom.xml                                         |   2 +-
 start-build-env.sh                              |   8 +-
 437 files changed, 7828 insertions(+), 7627 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1045251/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1045251/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1045251/hadoop-hdds/common/src/main/resources/ozone-default.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1045251/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
----------------------------------------------------------------------
diff --cc hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
index b6a9bb9,b073d7b..245d76f
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
@@@ -93,8 -95,9 +95,9 @@@ public class DatanodeStateMachine imple
       // trick.
      commandDispatcher = CommandDispatcher.newBuilder()
          .addHandler(new CloseContainerCommandHandler())
 -        .addHandler(new DeleteBlocksCommandHandler(
 -            container.getContainerManager(), conf))
 +        .addHandler(new DeleteBlocksCommandHandler(container.getContainerSet(),
 +            conf))
+         .addHandler(new ReplicateContainerCommandHandler())
          .setConnectionManager(connectionManager)
          .setContainer(container)
          .setContext(context)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1045251/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1045251/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java
----------------------------------------------------------------------
diff --cc hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java
index 6809d57,0000000..5d6fc0a
mode 100644,000000..100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java
@@@ -1,163 -1,0 +1,163 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.hadoop.ozone.container.common.utils;
 +
 +import com.google.common.annotations.VisibleForTesting;
- import org.apache.commons.lang.StringUtils;
++import org.apache.commons.lang3.StringUtils;
 +import org.apache.hadoop.ozone.OzoneConsts;
 +import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
 +import org.apache.hadoop.ozone.container.common.DataNodeLayoutVersion;
 +import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 +import org.apache.hadoop.util.Time;
 +
 +import java.io.File;
 +import java.util.Properties;
 +import java.util.UUID;
 +
 +/**
 + * A util class for {@link HddsVolume}.
 + */
 +public final class HddsVolumeUtil {
 +
 +  // Private constructor for Utility class. Unused.
 +  private HddsVolumeUtil() {
 +  }
 +
 +  private static final String VERSION_FILE   = "VERSION";
 +  private static final String STORAGE_ID_PREFIX = "DS-";
 +
 +  public static File getVersionFile(File rootDir) {
 +    return new File(rootDir, VERSION_FILE);
 +  }
 +
 +  public static String generateUuid() {
 +    return STORAGE_ID_PREFIX + UUID.randomUUID();
 +  }
 +
 +  /**
 +   * Get hddsRoot from volume root. If volumeRoot points to hddsRoot, it is
 +   * returned as is.
 +   * For a volumeRoot /data/disk1, the hddsRoot is /data/disk1/hdds.
 +   * @param volumeRoot root of the volume.
 +   * @return hddsRoot of the volume.
 +   */
 +  public static String getHddsRoot(String volumeRoot) {
 +    if (volumeRoot.endsWith(HddsVolume.HDDS_VOLUME_DIR)) {
 +      return volumeRoot;
 +    } else {
 +      File hddsRoot = new File(volumeRoot, HddsVolume.HDDS_VOLUME_DIR);
 +      return hddsRoot.getPath();
 +    }
 +  }
 +
 +  /**
 +   * Returns storageID if it is valid. Throws an exception otherwise.
 +   */
 +  @VisibleForTesting
 +  public static String getStorageID(Properties props, File versionFile)
 +      throws InconsistentStorageStateException {
 +    return getProperty(props, OzoneConsts.STORAGE_ID, versionFile);
 +  }
 +
 +  /**
 +   * Returns clusterID if it is valid. It should match the clusterID from the
 +   * Datanode. Throws an exception otherwise.
 +   */
 +  @VisibleForTesting
 +  public static String getClusterID(Properties props, File versionFile,
 +      String clusterID) throws InconsistentStorageStateException {
 +    String cid = getProperty(props, OzoneConsts.CLUSTER_ID, versionFile);
 +
 +    if (clusterID == null) {
 +      return cid;
 +    }
 +    if (!clusterID.equals(cid)) {
 +      throw new InconsistentStorageStateException("Mismatched " +
 +          "ClusterIDs. Version File : " + versionFile + " has clusterID: " +
 +          cid + " and Datanode has clusterID: " + clusterID);
 +    }
 +    return cid;
 +  }
 +
 +  /**
 +   * Returns datanodeUuid if it is valid. It should match the UUID of the
 +   * Datanode. Throws an exception otherwise.
 +   */
 +  @VisibleForTesting
 +  public static String getDatanodeUUID(Properties props, File versionFile,
 +      String datanodeUuid)
 +      throws InconsistentStorageStateException {
 +    String datanodeID = getProperty(props, OzoneConsts.DATANODE_UUID,
 +        versionFile);
 +
 +    if (datanodeUuid != null && !datanodeUuid.equals(datanodeID)) {
 +      throw new InconsistentStorageStateException("Mismatched " +
 +          "DatanodeUUIDs. Version File : " + versionFile + " has datanodeUuid: "
 +          + datanodeID + " and Datanode has datanodeUuid: " + datanodeUuid);
 +    }
 +    return datanodeID;
 +  }
 +
 +  /**
 +   * Returns creationTime if it is valid. Throws an exception otherwise.
 +   */
 +  @VisibleForTesting
 +  public static long getCreationTime(Properties props, File versionFile)
 +      throws InconsistentStorageStateException {
 +    String cTimeStr = getProperty(props, OzoneConsts.CTIME, versionFile);
 +
 +    long cTime = Long.parseLong(cTimeStr);
 +    long currentTime = Time.now();
 +    if (cTime > currentTime || cTime < 0) {
 +      throw new InconsistentStorageStateException("Invalid Creation time in " +
 +          "Version File : " + versionFile + " - " + cTime + ". Current system" +
 +          " time is " + currentTime);
 +    }
 +    return cTime;
 +  }
 +
 +  /**
 +   * Returns layOutVersion if it is valid. Throws an exception otherwise.
 +   */
 +  @VisibleForTesting
 +  public static int getLayOutVersion(Properties props, File versionFile) throws
 +      InconsistentStorageStateException {
 +    String lvStr = getProperty(props, OzoneConsts.LAYOUTVERSION, versionFile);
 +
 +    int lv = Integer.parseInt(lvStr);
 +    if(DataNodeLayoutVersion.getLatestVersion().getVersion() != lv) {
 +      throw new InconsistentStorageStateException("Invalid layOutVersion. " +
 +          "Version file has layOutVersion as " + lv + " and latest Datanode " +
 +          "layOutVersion is " +
 +          DataNodeLayoutVersion.getLatestVersion().getVersion());
 +    }
 +    return lv;
 +  }
 +
 +  private static String getProperty(Properties props, String propName, File
 +      versionFile)
 +      throws InconsistentStorageStateException {
 +    String value = props.getProperty(propName);
 +    if (StringUtils.isBlank(value)) {
 +      throw new InconsistentStorageStateException("Invalid " + propName +
 +          ". Version File : " + versionFile + " has null or empty " + propName);
 +    }
 +    return value;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1045251/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
----------------------------------------------------------------------
diff --cc hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
index 9ac9930,adca8ea..ff5b9f1
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
@@@ -40,9 -44,9 +44,10 @@@ import org.apache.hadoop.hdds.protocol.
      .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
  import org.apache.hadoop.ipc.Server;
  import org.apache.hadoop.metrics2.util.MBeans;
 +import org.apache.hadoop.ozone.OzoneConsts;
  import org.apache.hadoop.ozone.protocol.StorageContainerNodeProtocol;
  import org.apache.hadoop.ozone.protocol.VersionResponse;
+ import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
  import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
  import org.apache.hadoop.ozone.protocol.commands.ReregisterCommand;
  import org.apache.hadoop.ozone.protocol.commands.SCMCommand;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1045251/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1045251/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
----------------------------------------------------------------------


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


[19/37] hadoop git commit: HDDS-183:Integrate Volumeset, ContainerSet and HddsDispatcher. Contributed by Bharat Viswanadham

Posted by bh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
index 4156f5a..9e25c59 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
@@ -1,72 +1,49 @@
-/**
- * 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.
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
  */
 
 package org.apache.hadoop.ozone.container.ozoneimpl;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
-import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
-import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
-import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
-import org.apache.hadoop.ozone.container.common.impl.Dispatcher;
-import org.apache.hadoop.ozone.container.common.impl.KeyManagerImpl;
-import org.apache.hadoop.ozone.container.common.interfaces.ChunkManager;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
-import org.apache.hadoop.ozone.container.common.interfaces.KeyManager;
-import org.apache.hadoop.ozone.container.common.statemachine.background
-    .BlockDeletingService;
 import org.apache.hadoop.ozone.container.common.transport.server.XceiverServer;
-import org.apache.hadoop.ozone.container.common.transport.server
-    .XceiverServerGrpc;
-import org.apache.hadoop.ozone.container.common.transport.server
-    .XceiverServerSpi;
-import org.apache.hadoop.ozone.container.common.transport.server.ratis
-    .XceiverServerRatis;
+import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerGrpc;
+import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi;
+import org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+
+import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.nio.file.Paths;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
+import java.io.*;
+import java.util.ArrayList;
+import java.util.Iterator;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
-    .OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
-    .OZONE_BLOCK_DELETING_SERVICE_INTERVAL_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.OzoneConsts.CONTAINER_ROOT_PREFIX;
 import static org.apache.hadoop.ozone.OzoneConsts.INVALID_PORT;
 
 /**
@@ -74,226 +51,123 @@ import static org.apache.hadoop.ozone.OzoneConsts.INVALID_PORT;
  * layer.
  */
 public class OzoneContainer {
-  public static final Logger LOG =
-      LoggerFactory.getLogger(OzoneContainer.class);
 
-  private final Configuration ozoneConfig;
-  private final ContainerDispatcher dispatcher;
-  private final ContainerManager manager;
+  public static final Logger LOG = LoggerFactory.getLogger(
+      OzoneContainer.class);
+
+  private final HddsDispatcher hddsDispatcher;
+  private final DatanodeDetails dnDetails;
+  private final OzoneConfiguration config;
+  private final VolumeSet volumeSet;
+  private final ContainerSet containerSet;
   private final XceiverServerSpi[] server;
-  private final ChunkManager chunkManager;
-  private final KeyManager keyManager;
-  private final BlockDeletingService blockDeletingService;
 
   /**
-   * Creates a network endpoint and enables Ozone container.
-   *
-   * @param ozoneConfig - Config
+   * Construct OzoneContainer object.
+   * @param datanodeDetails
+   * @param conf
+   * @throws DiskOutOfSpaceException
    * @throws IOException
    */
-  public OzoneContainer(
-      DatanodeDetails datanodeDetails, Configuration ozoneConfig)
-      throws IOException {
-    this.ozoneConfig = ozoneConfig;
-    List<StorageLocation> locations = new LinkedList<>();
-    String[] paths = ozoneConfig.getStrings(
-        OzoneConfigKeys.OZONE_METADATA_DIRS);
-    if (paths != null && paths.length > 0) {
-      for (String p : paths) {
-        locations.add(StorageLocation.parse(
-            Paths.get(p).resolve(CONTAINER_ROOT_PREFIX).toString()));
-      }
-    } else {
-      getDataDir(locations);
-    }
-
-    manager = new ContainerManagerImpl();
-    manager.init(this.ozoneConfig, locations, datanodeDetails);
-    this.chunkManager = new ChunkManagerImpl(manager);
-    manager.setChunkManager(this.chunkManager);
-
-    this.keyManager = new KeyManagerImpl(manager, ozoneConfig);
-    manager.setKeyManager(this.keyManager);
-
-    long svcInterval =
-        ozoneConfig.getTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL,
-        OZONE_BLOCK_DELETING_SERVICE_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS);
-    long serviceTimeout = ozoneConfig.getTimeDuration(
-        OZONE_BLOCK_DELETING_SERVICE_TIMEOUT,
-        OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS);
-    this.blockDeletingService = new BlockDeletingService(manager,
-        svcInterval, serviceTimeout, ozoneConfig);
-
-    this.dispatcher = new Dispatcher(manager, this.ozoneConfig);
-
-    boolean useGrpc = this.ozoneConfig.getBoolean(
+  public OzoneContainer(DatanodeDetails datanodeDetails, OzoneConfiguration
+      conf) throws IOException {
+    this.dnDetails = datanodeDetails;
+    this.config = conf;
+    this.volumeSet = new VolumeSet(datanodeDetails, conf);
+    this.containerSet = new ContainerSet();
+    boolean useGrpc = this.config.getBoolean(
         ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_KEY,
         ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_DEFAULT);
+    buildContainerSet();
+    hddsDispatcher = new HddsDispatcher(config, containerSet, volumeSet);
     server = new XceiverServerSpi[]{
-        useGrpc ? new XceiverServerGrpc(datanodeDetails,
-            this.ozoneConfig, this.dispatcher) :
+        useGrpc ? new XceiverServerGrpc(datanodeDetails, this.config, this
+            .hddsDispatcher) :
             new XceiverServer(datanodeDetails,
-                this.ozoneConfig, this.dispatcher),
-      XceiverServerRatis
-          .newXceiverServerRatis(datanodeDetails, this.ozoneConfig, dispatcher)
+                this.config, this.hddsDispatcher),
+        XceiverServerRatis.newXceiverServerRatis(datanodeDetails, this
+            .config, hddsDispatcher)
     };
-  }
 
-  /**
-   * Starts serving requests to ozone container.
-   *
-   * @throws IOException
-   */
-  public void start() throws IOException {
-    for (XceiverServerSpi serverinstance : server) {
-      serverinstance.start();
-    }
-    blockDeletingService.start();
-    dispatcher.init();
+
   }
 
+
   /**
-   * Stops the ozone container.
-   * <p>
-   * Shutdown logic is not very obvious from the following code. if you need to
-   * modify the logic, please keep these comments in mind. Here is the shutdown
-   * sequence.
-   * <p>
-   * 1. We shutdown the network ports.
-   * <p>
-   * 2. Now we need to wait for all requests in-flight to finish.
-   * <p>
-   * 3. The container manager lock is a read-write lock with "Fairness"
-   * enabled.
-   * <p>
-   * 4. This means that the waiting threads are served in a "first-come-first
-   * -served" manner. Please note that this applies to waiting threads only.
-   * <p>
-   * 5. Since write locks are exclusive, if we are waiting to get a lock it
-   * implies that we are waiting for in-flight operations to complete.
-   * <p>
-   * 6. if there are other write operations waiting on the reader-writer lock,
-   * fairness guarantees that they will proceed before the shutdown lock
-   * request.
-   * <p>
-   * 7. Since all operations either take a reader or writer lock of container
-   * manager, we are guaranteed that we are the last operation since we have
-   * closed the network port, and we wait until close is successful.
-   * <p>
-   * 8. We take the writer lock and call shutdown on each of the managers in
-   * reverse order. That is chunkManager, keyManager and containerManager is
-   * shutdown.
+   * Build's container map.
    */
-  public void stop() {
-    LOG.info("Attempting to stop container services.");
-    for(XceiverServerSpi serverinstance: server) {
-      serverinstance.stop();
+  public void buildContainerSet() {
+    Iterator<HddsVolume> volumeSetIterator = volumeSet.getVolumesList()
+        .iterator();
+    ArrayList<Thread> volumeThreads = new ArrayList<Thread>();
+
+    //TODO: diskchecker should be run before this, to see how disks are.
+    // And also handle disk failure tolerance need to be added
+    while (volumeSetIterator.hasNext()) {
+      HddsVolume volume = volumeSetIterator.next();
+      File hddsVolumeRootDir = volume.getHddsRootDir();
+      Thread thread = new Thread(new ContainerReader(hddsVolumeRootDir,
+          containerSet, config));
+      thread.start();
+      volumeThreads.add(thread);
     }
-    dispatcher.shutdown();
 
     try {
-      this.manager.writeLock();
-      this.chunkManager.shutdown();
-      this.keyManager.shutdown();
-      this.manager.shutdown();
-      this.blockDeletingService.shutdown();
-      LOG.info("container services shutdown complete.");
-    } catch (IOException ex) {
-      LOG.warn("container service shutdown error:", ex);
-    } finally {
-      this.manager.writeUnlock();
+      for (int i = 0; i < volumeThreads.size(); i++) {
+        volumeThreads.get(i).join();
+      }
+    } catch (InterruptedException ex) {
+      LOG.info("Volume Threads Interrupted exception", ex);
     }
+
   }
 
   /**
-   * Returns a paths to data dirs.
+   * Starts serving requests to ozone container.
    *
-   * @param pathList - List of paths.
    * @throws IOException
    */
-  private void getDataDir(List<StorageLocation> pathList) throws IOException {
-    for (String dir : ozoneConfig.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
-      StorageLocation location = StorageLocation.parse(dir);
-      pathList.add(location);
+  public void start() throws IOException {
+    LOG.info("Attempting to start container services.");
+    for (XceiverServerSpi serverinstance : server) {
+      serverinstance.start();
     }
+    hddsDispatcher.init();
   }
 
   /**
-   * Returns node report of container storage usage.
+   * Stop Container Service on the datanode.
    */
-  public NodeReportProto getNodeReport() throws IOException {
-    return this.manager.getNodeReport();
-  }
-
-  private int getPortbyType(HddsProtos.ReplicationType replicationType) {
-    for (XceiverServerSpi serverinstance : server) {
-      if (serverinstance.getServerType() == replicationType) {
-        return serverinstance.getIPCPort();
-      }
+  public void stop() {
+    //TODO: at end of container IO integration work.
+    LOG.info("Attempting to stop container services.");
+    for(XceiverServerSpi serverinstance: server) {
+      serverinstance.stop();
     }
-    return INVALID_PORT;
+    hddsDispatcher.shutdown();
   }
 
-  /**
-   * Returns the container server IPC port.
-   *
-   * @return Container server IPC port.
-   */
-  public int getContainerServerPort() {
-    return getPortbyType(HddsProtos.ReplicationType.STAND_ALONE);
-  }
 
-  /**
-   * Returns the Ratis container Server IPC port.
-   *
-   * @return Ratis port.
-   */
-  public int getRatisContainerServerPort() {
-    return getPortbyType(HddsProtos.ReplicationType.RATIS);
+  @VisibleForTesting
+  public ContainerSet getContainerSet() {
+    return containerSet;
   }
-
   /**
    * Returns container report.
    * @return - container report.
    * @throws IOException
    */
-  public ContainerReportsProto getContainerReport() throws IOException {
-    return this.manager.getContainerReport();
+  public StorageContainerDatanodeProtocolProtos.ContainerReportsProto
+      getContainerReport() throws IOException {
+    return this.containerSet.getContainerReport();
   }
 
-// TODO: remove getContainerReports
   /**
-   * Returns the list of closed containers.
-   * @return - List of closed containers.
+   * Submit ContainerRequest.
+   * @param request
+   * @param replicationType
    * @throws IOException
    */
-  public List<ContainerData> getClosedContainerReports() throws IOException {
-    return this.manager.getClosedContainerReports();
-  }
-
-  private XceiverServerSpi getRatisSerer() {
-    for (XceiverServerSpi serverInstance : server) {
-      if (serverInstance instanceof XceiverServerRatis) {
-        return serverInstance;
-      }
-    }
-    return null;
-  }
-
-  private XceiverServerSpi getStandaAloneSerer() {
-    for (XceiverServerSpi serverInstance : server) {
-      if (!(serverInstance instanceof XceiverServerRatis)) {
-        return serverInstance;
-      }
-    }
-    return null;
-  }
-
-  @VisibleForTesting
-  public ContainerManager getContainerManager() {
-    return this.manager;
-  }
-
   public void submitContainerRequest(
       ContainerProtos.ContainerCommandRequestProto request,
       HddsProtos.ReplicationType replicationType) throws IOException {
@@ -332,4 +206,66 @@ public class OzoneContainer {
           + " not supported over HearBeat Response");
     }
   }
-}
\ No newline at end of file
+
+  private XceiverServerSpi getRatisSerer() {
+    for (XceiverServerSpi serverInstance : server) {
+      if (serverInstance instanceof XceiverServerRatis) {
+        return serverInstance;
+      }
+    }
+    return null;
+  }
+
+  private XceiverServerSpi getStandaAloneSerer() {
+    for (XceiverServerSpi serverInstance : server) {
+      if (!(serverInstance instanceof XceiverServerRatis)) {
+        return serverInstance;
+      }
+    }
+    return null;
+  }
+
+  private int getPortbyType(HddsProtos.ReplicationType replicationType) {
+    for (XceiverServerSpi serverinstance : server) {
+      if (serverinstance.getServerType() == replicationType) {
+        return serverinstance.getIPCPort();
+      }
+    }
+    return INVALID_PORT;
+  }
+
+  /**
+   * Returns the container server IPC port.
+   *
+   * @return Container server IPC port.
+   */
+  public int getContainerServerPort() {
+    return getPortbyType(HddsProtos.ReplicationType.STAND_ALONE);
+  }
+
+  /**
+   * Returns the Ratis container Server IPC port.
+   *
+   * @return Ratis port.
+   */
+  public int getRatisContainerServerPort() {
+    return getPortbyType(HddsProtos.ReplicationType.RATIS);
+  }
+
+  /**
+   * Returns node report of container storage usage.
+   */
+  public StorageContainerDatanodeProtocolProtos.NodeReportProto getNodeReport()
+      throws IOException {
+    return volumeSet.getNodeReport();
+  }
+
+  @VisibleForTesting
+  public ContainerDispatcher getDispatcher() {
+    return this.hddsDispatcher;
+  }
+
+  public VolumeSet getVolumeSet() {
+    return volumeSet;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/VersionResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/VersionResponse.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/VersionResponse.java
index 83acf5b..4d328d3 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/VersionResponse.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/VersionResponse.java
@@ -105,6 +105,10 @@ public class VersionResponse {
             .addAllKeys(list).build();
   }
 
+  public String getValue(String key) {
+    return this.values.get(key);
+  }
+
   /**
    * Builder class.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java
index b63c5fb..a24f096 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java
@@ -25,16 +25,20 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolService;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol;
 import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolPB;
 import org.apache.hadoop.ozone.protocolPB
     .StorageContainerDatanodeProtocolServerSideTranslatorPB;
+import org.apache.hadoop.test.GenericTestUtils;
 
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
+
 /**
  * Test Endpoint class.
  */
@@ -109,8 +113,13 @@ public final class SCMTestUtils {
     }
   }
 
-  public static Configuration getConf() {
-    return new Configuration();
+  public static OzoneConfiguration getConf() {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    conf.set(HDDS_DATANODE_DIR_KEY, GenericTestUtils
+        .getRandomizedTempPath());
+    conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, GenericTestUtils
+        .getRandomizedTempPath());
+    return conf;
   }
 
   public static OzoneConfiguration getOzoneConf() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
index 14da960..8f4b0e3 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.StorageReportProto;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol;
 import org.apache.hadoop.ozone.protocol.VersionResponse;
 
@@ -151,7 +152,10 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
     return VersionResponse.newBuilder()
         .setVersion(versionInfo.getVersion())
         .addValue(VersionInfo.DESCRIPTION_KEY, versionInfo.getDescription())
+        .addValue(OzoneConsts.SCM_ID, UUID.randomUUID().toString())
+        .addValue(OzoneConsts.CLUSTER_ID, UUID.randomUUID().toString())
         .build().getProtobufMessage();
+
   }
 
   private void sleepIfNeeded() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
index 52f291b..249b0fe 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
@@ -42,8 +42,7 @@ public class TestKeyValueContainerData {
         .ContainerLifeCycleState.CLOSED;
     AtomicLong val = new AtomicLong(0);
 
-    KeyValueContainerData kvData = new KeyValueContainerData(containerType,
-        containerId);
+    KeyValueContainerData kvData = new KeyValueContainerData(containerId);
 
     assertEquals(containerType, kvData.getContainerType());
     assertEquals(containerId, kvData.getContainerId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
new file mode 100644
index 0000000..e1b7bd2
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.impl;
+
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * This class tests create/read .container files.
+ */
+public class TestContainerDataYaml {
+
+  @Test
+  public void testCreateContainerFile() throws IOException {
+    String path = new FileSystemTestHelper().getTestRootDir();
+    String containerPath = "1.container";
+
+    File filePath = new File(new FileSystemTestHelper().getTestRootDir());
+    filePath.mkdirs();
+
+    KeyValueContainerData keyValueContainerData = new KeyValueContainerData(Long.MAX_VALUE);
+    keyValueContainerData.setContainerDBType("RocksDB");
+    keyValueContainerData.setMetadataPath(path);
+    keyValueContainerData.setChunksPath(path);
+
+    File containerFile = new File(filePath, containerPath);
+
+    // Create .container file with ContainerData
+    ContainerDataYaml.createContainerFile(ContainerProtos.ContainerType
+            .KeyValueContainer, containerFile, keyValueContainerData);
+
+    //Check .container file exists or not.
+    assertTrue(containerFile.exists());
+
+    // Read from .container file, and verify data.
+    KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
+        .readContainerFile(containerFile);
+    assertEquals(Long.MAX_VALUE, kvData.getContainerId());
+    assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
+        .getContainerType());
+    assertEquals("RocksDB", kvData.getContainerDBType());
+    assertEquals(path, kvData.getMetadataPath());
+    assertEquals(path, kvData.getChunksPath());
+    assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN, kvData
+        .getState());
+    assertEquals(1, kvData.getLayOutVersion());
+    assertEquals(0, kvData.getMetadata().size());
+
+    // Update ContainerData.
+    kvData.addMetadata("VOLUME", "hdfs");
+    kvData.addMetadata("OWNER", "ozone");
+    kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED);
+
+
+    // Update .container file with new ContainerData.
+    containerFile = new File(filePath, containerPath);
+    ContainerDataYaml.createContainerFile(ContainerProtos.ContainerType
+            .KeyValueContainer, containerFile, kvData);
+
+    // Reading newly updated data from .container file
+    kvData =  (KeyValueContainerData) ContainerDataYaml.readContainerFile(
+        containerFile);
+
+    // verify data.
+    assertEquals(Long.MAX_VALUE, kvData.getContainerId());
+    assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
+        .getContainerType());
+    assertEquals("RocksDB", kvData.getContainerDBType());
+    assertEquals(path, kvData.getMetadataPath());
+    assertEquals(path, kvData.getChunksPath());
+    assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, kvData
+        .getState());
+    assertEquals(1, kvData.getLayOutVersion());
+    assertEquals(2, kvData.getMetadata().size());
+    assertEquals("hdfs", kvData.getMetadata().get("VOLUME"));
+    assertEquals("ozone", kvData.getMetadata().get("OWNER"));
+
+    FileUtil.fullyDelete(filePath);
+
+
+  }
+
+  @Test
+  public void testIncorrectContainerFile() throws IOException{
+    try {
+      String path = "incorrect.container";
+      //Get file from resources folder
+      ClassLoader classLoader = getClass().getClassLoader();
+      File file = new File(classLoader.getResource(path).getFile());
+      KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
+          .readContainerFile(file);
+      fail("testIncorrectContainerFile failed");
+    } catch (IllegalStateException ex) {
+      GenericTestUtils.assertExceptionContains("Unexpected " +
+          "ContainerLifeCycleState", ex);
+    }
+  }
+
+
+  @Test
+  public void testCheckBackWardCompatabilityOfContainerFile() throws
+      IOException {
+    // This test is for if we upgrade, and then .container files added by new
+    // server will have new fields added to .container file, after a while we
+    // decided to rollback. Then older ozone can read .container files
+    // created or not.
+
+    try {
+      String path = "additionalfields.container";
+      //Get file from resources folder
+      ClassLoader classLoader = getClass().getClassLoader();
+      File file = new File(classLoader.getResource(path).getFile());
+      KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
+          .readContainerFile(file);
+
+      //Checking the Container file data is consistent or not
+      assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, kvData
+          .getState());
+      assertEquals("RocksDB", kvData.getContainerDBType());
+      assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
+          .getContainerType());
+      assertEquals(9223372036854775807L, kvData.getContainerId());
+      assertEquals("/hdds/current/aed-fg4-hji-jkl/containerdir0/1", kvData
+          .getChunksPath());
+      assertEquals("/hdds/current/aed-fg4-hji-jkl/containerdir0/1", kvData
+          .getMetadataPath());
+      assertEquals(1, kvData.getLayOutVersion());
+      assertEquals(2, kvData.getMetadata().size());
+
+    } catch (Exception ex) {
+      fail("testCheckBackWardCompatabilityOfContainerFile failed");
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
index 5a29e8a..55d6773 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
@@ -53,8 +53,7 @@ public class TestContainerSet {
     ContainerProtos.ContainerLifeCycleState state = ContainerProtos
         .ContainerLifeCycleState.CLOSED;
 
-    KeyValueContainerData kvData = new KeyValueContainerData(
-        ContainerProtos.ContainerType.KeyValueContainer, containerId);
+    KeyValueContainerData kvData = new KeyValueContainerData(containerId);
     kvData.setState(state);
     KeyValueContainer keyValueContainer = new KeyValueContainer(kvData, new
         OzoneConfiguration());
@@ -164,8 +163,7 @@ public class TestContainerSet {
   private ContainerSet createContainerSet() throws StorageContainerException {
     ContainerSet containerSet = new ContainerSet();
     for (int i=0; i<10; i++) {
-      KeyValueContainerData kvData = new KeyValueContainerData(
-          ContainerProtos.ContainerType.KeyValueContainer, i);
+      KeyValueContainerData kvData = new KeyValueContainerData(i);
       if (i%2 == 0) {
         kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED);
       } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java
deleted file mode 100644
index 75c0139..0000000
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.impl;
-
-import org.apache.hadoop.fs.FileSystemTestHelper;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
-import org.apache.hadoop.ozone.container.keyvalue.KeyValueYaml;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * This class tests create/read .container files.
- */
-public class TestKeyValueYaml {
-
-  @Test
-  public void testCreateContainerFile() throws IOException {
-    String path = new FileSystemTestHelper().getTestRootDir();
-    String containerPath = "1.container";
-
-    File filePath = new File(new FileSystemTestHelper().getTestRootDir());
-    filePath.mkdirs();
-
-    KeyValueContainerData keyValueContainerData = new KeyValueContainerData(
-        ContainerProtos.ContainerType.KeyValueContainer, Long.MAX_VALUE);
-    keyValueContainerData.setContainerDBType("RocksDB");
-    keyValueContainerData.setMetadataPath(path);
-    keyValueContainerData.setChunksPath(path);
-
-    File containerFile = new File(filePath, containerPath);
-
-    // Create .container file with ContainerData
-    KeyValueYaml.createContainerFile(containerFile, keyValueContainerData);
-
-    //Check .container file exists or not.
-    assertTrue(containerFile.exists());
-
-    // Read from .container file, and verify data.
-    KeyValueContainerData kvData = KeyValueYaml.readContainerFile(
-        containerFile);
-    assertEquals(Long.MAX_VALUE, kvData.getContainerId());
-    assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
-        .getContainerType());
-    assertEquals("RocksDB", kvData.getContainerDBType());
-    assertEquals(path, kvData.getMetadataPath());
-    assertEquals(path, kvData.getChunksPath());
-    assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN, kvData
-        .getState());
-    assertEquals(1, kvData.getLayOutVersion());
-    assertEquals(0, kvData.getMetadata().size());
-
-    // Update ContainerData.
-    kvData.addMetadata("VOLUME", "hdfs");
-    kvData.addMetadata("OWNER", "ozone");
-    kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED);
-
-
-    // Update .container file with new ContainerData.
-    containerFile = new File(filePath, containerPath);
-    KeyValueYaml.createContainerFile(containerFile, kvData);
-
-    // Reading newly updated data from .container file
-    kvData =  KeyValueYaml.readContainerFile(containerFile);
-
-    // verify data.
-    assertEquals(Long.MAX_VALUE, kvData.getContainerId());
-    assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
-        .getContainerType());
-    assertEquals("RocksDB", kvData.getContainerDBType());
-    assertEquals(path, kvData.getMetadataPath());
-    assertEquals(path, kvData.getChunksPath());
-    assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, kvData
-        .getState());
-    assertEquals(1, kvData.getLayOutVersion());
-    assertEquals(2, kvData.getMetadata().size());
-    assertEquals("hdfs", kvData.getMetadata().get("VOLUME"));
-    assertEquals("ozone", kvData.getMetadata().get("OWNER"));
-
-    FileUtil.fullyDelete(filePath);
-
-
-  }
-
-  @Test
-  public void testIncorrectContainerFile() throws IOException{
-    try {
-      String path = "incorrect.container";
-      //Get file from resources folder
-      ClassLoader classLoader = getClass().getClassLoader();
-      File file = new File(classLoader.getResource(path).getFile());
-      KeyValueContainerData kvData = KeyValueYaml.readContainerFile(file);
-      fail("testIncorrectContainerFile failed");
-    } catch (IllegalStateException ex) {
-      GenericTestUtils.assertExceptionContains("Unexpected " +
-          "ContainerLifeCycleState", ex);
-    }
-  }
-
-
-  @Test
-  public void testCheckBackWardCompatabilityOfContainerFile() throws
-      IOException {
-    // This test is for if we upgrade, and then .container files added by new
-    // server will have new fields added to .container file, after a while we
-    // decided to rollback. Then older ozone can read .container files
-    // created or not.
-
-    try {
-      String path = "additionalfields.container";
-      //Get file from resources folder
-      ClassLoader classLoader = getClass().getClassLoader();
-      File file = new File(classLoader.getResource(path).getFile());
-      KeyValueContainerData kvData = KeyValueYaml.readContainerFile(file);
-
-      //Checking the Container file data is consistent or not
-      assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, kvData
-          .getState());
-      assertEquals("RocksDB", kvData.getContainerDBType());
-      assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
-          .getContainerType());
-      assertEquals(9223372036854775807L, kvData.getContainerId());
-      assertEquals("/hdds/current/aed-fg4-hji-jkl/containerdir0/1", kvData
-          .getChunksPath());
-      assertEquals("/hdds/current/aed-fg4-hji-jkl/containerdir0/1", kvData
-          .getMetadataPath());
-      assertEquals(1, kvData.getLayOutVersion());
-      assertEquals(2, kvData.getMetadata().size());
-
-    } catch (Exception ex) {
-      fail("testCheckBackWardCompatabilityOfContainerFile failed");
-    }
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
index 50927d1..6660e9b 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
@@ -52,7 +52,6 @@ public class TestHandler {
   private VolumeSet volumeSet;
   private Handler handler;
 
-  private final static String SCM_ID = UUID.randomUUID().toString();
   private final static String DATANODE_UUID = UUID.randomUUID().toString();
 
   @Before
@@ -61,12 +60,12 @@ public class TestHandler {
     this.containerSet = Mockito.mock(ContainerSet.class);
     this.volumeSet = Mockito.mock(VolumeSet.class);
 
-    this.dispatcher = new HddsDispatcher(conf, containerSet, volumeSet, SCM_ID);
+    this.dispatcher = new HddsDispatcher(conf, containerSet, volumeSet);
   }
 
   @Test
   public void testGetKeyValueHandler() throws Exception {
-    Handler kvHandler = dispatcher.getHandlerForContainerType(
+    Handler kvHandler = dispatcher.getHandler(
         ContainerProtos.ContainerType.KeyValueContainer);
 
     Assert.assertTrue("getHandlerForContainerType returned incorrect handler",
@@ -83,8 +82,7 @@ public class TestHandler {
     Assert.assertEquals("New ContainerType detected. Not an invalid " +
         "containerType", invalidContainerType, null);
 
-    Handler handler = dispatcher.getHandlerForContainerType(
-        invalidContainerType);
+    Handler handler = dispatcher.getHandler(invalidContainerType);
     Assert.assertEquals("Get Handler for Invalid ContainerType should " +
         "return null.", handler, null);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
index 4576db6..272bdb9 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
@@ -81,8 +81,7 @@ public class TestChunkManagerImpl {
     Mockito.when(volumeChoosingPolicy.chooseVolume(anyList(), anyLong()))
         .thenReturn(hddsVolume);
 
-    keyValueContainerData = new KeyValueContainerData(
-        ContainerProtos.ContainerType.KeyValueContainer, 1L);
+    keyValueContainerData = new KeyValueContainerData(1L);
 
     keyValueContainer = new KeyValueContainer(
         keyValueContainerData, config);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
index 722cece..fa7c66d 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
@@ -79,8 +79,7 @@ public class TestKeyManagerImpl {
     Mockito.when(volumeChoosingPolicy.chooseVolume(anyList(), anyLong()))
         .thenReturn(hddsVolume);
 
-    keyValueContainerData = new KeyValueContainerData(
-        ContainerProtos.ContainerType.KeyValueContainer, 1L);
+    keyValueContainerData = new KeyValueContainerData(1L);
 
     keyValueContainer = new KeyValueContainer(
         keyValueContainerData, config);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
index 006b82c..de5f432 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume
     .RoundRobinVolumeChoosingPolicy;
@@ -85,8 +86,7 @@ public class TestKeyValueContainer {
     Mockito.when(volumeChoosingPolicy.chooseVolume(anyList(), anyLong()))
         .thenReturn(hddsVolume);
 
-    keyValueContainerData = new KeyValueContainerData(
-        ContainerProtos.ContainerType.KeyValueContainer, 1L);
+    keyValueContainerData = new KeyValueContainerData(1L);
 
     keyValueContainer = new KeyValueContainer(
         keyValueContainerData, conf);
@@ -197,7 +197,8 @@ public class TestKeyValueContainer {
     File containerFile = KeyValueContainerLocationUtil.getContainerFile(
         containerMetaDataLoc, containerName);
 
-    keyValueContainerData = KeyValueYaml.readContainerFile(containerFile);
+    keyValueContainerData = (KeyValueContainerData) ContainerDataYaml
+        .readContainerFile(containerFile);
     assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED,
         keyValueContainerData.getState());
   }
@@ -237,7 +238,8 @@ public class TestKeyValueContainer {
     File containerFile = KeyValueContainerLocationUtil.getContainerFile(
         containerMetaDataLoc, containerName);
 
-    keyValueContainerData = KeyValueYaml.readContainerFile(containerFile);
+    keyValueContainerData = (KeyValueContainerData) ContainerDataYaml
+        .readContainerFile(containerFile);
     assertEquals(2, keyValueContainerData.getMetadata().size());
 
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
index f4dd41c..dbddf47 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
@@ -74,9 +74,10 @@ public class TestKeyValueHandler {
         .build();
     this.volumeSet = new VolumeSet(datanodeDetails, conf);
 
-    this.dispatcher = new HddsDispatcher(conf, containerSet, volumeSet, SCM_ID);
-    this.handler = (KeyValueHandler) dispatcher.getHandlerForContainerType(
+    this.dispatcher = new HddsDispatcher(conf, containerSet, volumeSet);
+    this.handler = (KeyValueHandler) dispatcher.getHandler(
         ContainerProtos.ContainerType.KeyValueContainer);
+    dispatcher.setScmId(UUID.randomUUID().toString());
   }
 
   @Test
@@ -87,8 +88,7 @@ public class TestKeyValueHandler {
     // Create mock HddsDispatcher and KeyValueHandler.
     this.handler = Mockito.mock(KeyValueHandler.class);
     this.dispatcher = Mockito.mock(HddsDispatcher.class);
-    Mockito.when(dispatcher.getHandlerForContainerType(any())).thenReturn
-        (handler);
+    Mockito.when(dispatcher.getHandler(any())).thenReturn(handler);
     Mockito.when(dispatcher.dispatch(any())).thenCallRealMethod();
     Mockito.when(dispatcher.getContainer(anyLong())).thenReturn(
         Mockito.mock(KeyValueContainer.class));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
new file mode 100644
index 0000000..cf4bb62
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.ozoneimpl;
+
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import java.util.Random;
+import java.util.UUID;
+
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * This class is used to test OzoneContainer.
+ */
+public class TestOzoneContainer {
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+
+  private OzoneConfiguration conf;
+  private String scmId = UUID.randomUUID().toString();
+  private VolumeSet volumeSet;
+  private RoundRobinVolumeChoosingPolicy volumeChoosingPolicy;
+  private KeyValueContainerData keyValueContainerData;
+  private KeyValueContainer keyValueContainer;
+  private final DatanodeDetails datanodeDetails = createDatanodeDetails();
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new OzoneConfiguration();
+    conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, folder.getRoot()
+        .getAbsolutePath() + "," + folder.newFolder().getAbsolutePath());
+    conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, folder.newFolder().getAbsolutePath());
+    volumeSet = new VolumeSet(datanodeDetails, conf);
+    volumeChoosingPolicy = new RoundRobinVolumeChoosingPolicy();
+
+    for (int i=0; i<10; i++) {
+      keyValueContainerData = new KeyValueContainerData(i);
+      keyValueContainer = new KeyValueContainer(
+          keyValueContainerData, conf);
+      keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
+    }
+  }
+
+  @Test
+  public void testBuildContainerMap() throws Exception {
+    OzoneContainer ozoneContainer = new
+        OzoneContainer(datanodeDetails, conf);
+    ContainerSet containerset = ozoneContainer.getContainerSet();
+    assertEquals(10, containerset.containerCount());
+  }
+
+
+  private DatanodeDetails createDatanodeDetails() {
+    Random random = new Random();
+    String ipAddress =
+        random.nextInt(256) + "." + random.nextInt(256) + "." + random
+            .nextInt(256) + "." + random.nextInt(256);
+
+    String uuid = UUID.randomUUID().toString();
+    String hostName = uuid;
+    DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.STANDALONE, 0);
+    DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.RATIS, 0);
+    DatanodeDetails.Port restPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.REST, 0);
+    DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
+    builder.setUuid(uuid)
+        .setHostName("localhost")
+        .setIpAddress(ipAddress)
+        .addPort(containerPort)
+        .addPort(ratisPort)
+        .addPort(restPort);
+    return builder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
index b339fb7..9ac9930 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.protocol.StorageContainerNodeProtocol;
 import org.apache.hadoop.ozone.protocol.VersionResponse;
 import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
@@ -703,6 +704,9 @@ public class SCMNodeManager
   public VersionResponse getVersion(SCMVersionRequestProto versionRequest) {
     return VersionResponse.newBuilder()
         .setVersion(this.version.getVersion())
+        .addValue(OzoneConsts.SCM_ID, this.scmManager.getScmStorage().getScmId())
+        .addValue(OzoneConsts.CLUSTER_ID, this.scmManager.getScmStorage()
+            .getClusterID())
         .build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
index 34779da..9db9e80 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
@@ -20,6 +20,7 @@ import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.scm.TestUtils;
 import org.apache.hadoop.hdds.scm.VersionInfo;
@@ -125,12 +126,14 @@ public class TestEndPoint {
    * how the state machine would make the call.
    */
   public void testGetVersionTask() throws Exception {
-    Configuration conf = SCMTestUtils.getConf();
+    OzoneConfiguration conf = SCMTestUtils.getConf();
     try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
         serverAddress, 1000)) {
+      OzoneContainer ozoneContainer = new OzoneContainer(getDatanodeDetails(),
+          conf);
       rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
       VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
-          conf);
+          conf, ozoneContainer);
       EndpointStateMachine.EndPointStates newState = versionTask.call();
 
       // if version call worked the endpoint should automatically move to the
@@ -149,14 +152,16 @@ public class TestEndPoint {
    * expect that versionTask should be able to handle it.
    */
   public void testGetVersionToInvalidEndpoint() throws Exception {
-    Configuration conf = SCMTestUtils.getConf();
+    OzoneConfiguration conf = SCMTestUtils.getConf();
     InetSocketAddress nonExistentServerAddress = SCMTestUtils
         .getReuseableAddress();
     try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
         nonExistentServerAddress, 1000)) {
       rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
-      VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
+      OzoneContainer ozoneContainer = new OzoneContainer(getDatanodeDetails(),
           conf);
+      VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
+          conf, ozoneContainer);
       EndpointStateMachine.EndPointStates newState = versionTask.call();
 
       // This version call did NOT work, so endpoint should remain in the same
@@ -175,13 +180,15 @@ public class TestEndPoint {
   public void testGetVersionAssertRpcTimeOut() throws Exception {
     final long rpcTimeout = 1000;
     final long tolerance = 100;
-    Configuration conf = SCMTestUtils.getConf();
+    OzoneConfiguration conf = SCMTestUtils.getConf();
 
     try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
         serverAddress, (int) rpcTimeout)) {
       rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
-      VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
+      OzoneContainer ozoneContainer = new OzoneContainer(getDatanodeDetails(),
           conf);
+      VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
+          conf, ozoneContainer);
 
       scmServerImpl.setRpcResponseDelay(1500);
       long start = Time.monotonicNow();
@@ -386,4 +393,5 @@ public class TestEndPoint {
     }
     return reportsBuilder.build();
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
index c937980..ad1e706 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
@@ -27,8 +27,10 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
-import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
@@ -163,8 +165,9 @@ public class TestStorageContainerManagerHelper {
     DatanodeDetails leadDN = container.getPipeline().getLeader();
     OzoneContainer containerServer =
         getContainerServerByDatanodeUuid(leadDN.getUuidString());
-    ContainerData containerData = containerServer.getContainerManager()
-        .readContainer(containerID);
+    KeyValueContainerData containerData = (KeyValueContainerData) containerServer
+        .getContainerSet()
+        .getContainer(containerID).getContainerData();
     return KeyUtils.getDB(containerData, conf);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
index 9e8cb46..b832dd2 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.ozone.client.ObjectStore;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientFactory;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
@@ -183,7 +183,7 @@ public class TestCloseContainerByPipeline {
     for (DatanodeDetails datanodeDetails : datanodes) {
       GenericTestUtils.waitFor(
           () -> isContainerClosed(cluster, containerID, datanodeDetails), 500,
-          5 * 1000);
+          15 * 1000);
       //double check if it's really closed (waitFor also throws an exception)
       Assert.assertTrue(isContainerClosed(cluster, containerID, datanodeDetails));
     }
@@ -204,7 +204,7 @@ public class TestCloseContainerByPipeline {
         if (datanode.equals(datanodeService.getDatanodeDetails())) {
           containerData =
               datanodeService.getDatanodeStateMachine().getContainer()
-                  .getContainerManager().readContainer(containerID);
+                  .getContainerSet().getContainer(containerID).getContainerData();
           if (!containerData.isOpen()) {
             // make sure the closeContainerHandler on the Datanode is invoked
             Assert.assertTrue(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
index efb7344..114bd04 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
 import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
@@ -104,8 +104,8 @@ public class TestCloseContainerHandler {
     ContainerData containerData;
     try {
       containerData = cluster.getHddsDatanodes().get(0)
-          .getDatanodeStateMachine().getContainer().getContainerManager()
-          .readContainer(containerID);
+          .getDatanodeStateMachine().getContainer().getContainerSet()
+          .getContainer(containerID).getContainerData();
       return !containerData.isOpen();
     } catch (StorageContainerException e) {
       throw new AssertionError(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
index 3f02036..18b325b 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
@@ -35,10 +35,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
 
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 import java.util.concurrent.CompletableFuture;
 
 /**
@@ -66,7 +63,11 @@ public class TestOzoneContainer {
       conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, pipeline.getLeader()
               .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
       conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, false);
-      container = new OzoneContainer(TestUtils.getDatanodeDetails(), conf);
+
+      container = new OzoneContainer(TestUtils.getDatanodeDetails(),
+          conf);
+      //Setting scmId, as we start manually ozone container.
+      container.getDispatcher().setScmId(UUID.randomUUID().toString());
       container.start();
 
       XceiverClient client = new XceiverClient(pipeline, conf);
@@ -392,7 +393,7 @@ public class TestOzoneContainer {
       response = client.sendCommand(request);
 
       Assert.assertNotNull(response);
-      Assert.assertEquals(ContainerProtos.Result.UNCLOSED_CONTAINER_IO,
+      Assert.assertEquals(ContainerProtos.Result.DELETE_ON_OPEN_CONTAINER,
           response.getResult());
       Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
index d4c572f..bd9259d 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.container.server;
 
+import org.apache.hadoop.ozone.container.common.interfaces.Handler;
 import org.apache.ratis.shaded.io.netty.channel.embedded.EmbeddedChannel;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
@@ -262,5 +263,14 @@ public class TestContainerServer {
     @Override
     public void shutdown() {
     }
+    @Override
+    public Handler getHandler(ContainerProtos.ContainerType containerType) {
+      return null;
+    }
+
+    @Override
+    public void setScmId(String scmId) {
+
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java
index bafba32..adce0ef 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java
@@ -27,8 +27,8 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.*;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
@@ -119,8 +119,8 @@ public class TestContainerReportWithKeys {
 
     ContainerData cd = getContainerData(keyInfo.getContainerID());
 
-    LOG.info("DN Container Data:  keyCount: {} used: {} ",
-        cd.getKeyCount(), cd.getBytesUsed());
+/*    LOG.info("DN Container Data:  keyCount: {} used: {} ",
+        cd.getKeyCount(), cd.getBytesUsed());*/
 
     ContainerInfo cinfo = scm.getContainerInfo(keyInfo.getContainerID());
 
@@ -132,9 +132,9 @@ public class TestContainerReportWithKeys {
   private static ContainerData getContainerData(long containerID) {
     ContainerData containerData;
     try {
-      ContainerManager containerManager = cluster.getHddsDatanodes().get(0)
-          .getDatanodeStateMachine().getContainer().getContainerManager();
-      containerData = containerManager.readContainer(containerID);
+      ContainerSet containerManager = cluster.getHddsDatanodes().get(0)
+          .getDatanodeStateMachine().getContainer().getContainerSet();
+      containerData = containerManager.getContainer(containerID).getContainerData();
     } catch (StorageContainerException e) {
       throw new AssertionError(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
index b86c577..cda54cb 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
@@ -44,9 +44,10 @@ import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.client.rpc.RpcClient;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.ozone.ksm.KeySpaceManager;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
@@ -698,13 +699,16 @@ public class TestKeys {
         List<KsmKeyLocationInfo> locations =
             keyInfo.getLatestVersionLocations().getLocationList();
         for (KsmKeyLocationInfo location : locations) {
-          KeyData keyData = new KeyData(location.getBlockID());
-          KeyData blockInfo = cm.getContainerManager()
-              .getKeyManager().getKey(keyData);
-          ContainerData containerData = cm.getContainerManager()
-              .readContainer(keyData.getContainerID());
-          File dataDir = ContainerUtils
-              .getDataDirectory(containerData).toFile();
+          KeyValueHandler  keyValueHandler = (KeyValueHandler) cm
+              .getDispatcher().getHandler(ContainerProtos.ContainerType
+                  .KeyValueContainer);
+          KeyValueContainer container = (KeyValueContainer) cm.getContainerSet()
+              .getContainer(location.getBlockID().getContainerID());
+          KeyData blockInfo = keyValueHandler
+              .getKeyManager().getKey(container, location.getBlockID());
+          KeyValueContainerData containerData = (KeyValueContainerData) container
+              .getContainerData();
+          File dataDir = new File(containerData.getChunksPath());
           for (ContainerProtos.ChunkInfo chunkInfo : blockInfo.getChunks()) {
             File chunkFile = dataDir.toPath()
                 .resolve(chunkInfo.getChunkName()).toFile();


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


[37/37] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDDS-48

Posted by bh...@apache.org.
Merge remote-tracking branch 'apache/trunk' into HDDS-48


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

Branch: refs/heads/trunk
Commit: 9bd5bef297b036b19f7be0c42c5477808ef8c070
Parents: 3584baf 2403231
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Jul 9 13:22:58 2018 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Jul 9 13:22:58 2018 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/main/conf/hadoop-env.sh   |   6 +-
 .../src/main/conf/hadoop-metrics2.properties    |   2 +-
 .../crypto/key/kms/KMSClientProvider.java       |   4 +-
 .../src/main/conf/kms-log4j.properties          |   4 +-
 .../src/test/resources/log4j.properties         |   4 +-
 hadoop-hdds/framework/pom.xml                   |   5 +
 .../hadoop/hdds/server/events/EventQueue.java   | 108 ++++++++------
 .../hadoop/hdds/server/events/EventWatcher.java |  43 +++++-
 .../hdds/server/events/EventWatcherMetrics.java |  79 ++++++++++
 .../server/events/SingleThreadExecutor.java     |  35 +++--
 .../hdds/server/events/TestEventQueue.java      |  35 +----
 .../hdds/server/events/TestEventWatcher.java    | 107 ++++++++++++--
 .../hadoop/yarn/client/AMRMClientUtils.java     |  91 ------------
 .../hadoop/yarn/server/AMRMClientRelayer.java   |   9 +-
 .../yarn/server/uam/UnmanagedAMPoolManager.java |  16 ++
 .../server/uam/UnmanagedApplicationManager.java |  40 ++---
 .../yarn/server/MockResourceManagerFacade.java  |  13 +-
 .../amrmproxy/FederationInterceptor.java        | 146 ++++++++++++++++---
 .../amrmproxy/BaseAMRMProxyTest.java            |   2 +
 .../amrmproxy/TestFederationInterceptor.java    |  17 +++
 20 files changed, 515 insertions(+), 251 deletions(-)
----------------------------------------------------------------------



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


[24/37] hadoop git commit: HDDS-205. Add metrics to HddsDispatcher. Contributed by Bharat Viswanadham.

Posted by bh...@apache.org.
HDDS-205. Add metrics to HddsDispatcher. Contributed by Bharat Viswanadham.


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

Branch: refs/heads/trunk
Commit: 44b091a8d776db954db1f9ddbba91f7b9445c8ec
Parents: 23ebe0f
Author: Hanisha Koneru <ha...@apache.org>
Authored: Tue Jul 3 09:50:25 2018 -0700
Committer: Hanisha Koneru <ha...@apache.org>
Committed: Tue Jul 3 09:50:25 2018 -0700

----------------------------------------------------------------------
 .../common/helpers/ContainerMetrics.java        |  2 +-
 .../ozone/container/common/impl/Dispatcher.java |  2 +-
 .../container/common/impl/HddsDispatcher.java   | 24 ++++++--
 .../container/common/interfaces/Handler.java    | 14 +++--
 .../container/keyvalue/KeyValueHandler.java     | 41 ++++++++++++--
 .../container/keyvalue/TestKeyValueHandler.java | 13 ++++-
 .../container/metrics/TestContainerMetrics.java | 59 +++++++++++---------
 7 files changed, 110 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/44b091a8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java
index 714db59..2879001 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java
@@ -94,7 +94,7 @@ public class ContainerMetrics {
                        new ContainerMetrics(intervals));
   }
 
-  public void incContainerOpcMetrics(ContainerProtos.Type type){
+  public void incContainerOpsMetrics(ContainerProtos.Type type) {
     numOps.incr();
     numOpsArray[type.ordinal()].incr();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44b091a8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
index c485caf..3ffe6e4 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
@@ -111,7 +111,7 @@ public class Dispatcher implements ContainerDispatcher {
     try {
       Preconditions.checkNotNull(msg);
       Type cmdType = msg.getCmdType();
-      metrics.incContainerOpcMetrics(cmdType);
+      metrics.incContainerOpsMetrics(cmdType);
       if ((cmdType == Type.CreateContainer) ||
           (cmdType == Type.DeleteContainer) ||
           (cmdType == Type.ReadContainer) ||

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44b091a8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
index cbb48ec..25700f9 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
@@ -23,6 +23,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.common.interfaces.Handler;
@@ -53,6 +54,7 @@ public class HddsDispatcher implements ContainerDispatcher {
   private final ContainerSet containerSet;
   private final VolumeSet volumeSet;
   private String scmID;
+  private ContainerMetrics metrics;
 
   /**
    * Constructs an OzoneContainer that receives calls from
@@ -60,16 +62,17 @@ public class HddsDispatcher implements ContainerDispatcher {
    */
   public HddsDispatcher(Configuration config, ContainerSet contSet,
       VolumeSet volumes) {
-    //TODO: initialize metrics
     this.conf = config;
     this.containerSet = contSet;
     this.volumeSet = volumes;
     this.handlers = Maps.newHashMap();
+    this.metrics = ContainerMetrics.create(conf);
     for (ContainerType containerType : ContainerType.values()) {
       handlers.put(containerType,
           Handler.getHandlerForContainerType(
-              containerType, conf, containerSet, volumeSet));
+              containerType, conf, containerSet, volumeSet, metrics));
     }
+
   }
 
   @Override
@@ -89,10 +92,14 @@ public class HddsDispatcher implements ContainerDispatcher {
 
     Container container = null;
     ContainerType containerType = null;
+    ContainerCommandResponseProto responseProto = null;
+    long startTime = System.nanoTime();
+    ContainerProtos.Type cmdType = msg.getCmdType();
     try {
       long containerID = getContainerID(msg);
 
-      if (msg.getCmdType() != ContainerProtos.Type.CreateContainer) {
+      metrics.incContainerOpsMetrics(cmdType);
+      if (cmdType != ContainerProtos.Type.CreateContainer) {
         container = getContainer(containerID);
         containerType = getContainerType(container);
       } else {
@@ -109,7 +116,11 @@ public class HddsDispatcher implements ContainerDispatcher {
           ContainerProtos.Result.CONTAINER_INTERNAL_ERROR);
       return ContainerUtils.logAndReturnError(LOG, ex, msg);
     }
-    return handler.handle(msg, container);
+    responseProto = handler.handle(msg, container);
+    if (responseProto != null) {
+      metrics.incContainerOpsLatencies(cmdType, System.nanoTime() - startTime);
+    }
+    return responseProto;
   }
 
   @Override
@@ -187,4 +198,9 @@ public class HddsDispatcher implements ContainerDispatcher {
   private ContainerType getContainerType(Container container) {
     return container.getContainerType();
   }
+
+  @VisibleForTesting
+  public void setMetricsForTesting(ContainerMetrics containerMetrics) {
+    this.metrics = containerMetrics;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44b091a8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
index 8069d71..57dd224 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.ozone.container.common.interfaces;
 
-import com.sun.jersey.spi.resource.Singleton;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
@@ -26,7 +26,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandResponseProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerType;
-import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
@@ -42,19 +42,22 @@ public class Handler {
   protected final ContainerSet containerSet;
   protected final VolumeSet volumeSet;
   protected String scmID;
+  protected final ContainerMetrics metrics;
 
   protected Handler(Configuration config, ContainerSet contSet,
-      VolumeSet volumeSet) {
+      VolumeSet volumeSet, ContainerMetrics containerMetrics) {
     conf = config;
     containerSet = contSet;
     this.volumeSet = volumeSet;
+    this.metrics = containerMetrics;
   }
 
   public static Handler getHandlerForContainerType(ContainerType containerType,
-      Configuration config, ContainerSet contSet, VolumeSet volumeSet) {
+      Configuration config, ContainerSet contSet, VolumeSet volumeSet,
+                                                   ContainerMetrics metrics) {
     switch (containerType) {
     case KeyValueContainer:
-      return KeyValueHandler.getInstance(config, contSet, volumeSet);
+      return KeyValueHandler.getInstance(config, contSet, volumeSet, metrics);
     default:
       throw new IllegalArgumentException("Handler for ContainerType: " +
         containerType + "doesn't exist.");
@@ -69,4 +72,5 @@ public class Handler {
   public void setScmID(String scmId) {
     this.scmID = scmId;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44b091a8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index ffe0f21..d174383 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.SmallFileUtils;
@@ -71,7 +72,20 @@ import java.util.List;
 import java.util.Map;
 
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.*;
+    .Result.CONTAINER_INTERNAL_ERROR;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.CLOSED_CONTAINER_IO;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.DELETE_ON_OPEN_CONTAINER;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.IO_EXCEPTION;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.INVALID_CONTAINER_STATE;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.GET_SMALL_FILE_ERROR;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.PUT_SMALL_FILE_ERROR;
+
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Stage;
 
@@ -94,16 +108,18 @@ public class KeyValueHandler extends Handler {
   // TODO : Add metrics and populate it.
 
   public static KeyValueHandler getInstance(Configuration config,
-      ContainerSet contSet, VolumeSet volSet) {
+                                            ContainerSet contSet,
+                                            VolumeSet volSet,
+                                            ContainerMetrics metrics) {
     if (INSTANCE == null) {
-      INSTANCE = new KeyValueHandler(config, contSet, volSet);
+      INSTANCE = new KeyValueHandler(config, contSet, volSet, metrics);
     }
     return INSTANCE;
   }
 
   private KeyValueHandler(Configuration config, ContainerSet contSet,
-      VolumeSet volSet) {
-    super(config, contSet, volSet);
+      VolumeSet volSet, ContainerMetrics metrics) {
+    super(config, contSet, volSet, metrics);
     containerType = ContainerType.KeyValueContainer;
     keyManager = new KeyManagerImpl(config);
     chunkManager = new ChunkManagerImpl();
@@ -342,6 +358,8 @@ public class KeyValueHandler extends Handler {
       Preconditions.checkNotNull(keyData);
 
       keyManager.putKey(kvContainer, keyData);
+      long numBytes = keyData.getProtoBufMessage().toByteArray().length;
+      metrics.incContainerBytesStats(Type.PutKey, numBytes);
     } catch (StorageContainerException ex) {
       return ContainerUtils.logAndReturnError(LOG, ex, request);
     } catch (IOException ex) {
@@ -370,6 +388,8 @@ public class KeyValueHandler extends Handler {
       BlockID blockID = BlockID.getFromProtobuf(
           request.getGetKey().getBlockID());
       responseData = keyManager.getKey(kvContainer, blockID);
+      long numBytes = responseData.getProtoBufMessage().toByteArray().length;
+      metrics.incContainerBytesStats(Type.GetKey, numBytes);
 
     } catch (StorageContainerException ex) {
       return ContainerUtils.logAndReturnError(LOG, ex, request);
@@ -434,6 +454,7 @@ public class KeyValueHandler extends Handler {
       Preconditions.checkNotNull(chunkInfo);
 
       data = chunkManager.readChunk(kvContainer, blockID, chunkInfo);
+      metrics.incContainerBytesStats(Type.ReadChunk, data.length);
     } catch (StorageContainerException ex) {
       return ContainerUtils.logAndReturnError(LOG, ex, request);
     } catch (IOException ex) {
@@ -507,6 +528,13 @@ public class KeyValueHandler extends Handler {
 
       chunkManager.writeChunk(kvContainer, blockID, chunkInfo, data,
           request.getWriteChunk().getStage());
+
+      // We should increment stats after writeChunk
+      if (request.getWriteChunk().getStage() == Stage.WRITE_DATA ||
+          request.getWriteChunk().getStage() == Stage.COMBINED) {
+        metrics.incContainerBytesStats(Type.WriteChunk, request.getWriteChunk()
+            .getChunkData().getLen());
+      }
     } catch (StorageContainerException ex) {
       return ContainerUtils.logAndReturnError(LOG, ex, request);
     } catch (IOException ex) {
@@ -555,6 +583,7 @@ public class KeyValueHandler extends Handler {
       chunks.add(chunkInfo.getProtoBufMessage());
       keyData.setChunks(chunks);
       keyManager.putKey(kvContainer, keyData);
+      metrics.incContainerBytesStats(Type.PutSmallFile, data.length);
 
     } catch (StorageContainerException ex) {
       return ContainerUtils.logAndReturnError(LOG, ex, request);
@@ -597,7 +626,7 @@ public class KeyValueHandler extends Handler {
         dataBuf = dataBuf.concat(current);
         chunkInfo = chunk;
       }
-
+      metrics.incContainerBytesStats(Type.GetSmallFile, dataBuf.size());
       return SmallFileUtils.getGetSmallFileResponseSuccess(request, dataBuf
           .toByteArray(), ChunkInfo.getFromProtoBuf(chunkInfo));
     } catch (StorageContainerException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44b091a8/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
index dbddf47..947ad51 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
@@ -34,11 +35,14 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestRule;
 import org.junit.rules.Timeout;
+
+import org.mockito.Mockito;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyLong;
-import org.mockito.Mockito;
+import static org.mockito.Mockito.doCallRealMethod;
 import static org.mockito.Mockito.times;
 
+
 import java.util.UUID;
 
 /**
@@ -93,10 +97,13 @@ public class TestKeyValueHandler {
     Mockito.when(dispatcher.getContainer(anyLong())).thenReturn(
         Mockito.mock(KeyValueContainer.class));
     Mockito.when(handler.handle(any(), any())).thenCallRealMethod();
+    doCallRealMethod().when(dispatcher).setMetricsForTesting(any());
+    dispatcher.setMetricsForTesting(Mockito.mock(ContainerMetrics.class));
 
     // Test Create Container Request handling
     ContainerCommandRequestProto createContainerRequest =
         getDummyCommandRequestProto(ContainerProtos.Type.CreateContainer);
+
     dispatcher.dispatch(createContainerRequest);
     Mockito.verify(handler, times(1)).handleCreateContainer(
         any(ContainerCommandRequestProto.class), any());
@@ -207,8 +214,8 @@ public class TestKeyValueHandler {
         any(ContainerCommandRequestProto.class), any());
   }
 
-  private ContainerCommandRequestProto getDummyCommandRequestProto
-      (ContainerProtos.Type cmdType) {
+  private ContainerCommandRequestProto getDummyCommandRequestProto(
+      ContainerProtos.Type cmdType) {
     ContainerCommandRequestProto request =
         ContainerProtos.ContainerCommandRequestProto.newBuilder()
             .setCmdType(cmdType)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44b091a8/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
index ccad6f8..ef4b423 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
@@ -20,9 +20,10 @@ package org.apache.hadoop.ozone.container.metrics;
 import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
 import static org.apache.hadoop.test.MetricsAsserts.assertQuantileGauges;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
-import static org.mockito.Mockito.mock;
 
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@@ -34,18 +35,19 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
-import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
-import org.apache.hadoop.ozone.container.common.impl.Dispatcher;
-import org.apache.hadoop.ozone.container.common.interfaces.ChunkManager;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.common.transport.server.XceiverServer;
 import org.apache.hadoop.hdds.scm.TestUtils;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.hdds.scm.XceiverClient;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Mockito;
+
+import java.io.File;
+import java.util.UUID;
 
 /**
  * Test for metrics published by storage containers.
@@ -57,7 +59,7 @@ public class TestContainerMetrics {
     XceiverServer server = null;
     XceiverClient client = null;
     long containerID = ContainerTestHelper.getTestContainerID();
-    String keyName = OzoneUtils.getRequestID();
+    String path = GenericTestUtils.getRandomizedTempPath();
 
     try {
       final int interval = 1;
@@ -70,22 +72,14 @@ public class TestContainerMetrics {
       conf.setInt(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY,
           interval);
 
-      // Since we are only testing for container metrics and we can just
-      // mock the ContainerManager and ChunkManager instances instead of
-      // starting the whole cluster.
-      ContainerManager containerManager = mock(ContainerManager.class);
-      ChunkManager chunkManager = mock(ChunkManager.class);
-      Mockito.doNothing().when(chunkManager).writeChunk(
-          Mockito.any(BlockID.class),
-          Mockito.any(ChunkInfo.class), Mockito.any(byte[].class),
-          Mockito.any(ContainerProtos.Stage.class));
-
-      Mockito.doReturn(chunkManager).when(containerManager).getChunkManager();
-      Mockito.doReturn(true).when(containerManager).isOpen(containerID);
-
-      Dispatcher dispatcher = new Dispatcher(containerManager, conf);
-      dispatcher.init();
       DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
+      conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, path);
+      VolumeSet volumeSet = new VolumeSet(datanodeDetails, conf);
+      ContainerSet containerSet = new ContainerSet();
+      HddsDispatcher dispatcher = new HddsDispatcher(conf, containerSet,
+          volumeSet);
+      dispatcher.setScmId(UUID.randomUUID().toString());
+
       server = new XceiverServer(datanodeDetails, conf, dispatcher);
       client = new XceiverClient(pipeline, conf);
 
@@ -102,6 +96,8 @@ public class TestContainerMetrics {
 
       // Write Chunk
       BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
+      ContainerTestHelper.getWriteChunkRequest(
+          pipeline, blockID, 1024);
       ContainerProtos.ContainerCommandRequestProto writeChunkRequest =
           ContainerTestHelper.getWriteChunkRequest(
               pipeline, blockID, 1024);
@@ -109,13 +105,21 @@ public class TestContainerMetrics {
       Assert.assertEquals(ContainerProtos.Result.SUCCESS,
           response.getResult());
 
+      //Read Chunk
+      ContainerProtos.ContainerCommandRequestProto readChunkRequest =
+          ContainerTestHelper.getReadChunkRequest(pipeline, writeChunkRequest
+              .getWriteChunk());
+      response = client.sendCommand(readChunkRequest);
+      Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
+
       MetricsRecordBuilder containerMetrics = getMetrics(
           "StorageContainerMetrics");
-      assertCounter("NumOps", 2L, containerMetrics);
+      assertCounter("NumOps", 3L, containerMetrics);
       assertCounter("numCreateContainer", 1L, containerMetrics);
       assertCounter("numWriteChunk", 1L, containerMetrics);
+      assertCounter("numReadChunk", 1L, containerMetrics);
       assertCounter("bytesWriteChunk", 1024L, containerMetrics);
-      assertCounter("LatencyWriteChunkNumOps", 1L, containerMetrics);
+      assertCounter("bytesReadChunk", 1024L, containerMetrics);
 
       String sec = interval + "s";
       Thread.sleep((interval + 1) * 1000);
@@ -127,6 +131,11 @@ public class TestContainerMetrics {
       if (server != null) {
         server.stop();
       }
+      // clean up volume dir
+      File file = new File(path);
+      if(file.exists()) {
+        FileUtil.fullyDelete(file);
+      }
     }
   }
 }
\ No newline at end of file


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


[15/37] hadoop git commit: HDDS-160:Refactor KeyManager, ChunkManager. Contributed by Bharat Viswanadham

Posted by bh...@apache.org.
HDDS-160:Refactor KeyManager, ChunkManager. Contributed by Bharat Viswanadham


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

Branch: refs/heads/trunk
Commit: ca192cb7c9d76163391e88461716938fdb41c4d3
Parents: 998e285
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Fri Jun 15 14:35:33 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Fri Jun 15 14:35:33 2018 -0700

----------------------------------------------------------------------
 .../common/interfaces/ChunkManager.java         |   2 +-
 .../container/keyvalue/ChunkManagerImpl.java    | 240 +++++++++++++++
 .../container/keyvalue/KeyManagerImpl.java      | 188 ++++++++++++
 .../container/keyvalue/helpers/ChunkUtils.java  | 295 +++++++++++++++++++
 .../container/keyvalue/helpers/KeyUtils.java    |  37 ++-
 .../keyvalue/interfaces/ChunkManager.java       |  80 +++++
 .../keyvalue/interfaces/KeyManager.java         |  76 +++++
 .../keyvalue/TestChunkManagerImpl.java          | 237 +++++++++++++++
 .../container/keyvalue/TestKeyManagerImpl.java  | 179 +++++++++++
 9 files changed, 1331 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca192cb7/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java
index c58fb9d..9de84da 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java
@@ -70,4 +70,4 @@ public interface ChunkManager {
    */
   void shutdown();
 
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca192cb7/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ChunkManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ChunkManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ChunkManagerImpl.java
new file mode 100644
index 0000000..6ee0fd3
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/ChunkManagerImpl.java
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils;
+import org.apache.hadoop.ozone.container.common.impl.ChunkLayOutVersion;
+import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.security.NoSuchAlgorithmException;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.CONTAINER_INTERNAL_ERROR;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.NO_SUCH_ALGORITHM;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNSUPPORTED_REQUEST;
+
+/**
+ * This class is for performing chunk related operations.
+ */
+public class ChunkManagerImpl implements ChunkManager {
+  static final Logger LOG = LoggerFactory.getLogger(ChunkManagerImpl.class);
+
+  /**
+   * writes a given chunk.
+   *
+   * @param container - Container for the chunk
+   * @param blockID - ID of the block
+   * @param info - ChunkInfo
+   * @param data - data of the chunk
+   * @param stage - Stage of the Chunk operation
+   * @throws StorageContainerException
+   */
+  public void writeChunk(Container container, BlockID blockID, ChunkInfo info,
+                         byte[] data, ContainerProtos.Stage stage)
+      throws StorageContainerException {
+
+    try {
+
+      KeyValueContainerData containerData = (KeyValueContainerData) container
+          .getContainerData();
+
+      File chunkFile = ChunkUtils.validateChunk(containerData, info);
+      File tmpChunkFile = getTmpChunkFile(chunkFile, info);
+
+      LOG.debug("writing chunk:{} chunk stage:{} chunk file:{} tmp chunk file",
+          info.getChunkName(), stage, chunkFile, tmpChunkFile);
+
+      switch (stage) {
+      case WRITE_DATA:
+        // Initially writes to temporary chunk file.
+        ChunkUtils.writeData(tmpChunkFile, info, data);
+        break;
+      case COMMIT_DATA:
+        // commit the data, means move chunk data from temporary chunk file
+        // to actual chunk file.
+        long sizeDiff = tmpChunkFile.length() - chunkFile.length();
+        commitChunk(tmpChunkFile, chunkFile);
+        containerData.incrBytesUsed(sizeDiff);
+        containerData.incrWriteCount();
+        containerData.incrWriteBytes(sizeDiff);
+        break;
+      case COMBINED:
+        // directly write to the chunk file
+        ChunkUtils.writeData(chunkFile, info, data);
+        containerData.incrBytesUsed(info.getLen());
+        containerData.incrWriteCount();
+        containerData.incrWriteBytes(info.getLen());
+        break;
+      default:
+        throw new IOException("Can not identify write operation.");
+      }
+    } catch (StorageContainerException ex) {
+      throw ex;
+    } catch (NoSuchAlgorithmException ex) {
+      LOG.error("write data failed. error: {}", ex);
+      throw new StorageContainerException("Internal error: ", ex,
+          NO_SUCH_ALGORITHM);
+    } catch (ExecutionException  | IOException ex) {
+      LOG.error("write data failed. error: {}", ex);
+      throw new StorageContainerException("Internal error: ", ex,
+          CONTAINER_INTERNAL_ERROR);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("write data failed. error: {}", e);
+      throw new StorageContainerException("Internal error: ", e,
+          CONTAINER_INTERNAL_ERROR);
+    }
+  }
+
+  /**
+   * reads the data defined by a chunk.
+   *
+   * @param container - Container for the chunk
+   * @param blockID - ID of the block.
+   * @param info - ChunkInfo.
+   * @return byte array
+   * @throws StorageContainerException
+   * TODO: Right now we do not support partial reads and writes of chunks.
+   * TODO: Explore if we need to do that for ozone.
+   */
+  public byte[] readChunk(Container container, BlockID blockID, ChunkInfo info)
+      throws StorageContainerException {
+    try {
+      KeyValueContainerData containerData = (KeyValueContainerData) container
+          .getContainerData();
+      ByteBuffer data;
+
+      // Checking here, which layout version the container is, and reading
+      // the chunk file in that format.
+      // In version1, we verify checksum if it is available and return data
+      // of the chunk file.
+      if (containerData.getLayOutVersion() == ChunkLayOutVersion
+          .getLatestVersion().getVersion()) {
+        File chunkFile = ChunkUtils.getChunkFile(containerData, info);
+        data = ChunkUtils.readData(chunkFile, info);
+        containerData.incrReadCount();
+        containerData.incrReadBytes(chunkFile.length());
+        return data.array();
+      }
+    } catch(NoSuchAlgorithmException ex) {
+      LOG.error("read data failed. error: {}", ex);
+      throw new StorageContainerException("Internal error: ",
+          ex, NO_SUCH_ALGORITHM);
+    } catch (ExecutionException ex) {
+      LOG.error("read data failed. error: {}", ex);
+      throw new StorageContainerException("Internal error: ",
+          ex, CONTAINER_INTERNAL_ERROR);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("read data failed. error: {}", e);
+      throw new StorageContainerException("Internal error: ",
+          e, CONTAINER_INTERNAL_ERROR);
+    }
+    return null;
+  }
+
+  /**
+   * Deletes a given chunk.
+   *
+   * @param container - Container for the chunk
+   * @param blockID - ID of the block
+   * @param info - Chunk Info
+   * @throws StorageContainerException
+   */
+  public void deleteChunk(Container container, BlockID blockID, ChunkInfo info)
+      throws StorageContainerException {
+    Preconditions.checkNotNull(blockID, "Block ID cannot be null.");
+    KeyValueContainerData containerData = (KeyValueContainerData) container
+        .getContainerData();
+    // Checking here, which layout version the container is, and performing
+    // deleting chunk operation.
+    // In version1, we have only chunk file.
+    if (containerData.getLayOutVersion() == ChunkLayOutVersion
+        .getLatestVersion().getVersion()) {
+      File chunkFile = ChunkUtils.getChunkFile(containerData, info);
+      if ((info.getOffset() == 0) && (info.getLen() == chunkFile.length())) {
+        FileUtil.fullyDelete(chunkFile);
+        containerData.decrBytesUsed(chunkFile.length());
+      } else {
+        LOG.error("Not Supported Operation. Trying to delete a " +
+            "chunk that is in shared file. chunk info : " + info.toString());
+        throw new StorageContainerException("Not Supported Operation. " +
+            "Trying to delete a chunk that is in shared file. chunk info : "
+            + info.toString(), UNSUPPORTED_REQUEST);
+      }
+    }
+  }
+
+  /**
+   * Shutdown the chunkManager.
+   *
+   * In the chunkManager we haven't acquired any resources, so nothing to do
+   * here.
+   */
+
+  public void shutdown() {
+    //TODO: need to revisit this during integration of container IO.
+  }
+
+  /**
+   * Returns the temporary chunkFile path.
+   * @param chunkFile
+   * @param info
+   * @return temporary chunkFile path
+   * @throws StorageContainerException
+   */
+  private File getTmpChunkFile(File chunkFile, ChunkInfo info)
+      throws StorageContainerException {
+    return new File(chunkFile.getParent(),
+        chunkFile.getName() +
+            OzoneConsts.CONTAINER_CHUNK_NAME_DELIMITER +
+            OzoneConsts.CONTAINER_TEMPORARY_CHUNK_PREFIX);
+  }
+
+  /**
+   * Commit the chunk by renaming the temporary chunk file to chunk file.
+   * @param tmpChunkFile
+   * @param chunkFile
+   * @throws IOException
+   */
+  private void commitChunk(File tmpChunkFile, File chunkFile) throws
+      IOException {
+    Files.move(tmpChunkFile.toPath(), chunkFile.toPath(),
+        StandardCopyOption.REPLACE_EXISTING);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca192cb7/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyManagerImpl.java
new file mode 100644
index 0000000..87565ce
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyManagerImpl.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue;
+
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Longs;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
+import org.apache.hadoop.ozone.container.common.helpers.KeyData;
+import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.keyvalue.interfaces.KeyManager;
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
+import org.apache.hadoop.utils.MetadataStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_KEY;
+
+/**
+ * This class is for performing key related operations on the KeyValue
+ * Container.
+ */
+public class KeyManagerImpl implements KeyManager {
+
+  static final Logger LOG = LoggerFactory.getLogger(KeyManagerImpl.class);
+
+  private Configuration config;
+
+  /**
+   * Constructs a key Manager.
+   *
+   * @param conf - Ozone configuration
+   */
+  public KeyManagerImpl(Configuration conf) {
+    Preconditions.checkNotNull(conf, "Config cannot be null");
+    this.config = conf;
+  }
+
+  /**
+   * Puts or overwrites a key.
+   *
+   * @param container - Container for which key need to be added.
+   * @param data     - Key Data.
+   * @throws IOException
+   */
+  public void putKey(Container container, KeyData data) throws IOException {
+    Preconditions.checkNotNull(data, "KeyData cannot be null for put " +
+        "operation.");
+    Preconditions.checkState(data.getContainerID() >= 0, "Container Id " +
+        "cannot be negative");
+    // We are not locking the key manager since LevelDb serializes all actions
+    // against a single DB. We rely on DB level locking to avoid conflicts.
+    MetadataStore db = KeyUtils.getDB((KeyValueContainerData) container
+        .getContainerData(), config);
+
+    // This is a post condition that acts as a hint to the user.
+    // Should never fail.
+    Preconditions.checkNotNull(db, "DB cannot be null here");
+    db.put(Longs.toByteArray(data.getLocalID()), data.getProtoBufMessage()
+        .toByteArray());
+  }
+
+  /**
+   * Gets an existing key.
+   *
+   * @param container - Container from which key need to be get.
+   * @param data - Key Data.
+   * @return Key Data.
+   * @throws IOException
+   */
+  public KeyData getKey(Container container, KeyData data) throws IOException {
+    Preconditions.checkNotNull(data, "Key data cannot be null");
+    Preconditions.checkNotNull(data.getContainerID(), "Container name cannot" +
+        " be null");
+    KeyValueContainerData containerData = (KeyValueContainerData) container
+        .getContainerData();
+    MetadataStore db = KeyUtils.getDB(containerData, config);
+    // This is a post condition that acts as a hint to the user.
+    // Should never fail.
+    Preconditions.checkNotNull(db, "DB cannot be null here");
+    byte[] kData = db.get(Longs.toByteArray(data.getLocalID()));
+    if (kData == null) {
+      throw new StorageContainerException("Unable to find the key.",
+          NO_SUCH_KEY);
+    }
+    ContainerProtos.KeyData keyData = ContainerProtos.KeyData.parseFrom(kData);
+    return KeyData.getFromProtoBuf(keyData);
+  }
+
+  /**
+   * Deletes an existing Key.
+   *
+   * @param container - Container from which key need to be deleted.
+   * @param blockID - ID of the block.
+   * @throws StorageContainerException
+   */
+  public void deleteKey(Container container, BlockID blockID) throws
+      IOException {
+    Preconditions.checkNotNull(blockID, "block ID cannot be null.");
+    Preconditions.checkState(blockID.getContainerID() >= 0,
+        "Container ID cannot be negative.");
+    Preconditions.checkState(blockID.getLocalID() >= 0,
+        "Local ID cannot be negative.");
+
+    KeyValueContainerData cData = (KeyValueContainerData) container
+        .getContainerData();
+    MetadataStore db = KeyUtils.getDB(cData, config);
+    // This is a post condition that acts as a hint to the user.
+    // Should never fail.
+    Preconditions.checkNotNull(db, "DB cannot be null here");
+    // Note : There is a race condition here, since get and delete
+    // are not atomic. Leaving it here since the impact is refusing
+    // to delete a key which might have just gotten inserted after
+    // the get check.
+    byte[] kKey = Longs.toByteArray(blockID.getLocalID());
+    byte[] kData = db.get(kKey);
+    if (kData == null) {
+      throw new StorageContainerException("Unable to find the key.",
+          NO_SUCH_KEY);
+    }
+    db.delete(kKey);
+  }
+
+  /**
+   * List keys in a container.
+   *
+   * @param container - Container from which keys need to be listed.
+   * @param startLocalID  - Key to start from, 0 to begin.
+   * @param count    - Number of keys to return.
+   * @return List of Keys that match the criteria.
+   */
+  public List<KeyData> listKey(Container container, long startLocalID, int
+      count) throws IOException {
+    Preconditions.checkNotNull(container, "container cannot be null");
+    Preconditions.checkState(startLocalID >= 0, "startLocal ID cannot be " +
+        "negative");
+    Preconditions.checkArgument(count > 0,
+        "Count must be a positive number.");
+    container.readLock();
+    List<KeyData> result = null;
+    KeyValueContainerData cData = (KeyValueContainerData) container
+        .getContainerData();
+    MetadataStore db = KeyUtils.getDB(cData, config);
+    result = new ArrayList<>();
+    byte[] startKeyInBytes = Longs.toByteArray(startLocalID);
+    List<Map.Entry<byte[], byte[]>> range = db.getSequentialRangeKVs(
+        startKeyInBytes, count, null);
+    for (Map.Entry<byte[], byte[]> entry : range) {
+      KeyData value = KeyUtils.getKeyData(entry.getValue());
+      KeyData data = new KeyData(value.getBlockID());
+      result.add(data);
+    }
+    return result;
+  }
+
+  /**
+   * Shutdown KeyValueContainerManager.
+   */
+  public void shutdown() {
+    KeyUtils.shutdownCache(ContainerCache.getInstance(config));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca192cb7/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
new file mode 100644
index 0000000..c837ccc
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
@@ -0,0 +1,295 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue.helpers;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
+import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
+import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.AsynchronousFileChannel;
+import java.nio.channels.FileLock;
+import java.nio.file.StandardOpenOption;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.*;
+
+/**
+ * Utility methods for chunk operations for KeyValue container.
+ */
+public final class ChunkUtils {
+
+  /** Never constructed. **/
+  private ChunkUtils() {
+
+  }
+
+  /**
+   * Writes the data in chunk Info to the specified location in the chunkfile.
+   *
+   * @param chunkFile - File to write data to.
+   * @param chunkInfo - Data stream to write.
+   * @param data - The data buffer.
+   * @throws StorageContainerException
+   */
+  public static void writeData(File chunkFile, ChunkInfo chunkInfo,
+                               byte[] data) throws
+      StorageContainerException, ExecutionException, InterruptedException,
+      NoSuchAlgorithmException {
+
+    Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
+    if (data.length != chunkInfo.getLen()) {
+      String err = String.format("data array does not match the length " +
+              "specified. DataLen: %d Byte Array: %d",
+          chunkInfo.getLen(), data.length);
+      log.error(err);
+      throw new StorageContainerException(err, INVALID_WRITE_SIZE);
+    }
+
+    AsynchronousFileChannel file = null;
+    FileLock lock = null;
+
+    try {
+      file =
+          AsynchronousFileChannel.open(chunkFile.toPath(),
+              StandardOpenOption.CREATE,
+              StandardOpenOption.WRITE,
+              StandardOpenOption.SPARSE,
+              StandardOpenOption.SYNC);
+      lock = file.lock().get();
+      if (chunkInfo.getChecksum() != null &&
+          !chunkInfo.getChecksum().isEmpty()) {
+        verifyChecksum(chunkInfo, data, log);
+      }
+      int size = file.write(ByteBuffer.wrap(data), chunkInfo.getOffset()).get();
+      if (size != data.length) {
+        log.error("Invalid write size found. Size:{}  Expected: {} ", size,
+            data.length);
+        throw new StorageContainerException("Invalid write size found. " +
+            "Size: " + size + " Expected: " + data.length, INVALID_WRITE_SIZE);
+      }
+    } catch (StorageContainerException ex) {
+      throw ex;
+    } catch(IOException e) {
+      throw new StorageContainerException(e, IO_EXCEPTION);
+
+    } finally {
+      if (lock != null) {
+        try {
+          lock.release();
+        } catch (IOException e) {
+          log.error("Unable to release lock ??, Fatal Error.");
+          throw new StorageContainerException(e, CONTAINER_INTERNAL_ERROR);
+
+        }
+      }
+      if (file != null) {
+        try {
+          file.close();
+        } catch (IOException e) {
+          throw new StorageContainerException("Error closing chunk file",
+              e, CONTAINER_INTERNAL_ERROR);
+        }
+      }
+    }
+  }
+
+  /**
+   * Reads data from an existing chunk file.
+   *
+   * @param chunkFile - file where data lives.
+   * @param data - chunk definition.
+   * @return ByteBuffer
+   * @throws StorageContainerException
+   * @throws ExecutionException
+   * @throws InterruptedException
+   */
+  public static ByteBuffer readData(File chunkFile, ChunkInfo data) throws
+      StorageContainerException, ExecutionException, InterruptedException,
+      NoSuchAlgorithmException {
+    Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
+
+    if (!chunkFile.exists()) {
+      log.error("Unable to find the chunk file. chunk info : {}",
+          data.toString());
+      throw new StorageContainerException("Unable to find the chunk file. " +
+          "chunk info " +
+          data.toString(), UNABLE_TO_FIND_CHUNK);
+    }
+
+    AsynchronousFileChannel file = null;
+    FileLock lock = null;
+    try {
+      file =
+          AsynchronousFileChannel.open(chunkFile.toPath(),
+              StandardOpenOption.READ);
+      lock = file.lock(data.getOffset(), data.getLen(), true).get();
+
+      ByteBuffer buf = ByteBuffer.allocate((int) data.getLen());
+      file.read(buf, data.getOffset()).get();
+
+      if (data.getChecksum() != null && !data.getChecksum().isEmpty()) {
+        verifyChecksum(data, buf.array(), log);
+      }
+
+      return buf;
+    } catch (IOException e) {
+      throw new StorageContainerException(e, IO_EXCEPTION);
+    } finally {
+      if (lock != null) {
+        try {
+          lock.release();
+        } catch (IOException e) {
+          log.error("I/O error is lock release.");
+        }
+      }
+      if (file != null) {
+        IOUtils.closeStream(file);
+      }
+    }
+  }
+
+  /**
+   * Verifies the checksum of a chunk against the data buffer.
+   *
+   * @param chunkInfo - Chunk Info.
+   * @param data - data buffer
+   * @param log - log
+   * @throws NoSuchAlgorithmException
+   * @throws StorageContainerException
+   */
+  private static void verifyChecksum(ChunkInfo chunkInfo, byte[] data, Logger
+      log) throws NoSuchAlgorithmException, StorageContainerException {
+    MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
+    sha.update(data);
+    if (!Hex.encodeHexString(sha.digest()).equals(
+        chunkInfo.getChecksum())) {
+      log.error("Checksum mismatch. Provided: {} , computed: {}",
+          chunkInfo.getChecksum(), DigestUtils.sha256Hex(sha.digest()));
+      throw new StorageContainerException("Checksum mismatch. Provided: " +
+          chunkInfo.getChecksum() + " , computed: " +
+          DigestUtils.sha256Hex(sha.digest()), CHECKSUM_MISMATCH);
+    }
+  }
+
+  /**
+   * Validates chunk data and returns a file object to Chunk File that we are
+   * expected to write data to.
+   *
+   * @param data - container data.
+   * @param info - chunk info.
+   * @return File
+   * @throws StorageContainerException
+   */
+  public static File validateChunk(KeyValueContainerData data, ChunkInfo info)
+      throws StorageContainerException {
+
+    Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
+
+    File chunkFile = getChunkFile(data, info);
+    if (isOverWriteRequested(chunkFile, info)) {
+      if (!isOverWritePermitted(info)) {
+        log.error("Rejecting write chunk request. Chunk overwrite " +
+            "without explicit request. {}", info.toString());
+        throw new StorageContainerException("Rejecting write chunk request. " +
+            "OverWrite flag required." + info.toString(),
+            OVERWRITE_FLAG_REQUIRED);
+      }
+    }
+    return chunkFile;
+  }
+
+  /**
+   * Validates that Path to chunk file exists.
+   *
+   * @param containerData - Container Data
+   * @param info - Chunk info
+   * @return - File.
+   * @throws StorageContainerException
+   */
+  public static File getChunkFile(KeyValueContainerData containerData,
+                                  ChunkInfo info) throws
+      StorageContainerException {
+
+    Preconditions.checkNotNull(containerData, "Container data can't be null");
+    Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
+
+    String chunksPath = containerData.getChunksPath();
+    if (chunksPath == null) {
+      log.error("Chunks path is null in the container data");
+      throw new StorageContainerException("Unable to get Chunks directory.",
+          UNABLE_TO_FIND_DATA_DIR);
+    }
+    File chunksLoc = new File(chunksPath);
+    if (!chunksLoc.exists()) {
+      log.error("Chunks path does not exist");
+      throw new StorageContainerException("Unable to get Chunks directory.",
+          UNABLE_TO_FIND_DATA_DIR);
+    }
+
+    return chunksLoc.toPath().resolve(info.getChunkName()).toFile();
+  }
+
+  /**
+   * Checks if we are getting a request to overwrite an existing range of
+   * chunk.
+   *
+   * @param chunkFile - File
+   * @param chunkInfo - Buffer to write
+   * @return bool
+   */
+  public static boolean isOverWriteRequested(File chunkFile, ChunkInfo
+      chunkInfo) {
+
+    if (!chunkFile.exists()) {
+      return false;
+    }
+
+    long offset = chunkInfo.getOffset();
+    return offset < chunkFile.length();
+  }
+
+  /**
+   * Overwrite is permitted if an only if the user explicitly asks for it. We
+   * permit this iff the key/value pair contains a flag called
+   * [OverWriteRequested, true].
+   *
+   * @param chunkInfo - Chunk info
+   * @return true if the user asks for it.
+   */
+  public static boolean isOverWritePermitted(ChunkInfo chunkInfo) {
+    String overWrite = chunkInfo.getMetadata().get(OzoneConsts.CHUNK_OVERWRITE);
+    return (overWrite != null) &&
+        (!overWrite.isEmpty()) &&
+        (Boolean.valueOf(overWrite));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca192cb7/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
index 7d9f0e6..d45f598 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
@@ -20,14 +20,19 @@ package org.apache.hadoop.ozone.container.keyvalue.helpers;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.container.common.helpers.KeyData;
 import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
 import org.apache.hadoop.utils.MetadataStore;
 
 import java.io.IOException;
 
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNABLE_TO_READ_METADATA_DB;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.NO_SUCH_KEY;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.UNABLE_TO_READ_METADATA_DB;
 
 /**
  * Utils functions to help key functions.
@@ -79,4 +84,32 @@ public final class KeyUtils {
     cache.removeDB(container.getContainerId());
   }
 
-}
+  /**
+   * Shutdown all DB Handles.
+   *
+   * @param cache - Cache for DB Handles.
+   */
+  @SuppressWarnings("unchecked")
+  public static void shutdownCache(ContainerCache cache)  {
+    cache.shutdownCache();
+  }
+
+  /**
+   * Parses the {@link KeyData} from a bytes array.
+   *
+   * @param bytes key data in bytes.
+   * @return key data.
+   * @throws IOException if the bytes array is malformed or invalid.
+   */
+  public static KeyData getKeyData(byte[] bytes) throws IOException {
+    try {
+      ContainerProtos.KeyData keyData = ContainerProtos.KeyData.parseFrom(
+          bytes);
+      KeyData data = KeyData.getFromProtoBuf(keyData);
+      return data;
+    } catch (IOException e) {
+      throw new StorageContainerException("Failed to parse key data from the" +
+          " bytes array.", NO_SUCH_KEY);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca192cb7/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java
new file mode 100644
index 0000000..7134be1
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java
@@ -0,0 +1,80 @@
+package org.apache.hadoop.ozone.container.keyvalue.interfaces;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+
+/**
+ * Chunk Manager allows read, write, delete and listing of chunks in
+ * a container.
+ */
+
+public interface ChunkManager {
+
+  /**
+   * writes a given chunk.
+   *
+   * @param container - Container for the chunk
+   * @param blockID - ID of the block.
+   * @param info - ChunkInfo.
+   * @param stage - Chunk Stage write.
+   * @throws StorageContainerException
+   */
+  void writeChunk(Container container, BlockID blockID, ChunkInfo info,
+                  byte[] data, ContainerProtos.Stage stage)
+      throws StorageContainerException;
+
+  /**
+   * reads the data defined by a chunk.
+   *
+   * @param container - Container for the chunk
+   * @param blockID - ID of the block.
+   * @param info - ChunkInfo.
+   * @return  byte array
+   * @throws StorageContainerException
+   *
+   * TODO: Right now we do not support partial reads and writes of chunks.
+   * TODO: Explore if we need to do that for ozone.
+   */
+  byte[] readChunk(Container container, BlockID blockID, ChunkInfo info) throws
+      StorageContainerException;
+
+  /**
+   * Deletes a given chunk.
+   *
+   * @param container - Container for the chunk
+   * @param blockID - ID of the block.
+   * @param info  - Chunk Info
+   * @throws StorageContainerException
+   */
+  void deleteChunk(Container container, BlockID blockID, ChunkInfo info) throws
+      StorageContainerException;
+
+  // TODO : Support list operations.
+
+  /**
+   * Shutdown the chunkManager.
+   */
+  void shutdown();
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca192cb7/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
new file mode 100644
index 0000000..ebda97e
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue.interfaces;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.container.common.helpers.KeyData;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * KeyManager is for performing key related operations on the container.
+ */
+public interface KeyManager {
+
+  /**
+   * Puts or overwrites a key.
+   *
+   * @param container - Container for which key need to be added.
+   * @param data     - Key Data.
+   * @throws IOException
+   */
+  void putKey(Container container, KeyData data) throws IOException;
+
+  /**
+   * Gets an existing key.
+   *
+   * @param container - Container from which key need to be get.
+   * @param data - Key Data.
+   * @return Key Data.
+   * @throws IOException
+   */
+  KeyData getKey(Container container, KeyData data) throws IOException;
+
+  /**
+   * Deletes an existing Key.
+   *
+   * @param container - Container from which key need to be deleted.
+   * @param blockID - ID of the block.
+   * @throws StorageContainerException
+   */
+  void deleteKey(Container container, BlockID blockID) throws IOException;
+
+  /**
+   * List keys in a container.
+   *
+   * @param container - Container from which keys need to be listed.
+   * @param startLocalID  - Key to start from, 0 to begin.
+   * @param count    - Number of keys to return.
+   * @return List of Keys that match the criteria.
+   */
+  List<KeyData> listKey(Container container, long startLocalID, int count) throws
+      IOException;
+
+  /**
+   * Shutdown ContainerManager.
+   */
+  void shutdown();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca192cb7/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
new file mode 100644
index 0000000..ca936c7
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue;
+
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
+import org.apache.hadoop.ozone.container.common.helpers.KeyData;
+import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.anyList;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.Mockito.mock;
+
+/**
+ * This class is used to test ChunkManager operations.
+ */
+public class TestChunkManagerImpl {
+
+  private OzoneConfiguration config;
+  private String scmId = UUID.randomUUID().toString();
+  private VolumeSet volumeSet;
+  private RoundRobinVolumeChoosingPolicy volumeChoosingPolicy;
+  private KeyValueContainerData keyValueContainerData;
+  private KeyValueContainer keyValueContainer;
+  private KeyData keyData;
+  private BlockID blockID;
+  private ChunkManagerImpl chunkManager;
+  private ChunkInfo chunkInfo;
+  private byte[] data;
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws Exception {
+    config = new OzoneConfiguration();
+
+    HddsVolume hddsVolume = new HddsVolume.Builder(folder.getRoot()
+        .getAbsolutePath()).conf(config).datanodeUuid(UUID.randomUUID()
+        .toString()).build();
+
+    volumeSet = mock(VolumeSet.class);
+
+    volumeChoosingPolicy = mock(RoundRobinVolumeChoosingPolicy.class);
+    Mockito.when(volumeChoosingPolicy.chooseVolume(anyList(), anyLong()))
+        .thenReturn(hddsVolume);
+
+    keyValueContainerData = new KeyValueContainerData(
+        ContainerProtos.ContainerType.KeyValueContainer, 1L);
+
+    keyValueContainer = new KeyValueContainer(
+        keyValueContainerData, config);
+
+    keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
+
+    data = "testing write chunks".getBytes();
+    // Creating KeyData
+    blockID = new BlockID(1L, 1L);
+    keyData = new KeyData(blockID);
+    keyData.addMetadata("VOLUME", "ozone");
+    keyData.addMetadata("OWNER", "hdfs");
+    List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
+    chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
+        .getLocalID(), 0), 0, data.length);
+    chunkList.add(chunkInfo.getProtoBufMessage());
+    keyData.setChunks(chunkList);
+
+    // Create a ChunkManager object.
+    chunkManager = new ChunkManagerImpl();
+
+  }
+
+  @Test
+  public void testWriteChunkStageWriteAndCommit() throws Exception {
+    //As in Setup, we try to create container, these paths should exist.
+    assertTrue(keyValueContainerData.getChunksPath() != null);
+    File chunksPath = new File(keyValueContainerData.getChunksPath());
+    assertTrue(chunksPath.exists());
+    // Initially chunks folder should be empty.
+    assertTrue(chunksPath.listFiles().length == 0);
+    chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
+        ContainerProtos.Stage.WRITE_DATA);
+    // Now a chunk file is being written with Stage WRITE_DATA, so it should
+    // create a temporary chunk file.
+    assertTrue(chunksPath.listFiles().length == 1);
+    chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
+        ContainerProtos.Stage.COMMIT_DATA);
+    // Old temp file should have been renamed to chunk file.
+    assertTrue(chunksPath.listFiles().length == 1);
+
+  }
+
+  @Test
+  public void testWriteChunkIncorrectLength() throws Exception {
+    try {
+      long randomLength = 200L;
+      chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
+          .getLocalID(), 0), 0, randomLength);
+      List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
+      chunkList.add(chunkInfo.getProtoBufMessage());
+      keyData.setChunks(chunkList);
+      chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
+          ContainerProtos.Stage.WRITE_DATA);
+      fail("testWriteChunkIncorrectLength failed");
+    } catch (StorageContainerException ex) {
+      GenericTestUtils.assertExceptionContains("data array does not match " +
+          "the length ", ex);
+      assertEquals(ContainerProtos.Result.INVALID_WRITE_SIZE, ex.getResult());
+    }
+  }
+
+  @Test
+  public void testWriteChunkStageCombinedData() throws Exception {
+    //As in Setup, we try to create container, these paths should exist.
+    assertTrue(keyValueContainerData.getChunksPath() != null);
+    File chunksPath = new File(keyValueContainerData.getChunksPath());
+    assertTrue(chunksPath.exists());
+    // Initially chunks folder should be empty.
+    assertTrue(chunksPath.listFiles().length == 0);
+    chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
+        ContainerProtos.Stage.COMBINED);
+    // Now a chunk file is being written with Stage WRITE_DATA, so it should
+    // create a temporary chunk file.
+    assertTrue(chunksPath.listFiles().length == 1);
+  }
+
+  @Test
+  public void testReadChunk() throws Exception {
+    chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
+        ContainerProtos.Stage.COMBINED);
+    byte[] expectedData = chunkManager.readChunk(keyValueContainer, blockID,
+        chunkInfo);
+    assertEquals(expectedData.length, data.length);
+    assertTrue(Arrays.equals(expectedData, data));
+  }
+
+  @Test
+  public void testDeleteChunk() throws Exception {
+    File chunksPath = new File(keyValueContainerData.getChunksPath());
+    chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
+        ContainerProtos.Stage.COMBINED);
+    assertTrue(chunksPath.listFiles().length == 1);
+    chunkManager.deleteChunk(keyValueContainer, blockID, chunkInfo);
+    assertTrue(chunksPath.listFiles().length == 0);
+  }
+
+  @Test
+  public void testDeleteChunkUnsupportedRequest() throws Exception {
+    try {
+      chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
+          ContainerProtos.Stage.COMBINED);
+      long randomLength = 200L;
+      chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
+          .getLocalID(), 0), 0, randomLength);
+      List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
+      chunkList.add(chunkInfo.getProtoBufMessage());
+      keyData.setChunks(chunkList);
+      chunkManager.deleteChunk(keyValueContainer, blockID, chunkInfo);
+      fail("testDeleteChunkUnsupportedRequest");
+    } catch (StorageContainerException ex) {
+      GenericTestUtils.assertExceptionContains("Not Supported Operation.", ex);
+      assertEquals(ContainerProtos.Result.UNSUPPORTED_REQUEST, ex.getResult());
+    }
+  }
+
+  @Test
+  public void testWriteChunkChecksumMismatch() throws Exception {
+    try {
+      chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
+          .getLocalID(), 0), 0, data.length);
+      //Setting checksum to some value.
+      chunkInfo.setChecksum("some garbage");
+      List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
+      chunkList.add(chunkInfo.getProtoBufMessage());
+      keyData.setChunks(chunkList);
+      chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
+          ContainerProtos.Stage.COMBINED);
+      fail("testWriteChunkChecksumMismatch failed");
+    } catch (StorageContainerException ex) {
+      GenericTestUtils.assertExceptionContains("Checksum mismatch.", ex);
+      assertEquals(ContainerProtos.Result.CHECKSUM_MISMATCH, ex.getResult());
+    }
+  }
+
+  @Test
+  public void testReadChunkFileNotExists() throws Exception {
+    try {
+      // trying to read a chunk, where chunk file does not exist
+      byte[] expectedData = chunkManager.readChunk(keyValueContainer, blockID,
+          chunkInfo);
+      fail("testReadChunkFileNotExists failed");
+    } catch (StorageContainerException ex) {
+      GenericTestUtils.assertExceptionContains("Unable to find the chunk " +
+          "file.", ex);
+      assertEquals(ContainerProtos.Result.UNABLE_TO_FIND_CHUNK, ex.getResult());
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca192cb7/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
new file mode 100644
index 0000000..a6f50c4
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
+import org.apache.hadoop.ozone.container.common.helpers.KeyData;
+import org.apache.hadoop.ozone.container.common.volume
+    .RoundRobinVolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.UUID;
+
+import static org.junit.Assert.*;
+import static org.mockito.ArgumentMatchers.anyList;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.Mockito.mock;
+
+/**
+ * This class is used to test key related operations on the container.
+ */
+public class TestKeyManagerImpl {
+
+  private OzoneConfiguration config;
+  private String scmId = UUID.randomUUID().toString();
+  private VolumeSet volumeSet;
+  private RoundRobinVolumeChoosingPolicy volumeChoosingPolicy;
+  private KeyValueContainerData keyValueContainerData;
+  private KeyValueContainer keyValueContainer;
+  private KeyData keyData;
+  private KeyManagerImpl keyValueContainerManager;
+  private BlockID blockID;
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+
+  @Before
+  public void setUp() throws Exception {
+    config = new OzoneConfiguration();
+
+    HddsVolume hddsVolume = new HddsVolume.Builder(folder.getRoot()
+        .getAbsolutePath()).conf(config).datanodeUuid(UUID.randomUUID()
+        .toString()).build();
+
+    volumeSet = mock(VolumeSet.class);
+
+    volumeChoosingPolicy = mock(RoundRobinVolumeChoosingPolicy.class);
+    Mockito.when(volumeChoosingPolicy.chooseVolume(anyList(), anyLong()))
+        .thenReturn(hddsVolume);
+
+    keyValueContainerData = new KeyValueContainerData(
+        ContainerProtos.ContainerType.KeyValueContainer, 1L);
+
+    keyValueContainer = new KeyValueContainer(
+        keyValueContainerData, config);
+
+    keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
+
+    // Creating KeyData
+    blockID = new BlockID(1L, 1L);
+    keyData = new KeyData(blockID);
+    keyData.addMetadata("VOLUME", "ozone");
+    keyData.addMetadata("OWNER", "hdfs");
+    List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
+    ChunkInfo info = new ChunkInfo(String.format("%d.data.%d", blockID
+        .getLocalID(), 0), 0, 1024);
+    chunkList.add(info.getProtoBufMessage());
+    keyData.setChunks(chunkList);
+
+    // Create KeyValueContainerManager
+    keyValueContainerManager = new KeyManagerImpl(config);
+
+  }
+
+  @Test
+  public void testPutAndGetKey() throws Exception {
+    //Put Key
+    keyValueContainerManager.putKey(keyValueContainer, keyData);
+
+    //Get Key
+    KeyData fromGetKeyData = keyValueContainerManager.getKey(keyValueContainer,
+        keyData);
+
+    assertEquals(keyData.getContainerID(), fromGetKeyData.getContainerID());
+    assertEquals(keyData.getLocalID(), fromGetKeyData.getLocalID());
+    assertEquals(keyData.getChunks().size(), fromGetKeyData.getChunks().size());
+    assertEquals(keyData.getMetadata().size(), fromGetKeyData.getMetadata()
+        .size());
+
+  }
+
+
+  @Test
+  public void testDeleteKey() throws Exception {
+    try {
+      //Put Key
+      keyValueContainerManager.putKey(keyValueContainer, keyData);
+      //Delete Key
+      keyValueContainerManager.deleteKey(keyValueContainer, blockID);
+    } catch (IOException ex) {
+      fail("testDeleteKey failed");
+    }
+  }
+
+  @Test
+  public void testListKey() throws Exception {
+    try {
+      keyValueContainerManager.putKey(keyValueContainer, keyData);
+      List<KeyData> listKeyData = keyValueContainerManager.listKey(
+          keyValueContainer, 1, 10);
+      assertNotNull(listKeyData);
+      assertTrue(listKeyData.size() == 1);
+
+      for (long i = 2; i <= 10; i++) {
+        blockID = new BlockID(1L, i);
+        keyData = new KeyData(blockID);
+        keyData.addMetadata("VOLUME", "ozone");
+        keyData.addMetadata("OWNER", "hdfs");
+        List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
+        ChunkInfo info = new ChunkInfo(String.format("%d.data.%d", blockID
+            .getLocalID(), 0), 0, 1024);
+        chunkList.add(info.getProtoBufMessage());
+        keyData.setChunks(chunkList);
+        keyValueContainerManager.putKey(keyValueContainer, keyData);
+      }
+
+      listKeyData = keyValueContainerManager.listKey(
+          keyValueContainer, 1, 10);
+      assertNotNull(listKeyData);
+      assertTrue(listKeyData.size() == 10);
+
+    } catch (IOException ex) {
+      fail("testListKey failed");
+    }
+  }
+
+  @Test
+  public void testGetNoSuchKey() throws Exception {
+    try {
+      keyData = new KeyData(new BlockID(1L, 2L));
+      keyValueContainerManager.getKey(keyValueContainer, keyData);
+      fail("testGetNoSuchKey failed");
+    } catch (StorageContainerException ex) {
+      GenericTestUtils.assertExceptionContains("Unable to find the key.", ex);
+      assertEquals(ContainerProtos.Result.NO_SUCH_KEY, ex.getResult());
+    }
+  }
+}


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


[29/37] hadoop git commit: HDDS-182:CleanUp Reimplemented classes. Contributed by Hansiha Koneru

Posted by bh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java
deleted file mode 100644
index 02572a8..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerManagerImpl.java
+++ /dev/null
@@ -1,1115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.impl;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.primitives.Longs;
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
-import org.apache.hadoop.hdds.scm.container.common.helpers
-    .StorageContainerException;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .ContainerLifeCycleState;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.StorageReportProto;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
-import org.apache.hadoop.ozone.container.common.helpers.KeyData;
-import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
-import org.apache.hadoop.ozone.container.common.interfaces.ChunkManager;
-import org.apache.hadoop.ozone.container.common.interfaces
-    .ContainerDeletionChoosingPolicy;
-import org.apache.hadoop.ozone.container.common.interfaces
-    .ContainerLocationManager;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
-import org.apache.hadoop.ozone.container.common.interfaces.KeyManager;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.utils.MetadataKeyFilters;
-import org.apache.hadoop.utils.MetadataStore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.security.DigestInputStream;
-import java.security.DigestOutputStream;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentNavigableMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.stream.Collectors;
-
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.CONTAINER_EXISTS;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.CONTAINER_INTERNAL_ERROR;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.CONTAINER_NOT_FOUND;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.ERROR_IN_COMPACT_DB;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.INVALID_CONFIG;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.IO_EXCEPTION;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.NO_SUCH_ALGORITHM;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.UNABLE_TO_READ_METADATA_DB;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.UNCLOSED_CONTAINER_IO;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.UNSUPPORTED_REQUEST;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
-    Result.INVALID_CONTAINER_STATE;
-import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
-
-/**
- * A Generic ContainerManagerImpl that will be called from Ozone
- * ContainerManagerImpl. This allows us to support delta changes to ozone
- * version without having to rewrite the containerManager.
- */
-public class ContainerManagerImpl implements ContainerManager {
-  static final Logger LOG =
-      LoggerFactory.getLogger(ContainerManagerImpl.class);
-
-  // TODO: consider primitive collection like eclipse-collections
-  // to avoid autoboxing overhead
-  private final ConcurrentSkipListMap<Long, ContainerData>
-      containerMap = new ConcurrentSkipListMap<>();
-
-  // Use a non-fair RW lock for better throughput, we may revisit this decision
-  // if this causes fairness issues.
-  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-  private ContainerLocationManager locationManager;
-  private ChunkManager chunkManager;
-  private KeyManager keyManager;
-  private Configuration conf;
-
-  private ContainerDeletionChoosingPolicy containerDeletionChooser;
-
-  /**
-   * Init call that sets up a container Manager.
-   *
-   * @param config - Configuration.
-   * @param containerDirs - List of Metadata Container locations.
-   * @param dnDetails - DatanodeDetails.
-   * @throws IOException
-   */
-  @Override
-  public void init(
-      Configuration config, List<StorageLocation> containerDirs,
-      DatanodeDetails dnDetails) throws IOException {
-    Preconditions.checkNotNull(config, "Config must not be null");
-    Preconditions.checkNotNull(containerDirs, "Container directories cannot " +
-        "be null");
-    Preconditions.checkNotNull(dnDetails, "Datanode Details cannot " +
-        "be null");
-
-    Preconditions.checkState(containerDirs.size() > 0, "Number of container" +
-        " directories must be greater than zero.");
-
-    this.conf = config;
-
-    readLock();
-    try {
-      containerDeletionChooser = ReflectionUtils.newInstance(conf.getClass(
-          ScmConfigKeys.OZONE_SCM_CONTAINER_DELETION_CHOOSING_POLICY,
-          TopNOrderedContainerDeletionChoosingPolicy.class,
-          ContainerDeletionChoosingPolicy.class), conf);
-
-      for (StorageLocation path : containerDirs) {
-        File directory = Paths.get(path.getNormalizedUri()).toFile();
-        if (!directory.exists() && !directory.mkdirs()) {
-          LOG.error("Container metadata directory doesn't exist "
-              + "and cannot be created. Path: {}", path.toString());
-          throw new StorageContainerException("Container metadata "
-              + "directory doesn't exist and cannot be created " + path
-              .toString(), INVALID_CONFIG);
-        }
-
-        // TODO: This will fail if any directory is invalid.
-        // We should fix this to handle invalid directories and continue.
-        // Leaving it this way to fail fast for time being.
-        if (!directory.isDirectory()) {
-          LOG.error("Invalid path to container metadata directory. path: {}",
-              path.toString());
-          throw new StorageContainerException("Invalid path to container " +
-              "metadata directory." + path, INVALID_CONFIG);
-        }
-        LOG.info("Loading containers under {}", path);
-        File[] files = directory.listFiles(new ContainerFilter());
-        if (files != null) {
-          for (File containerFile : files) {
-            LOG.debug("Loading container {}", containerFile);
-            String containerPath =
-                ContainerUtils.getContainerNameFromFile(containerFile);
-            Preconditions.checkNotNull(containerPath, "Container path cannot" +
-                " be null");
-            readContainerInfo(containerPath);
-          }
-        }
-      }
-
-      List<StorageLocation> dataDirs = new LinkedList<>();
-      for (String dir : config.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
-        StorageLocation location = StorageLocation.parse(dir);
-        dataDirs.add(location);
-      }
-      this.locationManager =
-          new ContainerLocationManagerImpl(containerDirs, dataDirs, config);
-    } finally {
-      readUnlock();
-    }
-  }
-
-  /**
-   * Reads the Container Info from a file and verifies that checksum match. If
-   * the checksums match, then that file is added to containerMap.
-   *
-   * @param containerName - Name which points to the persisted container.
-   * @throws StorageContainerException
-   */
-  private void readContainerInfo(String containerName)
-      throws StorageContainerException {
-    Preconditions.checkState(containerName.length() > 0,
-        "Container name length cannot be zero.");
-    FileInputStream containerStream = null;
-    DigestInputStream dis = null;
-    FileInputStream metaStream = null;
-    Path cPath = Paths.get(containerName).getFileName();
-    String keyName = null;
-    if (cPath != null) {
-      keyName = cPath.toString();
-    }
-    Preconditions.checkNotNull(keyName,
-        "Container Name  to container key mapping is null");
-
-    long containerID = Long.parseLong(keyName);
-    try {
-      String containerFileName = containerName.concat(CONTAINER_EXTENSION);
-
-      containerStream = new FileInputStream(containerFileName);
-
-      ContainerProtos.ContainerData containerDataProto =
-          ContainerProtos.ContainerData.parseDelimitedFrom(containerStream);
-      ContainerData containerData;
-      if (containerDataProto == null) {
-        // Sometimes container metadata might have been created but empty,
-        // when loading the info we get a null, this often means last time
-        // SCM was ending up at some middle phase causing that the metadata
-        // was not populated. Such containers are marked as inactive.
-        ContainerData cData = new ContainerData(containerID, conf,
-            ContainerLifeCycleState.INVALID);
-        containerMap.put(containerID, cData);
-        return;
-      }
-      containerData = ContainerData.getFromProtBuf(containerDataProto, conf);
-
-      // Initialize pending deletion blocks and deleted blocks count in
-      // in-memory containerData.
-      MetadataStore metadata = KeyUtils.getDB(containerData, conf);
-      List<Map.Entry<byte[], byte[]>> underDeletionBlocks = metadata
-          .getSequentialRangeKVs(null, Integer.MAX_VALUE,
-              MetadataKeyFilters.getDeletingKeyFilter());
-      byte[] transactionID = metadata.get(DFSUtil.string2Bytes(
-          OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX + containerID));
-      if (transactionID != null) {
-        containerData
-            .updateDeleteTransactionId(Longs.fromByteArray(transactionID));
-      }
-      containerData.incrPendingDeletionBlocks(underDeletionBlocks.size());
-
-      List<Map.Entry<byte[], byte[]>> liveKeys = metadata
-          .getRangeKVs(null, Integer.MAX_VALUE,
-              MetadataKeyFilters.getNormalKeyFilter());
-
-      // Get container bytesUsed upon loading container
-      // The in-memory state is updated upon key write or delete
-      // TODO: update containerDataProto and persist it into container MetaFile
-      long bytesUsed = 0;
-      bytesUsed = liveKeys.parallelStream().mapToLong(e-> {
-        KeyData keyData;
-        try {
-          keyData = KeyUtils.getKeyData(e.getValue());
-          return keyData.getSize();
-        } catch (IOException ex) {
-          return 0L;
-        }
-      }).sum();
-      containerData.setBytesUsed(bytesUsed);
-
-      containerMap.put(containerID, containerData);
-    } catch (IOException ex) {
-      LOG.error("read failed for file: {} ex: {}", containerName,
-          ex.getMessage());
-
-      // TODO : Add this file to a recovery Queue.
-
-      // Remember that this container is busted and we cannot use it.
-      ContainerData cData = new ContainerData(containerID, conf,
-          ContainerLifeCycleState.INVALID);
-      containerMap.put(containerID, cData);
-      throw new StorageContainerException("Unable to read container info",
-          UNABLE_TO_READ_METADATA_DB);
-    } finally {
-      IOUtils.closeStream(dis);
-      IOUtils.closeStream(containerStream);
-      IOUtils.closeStream(metaStream);
-    }
-  }
-
-  /**
-   * Creates a container with the given name.
-   *
-   * @param containerData - Container Name and metadata.
-   * @throws StorageContainerException - Exception
-   */
-  @Override
-  public void createContainer(ContainerData containerData)
-      throws StorageContainerException {
-    Preconditions.checkNotNull(containerData, "Container data cannot be null");
-    writeLock();
-    try {
-      if (containerMap.containsKey(containerData.getContainerID())) {
-        LOG.debug("container already exists. {}",
-            containerData.getContainerID());
-        throw new StorageContainerException("container already exists.",
-            CONTAINER_EXISTS);
-      }
-
-      // This is by design. We first write and close the
-      // container Info and metadata to a directory.
-      // Then read back and put that info into the containerMap.
-      // This allows us to make sure that our write is consistent.
-
-      writeContainerInfo(containerData, false);
-      File cFile = new File(containerData.getContainerPath());
-      readContainerInfo(ContainerUtils.getContainerNameFromFile(cFile));
-    } catch (NoSuchAlgorithmException ex) {
-      LOG.error("Internal error: We seem to be running a JVM without a " +
-          "needed hash algorithm.");
-      throw new StorageContainerException("failed to create container",
-          NO_SUCH_ALGORITHM);
-    } finally {
-      writeUnlock();
-    }
-
-  }
-
-  /**
-   * Writes a container to a chosen location and updates the container Map.
-   *
-   * The file formats of ContainerData and Container Meta is the following.
-   *
-   * message ContainerData {
-   * required string name = 1;
-   * repeated KeyValue metadata = 2;
-   * optional string dbPath = 3;
-   * optional string containerPath = 4;
-   * optional int64 bytesUsed = 5;
-   * optional int64 size = 6;
-   * }
-   *
-   * message ContainerMeta {
-   * required string fileName = 1;
-   * required string hash = 2;
-   * }
-   *
-   * @param containerData - container Data
-   * @param overwrite - Whether we are overwriting.
-   * @throws StorageContainerException, NoSuchAlgorithmException
-   */
-  private void writeContainerInfo(ContainerData containerData,
-      boolean  overwrite)
-      throws StorageContainerException, NoSuchAlgorithmException {
-
-    Preconditions.checkNotNull(this.locationManager,
-        "Internal error: location manager cannot be null");
-
-    FileOutputStream containerStream = null;
-    DigestOutputStream dos = null;
-    FileOutputStream metaStream = null;
-
-    try {
-      Path metadataPath = null;
-      Path location = (!overwrite) ? locationManager.getContainerPath():
-          Paths.get(containerData.getContainerPath()).getParent();
-      if (location == null) {
-        throw new StorageContainerException(
-            "Failed to get container file path.",
-            CONTAINER_INTERNAL_ERROR);
-      }
-
-      File containerFile = ContainerUtils.getContainerFile(containerData,
-          location);
-      String containerName = Long.toString(containerData.getContainerID());
-
-      if(!overwrite) {
-        ContainerUtils.verifyIsNewContainer(containerFile);
-        metadataPath = this.locationManager.getDataPath(containerName);
-        metadataPath = ContainerUtils.createMetadata(metadataPath,
-            containerName, conf);
-      }  else {
-        metadataPath = ContainerUtils.getMetadataDirectory(containerData);
-      }
-
-      containerStream = new FileOutputStream(containerFile);
-
-      MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
-
-      dos = new DigestOutputStream(containerStream, sha);
-      containerData.setDBPath(metadataPath.resolve(
-          ContainerUtils.getContainerDbFileName(containerName))
-          .toString());
-      containerData.setContainerPath(containerFile.toString());
-
-      if(containerData.getContainerDBType() == null) {
-        String impl = conf.getTrimmed(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL,
-            OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_DEFAULT);
-        containerData.setContainerDBType(impl);
-      }
-
-      ContainerProtos.ContainerData protoData = containerData
-          .getProtoBufMessage();
-      protoData.writeDelimitedTo(dos);
-
-    } catch (IOException ex) {
-      // TODO : we need to clean up partially constructed files
-      // The proper way to do would be for a thread
-      // to read all these 3 artifacts and make sure they are
-      // sane. That info needs to come from the replication
-      // pipeline, and if not consistent delete these file.
-
-      // In case of ozone this is *not* a deal breaker since
-      // SCM is guaranteed to generate unique container names.
-      // The saving grace is that we check if we have residue files
-      // lying around when creating a new container. We need to queue
-      // this information to a cleaner thread.
-
-      LOG.error("Creation of container failed. Name: {}, we might need to " +
-              "cleanup partially created artifacts. ",
-          containerData.getContainerID(), ex);
-      throw new StorageContainerException("Container creation failed. ",
-          ex, CONTAINER_INTERNAL_ERROR);
-    } finally {
-      IOUtils.closeStream(dos);
-      IOUtils.closeStream(containerStream);
-      IOUtils.closeStream(metaStream);
-    }
-  }
-
-  /**
-   * Deletes an existing container.
-   *
-   * @param containerID - ID of the container.
-   * @param forceDelete - whether this container should be deleted forcibly.
-   * @throws StorageContainerException
-   */
-  @Override
-  public void deleteContainer(long containerID,
-      boolean forceDelete) throws StorageContainerException {
-    Preconditions.checkState(containerID >= 0,
-        "Container ID cannot be negative.");
-    writeLock();
-    try {
-      if (isOpen(containerID)) {
-        throw new StorageContainerException(
-            "Deleting an open container is not allowed.",
-            UNCLOSED_CONTAINER_IO);
-      }
-
-      ContainerData containerData = containerMap.get(containerID);
-      if (containerData == null) {
-        LOG.debug("No such container. ID: {}", containerID);
-        throw new StorageContainerException("No such container. ID : " +
-            containerID, CONTAINER_NOT_FOUND);
-      }
-
-      if(!containerData.isValid()) {
-        LOG.debug("Invalid container data. ID: {}", containerID);
-        throw new StorageContainerException("Invalid container data. Name : " +
-            containerID, CONTAINER_NOT_FOUND);
-      }
-      ContainerUtils.removeContainer(containerData, conf, forceDelete);
-      containerMap.remove(containerID);
-    } catch (StorageContainerException e) {
-      throw e;
-    } catch (IOException e) {
-      // TODO : An I/O error during delete can leave partial artifacts on the
-      // disk. We will need the cleaner thread to cleanup this information.
-      String errMsg = String.format("Failed to cleanup container. ID: %d",
-          containerID);
-      LOG.error(errMsg, e);
-      throw new StorageContainerException(errMsg, e, IO_EXCEPTION);
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  /**
-   * A simple interface for container Iterations.
-   * <p/>
-   * This call make no guarantees about consistency of the data between
-   * different list calls. It just returns the best known data at that point of
-   * time. It is possible that using this iteration you can miss certain
-   * container from the listing.
-   *
-   * @param startContainerID -  Return containers with ID >= startContainerID.
-   * @param count - how many to return
-   * @param data - Actual containerData
-   * @throws StorageContainerException
-   */
-  @Override
-  public void listContainer(long startContainerID, long count,
-      List<ContainerData> data) throws StorageContainerException {
-    Preconditions.checkNotNull(data,
-        "Internal assertion: data cannot be null");
-    Preconditions.checkState(startContainerID >= 0,
-        "Start container ID cannot be negative");
-    Preconditions.checkState(count > 0,
-        "max number of containers returned " +
-            "must be positive");
-
-    readLock();
-    try {
-      ConcurrentNavigableMap<Long, ContainerData> map;
-      if (startContainerID == 0) {
-        map = containerMap.tailMap(containerMap.firstKey(), true);
-      } else {
-        map = containerMap.tailMap(startContainerID, false);
-      }
-
-      int currentCount = 0;
-      for (ContainerData entry : map.values()) {
-        if (currentCount < count) {
-          data.add(entry);
-          currentCount++;
-        } else {
-          return;
-        }
-      }
-    } finally {
-      readUnlock();
-    }
-  }
-
-  /**
-   * Get metadata about a specific container.
-   *
-   * @param containerID - ID of the container
-   * @return ContainerData - Container Data.
-   * @throws StorageContainerException
-   */
-  @Override
-  public ContainerData readContainer(long containerID)
-      throws StorageContainerException {
-    Preconditions.checkState(containerID >= 0,
-        "Container ID cannot be negative.");
-    if (!containerMap.containsKey(containerID)) {
-      throw new StorageContainerException("Unable to find the container. ID: "
-          + containerID, CONTAINER_NOT_FOUND);
-    }
-    ContainerData cData = containerMap.get(containerID);
-    if (cData == null) {
-      throw new StorageContainerException("Invalid container data. ID: "
-          + containerID, CONTAINER_INTERNAL_ERROR);
-    }
-    return cData;
-  }
-
-  /**
-   * Closes a open container, if it is already closed or does not exist a
-   * StorageContainerException is thrown.
-   *
-   * @param containerID - ID of the container.
-   * @throws StorageContainerException
-   */
-  @Override
-  public void closeContainer(long containerID)
-      throws StorageContainerException, NoSuchAlgorithmException {
-    ContainerData containerData = readContainer(containerID);
-    containerData.closeContainer();
-    writeContainerInfo(containerData, true);
-    MetadataStore db = KeyUtils.getDB(containerData, conf);
-
-    // It is ok if this operation takes a bit of time.
-    // Close container is not expected to be instantaneous.
-    try {
-      db.compactDB();
-    } catch (IOException e) {
-      LOG.error("Error in DB compaction while closing container", e);
-      throw new StorageContainerException(e, ERROR_IN_COMPACT_DB);
-    }
-
-    // Active is different from closed. Closed means it is immutable, active
-    // false means we have some internal error that is happening to this
-    // container. This is a way to track damaged containers if we have an
-    // I/O failure, this allows us to take quick action in case of container
-    // issues.
-
-    containerMap.put(containerID, containerData);
-  }
-
-  @Override
-  public void updateContainer(long containerID, ContainerData data,
-      boolean forceUpdate) throws StorageContainerException {
-    Preconditions.checkState(containerID >= 0,
-        "Container ID cannot be negative.");
-    Preconditions.checkNotNull(data, "Container data cannot be null");
-    FileOutputStream containerStream = null;
-    DigestOutputStream dos = null;
-    MessageDigest sha = null;
-    File containerFileBK = null, containerFile = null;
-    boolean deleted = false;
-
-    if(!containerMap.containsKey(containerID)) {
-      throw new StorageContainerException("Container doesn't exist. Name :"
-          + containerID, CONTAINER_NOT_FOUND);
-    }
-
-    try {
-      sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
-    } catch (NoSuchAlgorithmException e) {
-      throw new StorageContainerException("Unable to create Message Digest,"
-          + " usually this is a java configuration issue.",
-          NO_SUCH_ALGORITHM);
-    }
-
-    try {
-      Path location = locationManager.getContainerPath();
-      ContainerData orgData = containerMap.get(containerID);
-      if (orgData == null) {
-        // updating a invalid container
-        throw new StorageContainerException("Update a container with invalid" +
-            "container meta data", CONTAINER_INTERNAL_ERROR);
-      }
-
-      if (!forceUpdate && !orgData.isOpen()) {
-        throw new StorageContainerException(
-            "Update a closed container is not allowed. ID: " + containerID,
-            UNSUPPORTED_REQUEST);
-      }
-
-      containerFile = ContainerUtils.getContainerFile(orgData, location);
-      // If forceUpdate is true, there is no need to check
-      // whether the container file exists.
-      if (!forceUpdate) {
-        if (!containerFile.exists() || !containerFile.canWrite()) {
-          throw new StorageContainerException(
-              "Container file not exists or corrupted. ID: " + containerID,
-              CONTAINER_INTERNAL_ERROR);
-        }
-
-        // Backup the container file
-        containerFileBK = File.createTempFile(
-            "tmp_" + System.currentTimeMillis() + "_",
-            containerFile.getName(), containerFile.getParentFile());
-        FileUtils.copyFile(containerFile, containerFileBK);
-
-        deleted = containerFile.delete();
-        containerStream = new FileOutputStream(containerFile);
-        dos = new DigestOutputStream(containerStream, sha);
-
-        ContainerProtos.ContainerData protoData = data.getProtoBufMessage();
-        protoData.writeDelimitedTo(dos);
-      }
-
-      // Update the in-memory map
-      containerMap.replace(containerID, data);
-    } catch (IOException e) {
-      // Restore the container file from backup
-      if(containerFileBK != null && containerFileBK.exists() && deleted) {
-        if(containerFile.delete()
-            && containerFileBK.renameTo(containerFile)) {
-          throw new StorageContainerException("Container update failed,"
-              + " container data restored from the backup.",
-              CONTAINER_INTERNAL_ERROR);
-        } else {
-          throw new StorageContainerException(
-              "Failed to restore container data from the backup. ID: "
-                  + containerID, CONTAINER_INTERNAL_ERROR);
-        }
-      } else {
-        throw new StorageContainerException(
-            e.getMessage(), CONTAINER_INTERNAL_ERROR);
-      }
-    } finally {
-      if (containerFileBK != null && containerFileBK.exists()) {
-        if(!containerFileBK.delete()) {
-          LOG.warn("Unable to delete container file backup : {}.",
-              containerFileBK.getAbsolutePath());
-        }
-      }
-      IOUtils.closeStream(dos);
-      IOUtils.closeStream(containerStream);
-    }
-  }
-
-  @VisibleForTesting
-  protected File getContainerFile(ContainerData data) throws IOException {
-    return ContainerUtils.getContainerFile(data,
-        this.locationManager.getContainerPath());
-  }
-
-  /**
-   * Checks if a container exists.
-   *
-   * @param containerID - ID of the container.
-   * @return true if the container is open false otherwise.
-   * @throws StorageContainerException - Throws Exception if we are not able to
-   *                                   find the container.
-   */
-  @Override
-  public boolean isOpen(long containerID) throws StorageContainerException {
-    final ContainerData containerData = containerMap.get(containerID);
-    if (containerData == null) {
-      throw new StorageContainerException(
-          "Container not found: " + containerID, CONTAINER_NOT_FOUND);
-    }
-    return containerData.isOpen();
-  }
-
-  /**
-   * Returns LifeCycle State of the container.
-   * @param containerID - Id of the container
-   * @return LifeCycle State of the container
-   * @throws StorageContainerException
-   */
-  private HddsProtos.LifeCycleState getState(long containerID)
-      throws StorageContainerException {
-    LifeCycleState state;
-    final ContainerData data = containerMap.get(containerID);
-    if (data == null) {
-      throw new StorageContainerException(
-          "Container status not found: " + containerID, CONTAINER_NOT_FOUND);
-    }
-    switch (data.getState()) {
-    case OPEN:
-      state = LifeCycleState.OPEN;
-      break;
-    case CLOSING:
-      state = LifeCycleState.CLOSING;
-      break;
-    case CLOSED:
-      state = LifeCycleState.CLOSED;
-      break;
-    default:
-      throw new StorageContainerException(
-          "Invalid Container state found: " + containerID,
-          INVALID_CONTAINER_STATE);
-    }
-
-    return state;
-  }
-
-  /**
-   * Supports clean shutdown of container.
-   *
-   * @throws IOException
-   */
-  @Override
-  public void shutdown() throws IOException {
-    Preconditions.checkState(this.hasWriteLock(),
-        "Assumption that we are holding the lock violated.");
-    this.containerMap.clear();
-    this.locationManager.shutdown();
-  }
-
-
-  @VisibleForTesting
-  public ConcurrentSkipListMap<Long, ContainerData> getContainerMap() {
-    return containerMap;
-  }
-
-  /**
-   * Acquire read lock.
-   */
-  @Override
-  public void readLock() {
-    this.lock.readLock().lock();
-
-  }
-
-  @Override
-  public void readLockInterruptibly() throws InterruptedException {
-    this.lock.readLock().lockInterruptibly();
-  }
-
-  /**
-   * Release read lock.
-   */
-  @Override
-  public void readUnlock() {
-    this.lock.readLock().unlock();
-  }
-
-  /**
-   * Check if the current thread holds read lock.
-   */
-  @Override
-  public boolean hasReadLock() {
-    return this.lock.readLock().tryLock();
-  }
-
-  /**
-   * Acquire write lock.
-   */
-  @Override
-  public void writeLock() {
-    this.lock.writeLock().lock();
-  }
-
-  /**
-   * Acquire write lock, unless interrupted while waiting.
-   */
-  @Override
-  public void writeLockInterruptibly() throws InterruptedException {
-    this.lock.writeLock().lockInterruptibly();
-
-  }
-
-  /**
-   * Release write lock.
-   */
-  @Override
-  public void writeUnlock() {
-    this.lock.writeLock().unlock();
-
-  }
-
-  /**
-   * Check if the current thread holds write lock.
-   */
-  @Override
-  public boolean hasWriteLock() {
-    return this.lock.writeLock().isHeldByCurrentThread();
-  }
-
-  public ChunkManager getChunkManager() {
-    return this.chunkManager;
-  }
-
-  /**
-   * Sets the chunk Manager.
-   *
-   * @param chunkManager - Chunk Manager
-   */
-  public void setChunkManager(ChunkManager chunkManager) {
-    this.chunkManager = chunkManager;
-  }
-
-  /**
-   * Gets the Key Manager.
-   *
-   * @return KeyManager.
-   */
-  @Override
-  public KeyManager getKeyManager() {
-    return this.keyManager;
-  }
-
-  /**
-   * Get the node report.
-   * @return node report.
-   */
-  @Override
-  public NodeReportProto getNodeReport() throws IOException {
-    StorageLocationReport[] reports = locationManager.getLocationReport();
-    NodeReportProto.Builder nrb = NodeReportProto.newBuilder();
-    for (int i = 0; i < reports.length; i++) {
-      StorageReportProto.Builder srb = StorageReportProto.newBuilder();
-      nrb.addStorageReport(reports[i].getProtoBufMessage());
-    }
-    return nrb.build();
-  }
-
-
-  /**
-   * Gets container reports.
-   *
-   * @return List of all closed containers.
-   * @throws IOException
-   */
-  @Override
-  public List<ContainerData> getClosedContainerReports() throws IOException {
-    LOG.debug("Starting container report iteration.");
-    // No need for locking since containerMap is a ConcurrentSkipListMap
-    // And we can never get the exact state since close might happen
-    // after we iterate a point.
-    return containerMap.entrySet().stream()
-        .filter(containerData ->
-            containerData.getValue().isClosed())
-        .map(containerData -> containerData.getValue())
-        .collect(Collectors.toList());
-  }
-
-  /**
-   * Get container report.
-   *
-   * @return The container report.
-   * @throws IOException
-   */
-  @Override
-  public ContainerReportsProto getContainerReport() throws IOException {
-    LOG.debug("Starting container report iteration.");
-    // No need for locking since containerMap is a ConcurrentSkipListMap
-    // And we can never get the exact state since close might happen
-    // after we iterate a point.
-    List<ContainerData> containers = containerMap.values().stream()
-        .collect(Collectors.toList());
-
-    ContainerReportsProto.Builder crBuilder =
-        ContainerReportsProto.newBuilder();
-
-    for (ContainerData container: containers) {
-      long containerId = container.getContainerID();
-      StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder =
-          StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder();
-      ciBuilder.setContainerID(container.getContainerID())
-          .setSize(container.getMaxSize())
-          .setUsed(container.getBytesUsed())
-          .setKeyCount(container.getKeyCount())
-          .setReadCount(container.getReadCount())
-          .setWriteCount(container.getWriteCount())
-          .setReadBytes(container.getReadBytes())
-          .setWriteBytes(container.getWriteBytes())
-          .setState(getState(containerId))
-          .setDeleteTransactionId(container.getDeleteTransactionId());
-
-      crBuilder.addReports(ciBuilder.build());
-    }
-
-    return crBuilder.build();
-  }
-
-  /**
-   * Sets the Key Manager.
-   *
-   * @param keyManager - Key Manager.
-   */
-  @Override
-  public void setKeyManager(KeyManager keyManager) {
-    this.keyManager = keyManager;
-  }
-
-  /**
-   * Filter out only container files from the container metadata dir.
-   */
-  private static class ContainerFilter implements FilenameFilter {
-    /**
-     * Tests if a specified file should be included in a file list.
-     *
-     * @param dir the directory in which the file was found.
-     * @param name the name of the file.
-     * @return <code>true</code> if and only if the name should be included in
-     * the file list; <code>false</code> otherwise.
-     */
-    @Override
-    public boolean accept(File dir, String name) {
-      return name.endsWith(CONTAINER_EXTENSION);
-    }
-  }
-
-  @Override
-  public List<ContainerData> chooseContainerForBlockDeletion(
-      int count) throws StorageContainerException {
-    readLock();
-    try {
-      return containerDeletionChooser.chooseContainerForBlockDeletion(
-          count, containerMap);
-    } finally {
-      readUnlock();
-    }
-  }
-
-  @VisibleForTesting
-  public ContainerDeletionChoosingPolicy getContainerDeletionChooser() {
-    return containerDeletionChooser;
-  }
-
-  @Override
-  public void incrPendingDeletionBlocks(int numBlocks, long containerId) {
-    writeLock();
-    try {
-      ContainerData cData = containerMap.get(containerId);
-      cData.incrPendingDeletionBlocks(numBlocks);
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  @Override
-  public void decrPendingDeletionBlocks(int numBlocks, long containerId) {
-    writeLock();
-    try {
-      ContainerData cData = containerMap.get(containerId);
-      cData.decrPendingDeletionBlocks(numBlocks);
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  /**
-   * Increase the read count of the container.
-   *
-   * @param containerId - ID of the container.
-   */
-  @Override
-  public void incrReadCount(long containerId) {
-    ContainerData cData = containerMap.get(containerId);
-    cData.incrReadCount();
-  }
-
-  public long getReadCount(long containerId) {
-    ContainerData cData = containerMap.get(containerId);
-    return cData.getReadCount();
-  }
-
-  /**
-   * Increase the read counter for bytes read from the container.
-   *
-   * @param containerId - ID of the container.
-   * @param readBytes     - bytes read from the container.
-   */
-  @Override
-  public void incrReadBytes(long containerId, long readBytes) {
-    ContainerData cData = containerMap.get(containerId);
-    cData.incrReadBytes(readBytes);
-  }
-
-  /**
-   * Returns number of bytes read from the container.
-   * @param containerId
-   * @return
-   */
-  public long getReadBytes(long containerId) {
-    readLock();
-    try {
-      ContainerData cData = containerMap.get(containerId);
-      return cData.getReadBytes();
-    } finally {
-      readUnlock();
-    }
-  }
-
-  /**
-   * Increase the write count of the container.
-   *
-   * @param containerId - Name of the container.
-   */
-  @Override
-  public void incrWriteCount(long containerId) {
-    ContainerData cData = containerMap.get(containerId);
-    cData.incrWriteCount();
-  }
-
-  public long getWriteCount(long containerId) {
-    ContainerData cData = containerMap.get(containerId);
-    return cData.getWriteCount();
-  }
-
-  /**
-   * Increse the write counter for bytes write into the container.
-   *
-   * @param containerId   - ID of the container.
-   * @param writeBytes    - bytes write into the container.
-   */
-  @Override
-  public void incrWriteBytes(long containerId, long writeBytes) {
-    ContainerData cData = containerMap.get(containerId);
-    cData.incrWriteBytes(writeBytes);
-  }
-
-  public long getWriteBytes(long containerId) {
-    ContainerData cData = containerMap.get(containerId);
-    return cData.getWriteBytes();
-  }
-
-  /**
-   * Increase the bytes used by the container.
-   *
-   * @param containerId   - ID of the container.
-   * @param used          - additional bytes used by the container.
-   * @return the current bytes used.
-   */
-  @Override
-  public long incrBytesUsed(long containerId, long used) {
-    ContainerData cData = containerMap.get(containerId);
-    return cData.incrBytesUsed(used);
-  }
-
-  /**
-   * Decrease the bytes used by the container.
-   *
-   * @param containerId   - ID of the container.
-   * @param used          - additional bytes reclaimed by the container.
-   * @return the current bytes used.
-   */
-  @Override
-  public long decrBytesUsed(long containerId, long used) {
-    ContainerData cData = containerMap.get(containerId);
-    return cData.decrBytesUsed(used);
-  }
-
-  public long getBytesUsed(long containerId) {
-    ContainerData cData = containerMap.get(containerId);
-    return cData.getBytesUsed();
-  }
-
-  /**
-   * Get the number of keys in the container.
-   *
-   * @param containerId - ID of the container.
-   * @return the current key count.
-   */
-  @Override
-  public long getNumKeys(long containerId) {
-    ContainerData cData = containerMap.get(containerId);
-    return cData.getKeyCount();
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
index 18a7839..bcba8c8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.container.common.impl;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto
@@ -63,7 +64,7 @@ public class ContainerSet {
       StorageContainerException {
     Preconditions.checkNotNull(container, "container cannot be null");
 
-    long containerId = container.getContainerData().getContainerId();
+    long containerId = container.getContainerData().getContainerID();
     if(containerMap.putIfAbsent(containerId, container) == null) {
       LOG.debug("Container with container Id {} is added to containerMap",
           containerId);
@@ -133,6 +134,13 @@ public class ContainerSet {
     return containerMap.entrySet().iterator();
   }
 
+  /**
+   * Return a copy of the containerMap
+   * @return containerMap
+   */
+  public Map<Long, Container> getContainerMap() {
+    return ImmutableMap.copyOf(containerMap);
+  }
 
   /**
    * A simple interface for container Iterations.
@@ -196,7 +204,7 @@ public class ContainerSet {
 
 
     for (Container container: containers) {
-      long containerId = container.getContainerData().getContainerId();
+      long containerId = container.getContainerData().getContainerID();
       ContainerInfo.Builder ciBuilder = ContainerInfo.newBuilder();
       ContainerData containerData = container.getContainerData();
       ciBuilder.setContainerID(containerId)
@@ -234,9 +242,14 @@ public class ContainerSet {
       break;
     default:
       throw new StorageContainerException("Invalid Container state found: " +
-          containerData.getContainerId(), INVALID_CONTAINER_STATE);
+          containerData.getContainerID(), INVALID_CONTAINER_STATE);
     }
     return state;
   }
 
+  // TODO: Implement BlockDeletingService
+  public List<ContainerData> chooseContainerForBlockDeletion(
+      int count) throws StorageContainerException {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerStorageLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerStorageLocation.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerStorageLocation.java
deleted file mode 100644
index 7431baa..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerStorageLocation.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.impl;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CachingGetSpaceUsed;
-import org.apache.hadoop.fs.DF;
-import org.apache.hadoop.fs.GetSpaceUsed;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.util.ShutdownHookManager;
-import org.apache.hadoop.util.Time;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.net.URI;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Paths;
-import java.util.Scanner;
-
-import static org.apache.hadoop.util.RunJar.SHUTDOWN_HOOK_PRIORITY;
-
-/**
- * Class that wraps the space usage of the Datanode Container Storage Location
- * by SCM containers.
- */
-public class ContainerStorageLocation {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(ContainerStorageLocation.class);
-
-  private static final String DU_CACHE_FILE = "scmUsed";
-  private volatile boolean scmUsedSaved = false;
-
-  private final StorageLocation dataLocation;
-  private final String storageUuId;
-  private final DF usage;
-  private final GetSpaceUsed scmUsage;
-  private final File scmUsedFile;
-
-  public ContainerStorageLocation(StorageLocation dataLoc, Configuration conf)
-      throws IOException {
-    this.dataLocation = dataLoc;
-    this.storageUuId = DatanodeStorage.generateUuid();
-    File dataDir = Paths.get(dataLoc.getNormalizedUri()).resolve(
-        OzoneConsts.CONTAINER_PREFIX).toFile();
-    // Initialize container data root if it does not exist as required by DF/DU
-    if (!dataDir.exists()) {
-      if (!dataDir.mkdirs()) {
-        LOG.error("Unable to create the container storage location at : {}",
-            dataDir);
-        throw new IllegalArgumentException("Unable to create the container" +
-            " storage location at : " + dataDir);
-      }
-    }
-    scmUsedFile = new File(dataDir, DU_CACHE_FILE);
-    // get overall disk usage
-    this.usage = new DF(dataDir, conf);
-    // get SCM specific usage
-    this.scmUsage = new CachingGetSpaceUsed.Builder().setPath(dataDir)
-        .setConf(conf)
-        .setInitialUsed(loadScmUsed())
-        .build();
-
-    // Ensure scm usage is saved during shutdown.
-    ShutdownHookManager.get().addShutdownHook(
-        new Runnable() {
-          @Override
-          public void run() {
-            if (!scmUsedSaved) {
-              saveScmUsed();
-            }
-          }
-        }, SHUTDOWN_HOOK_PRIORITY);
-  }
-
-  public URI getNormalizedUri() {
-    return dataLocation.getNormalizedUri();
-  }
-
-  public String getStorageUuId() {
-    return storageUuId;
-  }
-  public long getCapacity() {
-    long capacity = usage.getCapacity();
-    return (capacity > 0) ? capacity : 0;
-  }
-
-  public long getAvailable() throws IOException {
-    long remaining = getCapacity() - getScmUsed();
-    long available = usage.getAvailable();
-    if (remaining > available) {
-      remaining = available;
-    }
-    return (remaining > 0) ? remaining : 0;
-  }
-
-  public long getScmUsed() throws IOException{
-    return scmUsage.getUsed();
-  }
-
-  public String getStorageLocation() {
-    return getNormalizedUri().getRawPath();
-  }
-
-  public StorageType getStorageType() {
-    return dataLocation.getStorageType();
-  }
-
-  public void shutdown() {
-    saveScmUsed();
-    scmUsedSaved = true;
-
-    if (scmUsage instanceof CachingGetSpaceUsed) {
-      IOUtils.cleanupWithLogger(null, ((CachingGetSpaceUsed) scmUsage));
-    }
-  }
-
-  /**
-   * Read in the cached DU value and return it if it is less than 600 seconds
-   * old (DU update interval). Slight imprecision of scmUsed is not critical
-   * and skipping DU can significantly shorten the startup time.
-   * If the cached value is not available or too old, -1 is returned.
-   */
-  long loadScmUsed() {
-    long cachedScmUsed;
-    long mtime;
-    Scanner sc;
-
-    try {
-      sc = new Scanner(scmUsedFile, "UTF-8");
-    } catch (FileNotFoundException fnfe) {
-      return -1;
-    }
-
-    try {
-      // Get the recorded scmUsed from the file.
-      if (sc.hasNextLong()) {
-        cachedScmUsed = sc.nextLong();
-      } else {
-        return -1;
-      }
-      // Get the recorded mtime from the file.
-      if (sc.hasNextLong()) {
-        mtime = sc.nextLong();
-      } else {
-        return -1;
-      }
-
-      // Return the cached value if mtime is okay.
-      if (mtime > 0 && (Time.now() - mtime < 600000L)) {
-        LOG.info("Cached ScmUsed found for {} : {} ", dataLocation,
-            cachedScmUsed);
-        return cachedScmUsed;
-      }
-      return -1;
-    } finally {
-      sc.close();
-    }
-  }
-
-  /**
-   * Write the current scmUsed to the cache file.
-   */
-  void saveScmUsed() {
-    if (scmUsedFile.exists() && !scmUsedFile.delete()) {
-      LOG.warn("Failed to delete old scmUsed file in {}.", dataLocation);
-    }
-    OutputStreamWriter out = null;
-    try {
-      long used = getScmUsed();
-      if (used > 0) {
-        out = new OutputStreamWriter(new FileOutputStream(scmUsedFile),
-            StandardCharsets.UTF_8);
-        // mtime is written last, so that truncated writes won't be valid.
-        out.write(Long.toString(used) + " " + Long.toString(Time.now()));
-        out.flush();
-        out.close();
-        out = null;
-      }
-    } catch (IOException ioe) {
-      // If write failed, the volume might be bad. Since the cache file is
-      // not critical, log the error and continue.
-      LOG.warn("Failed to write scmUsed to " + scmUsedFile, ioe);
-    } finally {
-      IOUtils.cleanupWithLogger(null, out);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
deleted file mode 100644
index 3ffe6e4..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
+++ /dev/null
@@ -1,695 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.impl;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.ozone.container.common.interfaces.Handler;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.scm.container.common.helpers
-    .StorageContainerException;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .ContainerCommandRequestProto;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .ContainerCommandResponseProto;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type;
-import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
-import org.apache.hadoop.ozone.container.common.helpers.ChunkUtils;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
-import org.apache.hadoop.ozone.container.keyvalue.helpers.SmallFileUtils;
-import org.apache.hadoop.ozone.container.common.helpers.KeyData;
-import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.security.NoSuchAlgorithmException;
-import java.util.LinkedList;
-import java.util.List;
-
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.CLOSED_CONTAINER_IO;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.GET_SMALL_FILE_ERROR;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.NO_SUCH_ALGORITHM;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.PUT_SMALL_FILE_ERROR;
-
-/**
- * Ozone Container dispatcher takes a call from the netty server and routes it
- * to the right handler function.
- */
-public class Dispatcher implements ContainerDispatcher {
-  static final Logger LOG = LoggerFactory.getLogger(Dispatcher.class);
-
-  private final ContainerManager containerManager;
-  private ContainerMetrics metrics;
-  private Configuration conf;
-
-  /**
-   * Constructs an OzoneContainer that receives calls from
-   * XceiverServerHandler.
-   *
-   * @param containerManager - A class that manages containers.
-   */
-  public Dispatcher(ContainerManager containerManager, Configuration config) {
-    Preconditions.checkNotNull(containerManager);
-    this.containerManager = containerManager;
-    this.metrics = null;
-    this.conf = config;
-  }
-
-  @Override
-  public void init() {
-    this.metrics = ContainerMetrics.create(conf);
-  }
-
-  @Override
-  public void shutdown() {
-  }
-
-  @Override
-  public Handler getHandler(ContainerProtos.ContainerType containerType) {
-    return null;
-  }
-
-  @Override
-  public void setScmId(String scmId) {
-    // DO nothing, this will be removed when cleanup.
-  }
-
-  @Override
-  public ContainerCommandResponseProto dispatch(
-      ContainerCommandRequestProto msg) {
-    LOG.trace("Command {}, trace ID: {} ", msg.getCmdType().toString(),
-        msg.getTraceID());
-    long startNanos = System.nanoTime();
-    ContainerCommandResponseProto resp = null;
-    try {
-      Preconditions.checkNotNull(msg);
-      Type cmdType = msg.getCmdType();
-      metrics.incContainerOpsMetrics(cmdType);
-      if ((cmdType == Type.CreateContainer) ||
-          (cmdType == Type.DeleteContainer) ||
-          (cmdType == Type.ReadContainer) ||
-          (cmdType == Type.ListContainer) ||
-          (cmdType == Type.UpdateContainer) ||
-          (cmdType == Type.CloseContainer)) {
-        resp = containerProcessHandler(msg);
-      }
-
-      if ((cmdType == Type.PutKey) ||
-          (cmdType == Type.GetKey) ||
-          (cmdType == Type.DeleteKey) ||
-          (cmdType == Type.ListKey)) {
-        resp = keyProcessHandler(msg);
-      }
-
-      if ((cmdType == Type.WriteChunk) ||
-          (cmdType == Type.ReadChunk) ||
-          (cmdType == Type.DeleteChunk)) {
-        resp = chunkProcessHandler(msg);
-      }
-
-      if ((cmdType == Type.PutSmallFile) ||
-          (cmdType == Type.GetSmallFile)) {
-        resp = smallFileHandler(msg);
-      }
-
-      if (resp != null) {
-        metrics.incContainerOpsLatencies(cmdType,
-            System.nanoTime() - startNanos);
-        return resp;
-      }
-
-      return ContainerUtils.unsupportedRequest(msg);
-    } catch (StorageContainerException e) {
-      // This useful since the trace ID will allow us to correlate failures.
-      return ContainerUtils.logAndReturnError(LOG, e, msg);
-    }
-  }
-
-  public ContainerMetrics getContainerMetrics() {
-    return metrics;
-  }
-
-  /**
-   * Handles the all Container related functionality.
-   *
-   * @param msg - command
-   * @return - response
-   * @throws StorageContainerException
-   */
-  private ContainerCommandResponseProto containerProcessHandler(
-      ContainerCommandRequestProto msg) throws StorageContainerException {
-    try {
-
-      switch (msg.getCmdType()) {
-      case CreateContainer:
-        return handleCreateContainer(msg);
-
-      case DeleteContainer:
-        return handleDeleteContainer(msg);
-
-      case ListContainer:
-        // TODO : Support List Container.
-        return ContainerUtils.unsupportedRequest(msg);
-
-      case UpdateContainer:
-        return handleUpdateContainer(msg);
-
-      case ReadContainer:
-        return handleReadContainer(msg);
-
-      case CloseContainer:
-        return handleCloseContainer(msg);
-
-      default:
-        return ContainerUtils.unsupportedRequest(msg);
-      }
-    } catch (StorageContainerException e) {
-      return ContainerUtils.logAndReturnError(LOG, e, msg);
-    } catch (IOException ex) {
-      LOG.warn("Container operation failed. " +
-              "Container: {} Operation: {}  trace ID: {} Error: {}",
-          msg.getCreateContainer().getContainerID(),
-          msg.getCmdType().name(),
-          msg.getTraceID(),
-          ex.toString(), ex);
-
-      // TODO : Replace with finer error codes.
-      return ContainerUtils.getContainerCommandResponse(msg,
-          ContainerProtos.Result.CONTAINER_INTERNAL_ERROR,
-          ex.toString()).build();
-    }
-  }
-
-  /**
-   * Handles the all key related functionality.
-   *
-   * @param msg - command
-   * @return - response
-   * @throws StorageContainerException
-   */
-  private ContainerCommandResponseProto keyProcessHandler(
-      ContainerCommandRequestProto msg) throws StorageContainerException {
-    try {
-      switch (msg.getCmdType()) {
-      case PutKey:
-        return handlePutKey(msg);
-
-      case GetKey:
-        return handleGetKey(msg);
-
-      case DeleteKey:
-        return handleDeleteKey(msg);
-
-      case ListKey:
-        return ContainerUtils.unsupportedRequest(msg);
-
-      default:
-        return ContainerUtils.unsupportedRequest(msg);
-
-      }
-    } catch (StorageContainerException e) {
-      return ContainerUtils.logAndReturnError(LOG, e, msg);
-    } catch (IOException ex) {
-      LOG.warn("Container operation failed. " +
-              "Container: {} Operation: {}  trace ID: {} Error: {}",
-          msg.getCreateContainer().getContainerID(),
-          msg.getCmdType().name(),
-          msg.getTraceID(),
-          ex.toString(), ex);
-
-      // TODO : Replace with finer error codes.
-      return ContainerUtils.getContainerCommandResponse(msg,
-          ContainerProtos.Result.CONTAINER_INTERNAL_ERROR,
-          ex.toString()).build();
-    }
-  }
-
-  /**
-   * Handles the all chunk related functionality.
-   *
-   * @param msg - command
-   * @return - response
-   * @throws StorageContainerException
-   */
-  private ContainerCommandResponseProto chunkProcessHandler(
-      ContainerCommandRequestProto msg) throws StorageContainerException {
-    try {
-      switch (msg.getCmdType()) {
-      case WriteChunk:
-        return handleWriteChunk(msg);
-
-      case ReadChunk:
-        return handleReadChunk(msg);
-
-      case DeleteChunk:
-        return handleDeleteChunk(msg);
-
-      case ListChunk:
-        return ContainerUtils.unsupportedRequest(msg);
-
-      default:
-        return ContainerUtils.unsupportedRequest(msg);
-      }
-    } catch (StorageContainerException e) {
-      return ContainerUtils.logAndReturnError(LOG, e, msg);
-    } catch (IOException ex) {
-      LOG.warn("Container operation failed. " +
-              "Container: {} Operation: {}  trace ID: {} Error: {}",
-          msg.getCreateContainer().getContainerID(),
-          msg.getCmdType().name(),
-          msg.getTraceID(),
-          ex.toString(), ex);
-
-      // TODO : Replace with finer error codes.
-      return ContainerUtils.getContainerCommandResponse(msg,
-          ContainerProtos.Result.CONTAINER_INTERNAL_ERROR,
-          ex.toString()).build();
-    }
-  }
-
-  /**
-   * Dispatch calls to small file hanlder.
-   * @param msg - request
-   * @return response
-   * @throws StorageContainerException
-   */
-  private ContainerCommandResponseProto smallFileHandler(
-      ContainerCommandRequestProto msg) throws StorageContainerException {
-    switch (msg.getCmdType()) {
-    case PutSmallFile:
-      return handlePutSmallFile(msg);
-    case GetSmallFile:
-      return handleGetSmallFile(msg);
-    default:
-      return ContainerUtils.unsupportedRequest(msg);
-    }
-  }
-
-  /**
-   * Update an existing container with the new container data.
-   *
-   * @param msg Request
-   * @return ContainerCommandResponseProto
-   * @throws IOException
-   */
-  private ContainerCommandResponseProto handleUpdateContainer(
-      ContainerCommandRequestProto msg)
-      throws IOException {
-    if (!msg.hasUpdateContainer()) {
-      LOG.debug("Malformed read container request. trace ID: {}",
-          msg.getTraceID());
-      return ContainerUtils.malformedRequest(msg);
-    }
-    long containerID = msg.getUpdateContainer().getContainerID();
-
-    ContainerData data = new ContainerData(msg.getUpdateContainer()
-        .getContainerID(), conf);
-    boolean forceUpdate = msg.getUpdateContainer().getForceUpdate();
-    this.containerManager.updateContainer(containerID,
-        data, forceUpdate);
-    return ContainerUtils.getSuccessResponse(msg);
-  }
-
-  /**
-   * Calls into container logic and returns appropriate response.
-   *
-   * @param msg - Request
-   * @return ContainerCommandResponseProto
-   * @throws IOException
-   */
-  private ContainerCommandResponseProto handleReadContainer(
-      ContainerCommandRequestProto msg)
-      throws IOException {
-
-    if (!msg.hasReadContainer()) {
-      LOG.debug("Malformed read container request. trace ID: {}",
-          msg.getTraceID());
-      return ContainerUtils.malformedRequest(msg);
-    }
-
-    long containerID = msg.getReadContainer().getContainerID();
-    ContainerData container = this.containerManager.
-        readContainer(containerID);
-    return ContainerUtils.getReadContainerResponse(msg, container);
-  }
-
-  /**
-   * Calls into container logic and returns appropriate response.
-   *
-   * @param msg - Request
-   * @return Response.
-   * @throws IOException
-   */
-  private ContainerCommandResponseProto handleDeleteContainer(
-      ContainerCommandRequestProto msg) throws IOException {
-
-    if (!msg.hasDeleteContainer()) {
-      LOG.debug("Malformed delete container request. trace ID: {}",
-          msg.getTraceID());
-      return ContainerUtils.malformedRequest(msg);
-    }
-
-    long containerID = msg.getDeleteContainer().getContainerID();
-    boolean forceDelete = msg.getDeleteContainer().getForceDelete();
-    this.containerManager.deleteContainer(containerID, forceDelete);
-    return ContainerUtils.getSuccessResponse(msg);
-  }
-
-  /**
-   * Calls into container logic and returns appropriate response.
-   *
-   * @param msg - Request
-   * @return Response.
-   * @throws IOException
-   */
-  private ContainerCommandResponseProto handleCreateContainer(
-      ContainerCommandRequestProto msg) throws IOException {
-    if (!msg.hasCreateContainer()) {
-      LOG.debug("Malformed create container request. trace ID: {}",
-          msg.getTraceID());
-      return ContainerUtils.malformedRequest(msg);
-    }
-    ContainerData cData = new ContainerData(
-        msg.getCreateContainer().getContainerID(), conf);
-
-    this.containerManager.createContainer(cData);
-    return ContainerUtils.getSuccessResponse(msg);
-  }
-
-  /**
-   * closes an open container.
-   *
-   * @param msg -
-   * @return
-   * @throws IOException
-   */
-  private ContainerCommandResponseProto handleCloseContainer(
-      ContainerCommandRequestProto msg) throws IOException {
-    try {
-      if (!msg.hasCloseContainer()) {
-        LOG.debug("Malformed close Container request. trace ID: {}",
-            msg.getTraceID());
-        return ContainerUtils.malformedRequest(msg);
-      }
-      long containerID = msg.getCloseContainer().getContainerID();
-      if (!this.containerManager.isOpen(containerID)) {
-        throw new StorageContainerException("Attempting to close a closed " +
-            "container.", CLOSED_CONTAINER_IO);
-      }
-      this.containerManager.closeContainer(containerID);
-      return ContainerUtils.getSuccessResponse(msg);
-    } catch (NoSuchAlgorithmException e) {
-      throw new StorageContainerException("No such Algorithm", e,
-          NO_SUCH_ALGORITHM);
-    }
-  }
-
-  /**
-   * Calls into chunk manager to write a chunk.
-   *
-   * @param msg - Request.
-   * @return Response.
-   * @throws IOException
-   */
-  private ContainerCommandResponseProto handleWriteChunk(
-      ContainerCommandRequestProto msg) throws IOException {
-    if (!msg.hasWriteChunk()) {
-      LOG.debug("Malformed write chunk request. trace ID: {}",
-          msg.getTraceID());
-      return ContainerUtils.malformedRequest(msg);
-    }
-    BlockID blockID = BlockID.getFromProtobuf(
-        msg.getWriteChunk().getBlockID());
-    if (!this.containerManager.isOpen(blockID.getContainerID())) {
-      throw new StorageContainerException("Write to closed container.",
-          CLOSED_CONTAINER_IO);
-    }
-
-    ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(msg.getWriteChunk()
-        .getChunkData());
-    Preconditions.checkNotNull(chunkInfo);
-    byte[] data = null;
-    if (msg.getWriteChunk().getStage() == ContainerProtos.Stage.WRITE_DATA
-        || msg.getWriteChunk().getStage() == ContainerProtos.Stage.COMBINED) {
-      data = msg.getWriteChunk().getData().toByteArray();
-      metrics.incContainerBytesStats(Type.WriteChunk, data.length);
-
-    }
-    this.containerManager.getChunkManager()
-        .writeChunk(blockID, chunkInfo,
-            data, msg.getWriteChunk().getStage());
-
-    return ChunkUtils.getChunkResponse(msg);
-  }
-
-  /**
-   * Calls into chunk manager to read a chunk.
-   *
-   * @param msg - Request.
-   * @return - Response.
-   * @throws IOException
-   */
-  private ContainerCommandResponseProto handleReadChunk(
-      ContainerCommandRequestProto msg) throws IOException {
-    if (!msg.hasReadChunk()) {
-      LOG.debug("Malformed read chunk request. trace ID: {}",
-          msg.getTraceID());
-      return ContainerUtils.malformedRequest(msg);
-    }
-    BlockID blockID = BlockID.getFromProtobuf(
-        msg.getReadChunk().getBlockID());
-    ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(msg.getReadChunk()
-        .getChunkData());
-    Preconditions.checkNotNull(chunkInfo);
-    byte[] data = this.containerManager.getChunkManager().
-        readChunk(blockID, chunkInfo);
-    metrics.incContainerBytesStats(Type.ReadChunk, data.length);
-    return ChunkUtils.getReadChunkResponse(msg, data, chunkInfo);
-  }
-
-  /**
-   * Calls into chunk manager to write a chunk.
-   *
-   * @param msg - Request.
-   * @return Response.
-   * @throws IOException
-   */
-  private ContainerCommandResponseProto handleDeleteChunk(
-      ContainerCommandRequestProto msg) throws IOException {
-    if (!msg.hasDeleteChunk()) {
-      LOG.debug("Malformed delete chunk request. trace ID: {}",
-          msg.getTraceID());
-      return ContainerUtils.malformedRequest(msg);
-    }
-
-    BlockID blockID = BlockID.getFromProtobuf(msg.getDeleteChunk()
-        .getBlockID());
-    long containerID = blockID.getContainerID();
-    if (!this.containerManager.isOpen(containerID)) {
-      throw new StorageContainerException("Write to closed container.",
-          CLOSED_CONTAINER_IO);
-    }
-    ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(msg.getDeleteChunk()
-        .getChunkData());
-    Preconditions.checkNotNull(chunkInfo);
-
-    this.containerManager.getChunkManager().deleteChunk(blockID,
-        chunkInfo);
-    return ChunkUtils.getChunkResponse(msg);
-  }
-
-  /**
-   * Put Key handler.
-   *
-   * @param msg - Request.
-   * @return - Response.
-   * @throws IOException
-   */
-  private ContainerCommandResponseProto handlePutKey(
-      ContainerCommandRequestProto msg) throws IOException {
-    if (!msg.hasPutKey()) {
-      LOG.debug("Malformed put key request. trace ID: {}",
-          msg.getTraceID());
-      return ContainerUtils.malformedRequest(msg);
-    }
-    BlockID blockID = BlockID.getFromProtobuf(
-        msg.getPutKey().getKeyData().getBlockID());
-    long containerID = blockID.getContainerID();
-    if (!this.containerManager.isOpen(containerID)) {
-      throw new StorageContainerException("Write to closed container.",
-          CLOSED_CONTAINER_IO);
-    }
-    KeyData keyData = KeyData.getFromProtoBuf(msg.getPutKey().getKeyData());
-    Preconditions.checkNotNull(keyData);
-    this.containerManager.getKeyManager().putKey(keyData);
-    long numBytes = keyData.getProtoBufMessage().toByteArray().length;
-    metrics.incContainerBytesStats(Type.PutKey, numBytes);
-    return KeyUtils.getKeyResponse(msg);
-  }
-
-  /**
-   * Handle Get Key.
-   *
-   * @param msg - Request.
-   * @return - Response.
-   * @throws IOException
-   */
-  private ContainerCommandResponseProto handleGetKey(
-      ContainerCommandRequestProto msg) throws IOException {
-    if (!msg.hasGetKey()) {
-      LOG.debug("Malformed get key request. trace ID: {}",
-          msg.getTraceID());
-      return ContainerUtils.malformedRequest(msg);
-    }
-    KeyData keyData = new KeyData(
-        BlockID.getFromProtobuf(msg.getGetKey().getBlockID()));
-    Preconditions.checkNotNull(keyData);
-    KeyData responseData =
-        this.containerManager.getKeyManager().getKey(keyData);
-    long numBytes = responseData.getProtoBufMessage().toByteArray().length;
-    metrics.incContainerBytesStats(Type.GetKey, numBytes);
-    return KeyUtils.getKeyDataResponse(msg, responseData);
-  }
-
-  /**
-   * Handle Delete Key.
-   *
-   * @param msg - Request.
-   * @return - Response.
-   * @throws IOException
-   */
-  private ContainerCommandResponseProto handleDeleteKey(
-      ContainerCommandRequestProto msg) throws IOException {
-    if (!msg.hasDeleteKey()) {
-      LOG.debug("Malformed delete key request. trace ID: {}",
-          msg.getTraceID());
-      return ContainerUtils.malformedRequest(msg);
-    }
-    BlockID blockID = BlockID.getFromProtobuf(msg.getDeleteKey()
-        .getBlockID());
-    Preconditions.checkNotNull(blockID);
-    long containerID = blockID.getContainerID();
-    if (!this.containerManager.isOpen(containerID)) {
-      throw new StorageContainerException("Write to closed container.",
-          CLOSED_CONTAINER_IO);
-    }
-    this.containerManager.getKeyManager().deleteKey(blockID);
-    return KeyUtils.getKeyResponse(msg);
-  }
-
-  /**
-   * Handles writing a chunk and associated key using single RPC.
-   *
-   * @param msg - Message.
-   * @return ContainerCommandResponseProto
-   * @throws StorageContainerException
-   */
-  private ContainerCommandResponseProto handlePutSmallFile(
-      ContainerCommandRequestProto msg) throws StorageContainerException {
-
-    if (!msg.hasPutSmallFile()) {
-      LOG.debug("Malformed put small file request. trace ID: {}",
-          msg.getTraceID());
-      return ContainerUtils.malformedRequest(msg);
-    }
-    try {
-
-      BlockID blockID = BlockID.getFromProtobuf(msg.
-          getPutSmallFile().getKey().getKeyData().getBlockID());
-      long containerID = blockID.getContainerID();
-
-      if (!this.containerManager.isOpen(containerID)) {
-        throw new StorageContainerException("Write to closed container.",
-            CLOSED_CONTAINER_IO);
-      }
-      KeyData keyData = KeyData.getFromProtoBuf(msg.getPutSmallFile().getKey()
-          .getKeyData());
-      ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(msg.getPutSmallFile()
-          .getChunkInfo());
-      byte[] data = msg.getPutSmallFile().getData().toByteArray();
-
-      metrics.incContainerBytesStats(Type.PutSmallFile, data.length);
-      this.containerManager.getChunkManager().writeChunk(blockID,
-          chunkInfo, data, ContainerProtos.Stage.COMBINED);
-      List<ContainerProtos.ChunkInfo> chunks = new LinkedList<>();
-      chunks.add(chunkInfo.getProtoBufMessage());
-      keyData.setChunks(chunks);
-      this.containerManager.getKeyManager().putKey(keyData);
-      return SmallFileUtils.getPutFileResponseSuccess(msg);
-    } catch (StorageContainerException e) {
-      return ContainerUtils.logAndReturnError(LOG, e, msg);
-    } catch (IOException e) {
-      throw new StorageContainerException("Put Small File Failed.", e,
-          PUT_SMALL_FILE_ERROR);
-    }
-  }
-
-  /**
-   * Handles getting a data stream using a key. This helps in reducing the RPC
-   * overhead for small files.
-   *
-   * @param msg - ContainerCommandRequestProto
-   * @return ContainerCommandResponseProto
-   * @throws StorageContainerException
-   */
-  private ContainerCommandResponseProto handleGetSmallFile(
-      ContainerCommandRequestProto msg) throws StorageContainerException {
-    ByteString dataBuf = ByteString.EMPTY;
-    if (!msg.hasGetSmallFile()) {
-      LOG.debug("Malformed get small file request. trace ID: {}",
-          msg.getTraceID());
-      return ContainerUtils.malformedRequest(msg);
-    }
-    try {
-      long bytes = 0;
-      KeyData keyData = new KeyData(BlockID.getFromProtobuf(
-          msg.getGetSmallFile().getKey().getBlockID()));
-      KeyData data = this.containerManager.getKeyManager().getKey(keyData);
-      ContainerProtos.ChunkInfo c = null;
-      for (ContainerProtos.ChunkInfo chunk : data.getChunks()) {
-        bytes += chunk.getSerializedSize();
-        ByteString current =
-            ByteString.copyFrom(this.containerManager.getChunkManager()
-                .readChunk(keyData.getBlockID(),
-                    ChunkInfo.getFromProtoBuf(chunk)));
-        dataBuf = dataBuf.concat(current);
-        c = chunk;
-      }
-      metrics.incContainerBytesStats(Type.GetSmallFile, bytes);
-      return SmallFileUtils.getGetSmallFileResponseSuccess(
-          msg, dataBuf.toByteArray(), ChunkInfo.getFromProtoBuf(c));
-    } catch (StorageContainerException e) {
-      return ContainerUtils.logAndReturnError(LOG, e, msg);
-    } catch (IOException e) {
-      throw new StorageContainerException("Get Small File Failed", e,
-          GET_SMALL_FILE_ERROR);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyManagerImpl.java
deleted file mode 100644
index 40ae1c7..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyManagerImpl.java
+++ /dev/null
@@ -1,204 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.impl;
-
-import com.google.common.base.Preconditions;
-import com.google.common.primitives.Longs;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.scm.container.common.helpers
-    .StorageContainerException;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
-import org.apache.hadoop.ozone.container.common.helpers.KeyData;
-import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
-import org.apache.hadoop.ozone.container.common.interfaces.KeyManager;
-import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
-import org.apache.hadoop.utils.MetadataStore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.NO_SUCH_KEY;
-
-/**
- * Key Manager impl.
- */
-public class KeyManagerImpl implements KeyManager {
-  static final Logger LOG =
-      LoggerFactory.getLogger(KeyManagerImpl.class);
-
-  private static final float LOAD_FACTOR = 0.75f;
-  private final ContainerManager containerManager;
-  private final Configuration conf;
-
-  /**
-   * Constructs a key Manager.
-   *
-   * @param containerManager - Container Manager.
-   */
-  public KeyManagerImpl(ContainerManager containerManager, Configuration conf) {
-    Preconditions.checkNotNull(containerManager, "Container manager cannot be" +
-        " null");
-    Preconditions.checkNotNull(conf, "Config cannot be null");
-    this.containerManager = containerManager;
-    this.conf = conf;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void putKey(KeyData data) throws IOException {
-    Preconditions.checkNotNull(data,
-        "KeyData cannot be null for put operation.");
-    Preconditions.checkState(data.getContainerID() >= 0,
-        "Container ID cannot be negative");
-    containerManager.readLock();
-    try {
-      // We are not locking the key manager since LevelDb serializes all actions
-      // against a single DB. We rely on DB level locking to avoid conflicts.
-      ContainerData cData = containerManager.readContainer(
-          data.getContainerID());
-      MetadataStore db = KeyUtils.getDB(cData, conf);
-
-      // This is a post condition that acts as a hint to the user.
-      // Should never fail.
-      Preconditions.checkNotNull(db, "DB cannot be null here");
-      db.put(Longs.toByteArray(data.getLocalID()), data
-          .getProtoBufMessage().toByteArray());
-    } finally {
-      containerManager.readUnlock();
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public KeyData getKey(KeyData data) throws IOException {
-    containerManager.readLock();
-    try {
-      Preconditions.checkNotNull(data, "Key data cannot be null");
-      Preconditions.checkNotNull(data.getContainerID(),
-          "Container name cannot be null");
-      ContainerData cData = containerManager.readContainer(data
-          .getContainerID());
-      MetadataStore db = KeyUtils.getDB(cData, conf);
-
-      // This is a post condition that acts as a hint to the user.
-      // Should never fail.
-      Preconditions.checkNotNull(db, "DB cannot be null here");
-
-      byte[] kData = db.get(Longs.toByteArray(data.getLocalID()));
-      if (kData == null) {
-        throw new StorageContainerException("Unable to find the key.",
-            NO_SUCH_KEY);
-      }
-      ContainerProtos.KeyData keyData =
-          ContainerProtos.KeyData.parseFrom(kData);
-      return KeyData.getFromProtoBuf(keyData);
-    } finally {
-      containerManager.readUnlock();
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void deleteKey(BlockID blockID)
-      throws IOException {
-    Preconditions.checkNotNull(blockID, "block ID cannot be null.");
-    Preconditions.checkState(blockID.getContainerID() >= 0,
-        "Container ID cannot be negative.");
-    Preconditions.checkState(blockID.getLocalID() >= 0,
-        "Local ID cannot be negative.");
-
-    containerManager.readLock();
-    try {
-
-      ContainerData cData = containerManager
-          .readContainer(blockID.getContainerID());
-      MetadataStore db = KeyUtils.getDB(cData, conf);
-
-      // This is a post condition that acts as a hint to the user.
-      // Should never fail.
-      Preconditions.checkNotNull(db, "DB cannot be null here");
-      // Note : There is a race condition here, since get and delete
-      // are not atomic. Leaving it here since the impact is refusing
-      // to delete a key which might have just gotten inserted after
-      // the get check.
-
-      byte[] kKey = Longs.toByteArray(blockID.getLocalID());
-      byte[] kData = db.get(kKey);
-      if (kData == null) {
-        throw new StorageContainerException("Unable to find the key.",
-            NO_SUCH_KEY);
-      }
-      db.delete(kKey);
-    } finally {
-      containerManager.readUnlock();
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public List<KeyData> listKey(
-      long containerID, long startLocalID, int count)
-      throws IOException {
-    Preconditions.checkState(containerID >= 0,
-        "Container ID cannot be negative");
-    Preconditions.checkState(startLocalID >= 0,
-        "startLocal ID cannot be negative");
-    Preconditions.checkArgument(count > 0,
-        "Count must be a positive number.");
-    ContainerData cData = containerManager.readContainer(containerID);
-    MetadataStore db = KeyUtils.getDB(cData, conf);
-
-    List<KeyData> result = new ArrayList<>();
-    byte[] startKeyInBytes = Longs.toByteArray(startLocalID);
-    List<Map.Entry<byte[], byte[]>> range =
-        db.getSequentialRangeKVs(startKeyInBytes, count, null);
-    for (Map.Entry<byte[], byte[]> entry : range) {
-      KeyData value = KeyUtils.getKeyData(entry.getValue());
-      KeyData data = new KeyData(value.getBlockID());
-      result.add(data);
-    }
-    return result;
-  }
-
-  /**
-   * Shutdown keyManager.
-   */
-  @Override
-  public void shutdown() {
-    Preconditions.checkState(this.containerManager.hasWriteLock(), "asserts " +
-        "that we are holding the container manager lock when shutting down.");
-    KeyUtils.shutdownCache(ContainerCache.getInstance(conf));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
index 97fdb9e..83d746b 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
@@ -21,7 +21,6 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 import org.apache.hadoop.ozone.container.common.interfaces
     .ContainerDeletionChoosingPolicy;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
index 9a109e8..68074fc 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.ozone.container.common.impl;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 import org.apache.hadoop.ozone.container.common.interfaces
     .ContainerDeletionChoosingPolicy;
 import org.slf4j.Logger;


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


[36/37] hadoop git commit: HDDS-48. Fix branch after merging from trunk.

Posted by bh...@apache.org.
HDDS-48. Fix branch after merging from trunk.


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

Branch: refs/heads/trunk
Commit: 3584baf2642816a453402a717a05d16754a6ac52
Parents: c275a9a
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Mon Jul 9 12:30:59 2018 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Jul 9 13:22:30 2018 -0700

----------------------------------------------------------------------
 .../commandhandler/TestBlockDeletion.java       | 32 +++++++++++---------
 .../org/apache/hadoop/ozone/scm/TestSCMCli.java |  4 +--
 2 files changed, 19 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3584baf2/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
index 62059ec..c60c6c4 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
@@ -34,9 +34,10 @@ import org.apache.hadoop.ozone.client.OzoneBucket;
 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.container.common.helpers.ContainerData;
-import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
-import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
 import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
@@ -46,6 +47,7 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.utils.MetadataStore;
 import org.junit.Assert;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import java.io.File;
@@ -56,10 +58,11 @@ import java.util.function.Consumer;
 
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
 
+@Ignore("Need to be fixed according to ContainerIO")
 public class TestBlockDeletion {
   private static OzoneConfiguration conf = null;
   private static ObjectStore store;
-  private static ContainerManagerImpl dnContainerManager = null;
+  private static ContainerSet dnContainerManager = null;
   private static StorageContainerManager scm = null;
   private static OzoneManager om = null;
   private static Set<Long> containerIdsWithDeletedBlocks;
@@ -85,9 +88,8 @@ public class TestBlockDeletion {
         MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
     cluster.waitForClusterToBeReady();
     store = OzoneClientFactory.getRpcClient(conf).getObjectStore();
-    dnContainerManager =
-        (ContainerManagerImpl) cluster.getHddsDatanodes().get(0)
-            .getDatanodeStateMachine().getContainer().getContainerManager();
+    dnContainerManager = cluster.getHddsDatanodes().get(0)
+        .getDatanodeStateMachine().getContainer().getContainerSet();
     om = cluster.getOzoneManager();
     scm = cluster.getStorageContainerManager();
     containerIdsWithDeletedBlocks = new HashSet<>();
@@ -148,8 +150,8 @@ public class TestBlockDeletion {
         Assert.assertEquals(
             scm.getContainerInfo(containerId).getDeleteTransactionId(), 0);
       }
-      Assert.assertEquals(dnContainerManager.readContainer(containerId)
-              .getDeleteTransactionId(),
+      Assert.assertEquals(dnContainerManager.getContainer(containerId)
+              .getContainerData().getDeleteTransactionId(),
           scm.getContainerInfo(containerId).getDeleteTransactionId());
     }
   }
@@ -159,9 +161,9 @@ public class TestBlockDeletion {
       throws IOException {
     return performOperationOnKeyContainers((blockID) -> {
       try {
-        MetadataStore db = KeyUtils.getDB(
-            dnContainerManager.getContainerMap().get(blockID.getContainerID()),
-            conf);
+        MetadataStore db = KeyUtils.getDB((KeyValueContainerData)
+                dnContainerManager.getContainer(blockID.getContainerID())
+                    .getContainerData(), conf);
         Assert.assertNotNull(db.get(Longs.toByteArray(blockID.getLocalID())));
       } catch (IOException e) {
         e.printStackTrace();
@@ -174,9 +176,9 @@ public class TestBlockDeletion {
       throws IOException {
     return performOperationOnKeyContainers((blockID) -> {
       try {
-        MetadataStore db = KeyUtils.getDB(
-            dnContainerManager.getContainerMap().get(blockID.getContainerID()),
-            conf);
+        MetadataStore db = KeyUtils.getDB((KeyValueContainerData)
+            dnContainerManager.getContainer(blockID.getContainerID())
+                .getContainerData(), conf);
         Assert.assertNull(db.get(Longs.toByteArray(blockID.getLocalID())));
         Assert.assertNull(db.get(DFSUtil.string2Bytes(
             OzoneConsts.DELETING_KEY_PREFIX + blockID.getLocalID())));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3584baf2/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java
index cc11feb..722c1a5 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java
@@ -338,8 +338,8 @@ public class TestSCMCli {
 
     openStatus = data.isOpen() ? "OPEN" : "CLOSED";
     expected = String
-        .format(formatStr, container.getContainerID(), openStatus,
-            data.getDbFile().getPath(), data.getContainerPath(), "",
+        .format(formatStr, container.getContainerInfo().getContainerID(),
+            openStatus, data.getDbFile().getPath(), data.getContainerPath(), "",
             datanodeDetails.getHostName(), datanodeDetails.getHostName());
     assertEquals(expected, out.toString());
   }


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


[26/37] hadoop git commit: HDDS-229:Remove singleton for Handler. Contributed by Bharat Viswanadham

Posted by bh...@apache.org.
HDDS-229:Remove singleton for Handler. Contributed by Bharat Viswanadham


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

Branch: refs/heads/trunk
Commit: a404164939c898aaf1ced90747b1011ed722e99a
Parents: e1f4b3b
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Thu Jul 5 11:26:00 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Thu Jul 5 11:26:00 2018 -0700

----------------------------------------------------------------------
 .../container/common/interfaces/Handler.java    |  2 +-
 .../container/keyvalue/KeyValueHandler.java     | 22 ++++----------------
 2 files changed, 5 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4041649/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
index 57dd224..2725f59 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
@@ -57,7 +57,7 @@ public class Handler {
                                                    ContainerMetrics metrics) {
     switch (containerType) {
     case KeyValueContainer:
-      return KeyValueHandler.getInstance(config, contSet, volumeSet, metrics);
+      return new KeyValueHandler(config, contSet, volumeSet, metrics);
     default:
       throw new IllegalArgumentException("Handler for ContainerType: " +
         containerType + "doesn't exist.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4041649/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index b615acd..8166a83 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.ozone.container.keyvalue;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
-import com.sun.jersey.spi.resource.Singleton;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
@@ -93,33 +92,19 @@ import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
 /**
  * Handler for KeyValue Container type.
  */
-@Singleton
 public class KeyValueHandler extends Handler {
 
   private static final Logger LOG = LoggerFactory.getLogger(
       KeyValueHandler.class);
 
-  private static volatile KeyValueHandler INSTANCE = null; // Singleton class
-
   private final ContainerType containerType;
   private final KeyManager keyManager;
   private final ChunkManager chunkManager;
   private VolumeChoosingPolicy volumeChoosingPolicy;
   private final int maxContainerSizeGB;
 
-  // TODO : Add metrics and populate it.
-
-  public static KeyValueHandler getInstance(Configuration config,
-                                            ContainerSet contSet,
-                                            VolumeSet volSet,
-                                            ContainerMetrics metrics) {
-    if (INSTANCE == null) {
-      INSTANCE = new KeyValueHandler(config, contSet, volSet, metrics);
-    }
-    return INSTANCE;
-  }
 
-  private KeyValueHandler(Configuration config, ContainerSet contSet,
+  public KeyValueHandler(Configuration config, ContainerSet contSet,
       VolumeSet volSet, ContainerMetrics metrics) {
     super(config, contSet, volSet, metrics);
     containerType = ContainerType.KeyValueContainer;
@@ -127,8 +112,9 @@ public class KeyValueHandler extends Handler {
     chunkManager = new ChunkManagerImpl();
     // TODO: Add supoort for different volumeChoosingPolicies.
     volumeChoosingPolicy = new RoundRobinVolumeChoosingPolicy();
-    maxContainerSizeGB = config.getInt(ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
-        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+    maxContainerSizeGB = config.getInt(ScmConfigKeys
+            .OZONE_SCM_CONTAINER_SIZE_GB, ScmConfigKeys
+        .OZONE_SCM_CONTAINER_SIZE_DEFAULT);
   }
 
   @Override


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


[03/37] hadoop git commit: HDDS-116. Implement VolumeSet to manage disk volumes. Contributed by Hanisha Koneru.

Posted by bh...@apache.org.
HDDS-116. Implement VolumeSet to manage disk volumes. Contributed by Hanisha Koneru.


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

Branch: refs/heads/trunk
Commit: 59777185fc38b9f9b7428c5f3f7eb6c52796317d
Parents: 6cd19b4
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu May 31 10:29:25 2018 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu May 31 10:29:25 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdds/scm/ScmConfigKeys.java   |   1 +
 .../impl/RoundRobinVolumeChoosingPolicy.java    |  82 ++++++
 .../ozone/container/common/impl/VolumeInfo.java | 125 ++++++++++
 .../ozone/container/common/impl/VolumeSet.java  | 250 +++++++++++++++++++
 .../common/interfaces/VolumeChoosingPolicy.java |  46 ++++
 .../TestRoundRobinVolumeChoosingPolicy.java     | 100 ++++++++
 .../common/interfaces/TestVolumeSet.java        | 138 ++++++++++
 7 files changed, 742 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/59777185/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
index 85407e6..b6b95eb 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
@@ -141,6 +141,7 @@ public final class ScmConfigKeys {
   public static final String HDDS_REST_HTTP_ADDRESS_KEY =
       "hdds.rest.http-address";
   public static final String HDDS_REST_HTTP_ADDRESS_DEFAULT = "0.0.0.0:9880";
+  public static final String HDDS_DATANODE_DIR_KEY = "hdds.datanode.dir";
   public static final String HDDS_REST_CSRF_ENABLED_KEY =
       "hdds.rest.rest-csrf.enabled";
   public static final boolean HDDS_REST_CSRF_ENABLED_DEFAULT = false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/59777185/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RoundRobinVolumeChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RoundRobinVolumeChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RoundRobinVolumeChoosingPolicy.java
new file mode 100644
index 0000000..0a20bf2
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RoundRobinVolumeChoosingPolicy.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.common.impl;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
+import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Choose volumes in round-robin order.
+ * Use fine-grained locks to synchronize volume choosing.
+ */
+public class RoundRobinVolumeChoosingPolicy implements VolumeChoosingPolicy {
+
+  public static final Log LOG = LogFactory.getLog(
+		RoundRobinVolumeChoosingPolicy.class);
+
+  // Stores the index of the next volume to be returned.
+  private AtomicInteger nextVolumeIndex = new AtomicInteger(0);
+
+  @Override
+  public VolumeInfo chooseVolume(List<VolumeInfo> volumes,
+      long maxContainerSize) throws IOException {
+
+    // No volumes available to choose from
+    if (volumes.size() < 1) {
+      throw new DiskOutOfSpaceException("No more available volumes");
+    }
+
+    // since volumes could've been removed because of the failure
+    // make sure we are not out of bounds
+    int nextIndex = nextVolumeIndex.get();
+    int currentVolumeIndex = nextIndex < volumes.size() ? nextIndex : 0;
+
+    int startVolumeIndex = currentVolumeIndex;
+    long maxAvailable = 0;
+
+    while (true) {
+      final VolumeInfo volume = volumes.get(currentVolumeIndex);
+      long availableVolumeSize = volume.getAvailable();
+
+      currentVolumeIndex = (currentVolumeIndex + 1) % volumes.size();
+
+      if (availableVolumeSize > maxContainerSize) {
+        nextVolumeIndex.compareAndSet(nextIndex, currentVolumeIndex);
+        return volume;
+      }
+
+      if (availableVolumeSize > maxAvailable) {
+        maxAvailable = availableVolumeSize;
+      }
+
+      if (currentVolumeIndex == startVolumeIndex) {
+        throw new DiskOutOfSpaceException("Out of space: "
+            + "The volume with the most available space (=" + maxAvailable
+            + " B) is less than the container size (=" + maxContainerSize
+            + " B).");
+      }
+
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/59777185/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeInfo.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeInfo.java
new file mode 100644
index 0000000..1b5a7aa
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeInfo.java
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.impl;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Stores information about a disk/volume.
+ */
+public class VolumeInfo {
+
+  private static final Logger LOG = LoggerFactory.getLogger(VolumeInfo.class);
+
+  private final Path rootDir;
+  private final StorageType storageType;
+  private VolumeState state;
+  // Capacity configured. This is useful when we want to
+  // limit the visible capacity for tests. If negative, then we just
+  // query from the filesystem.
+  private long configuredCapacity;
+  private volatile AtomicLong scmUsed = new AtomicLong(0);
+
+  public static class Builder {
+    private final Path rootDir;
+    private StorageType storageType;
+    private VolumeState state;
+    private long configuredCapacity;
+
+    public Builder(Path rootDir) {
+      this.rootDir = rootDir;
+    }
+
+    public Builder(String rootDirStr) {
+      this.rootDir = new Path(rootDirStr);
+    }
+
+    public Builder storageType(StorageType storageType) {
+      this.storageType = storageType;
+      return this;
+    }
+
+    public Builder volumeState(VolumeState state) {
+      this.state = state;
+      return this;
+    }
+
+    public Builder configuredCapacity(long configuredCapacity) {
+      this.configuredCapacity = configuredCapacity;
+      return this;
+    }
+
+    public VolumeInfo build() throws IOException {
+      return new VolumeInfo(this);
+    }
+  }
+
+  private VolumeInfo(Builder b) {
+
+    this.rootDir = b.rootDir;
+
+    this.storageType = (b.storageType != null ?
+        b.storageType : StorageType.DEFAULT);
+
+    this.configuredCapacity = (b.configuredCapacity != 0 ?
+        b.configuredCapacity : -1);
+
+    this.state = (b.state != null ? b.state : VolumeState.NOT_FORMATTED);
+
+    LOG.info("Creating Volume : " + rootDir + " of  storage type : " +
+        storageType + " and capacity : " + configuredCapacity);
+  }
+
+  public void addSpaceUsed(long spaceUsed) {
+    this.scmUsed.getAndAdd(spaceUsed);
+  }
+
+  public long getAvailable() {
+    return configuredCapacity - scmUsed.get();
+  }
+
+  public void setState(VolumeState state) {
+    this.state = state;
+  }
+
+  public boolean isFailed() {
+    return (state == VolumeState.FAILED);
+  }
+
+  public Path getRootDir() {
+    return this.rootDir;
+  }
+
+  public StorageType getStorageType() {
+    return this.storageType;
+  }
+
+  public enum VolumeState {
+    NORMAL,
+    FAILED,
+    NON_EXISTENT,
+    NOT_FORMATTED,
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/59777185/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeSet.java
new file mode 100644
index 0000000..27fd657
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeSet.java
@@ -0,0 +1,250 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.ozone.container.common.impl.VolumeInfo.VolumeState;
+import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
+import org.apache.hadoop.util.AutoCloseableLock;
+import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
+import org.apache.hadoop.util.InstrumentedLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.EnumMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * VolumeSet to manage volumes in a DataNode.
+ */
+public class VolumeSet {
+
+  private static final Logger LOG = LoggerFactory.getLogger(VolumeSet.class);
+
+  private Configuration conf;
+  /**
+   * {@link VolumeSet#volumeList} maintains a list of active volumes in the
+   * DataNode. Each volume has one-to-one mapping with a volumeInfo object.
+   */
+  private List<VolumeInfo> volumeList;
+  /**
+   * {@link VolumeSet#failedVolumeList} maintains a list of volumes which have
+   * failed. This list is mutually exclusive to {@link VolumeSet#volumeList}.
+   */
+  private List<VolumeInfo> failedVolumeList;
+  /**
+   * {@link VolumeSet#volumeMap} maintains a map of all volumes in the
+   * DataNode irrespective of their state.
+   */
+  private Map<Path, VolumeInfo> volumeMap;
+  /**
+   * {@link VolumeSet#volumeStateMap} maintains a list of volumes per
+   * StorageType.
+   */
+  private EnumMap<StorageType, List<VolumeInfo>> volumeStateMap;
+
+  /**
+   * Lock to synchronize changes to the VolumeSet. Any update to
+   * {@link VolumeSet#volumeList}, {@link VolumeSet#failedVolumeList},
+   * {@link VolumeSet#volumeMap} or {@link VolumeSet#volumeStateMap} should
+   * be done after acquiring this lock.
+   */
+  private final AutoCloseableLock volumeSetLock;
+
+  public VolumeSet(Configuration conf) throws DiskOutOfSpaceException {
+    this.conf = conf;
+    this.volumeSetLock = new AutoCloseableLock(
+        new InstrumentedLock(getClass().getName(), LOG,
+            new ReentrantLock(true),
+            conf.getTimeDuration(
+                DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
+                DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_DEFAULT,
+                TimeUnit.MILLISECONDS),
+            300));
+
+    initializeVolumeSet();
+  }
+
+  // Add DN volumes configured through ConfigKeys to volumeMap.
+  private void initializeVolumeSet() throws DiskOutOfSpaceException {
+    volumeList = new ArrayList<>();
+    failedVolumeList = new ArrayList<>();
+    volumeMap = new ConcurrentHashMap<>();
+    volumeStateMap = new EnumMap<>(StorageType.class);
+
+    Collection<String> datanodeDirs = conf.getTrimmedStringCollection(
+        HDDS_DATANODE_DIR_KEY);
+    if (datanodeDirs.isEmpty()) {
+      datanodeDirs = conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY);
+    }
+    if (datanodeDirs.isEmpty()) {
+      throw new IllegalArgumentException("No location configured in either "
+          + HDDS_DATANODE_DIR_KEY + " or " + DFS_DATANODE_DATA_DIR_KEY);
+    }
+
+    for (StorageType storageType : StorageType.values()) {
+      volumeStateMap.put(storageType, new ArrayList<VolumeInfo>());
+    }
+
+    for (String dir : datanodeDirs) {
+      try {
+        VolumeInfo volumeInfo = getVolumeInfo(dir);
+
+        volumeList.add(volumeInfo);
+        volumeMap.put(volumeInfo.getRootDir(), volumeInfo);
+        volumeStateMap.get(volumeInfo.getStorageType()).add(volumeInfo);
+      } catch (IOException e) {
+        LOG.error("Failed to parse the storage location: " + dir, e);
+      }
+    }
+
+    if (volumeList.size() == 0) {
+      throw new DiskOutOfSpaceException("No storage location configured");
+    }
+  }
+
+  public void acquireLock() {
+    volumeSetLock.acquire();
+  }
+
+  public void releaseLock() {
+    volumeSetLock.release();
+  }
+
+  private VolumeInfo getVolumeInfo(String rootDir) throws IOException {
+    StorageLocation location = StorageLocation.parse(rootDir);
+    StorageType storageType = location.getStorageType();
+
+    VolumeInfo.Builder volumeBuilder = new VolumeInfo.Builder(rootDir);
+    volumeBuilder.storageType(storageType);
+    return volumeBuilder.build();
+  }
+
+  // Add a volume to VolumeSet
+  public void addVolume(String dataDir) throws IOException {
+    Path dirPath = new Path(dataDir);
+
+    try (AutoCloseableLock lock = volumeSetLock.acquire()) {
+      if (volumeMap.containsKey(dirPath)) {
+        VolumeInfo volumeInfo = volumeMap.get(dirPath);
+        if (volumeInfo.isFailed()) {
+          volumeInfo.setState(VolumeState.NORMAL);
+          failedVolumeList.remove(volumeInfo);
+          volumeList.add(volumeInfo);
+        } else {
+          LOG.warn("Volume : " + volumeInfo.getRootDir() + " already " +
+              "exists in VolumeMap");
+        }
+      } else {
+        VolumeInfo volumeInfo = getVolumeInfo(dataDir);
+
+        volumeList.add(volumeInfo);
+        volumeMap.put(volumeInfo.getRootDir(), volumeInfo);
+        volumeStateMap.get(volumeInfo.getStorageType()).add(volumeInfo);
+      }
+    }
+  }
+
+  // Mark a volume as failed
+  public void failVolume(String dataDir) {
+    Path dirPath = new Path(dataDir);
+
+    try (AutoCloseableLock lock = volumeSetLock.acquire()) {
+      if (volumeMap.containsKey(dirPath)) {
+        VolumeInfo volumeInfo = volumeMap.get(dirPath);
+        if (!volumeInfo.isFailed()) {
+          volumeInfo.setState(VolumeState.FAILED);
+          volumeList.remove(volumeInfo);
+          failedVolumeList.add(volumeInfo);
+        }
+      } else {
+        LOG.warn("Volume : " + dataDir + " does not exist in VolumeMap");
+      }
+    }
+  }
+
+  // Remove a volume from the VolumeSet completely.
+  public void removeVolume(String dataDir) throws IOException {
+    Path dirPath = new Path(dataDir);
+
+    try (AutoCloseableLock lock = volumeSetLock.acquire()) {
+      if (volumeMap.containsKey(dirPath)) {
+        VolumeInfo volumeInfo = volumeMap.get(dirPath);
+        if (!volumeInfo.isFailed()) {
+          volumeList.remove(volumeInfo);
+        } else {
+          failedVolumeList.remove(volumeInfo);
+        }
+        volumeMap.remove(dirPath);
+        volumeStateMap.get(volumeInfo.getStorageType()).remove(volumeInfo);
+      } else {
+        LOG.warn("Volume: " + dataDir + " does not exist in " + "volumeMap.");
+      }
+    }
+  }
+
+  /**
+   * Return an iterator over {@link VolumeSet#volumeList}.
+   */
+  public Iterator<VolumeInfo> getIterator() {
+    return volumeList.iterator();
+  }
+
+  public VolumeInfo chooseVolume(long containerSize,
+      VolumeChoosingPolicy choosingPolicy) throws IOException {
+    return choosingPolicy.chooseVolume(volumeList, containerSize);
+  }
+
+  @VisibleForTesting
+  public List<VolumeInfo> getVolumesList() {
+    return ImmutableList.copyOf(volumeList);
+  }
+
+  @VisibleForTesting
+  public List<VolumeInfo> getFailedVolumesList() {
+    return ImmutableList.copyOf(failedVolumeList);
+  }
+
+  @VisibleForTesting
+  public Map<Path, VolumeInfo> getVolumeMap() {
+    return ImmutableMap.copyOf(volumeMap);
+  }
+
+  @VisibleForTesting
+  public Map<StorageType, List<VolumeInfo>> getVolumeStateMap() {
+    return ImmutableMap.copyOf(volumeStateMap);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/59777185/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/VolumeChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/VolumeChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/VolumeChoosingPolicy.java
new file mode 100644
index 0000000..b8cbcb6
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/VolumeChoosingPolicy.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.common.interfaces;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ozone.container.common.impl.VolumeInfo;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * This interface specifies the policy for choosing volumes to store replicas.
+ */
+@InterfaceAudience.Private
+public interface VolumeChoosingPolicy {
+
+  /**
+   * Choose a volume to place a container,
+   * given a list of volumes and the max container size sought for storage.
+   *
+   * The implementations of this interface must be thread-safe.
+   *
+   * @param volumes - a list of available volumes.
+   * @param maxContainerSize - the maximum size of the container for which a
+   *                         volume is sought.
+   * @return the chosen volume.
+   * @throws IOException when disks are unavailable or are full.
+   */
+  VolumeInfo chooseVolume(List<VolumeInfo> volumes, long maxContainerSize)
+      throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/59777185/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestRoundRobinVolumeChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestRoundRobinVolumeChoosingPolicy.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestRoundRobinVolumeChoosingPolicy.java
new file mode 100644
index 0000000..409db57
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestRoundRobinVolumeChoosingPolicy.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.impl;
+
+import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests {@link RoundRobinVolumeChoosingPolicy}.
+ */
+public class TestRoundRobinVolumeChoosingPolicy {
+
+  private RoundRobinVolumeChoosingPolicy policy;
+
+  @Before
+  public void setup() {
+   policy = ReflectionUtils.newInstance(
+       RoundRobinVolumeChoosingPolicy.class, null);
+  }
+
+  @Test
+  public void testRRVolumeChoosingPolicy() throws Exception {
+    final List<VolumeInfo> volumes = new ArrayList<>();
+
+    // First volume, with 100 bytes of space.
+    volumes.add(Mockito.mock(VolumeInfo.class));
+    Mockito.when(volumes.get(0).getAvailable()).thenReturn(100L);
+
+    // Second volume, with 200 bytes of space.
+    volumes.add(Mockito.mock(VolumeInfo.class));
+    Mockito.when(volumes.get(1).getAvailable()).thenReturn(200L);
+
+    // Test two rounds of round-robin choosing
+    Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 0));
+    Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 0));
+    Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 0));
+    Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 0));
+
+    // The first volume has only 100L space, so the policy should
+    // choose the second one in case we ask for more.
+    Assert.assertEquals(volumes.get(1),
+        policy.chooseVolume(volumes, 150));
+
+    // Fail if no volume has enough space available
+    try {
+      policy.chooseVolume(volumes, Long.MAX_VALUE);
+      Assert.fail();
+    } catch (IOException e) {
+      // Passed.
+    }
+  }
+
+  @Test
+  public void testRRPolicyExceptionMessage() throws Exception {
+    final List<VolumeInfo> volumes = new ArrayList<>();
+
+    // First volume, with 100 bytes of space.
+    volumes.add(Mockito.mock(VolumeInfo.class));
+    Mockito.when(volumes.get(0).getAvailable()).thenReturn(100L);
+
+    // Second volume, with 200 bytes of space.
+    volumes.add(Mockito.mock(VolumeInfo.class));
+    Mockito.when(volumes.get(1).getAvailable()).thenReturn(200L);
+
+    int blockSize = 300;
+    try {
+      policy.chooseVolume(volumes, blockSize);
+      Assert.fail("expected to throw DiskOutOfSpaceException");
+    } catch(DiskOutOfSpaceException e) {
+      Assert.assertEquals("Not returnig the expected message",
+          "Out of space: The volume with the most available space (=" + 200
+              + " B) is less than the container size (=" + blockSize + " B).",
+          e.getMessage());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/59777185/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestVolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestVolumeSet.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestVolumeSet.java
new file mode 100644
index 0000000..5a1bc79
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestVolumeSet.java
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.interfaces;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.ozone.container.common.impl.VolumeInfo;
+import org.apache.hadoop.ozone.container.common.impl.VolumeSet;
+import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests {@link VolumeSet} operations.
+ */
+public class TestVolumeSet {
+
+  private OzoneConfiguration conf;
+  protected VolumeSet volumeSet;
+  protected final String baseDir = MiniDFSCluster.getBaseDirectory();
+  protected final String volume1 = baseDir + "disk1";
+  protected final String volume2 = baseDir + "disk2";
+  private final List<String> volumes = new ArrayList<>();
+
+  private void initializeVolumeSet() throws Exception {
+    volumeSet = new VolumeSet(conf);
+  }
+
+  @Rule
+  public Timeout testTimeout = new Timeout(300_000);
+
+  @Before
+  public void setup() throws Exception {
+    conf = new OzoneConfiguration();
+    String dataDirKey = volume1 + "," + volume2;
+    volumes.add(volume1);
+    volumes.add(volume2);
+    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataDirKey);
+    initializeVolumeSet();
+  }
+
+  @Test
+  public void testVolumeSetInitialization() throws Exception {
+
+    List<VolumeInfo> volumesList = volumeSet.getVolumesList();
+
+    // VolumeSet initialization should add volume1 and volume2 to VolumeSet
+    assertEquals("VolumeSet intialization is incorrect",
+        volumesList.size(), volumes.size());
+    assertEquals(volume1, volumesList.get(0).getRootDir().toString());
+    assertEquals(volume2, volumesList.get(1).getRootDir().toString());
+  }
+
+  @Test
+  public void testAddVolume() throws Exception {
+
+    List<VolumeInfo> volumesList = volumeSet.getVolumesList();
+    assertEquals(2, volumeSet.getVolumesList().size());
+
+    // Add a volume to VolumeSet
+    String volume3 = baseDir + "disk3";
+    volumeSet.addVolume(volume3);
+
+    assertEquals(3, volumeSet.getVolumesList().size());
+    assertEquals("AddVolume did not add requested volume to VolumeSet",
+        volume3,
+        volumeSet.getVolumesList().get(2).getRootDir().toString());
+  }
+
+  @Test
+  public void testFailVolume() throws Exception {
+
+    //Fail a volume
+    volumeSet.failVolume(volume1);
+
+    // Failed volume should not show up in the volumeList
+    assertEquals(1, volumeSet.getVolumesList().size());
+
+    // Failed volume should be added to FailedVolumeList
+    assertEquals("Failed volume not present in FailedVolumeList",
+        1, volumeSet.getFailedVolumesList().size());
+    assertEquals("Failed Volume list did not match", volume1,
+        volumeSet.getFailedVolumesList().get(0).getRootDir().toString());
+
+    // Failed volume should exist in VolumeMap with isFailed flag set to true
+    Path volume1Path = new Path(volume1);
+    assertTrue(volumeSet.getVolumeMap().containsKey(volume1Path));
+    assertTrue(volumeSet.getVolumeMap().get(volume1Path).isFailed());
+  }
+
+  @Test
+  public void testRemoveVolume() throws Exception {
+
+    List<VolumeInfo> volumesList = volumeSet.getVolumesList();
+    assertEquals(2, volumeSet.getVolumesList().size());
+
+    // Remove a volume from VolumeSet
+    volumeSet.removeVolume(volume1);
+    assertEquals(1, volumeSet.getVolumesList().size());
+
+    // Attempting to remove a volume which does not exist in VolumeSet should
+    // log a warning.
+    LogCapturer logs = LogCapturer.captureLogs(
+        LogFactory.getLog(VolumeSet.class));
+    volumeSet.removeVolume(volume1);
+    assertEquals(1, volumeSet.getVolumesList().size());
+    String expectedLogMessage = "Volume: " + volume1 + " does not exist in "
+        + "volumeMap.";
+    assertTrue("Log output does not contain expected log message: "
+        + expectedLogMessage, logs.getOutput().contains(expectedLogMessage));
+  }
+}


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


[28/37] hadoop git commit: HDDS-182:CleanUp Reimplemented classes. Contributed by Hansiha Koneru

Posted by bh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java
deleted file mode 100644
index 9de84da..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ChunkManager.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.interfaces;
-
-import org.apache.hadoop.hdds.scm.container.common.helpers
-    .StorageContainerException;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
-
-/**
- * Chunk Manager allows read, write, delete and listing of chunks in
- * a container.
- */
-public interface ChunkManager {
-
-  /**
-   * writes a given chunk.
-   * @param blockID - ID of the block.
-   * @param info - ChunkInfo.
-   * @param stage - Chunk Stage write.
-   * @throws StorageContainerException
-   */
-  void writeChunk(BlockID blockID,
-      ChunkInfo info, byte[] data, ContainerProtos.Stage stage)
-      throws StorageContainerException;
-
-  /**
-   * reads the data defined by a chunk.
-   * @param blockID - ID of the block.
-   * @param info - ChunkInfo.
-   * @return  byte array
-   * @throws StorageContainerException
-   *
-   * TODO: Right now we do not support partial reads and writes of chunks.
-   * TODO: Explore if we need to do that for ozone.
-   */
-  byte[] readChunk(BlockID blockID, ChunkInfo info) throws
-      StorageContainerException;
-
-  /**
-   * Deletes a given chunk.
-   * @param blockID - ID of the block.
-   * @param info  - Chunk Info
-   * @throws StorageContainerException
-   */
-  void deleteChunk(BlockID blockID, ChunkInfo info) throws
-      StorageContainerException;
-
-  // TODO : Support list operations.
-
-  /**
-   * Shutdown the chunkManager.
-   */
-  void shutdown();
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
index a5559aa..f0f1b37 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.ozone.container.common.interfaces;
 
-
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerLifeCycleState;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java
index 1ed50fb..2538368 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.ozone.container.common.interfaces;
 
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 
 import java.util.List;
 import java.util.Map;
@@ -28,6 +28,7 @@ import java.util.Map;
  * This interface is used for choosing desired containers for
  * block deletion.
  */
+// TODO: Fix ContainerDeletionChoosingPolicy to work with new StorageLayer
 public interface ContainerDeletionChoosingPolicy {
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java
deleted file mode 100644
index 49b68dc..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerManager.java
+++ /dev/null
@@ -1,267 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.interfaces;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.scm.container.common.helpers
-    .StorageContainerException;
-import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
-import org.apache.hadoop.hdfs.util.RwLock;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
-
-import java.io.IOException;
-import java.security.NoSuchAlgorithmException;
-import java.util.List;
-
-/**
- * Interface for container operations.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public interface ContainerManager extends RwLock {
-
-  /**
-   * Init call that sets up a container Manager.
-   *
-   * @param config        - Configuration.
-   * @param containerDirs - List of Metadata Container locations.
-   * @param datanodeDetails - DatanodeDetails
-   * @throws StorageContainerException
-   */
-  void init(Configuration config, List<StorageLocation> containerDirs,
-            DatanodeDetails datanodeDetails) throws IOException;
-
-  /**
-   * Creates a container with the given name.
-   *
-   * @param containerData - Container Name and metadata.
-   * @throws StorageContainerException
-   */
-  void createContainer(ContainerData containerData)
-      throws StorageContainerException;
-
-  /**
-   * Deletes an existing container.
-   *
-   * @param containerID - ID of the container.
-   * @param forceDelete   - whether this container should be deleted forcibly.
-   * @throws StorageContainerException
-   */
-  void deleteContainer(long containerID,
-      boolean forceDelete) throws StorageContainerException;
-
-  /**
-   * Update an existing container.
-   *
-   * @param containerID ID of the container
-   * @param data container data
-   * @param forceUpdate if true, update container forcibly.
-   * @throws StorageContainerException
-   */
-  void updateContainer(long containerID, ContainerData data,
-      boolean forceUpdate) throws StorageContainerException;
-
-  /**
-   * As simple interface for container Iterations.
-   *
-   * @param startContainerID -  Return containers with ID >= startContainerID.
-   * @param count - how many to return
-   * @param data - Actual containerData
-   * @throws StorageContainerException
-   */
-  void listContainer(long startContainerID, long count,
-      List<ContainerData> data) throws StorageContainerException;
-
-  /**
-   * Choose containers for block deletion.
-   *
-   * @param count   - how many to return
-   * @throws StorageContainerException
-   */
-  List<ContainerData> chooseContainerForBlockDeletion(int count)
-      throws StorageContainerException;
-
-  /**
-   * Get metadata about a specific container.
-   *
-   * @param containerID - ID of the container.
-   * @return ContainerData - Container Data.
-   * @throws StorageContainerException
-   */
-  ContainerData readContainer(long containerID)
-      throws StorageContainerException;
-
-  /**
-   * Closes a open container, if it is already closed or does not exist a
-   * StorageContainerException is thrown.
-   * @param containerID - ID of the container.
-   * @throws StorageContainerException
-   */
-  void closeContainer(long containerID)
-      throws StorageContainerException, NoSuchAlgorithmException;
-
-  /**
-   * Checks if a container exists.
-   * @param containerID - ID of the container.
-   * @return true if the container is open false otherwise.
-   * @throws StorageContainerException  - Throws Exception if we are not
-   * able to find the container.
-   */
-  boolean isOpen(long containerID) throws StorageContainerException;
-
-  /**
-   * Supports clean shutdown of container.
-   *
-   * @throws StorageContainerException
-   */
-  void shutdown() throws IOException;
-
-  /**
-   * Sets the Chunk Manager.
-   *
-   * @param chunkManager - ChunkManager.
-   */
-  void setChunkManager(ChunkManager chunkManager);
-
-  /**
-   * Gets the Chunk Manager.
-   *
-   * @return ChunkManager.
-   */
-  ChunkManager getChunkManager();
-
-  /**
-   * Sets the Key Manager.
-   *
-   * @param keyManager - Key Manager.
-   */
-  void setKeyManager(KeyManager keyManager);
-
-  /**
-   * Gets the Key Manager.
-   *
-   * @return KeyManager.
-   */
-  KeyManager getKeyManager();
-
-  /**
-   * Get the Node Report of container storage usage.
-   * @return node report.
-   */
-  NodeReportProto getNodeReport() throws IOException;
-
-  /**
-   * Gets container report.
-   * @return container report.
-   * @throws IOException
-   */
-  ContainerReportsProto getContainerReport() throws IOException;
-
-  /**
-   * Gets container reports.
-   * @return List of all closed containers.
-   * @throws IOException
-   */
-  List<ContainerData> getClosedContainerReports() throws IOException;
-
-  /**
-   * Increase pending deletion blocks count number of specified container.
-   *
-   * @param numBlocks
-   *          increment  count number
-   * @param containerId
-   *          container id
-   */
-  void incrPendingDeletionBlocks(int numBlocks, long containerId);
-
-  /**
-   * Decrease pending deletion blocks count number of specified container.
-   *
-   * @param numBlocks
-   *          decrement count number
-   * @param containerId
-   *          container id
-   */
-  void decrPendingDeletionBlocks(int numBlocks, long containerId);
-
-  /**
-   * Increase the read count of the container.
-   * @param containerId - ID of the container.
-   */
-  void incrReadCount(long containerId);
-
-  /**
-   * Increse the read counter for bytes read from the container.
-   * @param containerId - ID of the container.
-   * @param readBytes - bytes read from the container.
-   */
-  void incrReadBytes(long containerId, long readBytes);
-
-
-  /**
-   * Increase the write count of the container.
-   * @param containerId - ID of the container.
-   */
-  void incrWriteCount(long containerId);
-
-  /**
-   * Increase the write counter for bytes write into the container.
-   * @param containerId - ID of the container.
-   * @param writeBytes - bytes write into the container.
-   */
-  void incrWriteBytes(long containerId, long writeBytes);
-
-  /**
-   * Increase the bytes used by the container.
-   * @param containerId - ID of the container.
-   * @param used - additional bytes used by the container.
-   * @return the current bytes used.
-   */
-  long incrBytesUsed(long containerId, long used);
-
-  /**
-   * Decrease the bytes used by the container.
-   * @param containerId - ID of the container.
-   * @param used - additional bytes reclaimed by the container.
-   * @return the current bytes used.
-   */
-  long decrBytesUsed(long containerId, long used);
-
-  /**
-   * Get the bytes used by the container.
-   * @param containerId - ID of the container.
-   * @return the current bytes used by the container.
-   */
-  long getBytesUsed(long containerId);
-
-  /**
-   * Get the number of keys in the container.
-   * @param containerId - ID of the container.
-   * @return the current key count.
-   */
-  long getNumKeys(long containerId);
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/KeyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/KeyManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/KeyManager.java
deleted file mode 100644
index 158ce38..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/KeyManager.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.hadoop.ozone.container.common.interfaces;
-
-import org.apache.hadoop.hdds.scm.container.common.helpers
-    .StorageContainerException;
-import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.ozone.container.common.helpers.KeyData;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * KeyManager deals with Key Operations in the container Level.
- */
-public interface KeyManager {
-  /**
-   * Puts or overwrites a key.
-   *
-   * @param data     - Key Data.
-   * @throws IOException
-   */
-  void putKey(KeyData data) throws IOException;
-
-  /**
-   * Gets an existing key.
-   *
-   * @param data - Key Data.
-   * @return Key Data.
-   * @throws IOException
-   */
-  KeyData getKey(KeyData data) throws IOException;
-
-  /**
-   * Deletes an existing Key.
-   *
-   * @param blockID - ID of the block.
-   * @throws StorageContainerException
-   */
-  void deleteKey(BlockID blockID)
-      throws IOException;
-
-  /**
-   * List keys in a container.
-   *
-   * @param containerID - ID of the container.
-   * @param startLocalID  - Key to start from, 0 to begin.
-   * @param count    - Number of keys to return.
-   * @return List of Keys that match the criteria.
-   */
-  List<KeyData> listKey(long containerID, long startLocalID,
-      int count) throws IOException;
-
-  /**
-   * Shutdown keyManager.
-   */
-  void shutdown();
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java
deleted file mode 100644
index 52cf2e0..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java
+++ /dev/null
@@ -1,247 +0,0 @@
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.ozone.container.common.statemachine.background;
-
-import com.google.common.collect.Lists;
-import org.apache.ratis.shaded.com.google.protobuf
-    .InvalidProtocolBufferException;
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.scm.container.common.helpers
-    .StorageContainerException;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
-import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
-import org.apache.hadoop.util.Time;
-import org.apache.hadoop.utils.BackgroundService;
-import org.apache.hadoop.utils.BackgroundTask;
-import org.apache.hadoop.utils.BackgroundTaskQueue;
-import org.apache.hadoop.utils.BackgroundTaskResult;
-import org.apache.hadoop.utils.BatchOperation;
-import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
-import org.apache.hadoop.utils.MetadataStore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.hadoop.ozone.OzoneConfigKeys
-    .OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
-    .OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL_DEFAULT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
-    .OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
-    .OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER_DEFAULT;
-
-/**
- * A per-datanode container block deleting service takes in charge
- * of deleting staled ozone blocks.
- */
-public class BlockDeletingService extends BackgroundService{
-
-  private static final Logger LOG =
-      LoggerFactory.getLogger(BlockDeletingService.class);
-
-  private final ContainerManager containerManager;
-  private final Configuration conf;
-
-  // Throttle number of blocks to delete per task,
-  // set to 1 for testing
-  private final int blockLimitPerTask;
-
-  // Throttle the number of containers to process concurrently at a time,
-  private final int containerLimitPerInterval;
-
-  // Task priority is useful when a to-delete block has weight.
-  private final static int TASK_PRIORITY_DEFAULT = 1;
-  // Core pool size for container tasks
-  private final static int BLOCK_DELETING_SERVICE_CORE_POOL_SIZE = 10;
-
-  public BlockDeletingService(ContainerManager containerManager,
-      long serviceInterval, long serviceTimeout, TimeUnit unit,
-      Configuration conf) {
-    super("BlockDeletingService", serviceInterval, unit,
-        BLOCK_DELETING_SERVICE_CORE_POOL_SIZE, serviceTimeout);
-    this.containerManager = containerManager;
-    this.conf = conf;
-    this.blockLimitPerTask = conf.getInt(
-        OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER,
-        OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER_DEFAULT);
-    this.containerLimitPerInterval = conf.getInt(
-        OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL,
-        OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL_DEFAULT);
-  }
-
-
-  @Override
-  public BackgroundTaskQueue getTasks() {
-    BackgroundTaskQueue queue = new BackgroundTaskQueue();
-    List<ContainerData> containers = Lists.newArrayList();
-    try {
-      // We at most list a number of containers a time,
-      // in case there are too many containers and start too many workers.
-      // We must ensure there is no empty container in this result.
-      // The chosen result depends on what container deletion policy is
-      // configured.
-      containers = containerManager.chooseContainerForBlockDeletion(
-          containerLimitPerInterval);
-      LOG.info("Plan to choose {} containers for block deletion, "
-          + "actually returns {} valid containers.",
-          containerLimitPerInterval, containers.size());
-
-      for(ContainerData container : containers) {
-        BlockDeletingTask containerTask =
-            new BlockDeletingTask(container, TASK_PRIORITY_DEFAULT);
-        queue.add(containerTask);
-      }
-    } catch (StorageContainerException e) {
-      LOG.warn("Failed to initiate block deleting tasks, "
-          + "caused by unable to get containers info. "
-          + "Retry in next interval. ", e);
-    } catch (Exception e) {
-      // In case listContainer call throws any uncaught RuntimeException.
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Unexpected error occurs during deleting blocks.", e);
-      }
-    }
-    return queue;
-  }
-
-  private static class ContainerBackgroundTaskResult
-      implements BackgroundTaskResult {
-    private List<String> deletedBlockIds;
-
-    ContainerBackgroundTaskResult() {
-      deletedBlockIds = new LinkedList<>();
-    }
-
-    public void addBlockId(String blockId) {
-      deletedBlockIds.add(blockId);
-    }
-
-    public void addAll(List<String> blockIds) {
-      deletedBlockIds.addAll(blockIds);
-    }
-
-    public List<String> getDeletedBlocks() {
-      return deletedBlockIds;
-    }
-
-    @Override
-    public int getSize() {
-      return deletedBlockIds.size();
-    }
-  }
-
-  private class BlockDeletingTask
-      implements BackgroundTask<BackgroundTaskResult> {
-
-    private final int priority;
-    private final ContainerData containerData;
-
-    BlockDeletingTask(ContainerData containerName, int priority) {
-      this.priority = priority;
-      this.containerData = containerName;
-    }
-
-    @Override
-    public BackgroundTaskResult call() throws Exception {
-      ContainerBackgroundTaskResult crr = new ContainerBackgroundTaskResult();
-      long startTime = Time.monotonicNow();
-      // Scan container's db and get list of under deletion blocks
-      MetadataStore meta = KeyUtils.getDB(containerData, conf);
-      // # of blocks to delete is throttled
-      KeyPrefixFilter filter =
-          new KeyPrefixFilter().addFilter(OzoneConsts.DELETING_KEY_PREFIX);
-      List<Map.Entry<byte[], byte[]>> toDeleteBlocks =
-          meta.getSequentialRangeKVs(null, blockLimitPerTask, filter);
-      if (toDeleteBlocks.isEmpty()) {
-        LOG.debug("No under deletion block found in container : {}",
-            containerData.getContainerID());
-      }
-
-      List<String> succeedBlocks = new LinkedList<>();
-      LOG.debug("Container : {}, To-Delete blocks : {}",
-          containerData.getContainerID(), toDeleteBlocks.size());
-      File dataDir = ContainerUtils.getDataDirectory(containerData).toFile();
-      if (!dataDir.exists() || !dataDir.isDirectory()) {
-        LOG.error("Invalid container data dir {} : "
-            + "not exist or not a directory", dataDir.getAbsolutePath());
-        return crr;
-      }
-
-      toDeleteBlocks.forEach(entry -> {
-        String blockName = DFSUtil.bytes2String(entry.getKey());
-        LOG.debug("Deleting block {}", blockName);
-        try {
-          ContainerProtos.KeyData data =
-              ContainerProtos.KeyData.parseFrom(entry.getValue());
-          for (ContainerProtos.ChunkInfo chunkInfo : data.getChunksList()) {
-            File chunkFile = dataDir.toPath()
-                .resolve(chunkInfo.getChunkName()).toFile();
-            if (FileUtils.deleteQuietly(chunkFile)) {
-              LOG.debug("block {} chunk {} deleted", blockName,
-                  chunkFile.getAbsolutePath());
-            }
-          }
-          succeedBlocks.add(blockName);
-        } catch (InvalidProtocolBufferException e) {
-          LOG.error("Failed to parse block info for block {}", blockName, e);
-        }
-      });
-
-      // Once files are deleted... replace deleting entries with deleted entries
-      BatchOperation batch = new BatchOperation();
-      succeedBlocks.forEach(entry -> {
-        String blockId =
-            entry.substring(OzoneConsts.DELETING_KEY_PREFIX.length());
-        String deletedEntry = OzoneConsts.DELETED_KEY_PREFIX + blockId;
-        batch.put(DFSUtil.string2Bytes(deletedEntry),
-            DFSUtil.string2Bytes(blockId));
-        batch.delete(DFSUtil.string2Bytes(entry));
-      });
-      meta.writeBatch(batch);
-      // update count of pending deletion blocks in in-memory container status
-      containerManager.decrPendingDeletionBlocks(succeedBlocks.size(),
-          containerData.getContainerID());
-
-      if (!succeedBlocks.isEmpty()) {
-        LOG.info("Container: {}, deleted blocks: {}, task elapsed time: {}ms",
-            containerData.getContainerID(), succeedBlocks.size(),
-            Time.monotonicNow() - startTime);
-      }
-      crr.addAll(succeedBlocks);
-      return crr;
-    }
-
-    @Override
-    public int getPriority() {
-      return priority;
-    }
-  }
-}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java
index e35becd..692a9d1 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java
@@ -86,20 +86,17 @@ public class VolumeSet {
    */
   private final AutoCloseableLock volumeSetLock;
 
-  private final DatanodeDetails dnDetails;
-  private String datanodeUuid;
+  private final String datanodeUuid;
   private String clusterID;
 
-  public VolumeSet(DatanodeDetails datanodeDetails, Configuration conf)
+  public VolumeSet(String dnUuid, Configuration conf)
       throws DiskOutOfSpaceException {
-    this(datanodeDetails, null, conf);
+    this(dnUuid, null, conf);
   }
 
-  public VolumeSet(DatanodeDetails datanodeDetails, String clusterID,
-      Configuration conf)
+  public VolumeSet(String dnUuid, String clusterID, Configuration conf)
       throws DiskOutOfSpaceException {
-    this.dnDetails = datanodeDetails;
-    this.datanodeUuid = datanodeDetails.getUuidString();
+    this.datanodeUuid = dnUuid;
     this.clusterID = clusterID;
     this.conf = conf;
     this.volumeSetLock = new AutoCloseableLock(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index 474c625..95621e5 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -31,6 +31,8 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
@@ -112,7 +114,7 @@ public class KeyValueContainer implements Container {
           .getVolumesList(), maxSize);
       String containerBasePath = containerVolume.getHddsRootDir().toString();
 
-      long containerId = containerData.getContainerId();
+      long containerId = containerData.getContainerID();
       String containerName = Long.toString(containerId);
 
       containerMetaDataPath = KeyValueContainerLocationUtil
@@ -127,7 +129,7 @@ public class KeyValueContainer implements Container {
           containerMetaDataPath, containerName);
 
       // Check if it is new Container.
-      KeyValueContainerUtil.verifyIsNewContainer(containerMetaDataPath);
+      ContainerUtils.verifyIsNewContainer(containerMetaDataPath);
 
       //Create Metadata path chunks path and metadata db
       KeyValueContainerUtil.createContainerMetaData(containerMetaDataPath,
@@ -184,7 +186,7 @@ public class KeyValueContainer implements Container {
     File tempCheckSumFile = null;
     FileOutputStream containerCheckSumStream = null;
     Writer writer = null;
-    long containerId = containerData.getContainerId();
+    long containerId = containerData.getContainerID();
     try {
       tempContainerFile = createTempFile(containerFile);
       tempCheckSumFile = createTempFile(containerCheckSumFile);
@@ -238,7 +240,7 @@ public class KeyValueContainer implements Container {
 
     File containerBkpFile = null;
     File checkSumBkpFile = null;
-    long containerId = containerData.getContainerId();
+    long containerId = containerData.getContainerID();
 
     try {
       if (containerFile.exists() && containerCheckSumFile.exists()) {
@@ -251,8 +253,8 @@ public class KeyValueContainer implements Container {
       } else {
         containerData.setState(ContainerProtos.ContainerLifeCycleState.INVALID);
         throw new StorageContainerException("Container is an Inconsistent " +
-            "state, missing required files(.container, .chksm)",
-            INVALID_CONTAINER_STATE);
+            "state, missing required files(.container, .chksm). ContainerID: " +
+            containerId, INVALID_CONTAINER_STATE);
       }
     } catch (StorageContainerException ex) {
       throw ex;
@@ -303,7 +305,7 @@ public class KeyValueContainer implements Container {
   @Override
   public void delete(boolean forceDelete)
       throws StorageContainerException {
-    long containerId = containerData.getContainerId();
+    long containerId = containerData.getContainerID();
     try {
       KeyValueContainerUtil.removeContainer(containerData, config, forceDelete);
     } catch (StorageContainerException ex) {
@@ -326,11 +328,11 @@ public class KeyValueContainer implements Container {
     // complete this action
     try {
       writeLock();
-      long containerId = containerData.getContainerId();
+      long containerId = containerData.getContainerID();
       if(!containerData.isValid()) {
         LOG.debug("Invalid container data. Container Id: {}", containerId);
-        throw new StorageContainerException("Invalid container data. Name : " +
-            containerId, INVALID_CONTAINER_STATE);
+        throw new StorageContainerException("Invalid container data. " +
+            "ContainerID: " + containerId, INVALID_CONTAINER_STATE);
       }
       containerData.closeContainer();
       File containerFile = getContainerFile();
@@ -380,16 +382,16 @@ public class KeyValueContainer implements Container {
     // TODO: Now, when writing the updated data to .container file, we are
     // holding lock and writing data to disk. We can have async implementation
     // to flush the update container data to disk.
-    long containerId = containerData.getContainerId();
+    long containerId = containerData.getContainerID();
     if(!containerData.isValid()) {
-      LOG.debug("Invalid container data. ID: {}", containerId);
+      LOG.debug("Invalid container data. ContainerID: {}", containerId);
       throw new StorageContainerException("Invalid container data. " +
-          "Container Name : " + containerId, INVALID_CONTAINER_STATE);
+          "ContainerID: " + containerId, INVALID_CONTAINER_STATE);
     }
     if (!forceUpdate && !containerData.isOpen()) {
       throw new StorageContainerException(
-          "Updating a closed container is not allowed. ID: " + containerId,
-          UNSUPPORTED_REQUEST);
+          "Updating a closed container without force option is not allowed. " +
+              "ContainerID: " + containerId, UNSUPPORTED_REQUEST);
     }
     try {
       for (Map.Entry<String, String> entry : metadata.entrySet()) {
@@ -482,7 +484,7 @@ public class KeyValueContainer implements Container {
    */
   private File getContainerFile() {
     return new File(containerData.getMetadataPath(), containerData
-        .getContainerId() + OzoneConsts.CONTAINER_EXTENSION);
+        .getContainerID() + OzoneConsts.CONTAINER_EXTENSION);
   }
 
   /**
@@ -491,7 +493,7 @@ public class KeyValueContainer implements Container {
    */
   private File getContainerCheckSumFile() {
     return new File(containerData.getMetadataPath(), containerData
-        .getContainerId() + OzoneConsts.CONTAINER_FILE_CHECKSUM_EXTENSION);
+        .getContainerID() + OzoneConsts.CONTAINER_FILE_CHECKSUM_EXTENSION);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
index ed2c6af..d9ae38a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
@@ -18,16 +18,29 @@
 
 package org.apache.hadoop.ozone.container.keyvalue;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.yaml.snakeyaml.nodes.Tag;
 
 
 import java.io.File;
+import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.hadoop.ozone.OzoneConsts.CHUNKS_PATH;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_TYPE;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_ID;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_TYPE;
+import static org.apache.hadoop.ozone.OzoneConsts.LAYOUTVERSION;
+import static org.apache.hadoop.ozone.OzoneConsts.MAX_SIZE_GB;
+import static org.apache.hadoop.ozone.OzoneConsts.METADATA;
+import static org.apache.hadoop.ozone.OzoneConsts.METADATA_PATH;
+import static org.apache.hadoop.ozone.OzoneConsts.STATE;
+
 /**
  * This class represents the KeyValueContainer metadata, which is the
  * in-memory representation of container metadata and is represented on disk
@@ -36,12 +49,20 @@ import java.util.Map;
 public class KeyValueContainerData extends ContainerData {
 
   // Yaml Tag used for KeyValueContainerData.
-  public static final Tag YAML_TAG = new Tag("KeyValueContainerData");
+  public static final Tag KEYVALUE_YAML_TAG = new Tag("KeyValueContainerData");
 
   // Fields need to be stored in .container file.
-  private static final List<String> YAML_FIELDS = Lists.newArrayList(
-      "containerType", "containerId", "layOutVersion", "state", "metadata",
-      "metadataPath", "chunksPath", "containerDBType", "maxSizeGB");
+  private static final List<String> YAML_FIELDS =
+      Lists.newArrayList(
+          CONTAINER_TYPE,
+          CONTAINER_ID,
+          LAYOUTVERSION,
+          STATE,
+          METADATA,
+          METADATA_PATH,
+          CHUNKS_PATH,
+          CONTAINER_DB_TYPE,
+          MAX_SIZE_GB);
 
   // Path to Container metadata Level DB/RocksDB Store and .container file.
   private String metadataPath;
@@ -96,11 +117,11 @@ public class KeyValueContainerData extends ContainerData {
   public File getDbFile() {
     return dbFile;
   }
+
   /**
    * Returns container metadata path.
-   *
-   * @return - path
    */
+  @Override
   public String getMetadataPath() {
     return metadataPath;
   }
@@ -123,6 +144,14 @@ public class KeyValueContainerData extends ContainerData {
   }
 
   /**
+   * Returns container chunks path.
+   */
+  @Override
+  public String getDataPath() {
+    return chunksPath;
+  }
+
+  /**
    * Set chunks Path.
    * @param chunkPath - File path.
    */
@@ -181,7 +210,7 @@ public class KeyValueContainerData extends ContainerData {
   public ContainerProtos.ContainerData getProtoBufMessage() {
     ContainerProtos.ContainerData.Builder builder = ContainerProtos
         .ContainerData.newBuilder();
-    builder.setContainerID(this.getContainerId());
+    builder.setContainerID(this.getContainerID());
     builder.setDbPath(this.getDbFile().getPath());
     builder.setContainerPath(this.getMetadataPath());
     builder.setState(this.getState());
@@ -211,4 +240,41 @@ public class KeyValueContainerData extends ContainerData {
   public static List<String> getYamlFields() {
     return YAML_FIELDS;
   }
+
+  /**
+   * Constructs a KeyValueContainerData object from ProtoBuf classes.
+   *
+   * @param protoData - ProtoBuf Message
+   * @throws IOException
+   */
+  @VisibleForTesting
+  public static KeyValueContainerData getFromProtoBuf(
+      ContainerProtos.ContainerData protoData) throws IOException {
+    // TODO: Add containerMaxSize to ContainerProtos.ContainerData
+    KeyValueContainerData data = new KeyValueContainerData(
+        protoData.getContainerID(),
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+    for (int x = 0; x < protoData.getMetadataCount(); x++) {
+      data.addMetadata(protoData.getMetadata(x).getKey(),
+          protoData.getMetadata(x).getValue());
+    }
+
+    if (protoData.hasContainerPath()) {
+      data.setContainerPath(protoData.getContainerPath());
+    }
+
+    if (protoData.hasState()) {
+      data.setState(protoData.getState());
+    }
+
+    if (protoData.hasBytesUsed()) {
+      data.setBytesUsed(protoData.getBytesUsed());
+    }
+
+    if(protoData.hasContainerDBType()) {
+      data.setContainerDBType(protoData.getContainerDBType());
+    }
+
+    return data;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index 8166a83..b2c82f0 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -288,7 +288,7 @@ public class KeyValueHandler extends Handler {
             DELETE_ON_OPEN_CONTAINER);
       } else {
         containerSet.removeContainer(
-            kvContainer.getContainerData().getContainerId());
+            kvContainer.getContainerData().getContainerID());
         // Release the lock first.
         // Avoid holding write locks for disk operations
         kvContainer.writeUnlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
index 3529af8..62e328e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
@@ -33,8 +33,8 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
-import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerImpl;
 import org.apache.ratis.shaded.com.google.protobuf.ByteString;
 import org.apache.hadoop.ozone.container.common.volume.VolumeIOStats;
 import org.apache.hadoop.util.Time;
@@ -229,17 +229,16 @@ public final class ChunkUtils {
    * Validates chunk data and returns a file object to Chunk File that we are
    * expected to write data to.
    *
-   * @param data - container data.
+   * @param chunkFile - chunkFile to write data into.
    * @param info - chunk info.
-   * @return File
+   * @return boolean isOverwrite
    * @throws StorageContainerException
    */
-  public static File validateChunk(KeyValueContainerData data, ChunkInfo info)
-      throws StorageContainerException {
+  public static boolean validateChunkForOverwrite(File chunkFile,
+      ChunkInfo info) throws StorageContainerException {
 
     Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
 
-    File chunkFile = getChunkFile(data, info);
     if (isOverWriteRequested(chunkFile, info)) {
       if (!isOverWritePermitted(info)) {
         log.error("Rejecting write chunk request. Chunk overwrite " +
@@ -248,8 +247,9 @@ public final class ChunkUtils {
             "OverWrite flag required." + info.toString(),
             OVERWRITE_FLAG_REQUIRED);
       }
+      return true;
     }
-    return chunkFile;
+    return false;
   }
 
   /**
@@ -340,8 +340,8 @@ public final class ChunkUtils {
   public static ContainerCommandResponseProto getReadChunkResponse(
       ContainerCommandRequestProto msg, byte[] data, ChunkInfo info) {
     Preconditions.checkNotNull(msg);
-    Preconditions.checkNotNull("Chunk data is null", data);
-    Preconditions.checkNotNull("Chunk Info is null", info);
+    Preconditions.checkNotNull(data, "Chunk data is null");
+    Preconditions.checkNotNull(info, "Chunk Info is null");
 
     ReadChunkResponseProto.Builder response =
         ReadChunkResponseProto.newBuilder();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
index 714f445..5845fae 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
@@ -57,24 +57,25 @@ public final class KeyUtils {
    * add into cache. This function is called with containerManager
    * ReadLock held.
    *
-   * @param container container.
+   * @param containerData containerData.
    * @param conf configuration.
    * @return MetadataStore handle.
    * @throws StorageContainerException
    */
-  public static MetadataStore getDB(KeyValueContainerData container,
+  public static MetadataStore getDB(KeyValueContainerData containerData,
                                     Configuration conf) throws
       StorageContainerException {
-    Preconditions.checkNotNull(container);
+    Preconditions.checkNotNull(containerData);
     ContainerCache cache = ContainerCache.getInstance(conf);
     Preconditions.checkNotNull(cache);
-    Preconditions.checkNotNull(container.getDbFile());
+    Preconditions.checkNotNull(containerData.getDbFile());
     try {
-      return cache.getDB(container.getContainerId(), container
-          .getContainerDBType(), container.getDbFile().getAbsolutePath());
+      return cache.getDB(containerData.getContainerID(), containerData
+          .getContainerDBType(), containerData.getDbFile().getAbsolutePath());
     } catch (IOException ex) {
-      String message = String.format("Unable to open DB Path: " +
-          "%s. ex: %s", container.getDbFile(), ex.getMessage());
+      String message = String.format("Error opening DB. Container:%s " +
+          "ContainerPath:%s", containerData.getContainerID(), containerData
+          .getDbFile().getPath());
       throw new StorageContainerException(message, UNABLE_TO_READ_METADATA_DB);
     }
   }
@@ -89,7 +90,7 @@ public final class KeyUtils {
     Preconditions.checkNotNull(container);
     ContainerCache cache = ContainerCache.getInstance(conf);
     Preconditions.checkNotNull(cache);
-    cache.removeDB(container.getContainerId());
+    cache.removeDB(container.getContainerID());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
index 4c17dce..3c3c9cb 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.utils.MetadataKeyFilters;
 import org.apache.hadoop.utils.MetadataStore;
@@ -68,19 +69,6 @@ public final class KeyValueContainerUtil {
   private static final Logger LOG = LoggerFactory.getLogger(
       KeyValueContainerUtil.class);
 
-
-  public static void verifyIsNewContainer(File containerFile) throws
-      FileAlreadyExistsException {
-    Preconditions.checkNotNull(containerFile, "containerFile Should not be " +
-        "null");
-    if (containerFile.getParentFile().exists()) {
-      LOG.error("container already exists on disk. File: {}", containerFile
-          .toPath());
-      throw new FileAlreadyExistsException("container already exists on " +
-            "disk.");
-    }
-  }
-
   /**
    * creates metadata path, chunks path and  metadata DB for the specified
    * container.
@@ -271,7 +259,7 @@ public final class KeyValueContainerUtil {
     Preconditions.checkNotNull(dbFile, "dbFile cannot be null");
     Preconditions.checkNotNull(config, "ozone config cannot be null");
 
-    long containerId = containerData.getContainerId();
+    long containerId = containerData.getContainerID();
     String containerName = String.valueOf(containerId);
     File metadataPath = new File(containerData.getMetadataPath());
 
@@ -282,7 +270,7 @@ public final class KeyValueContainerUtil {
 
     // Verify Checksum
     String checksum = KeyValueContainerUtil.computeCheckSum(
-        containerData.getContainerId(), containerFile);
+        containerData.getContainerID(), containerFile);
     KeyValueContainerUtil.verifyCheckSum(containerId, checksumFile, checksum);
 
     containerData.setDbFile(dbFile);
@@ -305,4 +293,34 @@ public final class KeyValueContainerUtil {
     containerData.setKeyCount(liveKeys.size());
   }
 
+  /**
+   * Returns the path where data or chunks live for a given container.
+   *
+   * @param kvContainerData - KeyValueContainerData
+   * @return - Path to the chunks directory
+   */
+  public static Path getDataDirectory(KeyValueContainerData kvContainerData) {
+
+    String chunksPath = kvContainerData.getChunksPath();
+    Preconditions.checkNotNull(chunksPath);
+
+    return Paths.get(chunksPath);
+  }
+
+  /**
+   * Container metadata directory -- here is where the level DB and
+   * .container file lives.
+   *
+   * @param kvContainerData - KeyValueContainerData
+   * @return Path to the metadata directory
+   */
+  public static Path getMetadataDirectory(
+      KeyValueContainerData kvContainerData) {
+
+    String metadataPath = kvContainerData.getMetadataPath();
+    Preconditions.checkNotNull(metadataPath);
+
+    return Paths.get(metadataPath);
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java
index c3160a8..ce317bd 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java
@@ -76,7 +76,10 @@ public class ChunkManagerImpl implements ChunkManager {
       HddsVolume volume = containerData.getVolume();
       VolumeIOStats volumeIOStats = volume.getVolumeIOStats();
 
-      File chunkFile = ChunkUtils.validateChunk(containerData, info);
+      File chunkFile = ChunkUtils.getChunkFile(containerData, info);
+
+      boolean isOverwrite = ChunkUtils.validateChunkForOverwrite(
+          chunkFile, info);
       File tmpChunkFile = getTmpChunkFile(chunkFile, info);
 
       LOG.debug("writing chunk:{} chunk stage:{} chunk file:{} tmp chunk file",
@@ -101,8 +104,9 @@ public class ChunkManagerImpl implements ChunkManager {
       case COMBINED:
         // directly write to the chunk file
         ChunkUtils.writeData(chunkFile, info, data, volumeIOStats);
-        // Increment container stats here, as we directly write to chunk file.
-        containerData.incrBytesUsed(info.getLen());
+        if (!isOverwrite) {
+          containerData.incrBytesUsed(info.getLen());
+        }
         containerData.incrWriteCount();
         containerData.incrWriteBytes(info.getLen());
         break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
new file mode 100644
index 0000000..6aa54d1
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
@@ -0,0 +1,248 @@
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue.statemachine.background;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
+import org.apache.ratis.shaded.com.google.protobuf
+    .InvalidProtocolBufferException;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .StorageContainerException;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.utils.BackgroundService;
+import org.apache.hadoop.utils.BackgroundTask;
+import org.apache.hadoop.utils.BackgroundTaskQueue;
+import org.apache.hadoop.utils.BackgroundTaskResult;
+import org.apache.hadoop.utils.BatchOperation;
+import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
+import org.apache.hadoop.utils.MetadataStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys
+    .OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys
+    .OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConfigKeys
+    .OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER;
+import static org.apache.hadoop.ozone.OzoneConfigKeys
+    .OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER_DEFAULT;
+
+/**
+ * A per-datanode container block deleting service takes in charge
+ * of deleting staled ozone blocks.
+ */
+// TODO: Fix BlockDeletingService to work with new StorageLayer
+public class BlockDeletingService extends BackgroundService{
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(BlockDeletingService.class);
+
+  ContainerSet containerSet;
+  private final Configuration conf;
+
+  // Throttle number of blocks to delete per task,
+  // set to 1 for testing
+  private final int blockLimitPerTask;
+
+  // Throttle the number of containers to process concurrently at a time,
+  private final int containerLimitPerInterval;
+
+  // Task priority is useful when a to-delete block has weight.
+  private final static int TASK_PRIORITY_DEFAULT = 1;
+  // Core pool size for container tasks
+  private final static int BLOCK_DELETING_SERVICE_CORE_POOL_SIZE = 10;
+
+  public BlockDeletingService(ContainerSet containerSet,
+      long serviceInterval, long serviceTimeout, Configuration conf) {
+    super("BlockDeletingService", serviceInterval,
+        TimeUnit.MILLISECONDS, BLOCK_DELETING_SERVICE_CORE_POOL_SIZE,
+        serviceTimeout);
+    this.containerSet = containerSet;
+    this.conf = conf;
+    this.blockLimitPerTask = conf.getInt(
+        OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER,
+        OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER_DEFAULT);
+    this.containerLimitPerInterval = conf.getInt(
+        OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL,
+        OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL_DEFAULT);
+  }
+
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+    List<ContainerData> containers = Lists.newArrayList();
+    try {
+      // We at most list a number of containers a time,
+      // in case there are too many containers and start too many workers.
+      // We must ensure there is no empty container in this result.
+      // The chosen result depends on what container deletion policy is
+      // configured.
+      containers = containerSet.chooseContainerForBlockDeletion(
+          containerLimitPerInterval);
+      LOG.info("Plan to choose {} containers for block deletion, "
+          + "actually returns {} valid containers.",
+          containerLimitPerInterval, containers.size());
+
+      for(ContainerData container : containers) {
+        BlockDeletingTask containerTask =
+            new BlockDeletingTask(container, TASK_PRIORITY_DEFAULT);
+        queue.add(containerTask);
+      }
+    } catch (StorageContainerException e) {
+      LOG.warn("Failed to initiate block deleting tasks, "
+          + "caused by unable to get containers info. "
+          + "Retry in next interval. ", e);
+    } catch (Exception e) {
+      // In case listContainer call throws any uncaught RuntimeException.
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Unexpected error occurs during deleting blocks.", e);
+      }
+    }
+    return queue;
+  }
+
+  private static class ContainerBackgroundTaskResult
+      implements BackgroundTaskResult {
+    private List<String> deletedBlockIds;
+
+    ContainerBackgroundTaskResult() {
+      deletedBlockIds = new LinkedList<>();
+    }
+
+    public void addBlockId(String blockId) {
+      deletedBlockIds.add(blockId);
+    }
+
+    public void addAll(List<String> blockIds) {
+      deletedBlockIds.addAll(blockIds);
+    }
+
+    public List<String> getDeletedBlocks() {
+      return deletedBlockIds;
+    }
+
+    @Override
+    public int getSize() {
+      return deletedBlockIds.size();
+    }
+  }
+
+  private class BlockDeletingTask
+      implements BackgroundTask<BackgroundTaskResult> {
+
+    private final int priority;
+    private final ContainerData containerData;
+
+    BlockDeletingTask(ContainerData containerName, int priority) {
+      this.priority = priority;
+      this.containerData = containerName;
+    }
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      ContainerBackgroundTaskResult crr = new ContainerBackgroundTaskResult();
+      long startTime = Time.monotonicNow();
+      // Scan container's db and get list of under deletion blocks
+      MetadataStore meta = KeyUtils.getDB(
+          (KeyValueContainerData) containerData, conf);
+      // # of blocks to delete is throttled
+      KeyPrefixFilter filter =
+          new KeyPrefixFilter().addFilter(OzoneConsts.DELETING_KEY_PREFIX);
+      List<Map.Entry<byte[], byte[]>> toDeleteBlocks =
+          meta.getSequentialRangeKVs(null, blockLimitPerTask, filter);
+      if (toDeleteBlocks.isEmpty()) {
+        LOG.debug("No under deletion block found in container : {}",
+            containerData.getContainerID());
+      }
+
+      List<String> succeedBlocks = new LinkedList<>();
+      LOG.debug("Container : {}, To-Delete blocks : {}",
+          containerData.getContainerID(), toDeleteBlocks.size());
+      File dataDir = new File(containerData.getDataPath());
+      if (!dataDir.exists() || !dataDir.isDirectory()) {
+        LOG.error("Invalid container data dir {} : "
+            + "not exist or not a directory", dataDir.getAbsolutePath());
+        return crr;
+      }
+
+      toDeleteBlocks.forEach(entry -> {
+        String blockName = DFSUtil.bytes2String(entry.getKey());
+        LOG.debug("Deleting block {}", blockName);
+        try {
+          ContainerProtos.KeyData data =
+              ContainerProtos.KeyData.parseFrom(entry.getValue());
+          for (ContainerProtos.ChunkInfo chunkInfo : data.getChunksList()) {
+            File chunkFile = dataDir.toPath()
+                .resolve(chunkInfo.getChunkName()).toFile();
+            if (FileUtils.deleteQuietly(chunkFile)) {
+              LOG.debug("block {} chunk {} deleted", blockName,
+                  chunkFile.getAbsolutePath());
+            }
+          }
+          succeedBlocks.add(blockName);
+        } catch (InvalidProtocolBufferException e) {
+          LOG.error("Failed to parse block info for block {}", blockName, e);
+        }
+      });
+
+      // Once files are deleted... replace deleting entries with deleted entries
+      BatchOperation batch = new BatchOperation();
+      succeedBlocks.forEach(entry -> {
+        String blockId =
+            entry.substring(OzoneConsts.DELETING_KEY_PREFIX.length());
+        String deletedEntry = OzoneConsts.DELETED_KEY_PREFIX + blockId;
+        batch.put(DFSUtil.string2Bytes(deletedEntry),
+            DFSUtil.string2Bytes(blockId));
+        batch.delete(DFSUtil.string2Bytes(entry));
+      });
+      meta.writeBatch(batch);
+      // update count of pending deletion blocks in in-memory container status
+      containerData.decrPendingDeletionBlocks(succeedBlocks.size());
+
+      if (!succeedBlocks.isEmpty()) {
+        LOG.info("Container: {}, deleted blocks: {}, task elapsed time: {}ms",
+            containerData.getContainerID(), succeedBlocks.size(),
+            Time.monotonicNow() - startTime);
+      }
+      crr.addAll(succeedBlocks);
+      return crr;
+    }
+
+    @Override
+    public int getPriority() {
+      return priority;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/package-info.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/package-info.java
new file mode 100644
index 0000000..69d8042
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/package-info.java
@@ -0,0 +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
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.keyvalue.statemachine.background;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
index 9e25c59..667ea5c 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
@@ -73,7 +73,7 @@ public class OzoneContainer {
       conf) throws IOException {
     this.dnDetails = datanodeDetails;
     this.config = conf;
-    this.volumeSet = new VolumeSet(datanodeDetails, conf);
+    this.volumeSet = new VolumeSet(datanodeDetails.getUuidString(), conf);
     this.containerSet = new ContainerSet();
     boolean useGrpc = this.config.getBoolean(
         ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_KEY,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
index 16c4c2a..42db66d 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
@@ -47,7 +47,7 @@ public class TestKeyValueContainerData {
         MAXSIZE);
 
     assertEquals(containerType, kvData.getContainerType());
-    assertEquals(containerId, kvData.getContainerId());
+    assertEquals(containerId, kvData.getContainerID());
     assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN, kvData
         .getState());
     assertEquals(0, kvData.getMetadata().size());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
index 41d8315..eed5606 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
@@ -64,7 +64,7 @@ public class TestContainerDataYaml {
     // Read from .container file, and verify data.
     KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
         .readContainerFile(containerFile);
-    assertEquals(Long.MAX_VALUE, kvData.getContainerId());
+    assertEquals(Long.MAX_VALUE, kvData.getContainerID());
     assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
         .getContainerType());
     assertEquals("RocksDB", kvData.getContainerDBType());
@@ -92,7 +92,7 @@ public class TestContainerDataYaml {
         containerFile);
 
     // verify data.
-    assertEquals(Long.MAX_VALUE, kvData.getContainerId());
+    assertEquals(Long.MAX_VALUE, kvData.getContainerID());
     assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
         .getContainerType());
     assertEquals("RocksDB", kvData.getContainerDBType());
@@ -150,7 +150,7 @@ public class TestContainerDataYaml {
       assertEquals("RocksDB", kvData.getContainerDBType());
       assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
           .getContainerType());
-      assertEquals(9223372036854775807L, kvData.getContainerId());
+      assertEquals(9223372036854775807L, kvData.getContainerID());
       assertEquals("/hdds/current/aed-fg4-hji-jkl/containerdir0/1", kvData
           .getChunksPath());
       assertEquals("/hdds/current/aed-fg4-hji-jkl/containerdir0/1", kvData

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
index 6ec1fe4..ae670e0 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
@@ -74,7 +74,7 @@ public class TestContainerSet {
         .getContainer(containerId);
     KeyValueContainerData keyValueContainerData = (KeyValueContainerData)
         container.getContainerData();
-    assertEquals(containerId, keyValueContainerData.getContainerId());
+    assertEquals(containerId, keyValueContainerData.getContainerID());
     assertEquals(state, keyValueContainerData.getState());
     assertNull(containerSet.getContainer(1000L));
 
@@ -97,7 +97,7 @@ public class TestContainerSet {
     while(containerIterator.hasNext()) {
       Container kv = containerIterator.next();
       ContainerData containerData = kv.getContainerData();
-      long containerId = containerData.getContainerId();
+      long containerId = containerData.getContainerID();
       if (containerId%2 == 0) {
         assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED,
             containerData.getState());
@@ -117,7 +117,7 @@ public class TestContainerSet {
     while (containerMapIterator.hasNext()) {
       Container kv = containerMapIterator.next().getValue();
       ContainerData containerData = kv.getContainerData();
-      long containerId = containerData.getContainerId();
+      long containerId = containerData.getContainerID();
       if (containerId%2 == 0) {
         assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED,
             containerData.getState());
@@ -155,8 +155,8 @@ public class TestContainerSet {
     assertEquals(5, result.size());
 
     for(ContainerData containerData : result) {
-      assertTrue(containerData.getContainerId() >=2 && containerData
-          .getContainerId()<=6);
+      assertTrue(containerData.getContainerID() >=2 && containerData
+          .getContainerID()<=6);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestRoundRobinVolumeChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestRoundRobinVolumeChoosingPolicy.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestRoundRobinVolumeChoosingPolicy.java
index 41610af..a45a639 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestRoundRobinVolumeChoosingPolicy.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestRoundRobinVolumeChoosingPolicy.java
@@ -54,11 +54,7 @@ public class TestRoundRobinVolumeChoosingPolicy {
     conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataDirKey);
     policy = ReflectionUtils.newInstance(
         RoundRobinVolumeChoosingPolicy.class, null);
-    DatanodeDetails datanodeDetails = DatanodeDetails.newBuilder()
-        .setUuid(UUID.randomUUID().toString())
-        .setIpAddress(DUMMY_IP_ADDR)
-        .build();
-    VolumeSet volumeSet = new VolumeSet(datanodeDetails, conf);
+    VolumeSet volumeSet = new VolumeSet(UUID.randomUUID().toString(), conf);
     volumes = volumeSet.getVolumesList();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSet.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSet.java
index 61383de..41f75bd 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSet.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSet.java
@@ -53,11 +53,7 @@ public class TestVolumeSet {
   private static final String DUMMY_IP_ADDR = "0.0.0.0";
 
   private void initializeVolumeSet() throws Exception {
-    DatanodeDetails datanodeDetails = DatanodeDetails.newBuilder()
-        .setUuid(UUID.randomUUID().toString())
-        .setIpAddress(DUMMY_IP_ADDR)
-        .build();
-    volumeSet = new VolumeSet(datanodeDetails, conf);
+    volumeSet = new VolumeSet(UUID.randomUUID().toString(), conf);
   }
 
   @Rule

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
index e55ea57..4f00507 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
@@ -245,24 +245,6 @@ public class TestKeyValueContainer {
   }
 
   @Test
-  public void testUpdateContainerInvalidMetadata() throws IOException {
-    try {
-      keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
-      Map<String, String> metadata = new HashMap<>();
-      metadata.put("VOLUME", "ozone");
-      keyValueContainer.update(metadata, true);
-      //Trying to update again with same metadata
-      keyValueContainer.update(metadata, true);
-      fail("testUpdateContainerInvalidMetadata failed");
-    } catch (StorageContainerException ex) {
-      GenericTestUtils.assertExceptionContains("Container Metadata update " +
-          "error", ex);
-      assertEquals(ContainerProtos.Result.CONTAINER_METADATA_ERROR, ex
-          .getResult());
-    }
-  }
-
-  @Test
   public void testUpdateContainerUnsupportedRequest() throws Exception {
     try {
       keyValueContainerData.setState(ContainerProtos.ContainerLifeCycleState
@@ -275,7 +257,7 @@ public class TestKeyValueContainer {
       fail("testUpdateContainerUnsupportedRequest failed");
     } catch (StorageContainerException ex) {
       GenericTestUtils.assertExceptionContains("Updating a closed container " +
-          "is not allowed", ex);
+          "without force option is not allowed", ex);
       assertEquals(ContainerProtos.Result.UNSUPPORTED_REQUEST, ex
           .getResult());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
index 947ad51..a997145 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.container.keyvalue;
 
+import com.google.common.base.Supplier;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
@@ -30,6 +31,7 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -71,12 +73,7 @@ public class TestKeyValueHandler {
     conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, volume);
 
     this.containerSet = new ContainerSet();
-    DatanodeDetails datanodeDetails = DatanodeDetails.newBuilder()
-        .setUuid(DATANODE_UUID)
-        .setHostName("localhost")
-        .setIpAddress("127.0.0.1")
-        .build();
-    this.volumeSet = new VolumeSet(datanodeDetails, conf);
+    this.volumeSet = new VolumeSet(DATANODE_UUID, conf);
 
     this.dispatcher = new HddsDispatcher(conf, containerSet, volumeSet);
     this.handler = (KeyValueHandler) dispatcher.getHandler(
@@ -246,7 +243,7 @@ public class TestKeyValueHandler {
 
     // Verify that new container is added to containerSet.
     Container container = containerSet.getContainer(contId);
-    Assert.assertEquals(contId, container.getContainerData().getContainerId());
+    Assert.assertEquals(contId, container.getContainerData().getContainerID());
     Assert.assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN,
         container.getContainerState());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
index 26e1c77..27c6528 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
@@ -62,7 +62,7 @@ public class TestOzoneContainer {
     conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, folder.getRoot()
         .getAbsolutePath() + "," + folder.newFolder().getAbsolutePath());
     conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, folder.newFolder().getAbsolutePath());
-    volumeSet = new VolumeSet(datanodeDetails, conf);
+    volumeSet = new VolumeSet(datanodeDetails.getUuidString(), conf);
     volumeChoosingPolicy = new RoundRobinVolumeChoosingPolicy();
 
     for (int i=0; i<10; i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
index 7c12945..a87f655 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
@@ -19,8 +19,8 @@ package org.apache.hadoop.ozone.container.testutils;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
-import org.apache.hadoop.ozone.container.common.statemachine.background
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.keyvalue.statemachine.background
     .BlockDeletingService;
 
 import java.util.concurrent.CountDownLatch;
@@ -42,10 +42,9 @@ public class BlockDeletingServiceTestImpl
   private Thread testingThread;
   private AtomicInteger numOfProcessed = new AtomicInteger(0);
 
-  public BlockDeletingServiceTestImpl(ContainerManager containerManager,
+  public BlockDeletingServiceTestImpl(ContainerSet containerSet,
       int serviceInterval, Configuration conf) {
-    super(containerManager, serviceInterval, SERVICE_TIMEOUT_IN_MILLISECONDS,
-        TimeUnit.MILLISECONDS, conf);
+    super(containerSet, serviceInterval, SERVICE_TIMEOUT_IN_MILLISECONDS, conf);
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/test/resources/additionalfields.container
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/resources/additionalfields.container b/hadoop-hdds/container-service/src/test/resources/additionalfields.container
index 9027538..38c104a 100644
--- a/hadoop-hdds/container-service/src/test/resources/additionalfields.container
+++ b/hadoop-hdds/container-service/src/test/resources/additionalfields.container
@@ -1,7 +1,7 @@
 !<KeyValueContainerData>
 containerDBType: RocksDB
 chunksPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
-containerId: 9223372036854775807
+containerID: 9223372036854775807
 containerType: KeyValueContainer
 metadataPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
 layOutVersion: 1

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/test/resources/incorrect.container
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/resources/incorrect.container b/hadoop-hdds/container-service/src/test/resources/incorrect.container
index 6848484..abbb6aa 100644
--- a/hadoop-hdds/container-service/src/test/resources/incorrect.container
+++ b/hadoop-hdds/container-service/src/test/resources/incorrect.container
@@ -1,7 +1,7 @@
 !<KeyValueContainerData>
 containerDBType: RocksDB
 chunksPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
-containerId: 9223372036854775807
+containerID: 9223372036854775807
 containerType: KeyValueContainer
 metadataPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
 layOutVersion: 1

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
index b720549..d25b73e 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
@@ -58,6 +58,8 @@ public final class ContainerTestHelper {
       ContainerTestHelper.class);
   private static Random r = new Random();
 
+  public static final int CONTAINER_MAX_SIZE_GB = 1;
+
   /**
    * Never constructed.
    */


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


[25/37] hadoop git commit: HDDS-176. Add keyCount and container maximum size to ContainerData. Contributed by Bharat Viswanadham.

Posted by bh...@apache.org.
HDDS-176. Add keyCount and container maximum size to ContainerData. Contributed by Bharat Viswanadham.


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

Branch: refs/heads/trunk
Commit: e1f4b3b560a9ec2b34bb9ffbfe71fd3b0ac48120
Parents: 44b091a
Author: Hanisha Koneru <ha...@apache.org>
Authored: Tue Jul 3 09:53:41 2018 -0700
Committer: Hanisha Koneru <ha...@apache.org>
Committed: Tue Jul 3 09:53:41 2018 -0700

----------------------------------------------------------------------
 .../container/common/impl/ContainerData.java    | 50 +++++++++++++++++++-
 .../common/impl/ContainerDataYaml.java          | 10 ++--
 .../states/endpoint/VersionEndpointTask.java    |  3 --
 .../container/keyvalue/KeyValueContainer.java   |  8 +---
 .../keyvalue/KeyValueContainerData.java         | 20 +++++---
 .../container/keyvalue/KeyValueHandler.java     | 14 ++++--
 .../keyvalue/helpers/KeyValueContainerUtil.java |  1 +
 .../container/keyvalue/impl/KeyManagerImpl.java |  6 +++
 .../container/keyvalue/impl/package-info.java   | 22 +++++++++
 .../container/ozoneimpl/ContainerReader.java    | 21 +++++---
 .../common/TestKeyValueContainerData.java       |  8 +++-
 .../common/impl/TestContainerDataYaml.java      |  6 ++-
 .../container/common/impl/TestContainerSet.java |  4 +-
 .../keyvalue/TestChunkManagerImpl.java          |  2 +-
 .../container/keyvalue/TestKeyManagerImpl.java  |  7 ++-
 .../keyvalue/TestKeyValueContainer.java         |  2 +-
 .../container/ozoneimpl/TestOzoneContainer.java |  2 +-
 .../test/resources/additionalfields.container   |  1 +
 .../src/test/resources/incorrect.container      |  1 +
 19 files changed, 147 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
index 872d958..238fb09 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
@@ -53,12 +53,15 @@ public class ContainerData {
   // State of the Container
   private ContainerLifeCycleState state;
 
+  private final int maxSizeGB;
+
   /** parameters for read/write statistics on the container. **/
   private final AtomicLong readBytes;
   private final AtomicLong writeBytes;
   private final AtomicLong readCount;
   private final AtomicLong writeCount;
   private final AtomicLong bytesUsed;
+  private final AtomicLong keyCount;
 
   private HddsVolume volume;
 
@@ -67,8 +70,9 @@ public class ContainerData {
    * Creates a ContainerData Object, which holds metadata of the container.
    * @param type - ContainerType
    * @param containerId - ContainerId
+   * @param size - container maximum size
    */
-  public ContainerData(ContainerType type, long containerId) {
+  public ContainerData(ContainerType type, long containerId, int size) {
     this.containerType = type;
     this.containerId = containerId;
     this.layOutVersion = ChunkLayOutVersion.getLatestVersion().getVersion();
@@ -79,6 +83,8 @@ public class ContainerData {
     this.writeCount =  new AtomicLong(0L);
     this.writeBytes =  new AtomicLong(0L);
     this.bytesUsed = new AtomicLong(0L);
+    this.keyCount = new AtomicLong(0L);
+    this.maxSizeGB = size;
   }
 
   /**
@@ -86,9 +92,10 @@ public class ContainerData {
    * @param type - ContainerType
    * @param containerId - ContainerId
    * @param layOutVersion - Container layOutVersion
+   * @param size - Container maximum size
    */
   public ContainerData(ContainerType type, long containerId, int
-      layOutVersion) {
+      layOutVersion, int size) {
     this.containerType = type;
     this.containerId = containerId;
     this.layOutVersion = layOutVersion;
@@ -99,6 +106,8 @@ public class ContainerData {
     this.writeCount =  new AtomicLong(0L);
     this.writeBytes =  new AtomicLong(0L);
     this.bytesUsed = new AtomicLong(0L);
+    this.keyCount = new AtomicLong(0L);
+    this.maxSizeGB = size;
   }
 
   /**
@@ -134,6 +143,14 @@ public class ContainerData {
   }
 
   /**
+   * Return's maximum size of the container in GB.
+   * @return maxSizeGB
+   */
+  public int getMaxSizeGB() {
+    return maxSizeGB;
+  }
+
+  /**
    * Returns the layOutVersion of the actual container data format.
    * @return layOutVersion
    */
@@ -309,5 +326,34 @@ public class ContainerData {
     return volume;
   }
 
+  /**
+   * Increments the number of keys in the container.
+   */
+  public void incrKeyCount() {
+    this.keyCount.incrementAndGet();
+  }
+
+  /**
+   * Decrements number of keys in the container.
+   */
+  public void decrKeyCount() {
+    this.keyCount.decrementAndGet();
+  }
+
+  /**
+   * Returns number of keys in the container.
+   * @return key count
+   */
+  public long getKeyCount() {
+    return this.keyCount.get();
+  }
+
+  /**
+   * Set's number of keys in the container.
+   * @param count
+   */
+  public void setKeyCount(long count) {
+    this.keyCount.set(count);
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
index 6b8e6ee..4f4d588 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
@@ -32,6 +32,7 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.OutputStreamWriter;
 import java.io.File;
+import java.util.List;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.Map;
@@ -47,7 +48,6 @@ import org.yaml.snakeyaml.nodes.ScalarNode;
 import org.yaml.snakeyaml.nodes.Tag;
 import org.yaml.snakeyaml.representer.Representer;
 
-import static org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData.YAML_FIELDS;
 import static org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData.YAML_TAG;
 
 /**
@@ -150,10 +150,11 @@ public final class ContainerDataYaml {
       // When a new Container type is added, we need to add what fields need
       // to be filtered here
       if (type.equals(KeyValueContainerData.class)) {
+        List<String> yamlFields = KeyValueContainerData.getYamlFields();
         // filter properties
         for (Property prop : set) {
           String name = prop.getName();
-          if (YAML_FIELDS.contains(name)) {
+          if (yamlFields.contains(name)) {
             filtered.add(prop);
           }
         }
@@ -183,9 +184,12 @@ public final class ContainerDataYaml {
         long layOutVersion = (long) nodes.get("layOutVersion");
         int lv = (int) layOutVersion;
 
+        long size = (long) nodes.get("maxSizeGB");
+        int maxSize = (int) size;
+
         //When a new field is added, it needs to be added here.
         KeyValueContainerData kvData = new KeyValueContainerData((long) nodes
-            .get("containerId"), lv);
+            .get("containerId"), lv, maxSize);
         kvData.setContainerDBType((String)nodes.get("containerDBType"));
         kvData.setMetadataPath((String) nodes.get(
             "metadataPath"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
index e4cb4d5..d782b59 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
@@ -18,7 +18,6 @@ package org.apache.hadoop.ozone.container.common.states.endpoint;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -30,7 +29,6 @@ import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.ozone.protocol.VersionResponse;
 
 import java.io.IOException;
-import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
 
@@ -67,7 +65,6 @@ public class VersionEndpointTask implements
       rpcEndPoint.setVersion(response);
       VolumeSet volumeSet = ozoneContainer.getVolumeSet();
       Map<String, HddsVolume> volumeMap = volumeSet.getVolumeMap();
-      List<HddsProtos.KeyValue> keyValues =  versionResponse.getKeysList();
 
       String scmId = response.getValue(OzoneConsts.SCM_ID);
       String clusterId = response.getValue(OzoneConsts.CLUSTER_ID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index 72d50d6..474c625 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerLifeCycleState;
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.io.IOUtils;
@@ -84,7 +83,6 @@ public class KeyValueContainer implements Container {
   private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
 
   private final KeyValueContainerData containerData;
-  private long containerMaxSize;
   private Configuration config;
 
   public KeyValueContainer(KeyValueContainerData containerData, Configuration
@@ -95,9 +93,6 @@ public class KeyValueContainer implements Container {
         "be null");
     this.config = ozoneConfig;
     this.containerData = containerData;
-    this.containerMaxSize = (long) ozoneConfig.getInt(ScmConfigKeys
-        .OZONE_SCM_CONTAINER_SIZE_GB, ScmConfigKeys
-        .OZONE_SCM_CONTAINER_SIZE_DEFAULT) * 1024L * 1024L * 1024L;
   }
 
   @Override
@@ -111,9 +106,10 @@ public class KeyValueContainer implements Container {
     File containerMetaDataPath = null;
     //acquiring volumeset lock and container lock
     volumeSet.acquireLock();
+    long maxSize = (containerData.getMaxSizeGB() * 1024L * 1024L * 1024L);
     try {
       HddsVolume containerVolume = volumeChoosingPolicy.chooseVolume(volumeSet
-          .getVolumesList(), containerMaxSize);
+          .getVolumesList(), maxSize);
       String containerBasePath = containerVolume.getHddsRootDir().toString();
 
       long containerId = containerData.getContainerId();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
index 3b24468..ed2c6af 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
@@ -39,9 +39,9 @@ public class KeyValueContainerData extends ContainerData {
   public static final Tag YAML_TAG = new Tag("KeyValueContainerData");
 
   // Fields need to be stored in .container file.
-  public static final List<String> YAML_FIELDS = Lists.newArrayList(
+  private static final List<String> YAML_FIELDS = Lists.newArrayList(
       "containerType", "containerId", "layOutVersion", "state", "metadata",
-      "metadataPath", "chunksPath", "containerDBType");
+      "metadataPath", "chunksPath", "containerDBType", "maxSizeGB");
 
   // Path to Container metadata Level DB/RocksDB Store and .container file.
   private String metadataPath;
@@ -60,9 +60,10 @@ public class KeyValueContainerData extends ContainerData {
   /**
    * Constructs KeyValueContainerData object.
    * @param id - ContainerId
+   * @param size - maximum size of the container
    */
-  public KeyValueContainerData(long id) {
-    super(ContainerProtos.ContainerType.KeyValueContainer, id);
+  public KeyValueContainerData(long id, int size) {
+    super(ContainerProtos.ContainerType.KeyValueContainer, id, size);
     this.numPendingDeletionBlocks = 0;
   }
 
@@ -70,10 +71,11 @@ public class KeyValueContainerData extends ContainerData {
    * Constructs KeyValueContainerData object.
    * @param id - ContainerId
    * @param layOutVersion
+   * @param size - maximum size of the container
    */
-  public KeyValueContainerData(long id,
-                               int layOutVersion) {
-    super(ContainerProtos.ContainerType.KeyValueContainer, id, layOutVersion);
+  public KeyValueContainerData(long id, int layOutVersion, int size) {
+    super(ContainerProtos.ContainerType.KeyValueContainer, id, layOutVersion,
+        size);
     this.numPendingDeletionBlocks = 0;
   }
 
@@ -205,4 +207,8 @@ public class KeyValueContainerData extends ContainerData {
 
     return builder.build();
   }
+
+  public static List<String> getYamlFields() {
+    return YAML_FIELDS;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index d174383..b615acd 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .PutSmallFileRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Type;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
@@ -104,6 +105,7 @@ public class KeyValueHandler extends Handler {
   private final KeyManager keyManager;
   private final ChunkManager chunkManager;
   private VolumeChoosingPolicy volumeChoosingPolicy;
+  private final int maxContainerSizeGB;
 
   // TODO : Add metrics and populate it.
 
@@ -125,6 +127,8 @@ public class KeyValueHandler extends Handler {
     chunkManager = new ChunkManagerImpl();
     // TODO: Add supoort for different volumeChoosingPolicies.
     volumeChoosingPolicy = new RoundRobinVolumeChoosingPolicy();
+    maxContainerSizeGB = config.getInt(ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
   }
 
   @Override
@@ -207,7 +211,7 @@ public class KeyValueHandler extends Handler {
     }
 
     KeyValueContainerData newContainerData = new KeyValueContainerData(
-        containerID);
+        containerID, maxContainerSizeGB);
     // TODO: Add support to add metadataList to ContainerData. Add metadata
     // to container during creation.
     KeyValueContainer newContainer = new KeyValueContainer(
@@ -565,8 +569,8 @@ public class KeyValueHandler extends Handler {
     try {
       checkContainerOpen(kvContainer);
 
-      BlockID blockID = BlockID.getFromProtobuf(
-        putSmallFileReq.getKey().getKeyData().getBlockID());
+      BlockID blockID = BlockID.getFromProtobuf(putSmallFileReq.getKey()
+          .getKeyData().getBlockID());
       KeyData keyData = KeyData.getFromProtoBuf(
           putSmallFileReq.getKey().getKeyData());
       Preconditions.checkNotNull(keyData);
@@ -613,8 +617,8 @@ public class KeyValueHandler extends Handler {
     GetSmallFileRequestProto getSmallFileReq = request.getGetSmallFile();
 
     try {
-      BlockID blockID = BlockID.getFromProtobuf(
-        getSmallFileReq.getKey().getBlockID());
+      BlockID blockID = BlockID.getFromProtobuf(getSmallFileReq.getKey()
+          .getBlockID());
       KeyData responseData = keyManager.getKey(kvContainer, blockID);
 
       ContainerProtos.ChunkInfo chunkInfo = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
index 029e94d..4c17dce 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
@@ -302,6 +302,7 @@ public final class KeyValueContainerUtil {
       }
     }).sum();
     containerData.setBytesUsed(bytesUsed);
+    containerData.setKeyCount(liveKeys.size());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
index 40736e5..6a8897a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
@@ -84,6 +84,9 @@ public class KeyManagerImpl implements KeyManager {
     Preconditions.checkNotNull(db, "DB cannot be null here");
     db.put(Longs.toByteArray(data.getLocalID()), data.getProtoBufMessage()
         .toByteArray());
+
+    // Increment keycount here
+    container.getContainerData().incrKeyCount();
   }
 
   /**
@@ -148,6 +151,9 @@ public class KeyManagerImpl implements KeyManager {
           NO_SUCH_KEY);
     }
     db.delete(kKey);
+
+    // Decrement keycount here
+    container.getContainerData().decrKeyCount();
   }
 
   /**

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
index 68823bc..50a2d08 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
@@ -109,11 +109,9 @@ public class ContainerReader implements Runnable {
                     .getContainerFile(metadataPath, containerName);
                 File checksumFile = KeyValueContainerLocationUtil
                     .getContainerCheckSumFile(metadataPath, containerName);
-                File dbFile = KeyValueContainerLocationUtil
-                    .getContainerDBFile(metadataPath, containerName);
-                if (containerFile.exists() && checksumFile.exists() &&
-                    dbFile.exists()) {
-                  verifyContainerFile(containerFile, checksumFile, dbFile);
+                if (containerFile.exists() && checksumFile.exists()) {
+                  verifyContainerFile(containerName, containerFile,
+                      checksumFile);
                 } else {
                   LOG.error("Missing container metadata files for Container: " +
                       "{}", containerName);
@@ -129,8 +127,8 @@ public class ContainerReader implements Runnable {
     }
   }
 
-  private void verifyContainerFile(File containerFile, File checksumFile,
-                                   File dbFile) {
+  private void verifyContainerFile(String containerName, File containerFile,
+                                   File checksumFile) {
     try {
       ContainerData containerData =  ContainerDataYaml.readContainerFile(
           containerFile);
@@ -139,6 +137,15 @@ public class ContainerReader implements Runnable {
       case KeyValueContainer:
         KeyValueContainerData keyValueContainerData = (KeyValueContainerData)
             containerData;
+        File dbFile = KeyValueContainerLocationUtil
+            .getContainerDBFile(new File(containerFile.getParent()),
+                containerName);
+        if (!dbFile.exists()) {
+          LOG.error("Container DB file is missing for Container {}, skipping " +
+                  "this", containerName);
+          // Don't further process this container, as it is missing db file.
+          return;
+        }
         KeyValueContainerUtil.parseKeyValueContainerData(keyValueContainerData,
             containerFile, checksumFile, dbFile, config);
         KeyValueContainer keyValueContainer = new KeyValueContainer(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
index 249b0fe..16c4c2a 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
@@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicLong;
  */
 public class TestKeyValueContainerData {
 
+  private static final int MAXSIZE = 5;
   @Test
   public void testKeyValueData() {
     long containerId = 1L;
@@ -42,7 +43,8 @@ public class TestKeyValueContainerData {
         .ContainerLifeCycleState.CLOSED;
     AtomicLong val = new AtomicLong(0);
 
-    KeyValueContainerData kvData = new KeyValueContainerData(containerId);
+    KeyValueContainerData kvData = new KeyValueContainerData(containerId,
+        MAXSIZE);
 
     assertEquals(containerType, kvData.getContainerType());
     assertEquals(containerId, kvData.getContainerId());
@@ -54,6 +56,8 @@ public class TestKeyValueContainerData {
     assertEquals(val.get(), kvData.getWriteBytes());
     assertEquals(val.get(), kvData.getReadCount());
     assertEquals(val.get(), kvData.getWriteCount());
+    assertEquals(val.get(), kvData.getKeyCount());
+    assertEquals(MAXSIZE, kvData.getMaxSizeGB());
 
     kvData.setState(state);
     kvData.setContainerDBType(containerDBType);
@@ -63,6 +67,7 @@ public class TestKeyValueContainerData {
     kvData.incrWriteBytes(10);
     kvData.incrReadCount();
     kvData.incrWriteCount();
+    kvData.incrKeyCount();
 
     assertEquals(state, kvData.getState());
     assertEquals(containerDBType, kvData.getContainerDBType());
@@ -73,6 +78,7 @@ public class TestKeyValueContainerData {
     assertEquals(10, kvData.getWriteBytes());
     assertEquals(1, kvData.getReadCount());
     assertEquals(1, kvData.getWriteCount());
+    assertEquals(1, kvData.getKeyCount());
 
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
index e1b7bd2..41d8315 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
@@ -37,6 +37,7 @@ import static org.junit.Assert.fail;
  */
 public class TestContainerDataYaml {
 
+  private static final int MAXSIZE = 5;
   @Test
   public void testCreateContainerFile() throws IOException {
     String path = new FileSystemTestHelper().getTestRootDir();
@@ -45,7 +46,8 @@ public class TestContainerDataYaml {
     File filePath = new File(new FileSystemTestHelper().getTestRootDir());
     filePath.mkdirs();
 
-    KeyValueContainerData keyValueContainerData = new KeyValueContainerData(Long.MAX_VALUE);
+    KeyValueContainerData keyValueContainerData = new KeyValueContainerData(
+        Long.MAX_VALUE, MAXSIZE);
     keyValueContainerData.setContainerDBType("RocksDB");
     keyValueContainerData.setMetadataPath(path);
     keyValueContainerData.setChunksPath(path);
@@ -72,6 +74,7 @@ public class TestContainerDataYaml {
         .getState());
     assertEquals(1, kvData.getLayOutVersion());
     assertEquals(0, kvData.getMetadata().size());
+    assertEquals(MAXSIZE, kvData.getMaxSizeGB());
 
     // Update ContainerData.
     kvData.addMetadata("VOLUME", "hdfs");
@@ -101,6 +104,7 @@ public class TestContainerDataYaml {
     assertEquals(2, kvData.getMetadata().size());
     assertEquals("hdfs", kvData.getMetadata().get("VOLUME"));
     assertEquals("ozone", kvData.getMetadata().get("OWNER"));
+    assertEquals(MAXSIZE, kvData.getMaxSizeGB());
 
     FileUtil.fullyDelete(filePath);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
index 55d6773..6ec1fe4 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
@@ -53,7 +53,7 @@ public class TestContainerSet {
     ContainerProtos.ContainerLifeCycleState state = ContainerProtos
         .ContainerLifeCycleState.CLOSED;
 
-    KeyValueContainerData kvData = new KeyValueContainerData(containerId);
+    KeyValueContainerData kvData = new KeyValueContainerData(containerId, 5);
     kvData.setState(state);
     KeyValueContainer keyValueContainer = new KeyValueContainer(kvData, new
         OzoneConfiguration());
@@ -163,7 +163,7 @@ public class TestContainerSet {
   private ContainerSet createContainerSet() throws StorageContainerException {
     ContainerSet containerSet = new ContainerSet();
     for (int i=0; i<10; i++) {
-      KeyValueContainerData kvData = new KeyValueContainerData(i);
+      KeyValueContainerData kvData = new KeyValueContainerData(i, 5);
       if (i%2 == 0) {
         kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED);
       } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
index 6becf39..760d873 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
@@ -79,7 +79,7 @@ public class TestChunkManagerImpl {
     Mockito.when(volumeChoosingPolicy.chooseVolume(anyList(), anyLong()))
         .thenReturn(hddsVolume);
 
-    keyValueContainerData = new KeyValueContainerData(1L);
+    keyValueContainerData = new KeyValueContainerData(1L, 5);
 
     keyValueContainer = new KeyValueContainer(keyValueContainerData, config);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
index 62d9382..a90cf80 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
@@ -79,7 +79,7 @@ public class TestKeyManagerImpl {
     Mockito.when(volumeChoosingPolicy.chooseVolume(anyList(), anyLong()))
         .thenReturn(hddsVolume);
 
-    keyValueContainerData = new KeyValueContainerData(1L);
+    keyValueContainerData = new KeyValueContainerData(1L, 5);
 
     keyValueContainer = new KeyValueContainer(
         keyValueContainerData, config);
@@ -104,9 +104,11 @@ public class TestKeyManagerImpl {
 
   @Test
   public void testPutAndGetKey() throws Exception {
+    assertEquals(0, keyValueContainer.getContainerData().getKeyCount());
     //Put Key
     keyManager.putKey(keyValueContainer, keyData);
 
+    assertEquals(1, keyValueContainer.getContainerData().getKeyCount());
     //Get Key
     KeyData fromGetKeyData = keyManager.getKey(keyValueContainer,
         keyData.getBlockID());
@@ -123,10 +125,13 @@ public class TestKeyManagerImpl {
   @Test
   public void testDeleteKey() throws Exception {
     try {
+      assertEquals(0, keyValueContainer.getContainerData().getKeyCount());
       //Put Key
       keyManager.putKey(keyValueContainer, keyData);
+      assertEquals(1, keyValueContainer.getContainerData().getKeyCount());
       //Delete Key
       keyManager.deleteKey(keyValueContainer, blockID);
+      assertEquals(0, keyValueContainer.getContainerData().getKeyCount());
       try {
         keyManager.getKey(keyValueContainer, blockID);
         fail("testDeleteKey");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
index de5f432..e55ea57 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
@@ -86,7 +86,7 @@ public class TestKeyValueContainer {
     Mockito.when(volumeChoosingPolicy.chooseVolume(anyList(), anyLong()))
         .thenReturn(hddsVolume);
 
-    keyValueContainerData = new KeyValueContainerData(1L);
+    keyValueContainerData = new KeyValueContainerData(1L, 5);
 
     keyValueContainer = new KeyValueContainer(
         keyValueContainerData, conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
index cf4bb62..26e1c77 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
@@ -66,7 +66,7 @@ public class TestOzoneContainer {
     volumeChoosingPolicy = new RoundRobinVolumeChoosingPolicy();
 
     for (int i=0; i<10; i++) {
-      keyValueContainerData = new KeyValueContainerData(i);
+      keyValueContainerData = new KeyValueContainerData(i, 1);
       keyValueContainer = new KeyValueContainer(
           keyValueContainerData, conf);
       keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/test/resources/additionalfields.container
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/resources/additionalfields.container b/hadoop-hdds/container-service/src/test/resources/additionalfields.container
index b19ec44..9027538 100644
--- a/hadoop-hdds/container-service/src/test/resources/additionalfields.container
+++ b/hadoop-hdds/container-service/src/test/resources/additionalfields.container
@@ -5,6 +5,7 @@ containerId: 9223372036854775807
 containerType: KeyValueContainer
 metadataPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
 layOutVersion: 1
+maxSizeGB: 5
 metadata: {OWNER: ozone, VOLUME: hdfs}
 state: CLOSED
 aclEnabled: true
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1f4b3b5/hadoop-hdds/container-service/src/test/resources/incorrect.container
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/resources/incorrect.container b/hadoop-hdds/container-service/src/test/resources/incorrect.container
index 0d2bfd1..6848484 100644
--- a/hadoop-hdds/container-service/src/test/resources/incorrect.container
+++ b/hadoop-hdds/container-service/src/test/resources/incorrect.container
@@ -5,5 +5,6 @@ containerId: 9223372036854775807
 containerType: KeyValueContainer
 metadataPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
 layOutVersion: 1
+maxSizeGB: 5
 metadata: {OWNER: ozone, VOLUME: hdfs}
 state: INVALID
\ No newline at end of file


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


[14/37] hadoop git commit: HDDS-155:Implement KeyValueContainer and adopt new disk layout for the containers. Contributed by Bharat Viswanadham

Posted by bh...@apache.org.
HDDS-155:Implement KeyValueContainer and adopt new disk layout for the containers. Contributed by Bharat Viswanadham


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

Branch: refs/heads/trunk
Commit: 998e2850a3ceb961d66b9d3398a1afaad63a5cd8
Parents: 9a5552b
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Thu Jun 14 20:54:54 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Thu Jun 14 20:54:54 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/ozone/OzoneConsts.java    |   3 +
 .../org/apache/hadoop/ozone/common/Storage.java |   3 +-
 .../main/proto/DatanodeContainerProtocol.proto  |   5 +
 .../common/impl/KeyValueContainer.java          |  78 ---
 .../common/impl/KeyValueContainerData.java      |  21 +
 .../container/common/interfaces/Container.java  |  17 +-
 .../container/keyvalue/KeyValueContainer.java   | 544 +++++++++++++++++++
 .../keyvalue/KeyValueContainerLocationUtil.java | 140 +++++
 .../keyvalue/KeyValueContainerUtil.java         | 148 +++++
 .../container/keyvalue/helpers/KeyUtils.java    |  82 +++
 .../keyvalue/helpers/package-info.java          |  21 +
 .../ozone/container/keyvalue/package-info.java  |  21 +
 .../container/common/impl/TestContainerSet.java |   8 +-
 .../keyvalue/TestKeyValueContainer.java         | 281 ++++++++++
 14 files changed, 1285 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/998e2850/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index 36f830b..3b774a5 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -101,6 +101,9 @@ public final class OzoneConsts {
   public static final String DELETED_BLOCK_DB = "deletedBlock.db";
   public static final String KSM_DB_NAME = "ksm.db";
 
+  public static final String STORAGE_DIR_CHUNKS = "chunks";
+  public static final String CONTAINER_FILE_CHECKSUM_EXTENSION = ".chksm";
+
   /**
    * Supports Bucket Versioning.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/998e2850/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
index fb30d92..2ff4626 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
@@ -45,8 +45,9 @@ import java.util.Properties;
 public abstract class Storage {
   private static final Logger LOG = LoggerFactory.getLogger(Storage.class);
 
-  protected static final String STORAGE_DIR_CURRENT = "current";
+  public static final String STORAGE_DIR_CURRENT = "current";
   protected static final String STORAGE_FILE_VERSION = "VERSION";
+  public static final String CONTAINER_DIR = "containerdir";
 
   private final NodeType nodeType;
   private final File root;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/998e2850/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
index 72e1006..88645be 100644
--- a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
+++ b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
@@ -132,6 +132,11 @@ enum Result {
   DELETE_ON_OPEN_CONTAINER = 26;
   CLOSED_CONTAINER_RETRY = 27;
   INVALID_CONTAINER_STATE = 28;
+  DISK_OUT_OF_SPACE = 29;
+  CONTAINER_ALREADY_EXISTS = 30;
+  CONTAINER_METADATA_ERROR = 31;
+  CONTAINER_FILES_CREATE_ERROR = 32;
+  CONTAINER_CHECKSUM_ERROR = 33;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/998e2850/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java
deleted file mode 100644
index a35845d..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.impl;
-
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
-
-
-import org.apache.hadoop.ozone.container.common.interfaces.Container;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.security.NoSuchAlgorithmException;
-
-
-/**
- * Class to perform KeyValue Container operations.
- */
-public class KeyValueContainer implements Container {
-
-  static final Logger LOG =
-      LoggerFactory.getLogger(Container.class);
-
-  private KeyValueContainerData containerData;
-
-  public KeyValueContainer(KeyValueContainerData containerData) {
-    Preconditions.checkNotNull(containerData, "KeyValueContainerData cannot " +
-        "be null");
-    this.containerData = containerData;
-  }
-
-  @Override
-  public void create(ContainerData cData) throws StorageContainerException {
-
-
-  }
-
-  @Override
-  public void delete(boolean forceDelete)
-      throws StorageContainerException {
-
-  }
-
-  @Override
-  public void update(boolean forceUpdate)
-      throws StorageContainerException {
-
-  }
-
-  @Override
-  public ContainerData getContainerData()  {
-    return containerData;
-  }
-
-  @Override
-  public void close() throws StorageContainerException,
-      NoSuchAlgorithmException {
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/998e2850/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
index 0889913..b74bab2 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.ozone.container.common.impl;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 
 
+import java.io.File;
 import java.io.IOException;
 
 /**
@@ -42,6 +43,8 @@ public class KeyValueContainerData extends ContainerData {
   //Number of pending deletion blocks in container.
   private int numPendingDeletionBlocks;
 
+  private File dbFile = null;
+
   /**
    * Constructs KeyValueContainerData object.
    * @param type - containerType
@@ -63,6 +66,24 @@ public class KeyValueContainerData extends ContainerData {
     super(type, id, layOutVersion);
     this.numPendingDeletionBlocks = 0;
   }
+
+
+  /**
+   * Sets Container dbFile. This should be called only during creation of
+   * KeyValue container.
+   * @param containerDbFile
+   */
+  public void setDbFile(File containerDbFile) {
+    dbFile = containerDbFile;
+  }
+
+  /**
+   * Returns container DB file.
+   * @return dbFile
+   */
+  public File getDbFile() {
+    return dbFile;
+  }
   /**
    * Returns container metadata path.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/998e2850/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
index a680e6a..3b7e332 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
@@ -21,21 +21,26 @@ package org.apache.hadoop.ozone.container.common.interfaces;
 
 import org.apache.hadoop.hdds.scm.container.common.helpers.
     StorageContainerException;
+
+import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+
+import java.util.Map;
 
-import java.security.NoSuchAlgorithmException;
 
 /**
  * Interface for Container Operations.
  */
-public interface Container {
+public interface Container extends RwLock {
 
   /**
    * Creates a container.
    *
    * @throws StorageContainerException
    */
-  void create(ContainerData containerData) throws StorageContainerException;
+  void create(VolumeSet volumeSet, VolumeChoosingPolicy volumeChoosingPolicy,
+              String scmId) throws StorageContainerException;
 
   /**
    * Deletes the container.
@@ -48,10 +53,11 @@ public interface Container {
   /**
    * Update the container.
    *
+   * @param metaData
    * @param forceUpdate if true, update container forcibly.
    * @throws StorageContainerException
    */
-  void update(boolean forceUpdate)
+  void update(Map<String, String> metaData, boolean forceUpdate)
       throws StorageContainerException;
 
   /**
@@ -68,8 +74,7 @@ public interface Container {
    *
    * @throws StorageContainerException
    */
-  void close() throws StorageContainerException,
-      NoSuchAlgorithmException;
+  void close() throws StorageContainerException;
 
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/998e2850/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
new file mode 100644
index 0000000..740967b
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -0,0 +1,544 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue;
+
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+
+
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.common.impl.KeyValueYaml;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
+import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
+import org.apache.hadoop.utils.MetadataStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.CONTAINER_ALREADY_EXISTS;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.CONTAINER_CHECKSUM_ERROR;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.CONTAINER_METADATA_ERROR;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.CONTAINER_INTERNAL_ERROR;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.CONTAINER_FILES_CREATE_ERROR;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.DISK_OUT_OF_SPACE;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.ERROR_IN_COMPACT_DB;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.INVALID_CONTAINER_STATE;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.NO_SUCH_ALGORITHM;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.UNSUPPORTED_REQUEST;
+
+/**
+ * Class to perform KeyValue Container operations.
+ */
+public class KeyValueContainer implements Container {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Container.class);
+
+  // Use a non-fair RW lock for better throughput, we may revisit this decision
+  // if this causes fairness issues.
+  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+  private final KeyValueContainerData containerData;
+  private long containerMaxSize;
+  private Configuration config;
+
+  public KeyValueContainer(KeyValueContainerData containerData, Configuration
+      ozoneConfig) {
+    Preconditions.checkNotNull(containerData, "KeyValueContainerData cannot " +
+        "be null");
+    Preconditions.checkNotNull(ozoneConfig, "Ozone configuration cannot " +
+        "be null");
+    this.config = ozoneConfig;
+    this.containerData = containerData;
+    this.containerMaxSize = (long) ozoneConfig.getInt(ScmConfigKeys
+        .OZONE_SCM_CONTAINER_SIZE_GB, ScmConfigKeys
+        .OZONE_SCM_CONTAINER_SIZE_DEFAULT) * 1024L * 1024L * 1024L;
+  }
+
+  @Override
+  public void create(VolumeSet volumeSet, VolumeChoosingPolicy
+      volumeChoosingPolicy, String scmId) throws StorageContainerException {
+    Preconditions.checkNotNull(volumeChoosingPolicy, "VolumeChoosingPolicy " +
+        "cannot be null");
+    Preconditions.checkNotNull(volumeSet, "VolumeSet cannot be null");
+    Preconditions.checkNotNull(scmId, "scmId cannot be null");
+
+    File containerMetaDataPath = null;
+    try {
+      //acquiring volumeset lock and container lock
+      volumeSet.acquireLock();
+      HddsVolume containerVolume = volumeChoosingPolicy.chooseVolume(volumeSet
+          .getVolumesList(), containerMaxSize);
+      String containerBasePath = containerVolume.getHddsRootDir().toString();
+
+      long containerId = containerData.getContainerId();
+      String containerName = Long.toString(containerId);
+
+      containerMetaDataPath = KeyValueContainerLocationUtil
+          .getContainerMetaDataPath(containerBasePath, scmId, containerId);
+      File chunksPath = KeyValueContainerLocationUtil.getChunksLocationPath(
+          containerBasePath, scmId, containerId);
+      File containerFile = KeyValueContainerLocationUtil.getContainerFile(
+          containerMetaDataPath, containerName);
+      File containerCheckSumFile = KeyValueContainerLocationUtil
+          .getContainerCheckSumFile(containerMetaDataPath, containerName);
+      File dbFile = KeyValueContainerLocationUtil.getContainerDBFile(
+          containerMetaDataPath, containerName);
+
+      // Check if it is new Container.
+      KeyValueContainerUtil.verifyIsNewContainer(containerMetaDataPath);
+
+      //Create Metadata path chunks path and metadata db
+      KeyValueContainerUtil.createContainerMetaData(containerMetaDataPath,
+          chunksPath, dbFile, containerName, config);
+
+      String impl = config.getTrimmed(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL,
+          OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_DEFAULT);
+
+      //Set containerData for the KeyValueContainer.
+      containerData.setMetadataPath(containerMetaDataPath.getPath());
+      containerData.setChunksPath(chunksPath.getPath());
+      containerData.setContainerDBType(impl);
+      containerData.setDbFile(dbFile);
+
+      // Create .container file and .chksm file
+      createContainerFile(containerFile, containerCheckSumFile);
+
+
+    } catch (StorageContainerException ex) {
+      if (containerMetaDataPath != null && containerMetaDataPath.getParentFile()
+          .exists()) {
+        FileUtil.fullyDelete(containerMetaDataPath.getParentFile());
+      }
+      throw ex;
+    } catch (DiskOutOfSpaceException ex) {
+      throw new StorageContainerException("Container creation failed, due to " +
+          "disk out of space", ex, DISK_OUT_OF_SPACE);
+    } catch (FileAlreadyExistsException ex) {
+      throw new StorageContainerException("Container creation failed because " +
+          "ContainerFile already exists", ex, CONTAINER_ALREADY_EXISTS);
+    } catch (IOException ex) {
+      if (containerMetaDataPath != null && containerMetaDataPath.getParentFile()
+          .exists()) {
+        FileUtil.fullyDelete(containerMetaDataPath.getParentFile());
+      }
+      throw new StorageContainerException("Container creation failed.", ex,
+          CONTAINER_INTERNAL_ERROR);
+    } finally {
+      volumeSet.releaseLock();
+    }
+  }
+
+  /**
+   * Creates .container file and checksum file.
+   *
+   * @param containerFile
+   * @param containerCheckSumFile
+   * @throws StorageContainerException
+   */
+  private void createContainerFile(File containerFile, File
+      containerCheckSumFile) throws StorageContainerException {
+    File tempContainerFile = null;
+    File tempCheckSumFile = null;
+    FileOutputStream containerCheckSumStream = null;
+    Writer writer = null;
+    long containerId = containerData.getContainerId();
+    try {
+      tempContainerFile = createTempFile(containerFile);
+      tempCheckSumFile = createTempFile(containerCheckSumFile);
+      KeyValueYaml.createContainerFile(tempContainerFile, containerData);
+
+      //Compute Checksum for container file
+      String checksum = computeCheckSum(tempContainerFile);
+      containerCheckSumStream = new FileOutputStream(tempCheckSumFile);
+      writer = new OutputStreamWriter(containerCheckSumStream, "UTF-8");
+      writer.write(checksum);
+      writer.flush();
+
+      NativeIO.renameTo(tempContainerFile, containerFile);
+      NativeIO.renameTo(tempCheckSumFile, containerCheckSumFile);
+
+    } catch (IOException ex) {
+      throw new StorageContainerException("Error during creation of " +
+          "required files(.container, .chksm) for container. Container Name: "
+          + containerId, ex, CONTAINER_FILES_CREATE_ERROR);
+    } finally {
+      IOUtils.closeStream(containerCheckSumStream);
+      if (tempContainerFile != null && tempContainerFile.exists()) {
+        if (!tempContainerFile.delete()) {
+          LOG.warn("Unable to delete container temporary file: {}.",
+              tempContainerFile.getAbsolutePath());
+        }
+      }
+      if (tempCheckSumFile != null && tempCheckSumFile.exists()) {
+        if (!tempCheckSumFile.delete()) {
+          LOG.warn("Unable to delete container temporary checksum file: {}.",
+              tempContainerFile.getAbsolutePath());
+        }
+      }
+      try {
+        if (writer != null) {
+          writer.close();
+        }
+      } catch (IOException ex) {
+        LOG.warn("Error occurred during closing the writer.  Container " +
+            "Name:" + containerId);
+      }
+
+    }
+  }
+
+
+  private void updateContainerFile(File containerFile, File
+      containerCheckSumFile) throws StorageContainerException {
+
+    File containerBkpFile = null;
+    File checkSumBkpFile = null;
+    long containerId = containerData.getContainerId();
+
+    try {
+      if (containerFile.exists() && containerCheckSumFile.exists()) {
+        //Take backup of original files (.container and .chksm files)
+        containerBkpFile = new File(containerFile + ".bkp");
+        checkSumBkpFile = new File(containerCheckSumFile + ".bkp");
+        NativeIO.renameTo(containerFile, containerBkpFile);
+        NativeIO.renameTo(containerCheckSumFile, checkSumBkpFile);
+        createContainerFile(containerFile, containerCheckSumFile);
+      } else {
+        containerData.setState(ContainerProtos.ContainerLifeCycleState.INVALID);
+        throw new StorageContainerException("Container is an Inconsistent " +
+            "state, missing required files(.container, .chksm)",
+            INVALID_CONTAINER_STATE);
+      }
+    } catch (StorageContainerException ex) {
+      throw ex;
+    } catch (IOException ex) {
+      // Restore from back up files.
+      try {
+        if (containerBkpFile != null && containerBkpFile
+            .exists() && containerFile.delete()) {
+          LOG.info("update failed for container Name: {}, restoring container" +
+              " file", containerId);
+          NativeIO.renameTo(containerBkpFile, containerFile);
+        }
+        if (checkSumBkpFile != null && checkSumBkpFile.exists() &&
+            containerCheckSumFile.delete()) {
+          LOG.info("update failed for container Name: {}, restoring checksum" +
+              " file", containerId);
+          NativeIO.renameTo(checkSumBkpFile, containerCheckSumFile);
+        }
+        throw new StorageContainerException("Error during updating of " +
+            "required files(.container, .chksm) for container. Container Name: "
+            + containerId, ex, CONTAINER_FILES_CREATE_ERROR);
+      } catch (IOException e) {
+        containerData.setState(ContainerProtos.ContainerLifeCycleState.INVALID);
+        LOG.error("During restore failed for container Name: " +
+            containerId);
+        throw new StorageContainerException(
+            "Failed to restore container data from the backup. ID: "
+                + containerId, CONTAINER_FILES_CREATE_ERROR);
+      }
+    } finally {
+      if (containerBkpFile != null && containerBkpFile
+          .exists()) {
+        if(!containerBkpFile.delete()) {
+          LOG.warn("Unable to delete container backup file: {}",
+              containerBkpFile);
+        }
+      }
+      if (checkSumBkpFile != null && checkSumBkpFile.exists()) {
+        if(!checkSumBkpFile.delete()) {
+          LOG.warn("Unable to delete container checksum backup file: {}",
+              checkSumBkpFile);
+        }
+      }
+    }
+  }
+
+
+  /**
+   * Compute checksum of the .container file.
+   * @param containerFile
+   * @throws StorageContainerException
+   */
+  private String computeCheckSum(File containerFile) throws
+      StorageContainerException {
+
+    MessageDigest sha;
+    FileInputStream containerFileStream = null;
+    try {
+      sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
+    } catch (NoSuchAlgorithmException e) {
+      throw new StorageContainerException("Unable to create Message Digest,"
+          + " usually this is a java configuration issue.",
+          NO_SUCH_ALGORITHM);
+    }
+
+    try {
+      containerFileStream = new FileInputStream(containerFile);
+      byte[] byteArray = new byte[1024];
+      int bytesCount = 0;
+
+      while ((bytesCount = containerFileStream.read(byteArray)) != -1) {
+        sha.update(byteArray, 0, bytesCount);
+      }
+      String checksum = DigestUtils.sha256Hex(sha.digest());
+      return checksum;
+    } catch (IOException ex) {
+      throw new StorageContainerException("Error during update of " +
+          "check sum file. Container Name: " + containerData.getContainerId(),
+          ex, CONTAINER_CHECKSUM_ERROR);
+    } finally {
+      IOUtils.closeStream(containerFileStream);
+    }
+  }
+
+  @Override
+  public void delete(boolean forceDelete)
+      throws StorageContainerException {
+    long containerId = containerData.getContainerId();
+    try {
+      KeyValueContainerUtil.removeContainer(containerData, config, forceDelete);
+    } catch (StorageContainerException ex) {
+      throw ex;
+    } catch (IOException ex) {
+      // TODO : An I/O error during delete can leave partial artifacts on the
+      // disk. We will need the cleaner thread to cleanup this information.
+      String errMsg = String.format("Failed to cleanup container. ID: %d",
+          containerId);
+      LOG.error(errMsg, ex);
+      throw new StorageContainerException(errMsg, ex, CONTAINER_INTERNAL_ERROR);
+    }
+  }
+
+  @Override
+  public void close() throws StorageContainerException {
+
+    //TODO: writing .container file and compaction can be done
+    // asynchronously, otherwise rpc call for this will take a lot of time to
+    // complete this action
+    try {
+      writeLock();
+      long containerId = containerData.getContainerId();
+      if(!containerData.isValid()) {
+        LOG.debug("Invalid container data. Container Id: {}", containerId);
+        throw new StorageContainerException("Invalid container data. Name : " +
+            containerId, INVALID_CONTAINER_STATE);
+      }
+      containerData.closeContainer();
+      File containerFile = getContainerFile();
+      File containerCheckSumFile = getContainerCheckSumFile();
+
+      // update the new container data to .container File
+      updateContainerFile(containerFile, containerCheckSumFile);
+
+    } catch (StorageContainerException ex) {
+      throw ex;
+    } finally {
+      writeUnlock();
+    }
+
+    // It is ok if this operation takes a bit of time.
+    // Close container is not expected to be instantaneous.
+    try {
+      MetadataStore db = KeyUtils.getDB(containerData, config);
+      db.compactDB();
+    } catch (StorageContainerException ex) {
+      throw ex;
+    } catch (IOException ex) {
+      LOG.error("Error in DB compaction while closing container", ex);
+      throw new StorageContainerException(ex, ERROR_IN_COMPACT_DB);
+    }
+  }
+
+  @Override
+  public ContainerData getContainerData()  {
+    return containerData;
+  }
+
+
+  @Override
+  public void update(Map<String, String> metadata, boolean forceUpdate)
+      throws StorageContainerException {
+
+    // TODO: Now, when writing the updated data to .container file, we are
+    // holding lock and writing data to disk. We can have async implementation
+    // to flush the update container data to disk.
+    long containerId = containerData.getContainerId();
+    if(!containerData.isValid()) {
+      LOG.debug("Invalid container data. ID: {}", containerId);
+      throw new StorageContainerException("Invalid container data. " +
+          "Container Name : " + containerId, INVALID_CONTAINER_STATE);
+    }
+    if (!forceUpdate && !containerData.isOpen()) {
+      throw new StorageContainerException(
+          "Updating a closed container is not allowed. ID: " + containerId,
+          UNSUPPORTED_REQUEST);
+    }
+    try {
+      for (Map.Entry<String, String> entry : metadata.entrySet()) {
+        containerData.addMetadata(entry.getKey(), entry.getValue());
+      }
+    } catch (IOException ex) {
+      throw new StorageContainerException("Container Metadata update error" +
+          ". Container Name:" + containerId, ex, CONTAINER_METADATA_ERROR);
+    }
+    try {
+      writeLock();
+      String containerName = String.valueOf(containerId);
+      File containerFile = getContainerFile();
+      File containerCheckSumFile = getContainerCheckSumFile();
+      // update the new container data to .container File
+      updateContainerFile(containerFile, containerCheckSumFile);
+    } catch (StorageContainerException  ex) {
+      throw ex;
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  /**
+   * Acquire read lock.
+   */
+  public void readLock() {
+    this.lock.readLock().lock();
+
+  }
+
+  /**
+   * Release read lock.
+   */
+  public void readUnlock() {
+    this.lock.readLock().unlock();
+  }
+
+  /**
+   * Check if the current thread holds read lock.
+   */
+  public boolean hasReadLock() {
+    return this.lock.readLock().tryLock();
+  }
+
+  /**
+   * Acquire write lock.
+   */
+  public void writeLock() {
+    this.lock.writeLock().lock();
+  }
+
+  /**
+   * Release write lock.
+   */
+  public void writeUnlock() {
+    this.lock.writeLock().unlock();
+
+  }
+
+  /**
+   * Check if the current thread holds write lock.
+   */
+  public boolean hasWriteLock() {
+    return this.lock.writeLock().isHeldByCurrentThread();
+  }
+
+  /**
+   * Acquire read lock, unless interrupted while waiting.
+   * @throws InterruptedException
+   */
+  @Override
+  public void readLockInterruptibly() throws InterruptedException {
+    this.lock.readLock().lockInterruptibly();
+  }
+
+  /**
+   * Acquire write lock, unless interrupted while waiting.
+   * @throws InterruptedException
+   */
+  @Override
+  public void writeLockInterruptibly() throws InterruptedException {
+    this.lock.writeLock().lockInterruptibly();
+
+  }
+
+  /**
+   * Returns containerFile.
+   * @return .container File name
+   */
+  private File getContainerFile() {
+    return new File(containerData.getMetadataPath(), containerData
+        .getContainerId() + OzoneConsts.CONTAINER_EXTENSION);
+  }
+
+  /**
+   * Returns container checksum file.
+   * @return container checksum file
+   */
+  private File getContainerCheckSumFile() {
+    return new File(containerData.getMetadataPath(), containerData
+        .getContainerId() + OzoneConsts.CONTAINER_FILE_CHECKSUM_EXTENSION);
+  }
+
+  /**
+   * Creates a temporary file.
+   * @param file
+   * @return
+   * @throws IOException
+   */
+  private File createTempFile(File file) throws IOException{
+    return File.createTempFile("tmp_" + System.currentTimeMillis() + "_",
+        file.getName(), file.getParentFile());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/998e2850/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerLocationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerLocationUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerLocationUtil.java
new file mode 100644
index 0000000..2c15c94
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerLocationUtil.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.keyvalue;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.common.Storage;
+
+import java.io.File;
+
+/**
+ * Class which provides utility methods for container locations.
+ */
+public final class KeyValueContainerLocationUtil {
+
+  /* Never constructed. */
+  private KeyValueContainerLocationUtil() {
+
+  }
+  /**
+   * Returns Container Metadata Location.
+   * @param baseDir
+   * @param scmId
+   * @param containerId
+   * @return containerMetadata Path
+   */
+  public static File getContainerMetaDataPath(String baseDir, String scmId,
+                                              long containerId) {
+    String containerMetaDataPath = getBaseContainerLocation(baseDir, scmId,
+        containerId);
+    containerMetaDataPath = containerMetaDataPath + File.separator +
+        OzoneConsts.CONTAINER_META_PATH;
+    return new File(containerMetaDataPath);
+  }
+
+
+  /**
+   * Returns Container Chunks Location.
+   * @param baseDir
+   * @param scmId
+   * @param containerId
+   * @return chunksPath
+   */
+  public static File getChunksLocationPath(String baseDir, String scmId,
+                                           long containerId) {
+    String chunksPath = getBaseContainerLocation(baseDir, scmId, containerId)
+        + File.separator + OzoneConsts.STORAGE_DIR_CHUNKS;
+    return new File(chunksPath);
+  }
+
+  /**
+   * Returns base directory for specified container.
+   * @param baseDir
+   * @param scmId
+   * @param containerId
+   * @return base directory for container.
+   */
+  private static String getBaseContainerLocation(String baseDir, String scmId,
+                                        long containerId) {
+    Preconditions.checkNotNull(baseDir, "Base Directory cannot be null");
+    Preconditions.checkNotNull(scmId, "scmUuid cannot be null");
+    Preconditions.checkState(containerId >= 0,
+        "Container Id cannot be negative.");
+
+    String containerSubDirectory = getContainerSubDirectory(containerId);
+
+    String containerMetaDataPath = baseDir  + File.separator + scmId +
+        File.separator + Storage.STORAGE_DIR_CURRENT + File.separator +
+        containerSubDirectory + File.separator + containerId;
+
+    return containerMetaDataPath;
+  }
+
+  /**
+   * Returns subdirectory, where this container needs to be placed.
+   * @param containerId
+   * @return container sub directory
+   */
+  private static String getContainerSubDirectory(long containerId){
+    int directory = (int) ((containerId >> 9) & 0xFF);
+    return Storage.CONTAINER_DIR + directory;
+  }
+
+  /**
+   * Returns containerFile.
+   * @param containerMetaDataPath
+   * @param containerName
+   * @return .container File name
+   */
+  public static File getContainerFile(File containerMetaDataPath, String
+      containerName) {
+    Preconditions.checkNotNull(containerMetaDataPath);
+    Preconditions.checkNotNull(containerName);
+    return new File(containerMetaDataPath, containerName +
+        OzoneConsts.CONTAINER_EXTENSION);
+  }
+
+  /**
+   * Return containerDB File.
+   * @param containerMetaDataPath
+   * @param containerName
+   * @return containerDB File name
+   */
+  public static File getContainerDBFile(File containerMetaDataPath, String
+      containerName) {
+    Preconditions.checkNotNull(containerMetaDataPath);
+    Preconditions.checkNotNull(containerName);
+    return new File(containerMetaDataPath, containerName + OzoneConsts
+        .DN_CONTAINER_DB);
+  }
+
+  /**
+   * Returns container checksum file.
+   * @param containerMetaDataPath
+   * @param containerName
+   * @return container checksum file
+   */
+  public static File getContainerCheckSumFile(File containerMetaDataPath,
+                                              String containerName) {
+    Preconditions.checkNotNull(containerMetaDataPath);
+    Preconditions.checkNotNull(containerName);
+    return new File(containerMetaDataPath, containerName + OzoneConsts
+        .CONTAINER_FILE_CHECKSUM_EXTENSION);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/998e2850/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerUtil.java
new file mode 100644
index 0000000..55e2ab0
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerUtil.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.keyvalue;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
+import org.apache.hadoop.utils.MetadataStore;
+import org.apache.hadoop.utils.MetadataStoreBuilder;
+import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Class which defines utility methods for KeyValueContainer.
+ */
+
+public final class KeyValueContainerUtil {
+
+  /* Never constructed. */
+  private KeyValueContainerUtil() {
+
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      KeyValueContainerUtil.class);
+
+
+  public static void verifyIsNewContainer(File containerFile) throws
+      FileAlreadyExistsException {
+    Preconditions.checkNotNull(containerFile, "containerFile Should not be " +
+        "null");
+    if (containerFile.getParentFile().exists()) {
+      LOG.error("container already exists on disk. File: {}", containerFile
+          .toPath());
+      throw new FileAlreadyExistsException("container already exists on " +
+            "disk.");
+    }
+  }
+
+  /**
+   * creates metadata path, chunks path and  metadata DB for the specified
+   * container.
+   *
+   * @param containerMetaDataPath
+   * @throws IOException
+   */
+  public static void createContainerMetaData(File containerMetaDataPath, File
+      chunksPath, File dbFile, String containerName, Configuration conf) throws
+      IOException {
+    Preconditions.checkNotNull(containerMetaDataPath);
+    Preconditions.checkNotNull(containerName);
+    Preconditions.checkNotNull(conf);
+
+    if (!containerMetaDataPath.mkdirs()) {
+      LOG.error("Unable to create directory for metadata storage. Path: {}",
+          containerMetaDataPath);
+      throw new IOException("Unable to create directory for metadata storage." +
+          " Path: " + containerMetaDataPath);
+    }
+    MetadataStore store = MetadataStoreBuilder.newBuilder().setConf(conf)
+        .setCreateIfMissing(true).setDbFile(dbFile).build();
+
+    // we close since the SCM pre-creates containers.
+    // we will open and put Db handle into a cache when keys are being created
+    // in a container.
+
+    store.close();
+
+    if (!chunksPath.mkdirs()) {
+      LOG.error("Unable to create chunks directory Container {}",
+          chunksPath);
+      //clean up container metadata path and metadata db
+      FileUtils.deleteDirectory(containerMetaDataPath);
+      FileUtils.deleteDirectory(containerMetaDataPath.getParentFile());
+      throw new IOException("Unable to create directory for data storage." +
+          " Path: " + chunksPath);
+    }
+  }
+
+  /**
+   * remove Container if it is empty.
+   * <p/>
+   * There are three things we need to delete.
+   * <p/>
+   * 1. Container file and metadata file. 2. The Level DB file 3. The path that
+   * we created on the data location.
+   *
+   * @param containerData - Data of the container to remove.
+   * @param conf - configuration of the cluster.
+   * @param forceDelete - whether this container should be deleted forcibly.
+   * @throws IOException
+   */
+  public static void removeContainer(KeyValueContainerData containerData,
+                                     Configuration conf, boolean forceDelete)
+      throws IOException {
+    Preconditions.checkNotNull(containerData);
+    File containerMetaDataPath = new File(containerData
+        .getMetadataPath());
+    File chunksPath = new File(containerData.getChunksPath());
+
+    MetadataStore db = KeyUtils.getDB(containerData, conf);
+
+    // If the container is not empty and cannot be deleted forcibly,
+    // then throw a SCE to stop deleting.
+    if(!forceDelete && !db.isEmpty()) {
+      throw new StorageContainerException(
+          "Container cannot be deleted because it is not empty.",
+          ContainerProtos.Result.ERROR_CONTAINER_NOT_EMPTY);
+    }
+
+    // Close the DB connection and remove the DB handler from cache
+    KeyUtils.removeDB(containerData, conf);
+
+    // Delete the Container MetaData path.
+    FileUtils.deleteDirectory(containerMetaDataPath);
+
+    //Delete the Container Chunks Path.
+    FileUtils.deleteDirectory(chunksPath);
+
+    //Delete Container directory
+    FileUtils.deleteDirectory(containerMetaDataPath.getParentFile());
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/998e2850/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
new file mode 100644
index 0000000..7d9f0e6
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue.helpers;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
+import org.apache.hadoop.utils.MetadataStore;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNABLE_TO_READ_METADATA_DB;
+
+/**
+ * Utils functions to help key functions.
+ */
+public final class KeyUtils {
+
+  /** Never constructed. **/
+  private KeyUtils() {
+
+  }
+  /**
+   * Get a DB handler for a given container.
+   * If the handler doesn't exist in cache yet, first create one and
+   * add into cache. This function is called with containerManager
+   * ReadLock held.
+   *
+   * @param container container.
+   * @param conf configuration.
+   * @return MetadataStore handle.
+   * @throws StorageContainerException
+   */
+  public static MetadataStore getDB(KeyValueContainerData container,
+                                    Configuration conf) throws
+      StorageContainerException {
+    Preconditions.checkNotNull(container);
+    ContainerCache cache = ContainerCache.getInstance(conf);
+    Preconditions.checkNotNull(cache);
+    Preconditions.checkNotNull(container.getDbFile());
+    try {
+      return cache.getDB(container.getContainerId(), container
+          .getContainerDBType(), container.getDbFile().getAbsolutePath());
+    } catch (IOException ex) {
+      String message = String.format("Unable to open DB Path: " +
+          "%s. ex: %s", container.getDbFile(), ex.getMessage());
+      throw new StorageContainerException(message, UNABLE_TO_READ_METADATA_DB);
+    }
+  }
+  /**
+   * Remove a DB handler from cache.
+   *
+   * @param container - Container data.
+   * @param conf - Configuration.
+   */
+  public static void removeDB(KeyValueContainerData container, Configuration
+      conf) {
+    Preconditions.checkNotNull(container);
+    ContainerCache cache = ContainerCache.getInstance(conf);
+    Preconditions.checkNotNull(cache);
+    cache.removeDB(container.getContainerId());
+  }
+
+}

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/998e2850/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
index 235a32f..2c9c2c3 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
@@ -17,12 +17,14 @@
 
 package org.apache.hadoop.ozone.container.common.impl;
 
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -51,7 +53,8 @@ public class TestContainerSet {
     KeyValueContainerData kvData = new KeyValueContainerData(
         ContainerProtos.ContainerType.KeyValueContainer, containerId);
     kvData.setState(state);
-    KeyValueContainer keyValueContainer = new KeyValueContainer(kvData);
+    KeyValueContainer keyValueContainer = new KeyValueContainer(kvData, new
+        OzoneConfiguration());
 
     //addContainer
     boolean result = containerSet.addContainer(keyValueContainer);
@@ -160,7 +163,8 @@ public class TestContainerSet {
       } else {
         kvData.setState(ContainerProtos.ContainerLifeCycleState.OPEN);
       }
-      KeyValueContainer kv = new KeyValueContainer(kvData);
+      KeyValueContainer kv = new KeyValueContainer(kvData, new
+          OzoneConfiguration());
       containerSet.addContainer(kv);
     }
     return containerSet;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/998e2850/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
new file mode 100644
index 0000000..b24f601
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
@@ -0,0 +1,281 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+
+
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.common.impl.KeyValueYaml;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.DiskChecker;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import org.mockito.Mockito;
+
+import java.io.File;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+import static org.apache.ratis.util.Preconditions.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.anyList;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Class to test KeyValue Container operations.
+ */
+public class TestKeyValueContainer {
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+
+  private OzoneConfiguration conf;
+  private String scmId = UUID.randomUUID().toString();
+  private VolumeSet volumeSet;
+  private RoundRobinVolumeChoosingPolicy volumeChoosingPolicy;
+  private long containerId = 1L;
+  private String containerName = String.valueOf(containerId);
+  private KeyValueContainerData keyValueContainerData;
+  private KeyValueContainer keyValueContainer;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new OzoneConfiguration();
+    HddsVolume hddsVolume = new HddsVolume.Builder(folder.getRoot()
+        .getAbsolutePath()).conf(conf).datanodeUuid(UUID.randomUUID()
+        .toString()).build();
+
+    volumeSet = mock(VolumeSet.class);
+    volumeChoosingPolicy = mock(RoundRobinVolumeChoosingPolicy.class);
+    Mockito.when(volumeChoosingPolicy.chooseVolume(anyList(), anyLong()))
+        .thenReturn(hddsVolume);
+
+    keyValueContainerData = new KeyValueContainerData(
+        ContainerProtos.ContainerType.KeyValueContainer, 1L);
+
+    keyValueContainer = new KeyValueContainer(
+        keyValueContainerData, conf);
+
+  }
+
+  @Test
+  public void testCreateContainer() throws Exception {
+
+    // Create Container.
+    keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
+
+    keyValueContainerData = (KeyValueContainerData) keyValueContainer
+        .getContainerData();
+
+    String containerMetaDataPath = keyValueContainerData
+        .getMetadataPath();
+    String chunksPath = keyValueContainerData.getChunksPath();
+
+    // Check whether containerMetaDataPath and chunksPath exists or not.
+    assertTrue(containerMetaDataPath != null);
+    assertTrue(chunksPath != null);
+    File containerMetaDataLoc = new File(containerMetaDataPath);
+
+    //Check whether container file, check sum file and container db file exists
+    // or not.
+    assertTrue(KeyValueContainerLocationUtil.getContainerFile(
+        containerMetaDataLoc, containerName).exists(), ".Container File does" +
+        " not exist");
+    assertTrue(KeyValueContainerLocationUtil.getContainerCheckSumFile(
+        containerMetaDataLoc, containerName).exists(), "Container check sum " +
+        "File does" + " not exist");
+    assertTrue(KeyValueContainerLocationUtil.getContainerDBFile(
+        containerMetaDataLoc, containerName).exists(), "Container DB does " +
+        "not exist");
+  }
+
+  @Test
+  public void testDuplicateContainer() throws Exception {
+    try {
+      // Create Container.
+      keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
+      keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
+      fail("testDuplicateContainer failed");
+    } catch (StorageContainerException ex) {
+      GenericTestUtils.assertExceptionContains("ContainerFile already " +
+          "exists", ex);
+      assertEquals(ContainerProtos.Result.CONTAINER_ALREADY_EXISTS, ex
+          .getResult());
+    }
+  }
+
+  @Test
+  public void testDiskFullExceptionCreateContainer() throws Exception {
+
+    Mockito.when(volumeChoosingPolicy.chooseVolume(anyList(), anyLong()))
+        .thenThrow(DiskChecker.DiskOutOfSpaceException.class);
+    try {
+      keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
+      fail("testDiskFullExceptionCreateContainer failed");
+    } catch (StorageContainerException ex) {
+      GenericTestUtils.assertExceptionContains("disk out of space",
+          ex);
+      assertEquals(ContainerProtos.Result.DISK_OUT_OF_SPACE, ex.getResult());
+    }
+  }
+
+  @Test
+  public void testDeleteContainer() throws Exception {
+    keyValueContainerData.setState(ContainerProtos.ContainerLifeCycleState
+        .CLOSED);
+    keyValueContainer = new KeyValueContainer(
+        keyValueContainerData, conf);
+    keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
+    keyValueContainer.delete(true);
+
+    String containerMetaDataPath = keyValueContainerData
+        .getMetadataPath();
+    File containerMetaDataLoc = new File(containerMetaDataPath);
+
+    assertFalse("Container directory still exists", containerMetaDataLoc
+        .getParentFile().exists());
+
+    assertFalse("Container File still exists",
+        KeyValueContainerLocationUtil.getContainerFile(containerMetaDataLoc,
+            containerName).exists());
+    assertFalse("Container DB file still exists",
+        KeyValueContainerLocationUtil.getContainerDBFile(containerMetaDataLoc,
+            containerName).exists());
+  }
+
+
+  @Test
+  public void testCloseContainer() throws Exception {
+    keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
+    keyValueContainer.close();
+
+    keyValueContainerData = (KeyValueContainerData) keyValueContainer
+        .getContainerData();
+
+    assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED,
+        keyValueContainerData.getState());
+
+    //Check state in the .container file
+    String containerMetaDataPath = keyValueContainerData
+        .getMetadataPath();
+    File containerMetaDataLoc = new File(containerMetaDataPath);
+    File containerFile = KeyValueContainerLocationUtil.getContainerFile(
+        containerMetaDataLoc, containerName);
+
+    keyValueContainerData = KeyValueYaml.readContainerFile(containerFile);
+    assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED,
+        keyValueContainerData.getState());
+  }
+
+  @Test
+  public void testCloseInvalidContainer() throws Exception {
+    try {
+      keyValueContainerData.setState(ContainerProtos.ContainerLifeCycleState
+          .INVALID);
+      keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
+      keyValueContainer.close();
+      fail("testCloseInvalidContainer failed");
+    } catch (StorageContainerException ex) {
+      assertEquals(ContainerProtos.Result.INVALID_CONTAINER_STATE,
+          ex.getResult());
+      GenericTestUtils.assertExceptionContains("Invalid container data", ex);
+    }
+  }
+
+  @Test
+  public void testUpdateContainer() throws IOException {
+    keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
+    Map<String, String> metadata = new HashMap<>();
+    metadata.put("VOLUME", "ozone");
+    metadata.put("OWNER", "hdfs");
+    keyValueContainer.update(metadata, true);
+
+    keyValueContainerData = (KeyValueContainerData) keyValueContainer
+        .getContainerData();
+
+    assertEquals(2, keyValueContainerData.getMetadata().size());
+
+    //Check metadata in the .container file
+    String containerMetaDataPath = keyValueContainerData
+        .getMetadataPath();
+    File containerMetaDataLoc = new File(containerMetaDataPath);
+    File containerFile = KeyValueContainerLocationUtil.getContainerFile(
+        containerMetaDataLoc, containerName);
+
+    keyValueContainerData = KeyValueYaml.readContainerFile(containerFile);
+    assertEquals(2, keyValueContainerData.getMetadata().size());
+
+  }
+
+  @Test
+  public void testUpdateContainerInvalidMetadata() throws IOException {
+    try {
+      keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
+      Map<String, String> metadata = new HashMap<>();
+      metadata.put("VOLUME", "ozone");
+      keyValueContainer.update(metadata, true);
+      //Trying to update again with same metadata
+      keyValueContainer.update(metadata, true);
+      fail("testUpdateContainerInvalidMetadata failed");
+    } catch (StorageContainerException ex) {
+      GenericTestUtils.assertExceptionContains("Container Metadata update " +
+          "error", ex);
+      assertEquals(ContainerProtos.Result.CONTAINER_METADATA_ERROR, ex
+          .getResult());
+    }
+  }
+
+  @Test
+  public void testUpdateContainerUnsupportedRequest() throws Exception {
+    try {
+      keyValueContainerData.setState(ContainerProtos.ContainerLifeCycleState
+          .CLOSED);
+      keyValueContainer = new KeyValueContainer(keyValueContainerData, conf);
+      keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
+      Map<String, String> metadata = new HashMap<>();
+      metadata.put("VOLUME", "ozone");
+      keyValueContainer.update(metadata, false);
+      fail("testUpdateContainerUnsupportedRequest failed");
+    } catch (StorageContainerException ex) {
+      GenericTestUtils.assertExceptionContains("Updating a closed container " +
+          "is not allowed", ex);
+      assertEquals(ContainerProtos.Result.UNSUPPORTED_REQUEST, ex
+          .getResult());
+    }
+  }
+
+
+}


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


[30/37] hadoop git commit: HDDS-182:CleanUp Reimplemented classes. Contributed by Hansiha Koneru

Posted by bh...@apache.org.
HDDS-182:CleanUp Reimplemented classes. Contributed by Hansiha Koneru


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

Branch: refs/heads/trunk
Commit: 2c2351e87b60d3e8b50b94e9ca5ab78d7afae783
Parents: a404164
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Thu Jul 5 15:23:39 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Thu Jul 5 15:23:39 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/ozone/OzoneConsts.java    |   10 +
 .../container/common/helpers/ChunkUtils.java    |  343 ------
 .../container/common/helpers/ContainerData.java |  512 --------
 .../common/helpers/ContainerUtils.java          |  179 +--
 .../container/common/helpers/KeyUtils.java      |  148 ---
 .../container/common/impl/ChunkManagerImpl.java |  233 ----
 .../container/common/impl/ContainerData.java    |  131 +-
 .../common/impl/ContainerDataYaml.java          |   29 +-
 .../impl/ContainerLocationManagerImpl.java      |  158 ---
 .../common/impl/ContainerManagerImpl.java       | 1115 ------------------
 .../container/common/impl/ContainerSet.java     |   19 +-
 .../common/impl/ContainerStorageLocation.java   |  212 ----
 .../ozone/container/common/impl/Dispatcher.java |  695 -----------
 .../container/common/impl/KeyManagerImpl.java   |  204 ----
 .../RandomContainerDeletionChoosingPolicy.java  |    1 -
 ...NOrderedContainerDeletionChoosingPolicy.java |    1 -
 .../common/interfaces/ChunkManager.java         |   73 --
 .../container/common/interfaces/Container.java  |    1 -
 .../ContainerDeletionChoosingPolicy.java        |    3 +-
 .../common/interfaces/ContainerManager.java     |  267 -----
 .../container/common/interfaces/KeyManager.java |   73 --
 .../background/BlockDeletingService.java        |  247 ----
 .../statemachine/background/package-info.java   |   18 -
 .../container/common/volume/VolumeSet.java      |   13 +-
 .../container/keyvalue/KeyValueContainer.java   |   36 +-
 .../keyvalue/KeyValueContainerData.java         |   80 +-
 .../container/keyvalue/KeyValueHandler.java     |    2 +-
 .../container/keyvalue/helpers/ChunkUtils.java  |   18 +-
 .../container/keyvalue/helpers/KeyUtils.java    |   19 +-
 .../keyvalue/helpers/KeyValueContainerUtil.java |   48 +-
 .../keyvalue/impl/ChunkManagerImpl.java         |   10 +-
 .../background/BlockDeletingService.java        |  248 ++++
 .../statemachine/background/package-info.java   |   18 +
 .../container/ozoneimpl/OzoneContainer.java     |    2 +-
 .../common/TestKeyValueContainerData.java       |    2 +-
 .../common/impl/TestContainerDataYaml.java      |    6 +-
 .../container/common/impl/TestContainerSet.java |   10 +-
 .../TestRoundRobinVolumeChoosingPolicy.java     |    6 +-
 .../container/common/volume/TestVolumeSet.java  |    6 +-
 .../keyvalue/TestKeyValueContainer.java         |   20 +-
 .../container/keyvalue/TestKeyValueHandler.java |   11 +-
 .../container/ozoneimpl/TestOzoneContainer.java |    2 +-
 .../testutils/BlockDeletingServiceTestImpl.java |    9 +-
 .../test/resources/additionalfields.container   |    2 +-
 .../src/test/resources/incorrect.container      |    2 +-
 .../ozone/container/ContainerTestHelper.java    |    2 +
 .../common/TestBlockDeletingService.java        |  118 +-
 .../TestContainerDeletionChoosingPolicy.java    |   63 +-
 .../common/impl/TestContainerPersistence.java   |  478 ++++----
 .../container/metrics/TestContainerMetrics.java |    3 +-
 .../container/server/TestContainerServer.java   |   14 +-
 .../org/apache/hadoop/ozone/scm/TestSCMCli.java |   44 +-
 .../genesis/BenchMarkDatanodeDispatcher.java    |   24 +-
 53 files changed, 943 insertions(+), 5045 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index 27aa6ee..82d67b7 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -183,4 +183,14 @@ public final class OzoneConsts {
   private OzoneConsts() {
     // Never Constructed
   }
+
+  // YAML fields for .container files
+  public static final String CONTAINER_ID = "containerID";
+  public static final String CONTAINER_TYPE = "containerType";
+  public static final String STATE = "state";
+  public static final String METADATA = "metadata";
+  public static final String MAX_SIZE_GB = "maxSizeGB";
+  public static final String METADATA_PATH = "metadataPath";
+  public static final String CHUNKS_PATH = "chunksPath";
+  public static final String CONTAINER_DB_TYPE = "containerDBType";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java
deleted file mode 100644
index e0bf213..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkUtils.java
+++ /dev/null
@@ -1,343 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.hadoop.ozone.container.common.helpers;
-
-import com.google.common.base.Preconditions;
-import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.commons.codec.binary.Hex;
-import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.hadoop.hdds.scm.container.common.helpers
-    .StorageContainerException;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.AsynchronousFileChannel;
-import java.nio.channels.FileLock;
-import java.nio.file.StandardOpenOption;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.util.concurrent.ExecutionException;
-
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.CHECKSUM_MISMATCH;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.CONTAINER_INTERNAL_ERROR;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.CONTAINER_NOT_FOUND;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.INVALID_WRITE_SIZE;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.IO_EXCEPTION;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.OVERWRITE_FLAG_REQUIRED;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.UNABLE_TO_FIND_CHUNK;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.UNABLE_TO_FIND_DATA_DIR;
-
-/**
- * Set of utility functions used by the chunk Manager.
- */
-public final class ChunkUtils {
-
-  /* Never constructed. */
-  private ChunkUtils() {
-  }
-
-  /**
-   * Checks if we are getting a request to overwrite an existing range of
-   * chunk.
-   *
-   * @param chunkFile - File
-   * @param chunkInfo - Buffer to write
-   * @return bool
-   */
-  public static boolean isOverWriteRequested(File chunkFile, ChunkInfo
-      chunkInfo) {
-
-    if (!chunkFile.exists()) {
-      return false;
-    }
-
-    long offset = chunkInfo.getOffset();
-    return offset < chunkFile.length();
-  }
-
-  /**
-   * Overwrite is permitted if an only if the user explicitly asks for it. We
-   * permit this iff the key/value pair contains a flag called
-   * [OverWriteRequested, true].
-   *
-   * @param chunkInfo - Chunk info
-   * @return true if the user asks for it.
-   */
-  public static boolean isOverWritePermitted(ChunkInfo chunkInfo) {
-    String overWrite = chunkInfo.getMetadata().get(OzoneConsts.CHUNK_OVERWRITE);
-    return (overWrite != null) &&
-        (!overWrite.isEmpty()) &&
-        (Boolean.valueOf(overWrite));
-  }
-
-  /**
-   * Validates chunk data and returns a file object to Chunk File that we are
-   * expected to write data to.
-   *
-   * @param data - container data.
-   * @param info - chunk info.
-   * @return File
-   * @throws StorageContainerException
-   */
-  public static File validateChunk(ContainerData data,
-      ChunkInfo info) throws StorageContainerException {
-
-    Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
-
-    File chunkFile = getChunkFile(data, info);
-    if (ChunkUtils.isOverWriteRequested(chunkFile, info)) {
-      if (!ChunkUtils.isOverWritePermitted(info)) {
-        log.error("Rejecting write chunk request. Chunk overwrite " +
-            "without explicit request. {}", info.toString());
-        throw new StorageContainerException("Rejecting write chunk request. " +
-            "OverWrite flag required." + info.toString(),
-            OVERWRITE_FLAG_REQUIRED);
-      }
-    }
-    return chunkFile;
-  }
-
-  /**
-   * Validates that Path to chunk file exists.
-   *
-   * @param data - Container Data
-   * @param info - Chunk info
-   * @return - File.
-   * @throws StorageContainerException
-   */
-  public static File getChunkFile(ContainerData data,
-      ChunkInfo info) throws StorageContainerException {
-
-    Preconditions.checkNotNull(data, "Container data can't be null");
-    Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
-    if (data.getContainerID() < 0) {
-      log.error("Invalid container id: {}", data.getContainerID());
-      throw new StorageContainerException("Unable to find the container id:" +
-          " " +
-          data.getContainerID(), CONTAINER_NOT_FOUND);
-    }
-
-    File dataDir = ContainerUtils.getDataDirectory(data).toFile();
-    if (!dataDir.exists()) {
-      log.error("Unable to find the data directory: {}", dataDir);
-      throw new StorageContainerException("Unable to find the data directory:" +
-          " " + dataDir, UNABLE_TO_FIND_DATA_DIR);
-    }
-
-    return dataDir.toPath().resolve(info.getChunkName()).toFile();
-
-  }
-
-  /**
-   * Writes the data in chunk Info to the specified location in the chunkfile.
-   *
-   * @param chunkFile - File to write data to.
-   * @param chunkInfo - Data stream to write.
-   * @param data - The data buffer.
-   * @throws StorageContainerException
-   */
-  public static void writeData(File chunkFile, ChunkInfo chunkInfo,
-      byte[] data) throws
-      StorageContainerException, ExecutionException, InterruptedException,
-      NoSuchAlgorithmException {
-
-    Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
-    if (data.length != chunkInfo.getLen()) {
-      String err = String.format("data array does not match the length " +
-              "specified. DataLen: %d Byte Array: %d",
-          chunkInfo.getLen(), data.length);
-      log.error(err);
-      throw new StorageContainerException(err, INVALID_WRITE_SIZE);
-    }
-
-    AsynchronousFileChannel file = null;
-    FileLock lock = null;
-
-    try {
-      file =
-          AsynchronousFileChannel.open(chunkFile.toPath(),
-              StandardOpenOption.CREATE,
-              StandardOpenOption.WRITE,
-              StandardOpenOption.SPARSE,
-              StandardOpenOption.SYNC);
-      lock = file.lock().get();
-      if (chunkInfo.getChecksum() != null &&
-          !chunkInfo.getChecksum().isEmpty()) {
-        verifyChecksum(chunkInfo, data, log);
-      }
-      int size = file.write(ByteBuffer.wrap(data), chunkInfo.getOffset()).get();
-      if (size != data.length) {
-        log.error("Invalid write size found. Size:{}  Expected: {} ", size,
-            data.length);
-        throw new StorageContainerException("Invalid write size found. " +
-            "Size: " + size + " Expected: " + data.length, INVALID_WRITE_SIZE);
-      }
-    } catch (IOException e) {
-      throw new StorageContainerException(e, IO_EXCEPTION);
-
-    } finally {
-      if (lock != null) {
-        try {
-          lock.release();
-        } catch (IOException e) {
-          log.error("Unable to release lock ??, Fatal Error.");
-          throw new StorageContainerException(e, CONTAINER_INTERNAL_ERROR);
-
-        }
-      }
-      if (file != null) {
-        try {
-          file.close();
-        } catch (IOException e) {
-          throw new StorageContainerException("Error closing chunk file",
-              e, CONTAINER_INTERNAL_ERROR);
-        }
-      }
-    }
-  }
-
-  /**
-   * Verifies the checksum of a chunk against the data buffer.
-   *
-   * @param chunkInfo - Chunk Info.
-   * @param data - data buffer
-   * @param log - log
-   * @throws NoSuchAlgorithmException
-   * @throws StorageContainerException
-   */
-  private static void verifyChecksum(ChunkInfo chunkInfo, byte[] data, Logger
-      log) throws NoSuchAlgorithmException, StorageContainerException {
-    MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
-    sha.update(data);
-    if (!Hex.encodeHexString(sha.digest()).equals(
-        chunkInfo.getChecksum())) {
-      log.error("Checksum mismatch. Provided: {} , computed: {}",
-          chunkInfo.getChecksum(), DigestUtils.sha256Hex(sha.digest()));
-      throw new StorageContainerException("Checksum mismatch. Provided: " +
-          chunkInfo.getChecksum() + " , computed: " +
-          DigestUtils.sha256Hex(sha.digest()), CHECKSUM_MISMATCH);
-    }
-  }
-
-  /**
-   * Reads data from an existing chunk file.
-   *
-   * @param chunkFile - file where data lives.
-   * @param data - chunk definition.
-   * @return ByteBuffer
-   * @throws StorageContainerException
-   * @throws ExecutionException
-   * @throws InterruptedException
-   */
-  public static ByteBuffer readData(File chunkFile, ChunkInfo data) throws
-      StorageContainerException, ExecutionException, InterruptedException,
-      NoSuchAlgorithmException {
-    Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
-
-    if (!chunkFile.exists()) {
-      log.error("Unable to find the chunk file. chunk info : {}",
-          data.toString());
-      throw new StorageContainerException("Unable to find the chunk file. " +
-          "chunk info " +
-          data.toString(), UNABLE_TO_FIND_CHUNK);
-    }
-
-    AsynchronousFileChannel file = null;
-    FileLock lock = null;
-    try {
-      file =
-          AsynchronousFileChannel.open(chunkFile.toPath(),
-              StandardOpenOption.READ);
-      lock = file.lock(data.getOffset(), data.getLen(), true).get();
-
-      ByteBuffer buf = ByteBuffer.allocate((int) data.getLen());
-      file.read(buf, data.getOffset()).get();
-
-      if (data.getChecksum() != null && !data.getChecksum().isEmpty()) {
-        verifyChecksum(data, buf.array(), log);
-      }
-
-      return buf;
-    } catch (IOException e) {
-      throw new StorageContainerException(e, IO_EXCEPTION);
-    } finally {
-      if (lock != null) {
-        try {
-          lock.release();
-        } catch (IOException e) {
-          log.error("I/O error is lock release.");
-        }
-      }
-      if (file != null) {
-        IOUtils.closeStream(file);
-      }
-    }
-  }
-
-  /**
-   * Returns a CreateContainer Response. This call is used by create and delete
-   * containers which have null success responses.
-   *
-   * @param msg Request
-   * @return Response.
-   */
-  public static ContainerProtos.ContainerCommandResponseProto
-      getChunkResponse(ContainerProtos.ContainerCommandRequestProto msg) {
-    return ContainerUtils.getSuccessResponse(msg);
-  }
-
-  /**
-   * Gets a response to the read chunk calls.
-   *
-   * @param msg - Msg
-   * @param data - Data
-   * @param info - Info
-   * @return Response.
-   */
-  public static ContainerProtos.ContainerCommandResponseProto
-      getReadChunkResponse(ContainerProtos.ContainerCommandRequestProto msg,
-      byte[] data, ChunkInfo info) {
-    Preconditions.checkNotNull(msg);
-
-    ContainerProtos.ReadChunkResponseProto.Builder response =
-        ContainerProtos.ReadChunkResponseProto.newBuilder();
-    response.setChunkData(info.getProtoBufMessage());
-    response.setData(ByteString.copyFrom(data));
-    response.setBlockID(msg.getReadChunk().getBlockID());
-
-    ContainerProtos.ContainerCommandResponseProto.Builder builder =
-        ContainerUtils.getSuccessResponseBuilder(msg);
-    builder.setReadChunk(response);
-    return builder.build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerData.java
deleted file mode 100644
index 5767f76..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerData.java
+++ /dev/null
@@ -1,512 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.helpers;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .ContainerType;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .ContainerLifeCycleState;
-import org.apache.hadoop.ozone.OzoneConsts;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static java.lang.Math.max;
-
-/**
- * This class maintains the information about a container in the ozone world.
- * <p>
- * A container is a name, along with metadata- which is a set of key value
- * pair.
- */
-public class ContainerData {
-
-  private final Map<String, String> metadata;
-  private String dbPath;  // Path to Level DB Store.
-  // Path to Physical file system where container and checksum are stored.
-  private String containerFilePath;
-  private AtomicLong bytesUsed;
-  private long maxSize;
-  private long containerID;
-  private ContainerLifeCycleState state;
-  private ContainerType containerType;
-  private String containerDBType;
-
-
-  /**
-   * Number of pending deletion blocks in container.
-   */
-  private int numPendingDeletionBlocks;
-  private long deleteTransactionId;
-  private AtomicLong readBytes;
-  private AtomicLong writeBytes;
-  private AtomicLong readCount;
-  private AtomicLong writeCount;
-
-
-  /**
-   * Constructs a  ContainerData Object.
-   *
-   * @param containerID - ID
-   * @param conf - Configuration
-   */
-  public ContainerData(long containerID,
-      Configuration conf) {
-    this.metadata = new TreeMap<>();
-    this.maxSize = conf.getLong(ScmConfigKeys.SCM_CONTAINER_CLIENT_MAX_SIZE_KEY,
-        ScmConfigKeys.SCM_CONTAINER_CLIENT_MAX_SIZE_DEFAULT) * OzoneConsts.GB;
-    this.bytesUsed =  new AtomicLong(0L);
-    this.containerID = containerID;
-    this.state = ContainerLifeCycleState.OPEN;
-    this.numPendingDeletionBlocks = 0;
-    this.deleteTransactionId = 0;
-    this.readCount = new AtomicLong(0L);
-    this.readBytes =  new AtomicLong(0L);
-    this.writeCount =  new AtomicLong(0L);
-    this.writeBytes =  new AtomicLong(0L);
-  }
-
-  /**
-   * Constructs a  ContainerData Object.
-   *
-   * @param containerID - ID
-   * @param conf - Configuration
-   * @param state - ContainerLifeCycleState
-   * @param
-   */
-  public ContainerData(long containerID, Configuration conf,
-                       ContainerLifeCycleState state) {
-    this.metadata = new TreeMap<>();
-    this.maxSize = conf.getLong(ScmConfigKeys.SCM_CONTAINER_CLIENT_MAX_SIZE_KEY,
-        ScmConfigKeys.SCM_CONTAINER_CLIENT_MAX_SIZE_DEFAULT) * OzoneConsts.GB;
-    this.bytesUsed =  new AtomicLong(0L);
-    this.containerID = containerID;
-    this.state = state;
-    this.numPendingDeletionBlocks = 0;
-    this.deleteTransactionId = 0;
-    this.readCount = new AtomicLong(0L);
-    this.readBytes =  new AtomicLong(0L);
-    this.writeCount =  new AtomicLong(0L);
-    this.writeBytes =  new AtomicLong(0L);
-  }
-
-  /**
-   * Constructs a ContainerData object from ProtoBuf classes.
-   *
-   * @param protoData - ProtoBuf Message
-   * @throws IOException
-   */
-  public static ContainerData getFromProtBuf(
-      ContainerProtos.ContainerData protoData, Configuration conf)
-      throws IOException {
-    ContainerData data = new ContainerData(
-        protoData.getContainerID(), conf);
-    for (int x = 0; x < protoData.getMetadataCount(); x++) {
-      data.addMetadata(protoData.getMetadata(x).getKey(),
-          protoData.getMetadata(x).getValue());
-    }
-
-    if (protoData.hasContainerPath()) {
-      data.setContainerPath(protoData.getContainerPath());
-    }
-
-    if (protoData.hasDbPath()) {
-      data.setDBPath(protoData.getDbPath());
-    }
-
-    if (protoData.hasState()) {
-      data.setState(protoData.getState());
-    }
-
-    if (protoData.hasBytesUsed()) {
-      data.setBytesUsed(protoData.getBytesUsed());
-    }
-
-    if (protoData.hasSize()) {
-      data.setMaxSize(protoData.getSize());
-    }
-
-    if(protoData.hasContainerType()) {
-      data.setContainerType(protoData.getContainerType());
-    }
-
-    if(protoData.hasContainerDBType()) {
-      data.setContainerDBType(protoData.getContainerDBType());
-    }
-
-    return data;
-  }
-
-  public String getContainerDBType() {
-    return containerDBType;
-  }
-
-  public void setContainerDBType(String containerDBType) {
-    this.containerDBType = containerDBType;
-  }
-
-  /**
-   * Returns a ProtoBuf Message from ContainerData.
-   *
-   * @return Protocol Buffer Message
-   */
-  public ContainerProtos.ContainerData getProtoBufMessage() {
-    ContainerProtos.ContainerData.Builder builder = ContainerProtos
-        .ContainerData.newBuilder();
-    builder.setContainerID(this.getContainerID());
-
-    if (this.getDBPath() != null) {
-      builder.setDbPath(this.getDBPath());
-    }
-
-    if (this.getContainerPath() != null) {
-      builder.setContainerPath(this.getContainerPath());
-    }
-
-    builder.setState(this.getState());
-
-    for (Map.Entry<String, String> entry : metadata.entrySet()) {
-      ContainerProtos.KeyValue.Builder keyValBuilder =
-          ContainerProtos.KeyValue.newBuilder();
-      builder.addMetadata(keyValBuilder.setKey(entry.getKey())
-          .setValue(entry.getValue()).build());
-    }
-
-    if (this.getBytesUsed() >= 0) {
-      builder.setBytesUsed(this.getBytesUsed());
-    }
-
-    if (this.getKeyCount() >= 0) {
-      builder.setKeyCount(this.getKeyCount());
-    }
-
-    if (this.getMaxSize() >= 0) {
-      builder.setSize(this.getMaxSize());
-    }
-
-    if(this.getContainerType() != null) {
-      builder.setContainerType(containerType);
-    }
-
-    if(this.getContainerDBType() != null) {
-      builder.setContainerDBType(containerDBType);
-    }
-
-    return builder.build();
-  }
-
-  public void setContainerType(ContainerType containerType) {
-    this.containerType = containerType;
-  }
-
-  public ContainerType getContainerType() {
-    return this.containerType;
-  }
-  /**
-   * Adds metadata.
-   */
-  public void addMetadata(String key, String value) throws IOException {
-    synchronized (this.metadata) {
-      if (this.metadata.containsKey(key)) {
-        throw new IOException("This key already exists. Key " + key);
-      }
-      metadata.put(key, value);
-    }
-  }
-
-  /**
-   * Returns all metadata.
-   */
-  public Map<String, String> getAllMetadata() {
-    synchronized (this.metadata) {
-      return Collections.unmodifiableMap(this.metadata);
-    }
-  }
-
-  /**
-   * Returns value of a key.
-   */
-  public String getValue(String key) {
-    synchronized (this.metadata) {
-      return metadata.get(key);
-    }
-  }
-
-  /**
-   * Deletes a metadata entry from the map.
-   *
-   * @param key - Key
-   */
-  public void deleteKey(String key) {
-    synchronized (this.metadata) {
-      metadata.remove(key);
-    }
-  }
-
-  /**
-   * Returns path.
-   *
-   * @return - path
-   */
-  public String getDBPath() {
-    return dbPath;
-  }
-
-  /**
-   * Sets path.
-   *
-   * @param path - String.
-   */
-  public void setDBPath(String path) {
-    this.dbPath = path;
-  }
-
-  /**
-   * This function serves as the generic key for ContainerCache class. Both
-   * ContainerData and ContainerKeyData overrides this function to appropriately
-   * return the right name that can  be used in ContainerCache.
-   *
-   * @return String Name.
-   */
-    // TODO: check the ContainerCache class to see if
-  // we are using the ContainerID instead.
-   /*
-   public String getName() {
-    return getContainerID();
-  }*/
-
-  /**
-   * Get container file path.
-   * @return - Physical path where container file and checksum is stored.
-   */
-  public String getContainerPath() {
-    return containerFilePath;
-  }
-
-  /**
-   * Set container Path.
-   * @param containerPath - File path.
-   */
-  public void setContainerPath(String containerPath) {
-    this.containerFilePath = containerPath;
-  }
-
-  /**
-   * Get container ID.
-   * @return - container ID.
-   */
-  public synchronized long getContainerID() {
-    return containerID;
-  }
-
-  public synchronized void setState(ContainerLifeCycleState state) {
-    this.state = state;
-  }
-
-  public synchronized ContainerLifeCycleState getState() {
-    return this.state;
-  }
-
-  /**
-   * checks if the container is open.
-   * @return - boolean
-   */
-  public synchronized  boolean isOpen() {
-    return ContainerLifeCycleState.OPEN == state;
-  }
-
-  /**
-   * checks if the container is invalid.
-   * @return - boolean
-   */
-  public boolean isValid() {
-    return !(ContainerLifeCycleState.INVALID == state);
-  }
-
-  /**
-   * checks if the container is closed.
-   * @return - boolean
-   */
-  public synchronized  boolean isClosed() {
-    return ContainerLifeCycleState.CLOSED == state;
-  }
-
-  /**
-   * Marks this container as closed.
-   */
-  public synchronized void closeContainer() {
-    // TODO: closed or closing here
-    setState(ContainerLifeCycleState.CLOSED);
-
-  }
-
-  public void setMaxSize(long maxSize) {
-    this.maxSize = maxSize;
-  }
-
-  public long getMaxSize() {
-    return maxSize;
-  }
-
-  public long getKeyCount() {
-    return metadata.size();
-  }
-
-  public void setBytesUsed(long used) {
-    this.bytesUsed.set(used);
-  }
-
-  /**
-   * Get the number of bytes used by the container.
-   * @return the number of bytes used by the container.
-   */
-  public long getBytesUsed() {
-    return bytesUsed.get();
-  }
-
-  /**
-   * Increase the number of bytes used by the container.
-   * @param used number of bytes used by the container.
-   * @return the current number of bytes used by the container afert increase.
-   */
-  public long incrBytesUsed(long used) {
-    return this.bytesUsed.addAndGet(used);
-  }
-
-
-  /**
-   * Decrease the number of bytes used by the container.
-   * @param reclaimed the number of bytes reclaimed from the container.
-   * @return the current number of bytes used by the container after decrease.
-   */
-  public long decrBytesUsed(long reclaimed) {
-    return this.bytesUsed.addAndGet(-1L * reclaimed);
-  }
-
-  /**
-   * Increase the count of pending deletion blocks.
-   *
-   * @param numBlocks increment number
-   */
-  public void incrPendingDeletionBlocks(int numBlocks) {
-    this.numPendingDeletionBlocks += numBlocks;
-  }
-
-  /**
-   * Decrease the count of pending deletion blocks.
-   *
-   * @param numBlocks decrement number
-   */
-  public void decrPendingDeletionBlocks(int numBlocks) {
-    this.numPendingDeletionBlocks -= numBlocks;
-  }
-
-  /**
-   * Get the number of pending deletion blocks.
-   */
-  public int getNumPendingDeletionBlocks() {
-    return this.numPendingDeletionBlocks;
-  }
-
-  /**
-   * Sets deleteTransactionId to latest delete transactionId for the container.
-   *
-   * @param transactionId latest transactionId of the container.
-   */
-  public void updateDeleteTransactionId(long transactionId) {
-    deleteTransactionId = max(transactionId, deleteTransactionId);
-  }
-
-  /**
-   * Return the latest deleteTransactionId of the container.
-   */
-  public long getDeleteTransactionId() {
-    return deleteTransactionId;
-  }
-
-  /**
-   * Get the number of bytes read from the container.
-   * @return the number of bytes read from the container.
-   */
-  public long getReadBytes() {
-    return readBytes.get();
-  }
-
-  /**
-   * Increase the number of bytes read from the container.
-   * @param bytes number of bytes read.
-   */
-  public void incrReadBytes(long bytes) {
-    this.readBytes.addAndGet(bytes);
-  }
-
-  /**
-   * Get the number of times the container is read.
-   * @return the number of times the container is read.
-   */
-  public long getReadCount() {
-    return readCount.get();
-  }
-
-  /**
-   * Increase the number of container read count by 1.
-   */
-  public void incrReadCount() {
-    this.readCount.incrementAndGet();
-  }
-
-  /**
-   * Get the number of bytes write into the container.
-   * @return the number of bytes write into the container.
-   */
-  public long getWriteBytes() {
-    return writeBytes.get();
-  }
-
-  /**
-   * Increase the number of bytes write into the container.
-   * @param bytes the number of bytes write into the container.
-   */
-  public void incrWriteBytes(long bytes) {
-    this.writeBytes.addAndGet(bytes);
-  }
-
-  /**
-   * Get the number of writes into the container.
-   * @return the number of writes into the container.
-   */
-  public long getWriteCount() {
-    return writeCount.get();
-  }
-
-  /**
-   * Increase the number of writes into the container by 1.
-   */
-  public void incrWriteCount() {
-    this.writeCount.incrementAndGet();
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
index b975217..18a5231 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
@@ -19,8 +19,6 @@
 package org.apache.hadoop.ozone.container.common.helpers;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
@@ -34,10 +32,8 @@ import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.ozone.OzoneConsts;
 import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
-import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
-import org.apache.hadoop.ozone.container.common.interfaces.Container;
-import org.apache.hadoop.utils.MetadataStore;
-import org.apache.hadoop.utils.MetadataStoreBuilder;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -49,14 +45,6 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 
 import static org.apache.commons.io.FilenameUtils.removeExtension;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.CLOSED_CONTAINER_IO;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.INVALID_CONTAINER_STATE;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.SUCCESS;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.UNABLE_TO_FIND_DATA_DIR;
 
 /**
  * A set of helper functions to create proper responses.
@@ -203,30 +191,17 @@ public final class ContainerUtils {
    * @param containerFile - Container File to verify
    * @throws IOException
    */
-  public static void verifyIsNewContainer(File containerFile)
-      throws IOException {
-    Logger log = LoggerFactory.getLogger(ContainerManagerImpl.class);
-    if (containerFile.exists()) {
-      log.error("container already exists on disk. File: {}",
-          containerFile.toPath());
+  public static void verifyIsNewContainer(File containerFile) throws
+      FileAlreadyExistsException {
+    Logger log = LoggerFactory.getLogger(ContainerSet.class);
+    Preconditions.checkNotNull(containerFile, "containerFile Should not be " +
+        "null");
+    if (containerFile.getParentFile().exists()) {
+      log.error("Container already exists on disk. File: {}", containerFile
+          .toPath());
       throw new FileAlreadyExistsException("container already exists on " +
           "disk.");
     }
-
-    File parentPath = new File(containerFile.getParent());
-
-    if (!parentPath.exists() && !parentPath.mkdirs()) {
-      log.error("Unable to create parent path. Path: {}",
-          parentPath.toString());
-      throw new IOException("Unable to create container directory.");
-    }
-
-    if (!containerFile.createNewFile()) {
-      log.error("creation of a new container file failed. File: {}",
-          containerFile.toPath());
-      throw new IOException("creation of a new container file failed.");
-    }
-
   }
 
   public static String getContainerDbFileName(String containerName) {
@@ -234,53 +209,6 @@ public final class ContainerUtils {
   }
 
   /**
-   * creates a Metadata DB for the specified container.
-   *
-   * @param containerPath - Container Path.
-   * @throws IOException
-   */
-  public static Path createMetadata(Path containerPath, String containerName,
-      Configuration conf)
-      throws IOException {
-    Logger log = LoggerFactory.getLogger(ContainerManagerImpl.class);
-    Preconditions.checkNotNull(containerPath);
-    Path metadataPath = containerPath.resolve(OzoneConsts.CONTAINER_META_PATH);
-    if (!metadataPath.toFile().mkdirs()) {
-      log.error("Unable to create directory for metadata storage. Path: {}",
-          metadataPath);
-      throw new IOException("Unable to create directory for metadata storage." +
-          " Path: " + metadataPath);
-    }
-    MetadataStore store = MetadataStoreBuilder.newBuilder()
-        .setConf(conf)
-        .setCreateIfMissing(true)
-        .setDbFile(metadataPath
-            .resolve(getContainerDbFileName(containerName)).toFile())
-        .build();
-
-    // we close since the SCM pre-creates containers.
-    // we will open and put Db handle into a cache when keys are being created
-    // in a container.
-
-    store.close();
-
-    Path dataPath = containerPath.resolve(OzoneConsts.CONTAINER_DATA_PATH);
-    if (!dataPath.toFile().mkdirs()) {
-
-      // If we failed to create data directory, we cleanup the
-      // metadata directory completely. That is, we will delete the
-      // whole directory including LevelDB file.
-      log.error("Unable to create directory for data storage. cleaning up the" +
-              " container path: {} dataPath: {}",
-          containerPath, dataPath);
-      FileUtils.deleteDirectory(containerPath.toFile());
-      throw new IOException("Unable to create directory for data storage." +
-          " Path: " + dataPath);
-    }
-    return metadataPath;
-  }
-
-  /**
    * Returns container file location.
    *
    * @param containerData - Data
@@ -295,93 +223,6 @@ public final class ContainerUtils {
   }
 
   /**
-   * Container metadata directory -- here is where the level DB lives.
-   *
-   * @param cData - cData.
-   * @return Path to the parent directory where the DB lives.
-   */
-  public static Path getMetadataDirectory(ContainerData cData) {
-    Path dbPath = Paths.get(cData.getDBPath());
-    Preconditions.checkNotNull(dbPath);
-    Preconditions.checkState(dbPath.toString().length() > 0);
-    return dbPath.getParent();
-  }
-
-  /**
-   * Returns the path where data or chunks live for a given container.
-   *
-   * @param cData - cData container
-   * @return - Path
-   * @throws StorageContainerException
-   */
-  public static Path getDataDirectory(ContainerData cData)
-      throws StorageContainerException {
-    Path path = getMetadataDirectory(cData);
-    Preconditions.checkNotNull(path);
-    Path parentPath = path.getParent();
-    if (parentPath == null) {
-      throw new StorageContainerException("Unable to get Data directory."
-          + path, UNABLE_TO_FIND_DATA_DIR);
-    }
-    return parentPath.resolve(OzoneConsts.CONTAINER_DATA_PATH);
-  }
-
-  /**
-   * remove Container if it is empty.
-   * <p/>
-   * There are three things we need to delete.
-   * <p/>
-   * 1. Container file and metadata file. 2. The Level DB file 3. The path that
-   * we created on the data location.
-   *
-   * @param containerData - Data of the container to remove.
-   * @param conf - configuration of the cluster.
-   * @param forceDelete - whether this container should be deleted forcibly.
-   * @throws IOException
-   */
-  public static void removeContainer(ContainerData containerData,
-      Configuration conf, boolean forceDelete) throws IOException {
-    Preconditions.checkNotNull(containerData);
-    Path dbPath = Paths.get(containerData.getDBPath());
-
-    MetadataStore db = KeyUtils.getDB(containerData, conf);
-    // If the container is not empty and cannot be deleted forcibly,
-    // then throw a SCE to stop deleting.
-    if(!forceDelete && !db.isEmpty()) {
-      throw new StorageContainerException(
-          "Container cannot be deleted because it is not empty.",
-          Result.ERROR_CONTAINER_NOT_EMPTY);
-    }
-    // Close the DB connection and remove the DB handler from cache
-    KeyUtils.removeDB(containerData, conf);
-
-    // Delete the DB File.
-    FileUtils.forceDelete(dbPath.toFile());
-    dbPath = dbPath.getParent();
-
-    // Delete all Metadata in the Data directories for this containers.
-    if (dbPath != null) {
-      FileUtils.deleteDirectory(dbPath.toFile());
-      dbPath = dbPath.getParent();
-    }
-
-    // now delete the container directory, this means that all key data dirs
-    // will be removed too.
-    if (dbPath != null) {
-      FileUtils.deleteDirectory(dbPath.toFile());
-    }
-
-    // Delete the container metadata from the metadata locations.
-    String rootPath = getContainerNameFromFile(new File(containerData
-        .getContainerPath()));
-    Path containerPath = Paths.get(rootPath.concat(CONTAINER_EXTENSION));
-
-
-    FileUtils.forceDelete(containerPath.toFile());
-
-  }
-
-  /**
    * Persistent a {@link DatanodeDetails} to a local file.
    *
    * @throws IOException when read/write error occurs

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java
deleted file mode 100644
index a710864..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyUtils.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.hadoop.ozone.container.common.helpers;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.scm.container.common.helpers
-    .StorageContainerException;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
-import org.apache.hadoop.utils.MetadataStore;
-
-import java.io.IOException;
-import java.nio.charset.Charset;
-
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.NO_SUCH_KEY;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.UNABLE_TO_READ_METADATA_DB;
-
-/**
- * Utils functions to help key functions.
- */
-public final class KeyUtils {
-  public static final String ENCODING_NAME = "UTF-8";
-  public static final Charset ENCODING = Charset.forName(ENCODING_NAME);
-
-  /**
-   * Never Constructed.
-   */
-  private KeyUtils() {
-  }
-
-  /**
-   * Get a DB handler for a given container.
-   * If the handler doesn't exist in cache yet, first create one and
-   * add into cache. This function is called with containerManager
-   * ReadLock held.
-   *
-   * @param container container.
-   * @param conf configuration.
-   * @return MetadataStore handle.
-   * @throws StorageContainerException
-   */
-  public static MetadataStore getDB(ContainerData container,
-      Configuration conf) throws StorageContainerException {
-    Preconditions.checkNotNull(container);
-    ContainerCache cache = ContainerCache.getInstance(conf);
-    Preconditions.checkNotNull(cache);
-    try {
-      return cache.getDB(container.getContainerID(), container
-          .getContainerDBType(), container.getDBPath());
-    } catch (IOException ex) {
-      String message =
-          String.format("Unable to open DB. DB Name: %s, Path: %s. ex: %s",
-          container.getContainerID(), container.getDBPath(), ex.getMessage());
-      throw new StorageContainerException(message, UNABLE_TO_READ_METADATA_DB);
-    }
-  }
-
-  /**
-   * Remove a DB handler from cache.
-   *
-   * @param container - Container data.
-   * @param conf - Configuration.
-   */
-  public static void removeDB(ContainerData container,
-      Configuration conf) {
-    Preconditions.checkNotNull(container);
-    ContainerCache cache = ContainerCache.getInstance(conf);
-    Preconditions.checkNotNull(cache);
-    cache.removeDB(container.getContainerID());
-  }
-  /**
-   * Shutdown all DB Handles.
-   *
-   * @param cache - Cache for DB Handles.
-   */
-  @SuppressWarnings("unchecked")
-  public static void shutdownCache(ContainerCache cache)  {
-    cache.shutdownCache();
-  }
-
-  /**
-   * Returns successful keyResponse.
-   * @param msg - Request.
-   * @return Response.
-   */
-  public static ContainerProtos.ContainerCommandResponseProto
-      getKeyResponse(ContainerProtos.ContainerCommandRequestProto msg) {
-    return ContainerUtils.getSuccessResponse(msg);
-  }
-
-
-  public static ContainerProtos.ContainerCommandResponseProto
-      getKeyDataResponse(ContainerProtos.ContainerCommandRequestProto msg,
-      KeyData data) {
-    ContainerProtos.GetKeyResponseProto.Builder getKey = ContainerProtos
-        .GetKeyResponseProto.newBuilder();
-    getKey.setKeyData(data.getProtoBufMessage());
-    ContainerProtos.ContainerCommandResponseProto.Builder builder =
-        ContainerUtils.getSuccessResponseBuilder(msg);
-    builder.setGetKey(getKey);
-    return  builder.build();
-  }
-
-  /**
-   * Parses the key name from a bytes array.
-   * @param bytes key name in bytes.
-   * @return key name string.
-   */
-  public static String getKeyName(byte[] bytes) {
-    return new String(bytes, ENCODING);
-  }
-
-  /**
-   * Parses the {@link KeyData} from a bytes array.
-   *
-   * @param bytes key data in bytes.
-   * @return key data.
-   * @throws IOException if the bytes array is malformed or invalid.
-   */
-  public static KeyData getKeyData(byte[] bytes) throws IOException {
-    try {
-      ContainerProtos.KeyData kd = ContainerProtos.KeyData.parseFrom(bytes);
-      KeyData data = KeyData.getFromProtoBuf(kd);
-      return data;
-    } catch (IOException e) {
-      throw new StorageContainerException("Failed to parse key data from the" +
-              " bytes array.", NO_SUCH_KEY);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkManagerImpl.java
deleted file mode 100644
index fa82026..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkManagerImpl.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.hadoop.ozone.container.common.impl;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdds.scm.container.common.helpers
-    .StorageContainerException;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
-import org.apache.hadoop.ozone.container.common.helpers.ChunkUtils;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
-import org.apache.hadoop.ozone.container.common.interfaces.ChunkManager;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.file.Files;
-import java.nio.file.StandardCopyOption;
-import java.security.NoSuchAlgorithmException;
-import java.util.concurrent.ExecutionException;
-
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.CONTAINER_INTERNAL_ERROR;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.UNSUPPORTED_REQUEST;
-
-/**
- * An implementation of ChunkManager that is used by default in ozone.
- */
-public class ChunkManagerImpl implements ChunkManager {
-  static final Logger LOG =
-      LoggerFactory.getLogger(ChunkManagerImpl.class);
-
-  private final ContainerManager containerManager;
-
-  /**
-   * Constructs a ChunkManager.
-   *
-   * @param manager - ContainerManager.
-   */
-  public ChunkManagerImpl(ContainerManager manager) {
-    this.containerManager = manager;
-  }
-
-  /**
-   * writes a given chunk.
-   *
-   * @param blockID - ID of the block.
-   * @param info - ChunkInfo.
-   * @throws StorageContainerException
-   */
-  @Override
-  public void writeChunk(BlockID blockID, ChunkInfo info,
-      byte[] data, ContainerProtos.Stage stage)
-      throws StorageContainerException {
-    // we don't want container manager to go away while we are writing chunks.
-    containerManager.readLock();
-
-    // TODO : Take keyManager Write lock here.
-    try {
-      Preconditions.checkNotNull(blockID, "Block ID cannot be null.");
-      long containerID = blockID.getContainerID();
-      Preconditions.checkState(containerID >= 0,
-          "Container ID cannot be negative");
-      ContainerData container =
-          containerManager.readContainer(containerID);
-      File chunkFile = ChunkUtils.validateChunk(container, info);
-      File tmpChunkFile = getTmpChunkFile(chunkFile, info);
-
-      LOG.debug("writing chunk:{} chunk stage:{} chunk file:{} tmp chunk file",
-          info.getChunkName(), stage, chunkFile, tmpChunkFile);
-      switch (stage) {
-      case WRITE_DATA:
-        ChunkUtils.writeData(tmpChunkFile, info, data);
-        break;
-      case COMMIT_DATA:
-        commitChunk(tmpChunkFile, chunkFile, containerID, info.getLen());
-        break;
-      case COMBINED:
-        // directly write to the chunk file
-        long oldSize = chunkFile.length();
-        ChunkUtils.writeData(chunkFile, info, data);
-        long newSize = chunkFile.length();
-        containerManager.incrBytesUsed(containerID, newSize - oldSize);
-        containerManager.incrWriteCount(containerID);
-        containerManager.incrWriteBytes(containerID, info.getLen());
-        break;
-      default:
-        throw new IOException("Can not identify write operation.");
-      }
-    } catch (ExecutionException | NoSuchAlgorithmException | IOException e) {
-      LOG.error("write data failed. error: {}", e);
-      throw new StorageContainerException("Internal error: ", e,
-          CONTAINER_INTERNAL_ERROR);
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      LOG.error("write data failed. error: {}", e);
-      throw new StorageContainerException("Internal error: ", e,
-          CONTAINER_INTERNAL_ERROR);
-    } finally {
-      containerManager.readUnlock();
-    }
-  }
-
-  // Create a temporary file in the same container directory
-  // in the format "<chunkname>.tmp"
-  private static File getTmpChunkFile(File chunkFile, ChunkInfo info)
-      throws StorageContainerException {
-    return new File(chunkFile.getParent(),
-        chunkFile.getName() +
-            OzoneConsts.CONTAINER_CHUNK_NAME_DELIMITER +
-            OzoneConsts.CONTAINER_TEMPORARY_CHUNK_PREFIX);
-  }
-
-  // Commit the chunk by renaming the temporary chunk file to chunk file
-  private void commitChunk(File tmpChunkFile, File chunkFile,
-      long containerID, long chunkLen) throws IOException {
-    long sizeDiff = tmpChunkFile.length() - chunkFile.length();
-    // It is safe to replace here as the earlier chunk if existing should be
-    // caught as part of validateChunk
-    Files.move(tmpChunkFile.toPath(), chunkFile.toPath(),
-        StandardCopyOption.REPLACE_EXISTING);
-    containerManager.incrBytesUsed(containerID, sizeDiff);
-    containerManager.incrWriteCount(containerID);
-    containerManager.incrWriteBytes(containerID, chunkLen);
-  }
-
-  /**
-   * reads the data defined by a chunk.
-   *
-   * @param blockID - ID of the block.
-   * @param info - ChunkInfo.
-   * @return byte array
-   * @throws StorageContainerException
-   * TODO: Right now we do not support partial reads and writes of chunks.
-   * TODO: Explore if we need to do that for ozone.
-   */
-  @Override
-  public byte[] readChunk(BlockID blockID, ChunkInfo info)
-      throws StorageContainerException {
-    containerManager.readLock();
-    try {
-      Preconditions.checkNotNull(blockID, "Block ID cannot be null.");
-      long containerID = blockID.getContainerID();
-      Preconditions.checkState(containerID >= 0,
-          "Container ID cannot be negative");
-      ContainerData container =
-          containerManager.readContainer(containerID);
-      File chunkFile = ChunkUtils.getChunkFile(container, info);
-      ByteBuffer data =  ChunkUtils.readData(chunkFile, info);
-      containerManager.incrReadCount(containerID);
-      containerManager.incrReadBytes(containerID, chunkFile.length());
-      return data.array();
-    } catch (ExecutionException | NoSuchAlgorithmException e) {
-      LOG.error("read data failed. error: {}", e);
-      throw new StorageContainerException("Internal error: ",
-          e, CONTAINER_INTERNAL_ERROR);
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      LOG.error("read data failed. error: {}", e);
-      throw new StorageContainerException("Internal error: ",
-          e, CONTAINER_INTERNAL_ERROR);
-    } finally {
-      containerManager.readUnlock();
-    }
-  }
-
-  /**
-   * Deletes a given chunk.
-   *
-   * @param blockID - ID of the block.
-   * @param info - Chunk Info
-   * @throws StorageContainerException
-   */
-  @Override
-  public void deleteChunk(BlockID blockID, ChunkInfo info)
-      throws StorageContainerException {
-    containerManager.readLock();
-    try {
-      Preconditions.checkNotNull(blockID, "Block ID cannot be null.");
-      long containerID = blockID.getContainerID();
-      Preconditions.checkState(containerID >= 0,
-          "Container ID cannot be negative");
-
-      File chunkFile = ChunkUtils.getChunkFile(containerManager
-          .readContainer(containerID), info);
-      if ((info.getOffset() == 0) && (info.getLen() == chunkFile.length())) {
-        FileUtil.fullyDelete(chunkFile);
-        containerManager.decrBytesUsed(containerID, chunkFile.length());
-      } else {
-        LOG.error("Not Supported Operation. Trying to delete a " +
-            "chunk that is in shared file. chunk info : " + info.toString());
-        throw new StorageContainerException("Not Supported Operation. " +
-            "Trying to delete a chunk that is in shared file. chunk info : "
-            + info.toString(), UNSUPPORTED_REQUEST);
-      }
-    } finally {
-      containerManager.readUnlock();
-    }
-  }
-
-  /**
-   * Shutdown the chunkManager.
-   *
-   * In the chunkManager we haven't acquired any resources, so nothing to do
-   * here. This call is made with containerManager Writelock held.
-   */
-  @Override
-  public void shutdown() {
-    Preconditions.checkState(this.containerManager.hasWriteLock());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
index 238fb09..5638b60 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.ozone.container.common.impl;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
     ContainerType;
@@ -28,6 +29,7 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.Map;
 import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
@@ -41,7 +43,10 @@ public class ContainerData {
   private final ContainerType containerType;
 
   // Unique identifier for the container
-  private final long containerId;
+  private final long containerID;
+
+  // Path to container root dir.
+  private String containerPath;
 
   // Layout version of the container data
   private final int layOutVersion;
@@ -65,6 +70,10 @@ public class ContainerData {
 
   private HddsVolume volume;
 
+  /**
+   * Number of pending deletion blocks in container.
+   */
+  private final AtomicInteger numPendingDeletionBlocks;
 
   /**
    * Creates a ContainerData Object, which holds metadata of the container.
@@ -73,18 +82,8 @@ public class ContainerData {
    * @param size - container maximum size
    */
   public ContainerData(ContainerType type, long containerId, int size) {
-    this.containerType = type;
-    this.containerId = containerId;
-    this.layOutVersion = ChunkLayOutVersion.getLatestVersion().getVersion();
-    this.metadata = new TreeMap<>();
-    this.state = ContainerLifeCycleState.OPEN;
-    this.readCount = new AtomicLong(0L);
-    this.readBytes =  new AtomicLong(0L);
-    this.writeCount =  new AtomicLong(0L);
-    this.writeBytes =  new AtomicLong(0L);
-    this.bytesUsed = new AtomicLong(0L);
-    this.keyCount = new AtomicLong(0L);
-    this.maxSizeGB = size;
+    this(type, containerId,
+        ChunkLayOutVersion.getLatestVersion().getVersion(), size);
   }
 
   /**
@@ -94,10 +93,12 @@ public class ContainerData {
    * @param layOutVersion - Container layOutVersion
    * @param size - Container maximum size
    */
-  public ContainerData(ContainerType type, long containerId, int
-      layOutVersion, int size) {
+  public ContainerData(ContainerType type, long containerId,
+    int layOutVersion, int size) {
+    Preconditions.checkNotNull(type);
+
     this.containerType = type;
-    this.containerId = containerId;
+    this.containerID = containerId;
     this.layOutVersion = layOutVersion;
     this.metadata = new TreeMap<>();
     this.state = ContainerLifeCycleState.OPEN;
@@ -108,13 +109,30 @@ public class ContainerData {
     this.bytesUsed = new AtomicLong(0L);
     this.keyCount = new AtomicLong(0L);
     this.maxSizeGB = size;
+    this.numPendingDeletionBlocks = new AtomicInteger(0);
   }
 
   /**
-   * Returns the containerId.
+   * Returns the containerID.
    */
-  public long getContainerId() {
-    return containerId;
+  public long getContainerID() {
+    return containerID;
+  }
+
+  /**
+   * Returns the path to base dir of the container.
+   * @return Path to base dir.
+   */
+  public String getContainerPath() {
+    return containerPath;
+  }
+
+  /**
+   * Set the base dir path of the container.
+   * @param baseDir path to base dir
+   */
+  public void setContainerPath(String baseDir) {
+    this.containerPath = baseDir;
   }
 
   /**
@@ -163,9 +181,6 @@ public class ContainerData {
    */
   public void addMetadata(String key, String value) throws IOException {
     synchronized (this.metadata) {
-      if (this.metadata.containsKey(key)) {
-        throw new IOException("This key already exists. Key " + key);
-      }
       metadata.put(key, value);
     }
   }
@@ -299,7 +314,6 @@ public class ContainerData {
     return this.bytesUsed.addAndGet(used);
   }
 
-
   /**
    * Decrease the number of bytes used by the container.
    * @param reclaimed the number of bytes reclaimed from the container.
@@ -356,4 +370,75 @@ public class ContainerData {
     this.keyCount.set(count);
   }
 
+  /**
+   * Returns container metadata path.
+   */
+  public String getMetadataPath() {
+    return null;
+  }
+
+  /**
+   * Returns container data path.
+   */
+  public String getDataPath() {
+    return null;
+  }
+
+  /**
+   * Increase the count of pending deletion blocks.
+   *
+   * @param numBlocks increment number
+   */
+  public void incrPendingDeletionBlocks(int numBlocks) {
+    this.numPendingDeletionBlocks.addAndGet(numBlocks);
+  }
+
+  /**
+   * Decrease the count of pending deletion blocks.
+   *
+   * @param numBlocks decrement number
+   */
+  public void decrPendingDeletionBlocks(int numBlocks) {
+    this.numPendingDeletionBlocks.addAndGet(-1 * numBlocks);
+  }
+
+  /**
+   * Get the number of pending deletion blocks.
+   */
+  public int getNumPendingDeletionBlocks() {
+    return this.numPendingDeletionBlocks.get();
+  }
+
+  /**
+   * Returns a ProtoBuf Message from ContainerData.
+   *
+   * @return Protocol Buffer Message
+   */
+  public ContainerProtos.ContainerData getProtoBufMessage() {
+    ContainerProtos.ContainerData.Builder builder =
+        ContainerProtos.ContainerData.newBuilder();
+
+    builder.setContainerID(this.getContainerID());
+
+    if (this.containerPath != null) {
+      builder.setContainerPath(this.containerPath);
+    }
+
+    builder.setState(this.getState());
+
+    for (Map.Entry<String, String> entry : metadata.entrySet()) {
+      ContainerProtos.KeyValue.Builder keyValBuilder =
+          ContainerProtos.KeyValue.newBuilder();
+      builder.addMetadata(keyValBuilder.setKey(entry.getKey())
+          .setValue(entry.getValue()).build());
+    }
+
+    if (this.getBytesUsed() >= 0) {
+      builder.setBytesUsed(this.getBytesUsed());
+    }
+
+    builder.setContainerType(containerType);
+
+    return builder.build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
index 4f4d588..70d1615 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.ozone.container.common.impl;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.yaml.snakeyaml.Yaml;
 
@@ -48,7 +50,8 @@ import org.yaml.snakeyaml.nodes.ScalarNode;
 import org.yaml.snakeyaml.nodes.Tag;
 import org.yaml.snakeyaml.representer.Representer;
 
-import static org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData.YAML_TAG;
+import static org.apache.hadoop.ozone.container.keyvalue
+    .KeyValueContainerData.KEYVALUE_YAML_TAG;
 
 /**
  * Class for creating and reading .container files.
@@ -84,7 +87,7 @@ public final class ContainerDataYaml {
       Representer representer = new ContainerDataRepresenter();
       representer.setPropertyUtils(propertyUtils);
       representer.addClassTag(KeyValueContainerData.class,
-          KeyValueContainerData.YAML_TAG);
+          KeyValueContainerData.KEYVALUE_YAML_TAG);
 
       Constructor keyValueDataConstructor = new ContainerDataConstructor();
 
@@ -171,7 +174,8 @@ public final class ContainerDataYaml {
       //Adding our own specific constructors for tags.
       // When a new Container type is added, we need to add yamlConstructor
       // for that
-      this.yamlConstructors.put(YAML_TAG, new ConstructKeyValueContainerData());
+      this.yamlConstructors.put(
+          KEYVALUE_YAML_TAG, new ConstructKeyValueContainerData());
       this.yamlConstructors.put(Tag.INT, new ConstructLong());
     }
 
@@ -181,20 +185,21 @@ public final class ContainerDataYaml {
         Map<Object, Object> nodes = constructMapping(mnode);
 
         //Needed this, as TAG.INT type is by default converted to Long.
-        long layOutVersion = (long) nodes.get("layOutVersion");
+        long layOutVersion = (long) nodes.get(OzoneConsts.LAYOUTVERSION);
         int lv = (int) layOutVersion;
 
-        long size = (long) nodes.get("maxSizeGB");
+        long size = (long) nodes.get(OzoneConsts.MAX_SIZE_GB);
         int maxSize = (int) size;
 
         //When a new field is added, it needs to be added here.
         KeyValueContainerData kvData = new KeyValueContainerData((long) nodes
-            .get("containerId"), lv, maxSize);
-        kvData.setContainerDBType((String)nodes.get("containerDBType"));
+            .get(OzoneConsts.CONTAINER_ID), lv, maxSize);
+        kvData.setContainerDBType((String)nodes.get(
+            OzoneConsts.CONTAINER_DB_TYPE));
         kvData.setMetadataPath((String) nodes.get(
-            "metadataPath"));
-        kvData.setChunksPath((String) nodes.get("chunksPath"));
-        Map<String, String> meta = (Map) nodes.get("metadata");
+            OzoneConsts.METADATA_PATH));
+        kvData.setChunksPath((String) nodes.get(OzoneConsts.CHUNKS_PATH));
+        Map<String, String> meta = (Map) nodes.get(OzoneConsts.METADATA);
         meta.forEach((key, val) -> {
           try {
             kvData.addMetadata(key, val);
@@ -204,7 +209,7 @@ public final class ContainerDataYaml {
                 "for containerId " + (long) nodes.get("containerId"));
           }
         });
-        String state = (String) nodes.get("state");
+        String state = (String) nodes.get(OzoneConsts.STATE);
         switch (state) {
         case "OPEN":
           kvData.setState(ContainerProtos.ContainerLifeCycleState.OPEN);
@@ -218,7 +223,7 @@ public final class ContainerDataYaml {
         default:
           throw new IllegalStateException("Unexpected " +
               "ContainerLifeCycleState " + state + " for the containerId " +
-              (long) nodes.get("containerId"));
+              (long) nodes.get(OzoneConsts.CONTAINER_ID));
         }
         return kvData;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerLocationManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerLocationManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerLocationManagerImpl.java
deleted file mode 100644
index 5f5b81f..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerLocationManagerImpl.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.impl;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
-import org.apache.hadoop.metrics2.util.MBeans;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.container.common.interfaces
-    .ContainerLocationManager;
-import org.apache.hadoop.ozone.container.common.interfaces
-    .ContainerLocationManagerMXBean;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.management.ObjectName;
-import java.io.IOException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.LinkedList;
-import java.util.List;
-
-/**
- * A class that tells the ContainerManager where to place the containers.
- * Please note : There is *no* one-to-one correlation between metadata
- * Locations and data Locations.
- *
- *  For example : A user could map all container files to a
- *  SSD but leave data/metadata on bunch of other disks.
- */
-public class ContainerLocationManagerImpl implements ContainerLocationManager,
-    ContainerLocationManagerMXBean {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(ContainerLocationManagerImpl.class);
-
-  private final List<ContainerStorageLocation> dataLocations;
-  private int currentIndex;
-  private final List<StorageLocation> metadataLocations;
-  private final ObjectName jmxbean;
-
-  /**
-   * Constructs a Location Manager.
-   * @param metadataLocations  - Refers to the metadataLocations
-   * where we store the container metadata.
-   * @param dataDirs - metadataLocations where we store the actual
-   * data or chunk files.
-   * @param conf - configuration.
-   * @throws IOException
-   */
-  public ContainerLocationManagerImpl(List<StorageLocation> metadataLocations,
-      List<StorageLocation> dataDirs, Configuration conf)
-      throws IOException {
-    dataLocations = new LinkedList<>();
-    for (StorageLocation dataDir : dataDirs) {
-      dataLocations.add(new ContainerStorageLocation(dataDir, conf));
-    }
-    this.metadataLocations = metadataLocations;
-    jmxbean = MBeans.register("OzoneDataNode",
-        ContainerLocationManager.class.getSimpleName(), this);
-  }
-
-  /**
-   * Returns the path where the container should be placed from a set of
-   * metadataLocations.
-   *
-   * @return A path where we should place this container and metadata.
-   * @throws IOException
-   */
-  @Override
-  public Path getContainerPath()
-      throws IOException {
-    Preconditions.checkState(metadataLocations.size() > 0);
-    int index = currentIndex % metadataLocations.size();
-    return Paths.get(metadataLocations.get(index).getNormalizedUri());
-  }
-
-  /**
-   * Returns the path where the container Data file are stored.
-   *
-   * @return  a path where we place the LevelDB and data files of a container.
-   * @throws IOException
-   */
-  @Override
-  public Path getDataPath(String containerName) throws IOException {
-    Path currentPath = Paths.get(
-        dataLocations.get(currentIndex++ % dataLocations.size())
-            .getNormalizedUri());
-    currentPath = currentPath.resolve(OzoneConsts.CONTAINER_PREFIX);
-    return currentPath.resolve(containerName);
-  }
-
-  @Override
-  public StorageLocationReport[] getLocationReport() throws IOException {
-    boolean failed;
-    StorageLocationReport[] reports =
-        new StorageLocationReport[dataLocations.size()];
-    for (int idx = 0; idx < dataLocations.size(); idx++) {
-      ContainerStorageLocation loc = dataLocations.get(idx);
-      long scmUsed = 0;
-      long remaining = 0;
-      failed = false;
-      try {
-        scmUsed = loc.getScmUsed();
-        remaining = loc.getAvailable();
-      } catch (IOException ex) {
-        LOG.warn("Failed to get scmUsed and remaining for container " +
-            "storage location {}", loc.getNormalizedUri());
-        // reset scmUsed and remaining if df/du failed.
-        scmUsed = 0;
-        remaining = 0;
-        failed = true;
-      }
-
-      StorageLocationReport.Builder builder =
-          StorageLocationReport.newBuilder();
-      builder.setStorageLocation(loc.getStorageLocation())
-          .setId(loc.getStorageUuId())
-          .setFailed(failed)
-          .setCapacity(loc.getCapacity())
-          .setRemaining(remaining)
-          .setScmUsed(scmUsed)
-          .setStorageType(loc.getStorageType());
-      StorageLocationReport r = builder.build();
-      reports[idx] = r;
-    }
-    return reports;
-  }
-
-  /**
-   * Supports clean shutdown of container location du threads.
-   *
-   * @throws IOException
-   */
-  @Override
-  public void shutdown() throws IOException {
-    for (ContainerStorageLocation loc: dataLocations) {
-      loc.shutdown();
-    }
-    MBeans.unregister(jmxbean);
-  }
-}


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


[34/37] hadoop git commit: HDDS-213. Single lock to synchronize KeyValueContainer#update.

Posted by bh...@apache.org.
HDDS-213. Single lock to synchronize KeyValueContainer#update.


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

Branch: refs/heads/trunk
Commit: 44e19fc7f70b5c19f2b626fe247aea5d51ada51c
Parents: cb9574a
Author: Hanisha Koneru <ha...@apache.org>
Authored: Mon Jul 9 09:33:09 2018 -0700
Committer: Hanisha Koneru <ha...@apache.org>
Committed: Mon Jul 9 09:33:09 2018 -0700

----------------------------------------------------------------------
 .../container/common/impl/ContainerData.java    |  28 +++--
 .../common/impl/ContainerDataYaml.java          |  10 +-
 .../container/keyvalue/KeyValueContainer.java   | 124 +++++++------------
 .../container/ozoneimpl/ContainerReader.java    |  37 +++---
 4 files changed, 87 insertions(+), 112 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/44e19fc7/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
index 0d217e4..54b186b 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
@@ -182,12 +182,14 @@ public class ContainerData {
   }
 
   /**
-   * Adds metadata.
+   * Add/Update metadata.
+   * We should hold the container lock before updating the metadata as this
+   * will be persisted on disk. Unless, we are reconstructing ContainerData
+   * from protoBuf or from on disk .container file in which case lock is not
+   * required.
    */
-  public void addMetadata(String key, String value) throws IOException {
-    synchronized (this.metadata) {
-      metadata.put(key, value);
-    }
+  public void addMetadata(String key, String value) {
+    metadata.put(key, value);
   }
 
   /**
@@ -195,9 +197,19 @@ public class ContainerData {
    * @return metadata
    */
   public Map<String, String> getMetadata() {
-    synchronized (this.metadata) {
-      return Collections.unmodifiableMap(this.metadata);
-    }
+    return Collections.unmodifiableMap(this.metadata);
+  }
+
+  /**
+   * Set metadata.
+   * We should hold the container lock before updating the metadata as this
+   * will be persisted on disk. Unless, we are reconstructing ContainerData
+   * from protoBuf or from on disk .container file in which case lock is not
+   * required.
+   */
+  public void setMetadata(Map<String, String> metadataMap) {
+    metadata.clear();
+    metadata.putAll(metadataMap);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44e19fc7/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
index 70d1615..90af24f 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
@@ -200,15 +200,7 @@ public final class ContainerDataYaml {
             OzoneConsts.METADATA_PATH));
         kvData.setChunksPath((String) nodes.get(OzoneConsts.CHUNKS_PATH));
         Map<String, String> meta = (Map) nodes.get(OzoneConsts.METADATA);
-        meta.forEach((key, val) -> {
-          try {
-            kvData.addMetadata(key, val);
-          } catch (IOException e) {
-            throw new IllegalStateException("Unexpected " +
-                "Key Value Pair " + "(" + key + "," + val +")in the metadata " +
-                "for containerId " + (long) nodes.get("containerId"));
-          }
-        });
+        kvData.setMetadata(meta);
         String state = (String) nodes.get(OzoneConsts.STATE);
         switch (state) {
         case "OPEN":

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44e19fc7/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index b07b053..155a988 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.ozone.container.keyvalue;
 
 import com.google.common.base.Preconditions;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileUtil;
@@ -32,7 +34,6 @@ import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
-import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
@@ -59,8 +60,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.CONTAINER_ALREADY_EXISTS;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.CONTAINER_METADATA_ERROR;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.CONTAINER_INTERNAL_ERROR;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.CONTAINER_FILES_CREATE_ERROR;
@@ -146,7 +145,7 @@ public class KeyValueContainer implements Container {
       containerData.setVolume(containerVolume);
 
       // Create .container file and .chksm file
-      createContainerFile(containerFile, containerCheckSumFile);
+      writeToContainerFile(containerFile, containerCheckSumFile, true);
 
 
     } catch (StorageContainerException ex) {
@@ -177,36 +176,50 @@ public class KeyValueContainer implements Container {
    * Creates .container file and checksum file.
    *
    * @param containerFile
-   * @param containerCheckSumFile
+   * @param checksumFile
+   * @param isCreate true if we are creating a new container file and false if
+   *                we are updating an existing container file.
    * @throws StorageContainerException
    */
-  private void createContainerFile(File containerFile, File
-      containerCheckSumFile) throws StorageContainerException {
+  private void writeToContainerFile(File containerFile, File
+      checksumFile, boolean isCreate)
+      throws StorageContainerException {
     File tempContainerFile = null;
-    File tempCheckSumFile = null;
+    File tempChecksumFile = null;
     FileOutputStream containerCheckSumStream = null;
     Writer writer = null;
     long containerId = containerData.getContainerID();
     try {
       tempContainerFile = createTempFile(containerFile);
-      tempCheckSumFile = createTempFile(containerCheckSumFile);
+      tempChecksumFile = createTempFile(checksumFile);
       ContainerDataYaml.createContainerFile(ContainerProtos.ContainerType
               .KeyValueContainer, tempContainerFile, containerData);
 
       //Compute Checksum for container file
       String checksum = KeyValueContainerUtil.computeCheckSum(containerId,
           tempContainerFile);
-      containerCheckSumStream = new FileOutputStream(tempCheckSumFile);
+      containerCheckSumStream = new FileOutputStream(tempChecksumFile);
       writer = new OutputStreamWriter(containerCheckSumStream, "UTF-8");
       writer.write(checksum);
       writer.flush();
 
-      NativeIO.renameTo(tempContainerFile, containerFile);
-      NativeIO.renameTo(tempCheckSumFile, containerCheckSumFile);
+      if (isCreate) {
+        // When creating a new container, .container file should not exist
+        // already.
+        NativeIO.renameTo(tempContainerFile, containerFile);
+        NativeIO.renameTo(tempChecksumFile, checksumFile);
+      } else {
+        // When updating a container, the .container file should exist. If
+        // not, the container is in an inconsistent state.
+        Files.move(tempContainerFile.toPath(), containerFile.toPath(),
+            StandardCopyOption.REPLACE_EXISTING);
+        Files.move(tempChecksumFile.toPath(), checksumFile.toPath(),
+            StandardCopyOption.REPLACE_EXISTING);
+      }
 
     } catch (IOException ex) {
       throw new StorageContainerException("Error during creation of " +
-          "required files(.container, .chksm) for container. Container Name: "
+          "required files(.container, .chksm) for container. ContainerID: "
           + containerId, ex, CONTAINER_FILES_CREATE_ERROR);
     } finally {
       IOUtils.closeStream(containerCheckSumStream);
@@ -216,8 +229,8 @@ public class KeyValueContainer implements Container {
               tempContainerFile.getAbsolutePath());
         }
       }
-      if (tempCheckSumFile != null && tempCheckSumFile.exists()) {
-        if (!tempCheckSumFile.delete()) {
+      if (tempChecksumFile != null && tempChecksumFile.exists()) {
+        if (!tempChecksumFile.delete()) {
           LOG.warn("Unable to delete container temporary checksum file: {}.",
               tempContainerFile.getAbsolutePath());
         }
@@ -236,68 +249,24 @@ public class KeyValueContainer implements Container {
 
 
   private void updateContainerFile(File containerFile, File
-      containerCheckSumFile) throws StorageContainerException {
+      checksumFile) throws StorageContainerException {
 
-    File containerBkpFile = null;
-    File checkSumBkpFile = null;
     long containerId = containerData.getContainerID();
 
-    try {
-      if (containerFile.exists() && containerCheckSumFile.exists()) {
-        //Take backup of original files (.container and .chksm files)
-        containerBkpFile = new File(containerFile + ".bkp");
-        checkSumBkpFile = new File(containerCheckSumFile + ".bkp");
-        NativeIO.renameTo(containerFile, containerBkpFile);
-        NativeIO.renameTo(containerCheckSumFile, checkSumBkpFile);
-        createContainerFile(containerFile, containerCheckSumFile);
-      } else {
-        containerData.setState(ContainerProtos.ContainerLifeCycleState.INVALID);
-        throw new StorageContainerException("Container is an Inconsistent " +
-            "state, missing required files(.container, .chksm). ContainerID: " +
-            containerId, INVALID_CONTAINER_STATE);
-      }
-    } catch (StorageContainerException ex) {
-      throw ex;
-    } catch (IOException ex) {
-      // Restore from back up files.
+    if (containerFile.exists() && checksumFile.exists()) {
       try {
-        if (containerBkpFile != null && containerBkpFile
-            .exists() && containerFile.delete()) {
-          LOG.info("update failed for container Name: {}, restoring container" +
-              " file", containerId);
-          NativeIO.renameTo(containerBkpFile, containerFile);
-        }
-        if (checkSumBkpFile != null && checkSumBkpFile.exists() &&
-            containerCheckSumFile.delete()) {
-          LOG.info("update failed for container Name: {}, restoring checksum" +
-              " file", containerId);
-          NativeIO.renameTo(checkSumBkpFile, containerCheckSumFile);
-        }
-        throw new StorageContainerException("Error during updating of " +
-            "required files(.container, .chksm) for container. Container Name: "
-            + containerId, ex, CONTAINER_FILES_CREATE_ERROR);
+        writeToContainerFile(containerFile, checksumFile, false);
       } catch (IOException e) {
-        containerData.setState(ContainerProtos.ContainerLifeCycleState.INVALID);
-        LOG.error("During restore failed for container Name: " +
-            containerId);
-        throw new StorageContainerException(
-            "Failed to restore container data from the backup. ID: "
-                + containerId, CONTAINER_FILES_CREATE_ERROR);
-      }
-    } finally {
-      if (containerBkpFile != null && containerBkpFile
-          .exists()) {
-        if(!containerBkpFile.delete()) {
-          LOG.warn("Unable to delete container backup file: {}",
-              containerBkpFile);
-        }
-      }
-      if (checkSumBkpFile != null && checkSumBkpFile.exists()) {
-        if(!checkSumBkpFile.delete()) {
-          LOG.warn("Unable to delete container checksum backup file: {}",
-              checkSumBkpFile);
-        }
+        //TODO : Container update failure is not handled currently. Might
+        // lead to loss of .container file. When Update container feature
+        // support is added, this failure should also be handled.
+        throw new StorageContainerException("Container update failed. " +
+            "ContainerID: " + containerId, CONTAINER_FILES_CREATE_ERROR);
       }
+    } else {
+      throw new StorageContainerException("Container is an Inconsistent " +
+          "state, missing required files(.container, .chksm). ContainerID: " +
+          containerId, INVALID_CONTAINER_STATE);
     }
   }
 
@@ -393,22 +362,21 @@ public class KeyValueContainer implements Container {
           "Updating a closed container without force option is not allowed. " +
               "ContainerID: " + containerId, UNSUPPORTED_REQUEST);
     }
+
+    Map<String, String> oldMetadata = containerData.getMetadata();
     try {
+      writeLock();
       for (Map.Entry<String, String> entry : metadata.entrySet()) {
         containerData.addMetadata(entry.getKey(), entry.getValue());
       }
-    } catch (IOException ex) {
-      throw new StorageContainerException("Container Metadata update error" +
-          ". Container Name:" + containerId, ex, CONTAINER_METADATA_ERROR);
-    }
-    try {
-      writeLock();
-      String containerName = String.valueOf(containerId);
       File containerFile = getContainerFile();
       File containerCheckSumFile = getContainerCheckSumFile();
       // update the new container data to .container File
       updateContainerFile(containerFile, containerCheckSumFile);
     } catch (StorageContainerException  ex) {
+      // TODO:
+      // On error, reset the metadata.
+      containerData.setMetadata(oldMetadata);
       throw ex;
     } finally {
       writeUnlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44e19fc7/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
index b90efdc..06e49f0 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
@@ -109,25 +109,28 @@ public class ContainerReader implements Runnable {
         for (File containerTopDir : containerTopDirs) {
           if (containerTopDir.isDirectory()) {
             File[] containerDirs = containerTopDir.listFiles();
-            for (File containerDir : containerDirs) {
-              File metadataPath = new File(containerDir + File.separator +
-                  OzoneConsts.CONTAINER_META_PATH);
-              String containerName = containerDir.getName();
-              if (metadataPath.exists()) {
-                File containerFile = KeyValueContainerLocationUtil
-                    .getContainerFile(metadataPath, containerName);
-                File checksumFile = KeyValueContainerLocationUtil
-                    .getContainerCheckSumFile(metadataPath, containerName);
-                if (containerFile.exists() && checksumFile.exists()) {
-                  verifyContainerFile(containerName, containerFile,
-                      checksumFile);
+            if (containerDirs != null) {
+              for (File containerDir : containerDirs) {
+                File metadataPath = new File(containerDir + File.separator +
+                    OzoneConsts.CONTAINER_META_PATH);
+                String containerName = containerDir.getName();
+                if (metadataPath.exists()) {
+                  File containerFile = KeyValueContainerLocationUtil
+                      .getContainerFile(metadataPath, containerName);
+                  File checksumFile = KeyValueContainerLocationUtil
+                      .getContainerCheckSumFile(metadataPath, containerName);
+                  if (containerFile.exists() && checksumFile.exists()) {
+                    verifyContainerFile(containerName, containerFile,
+                        checksumFile);
+                  } else {
+                    LOG.error(
+                        "Missing container metadata files for Container: " +
+                            "{}", containerName);
+                  }
                 } else {
-                  LOG.error("Missing container metadata files for Container: " +
-                      "{}", containerName);
+                  LOG.error("Missing container metadata directory for " +
+                      "Container: {}", containerName);
                 }
-              } else {
-                LOG.error("Missing container metadata directory for " +
-                    "Container: {}", containerName);
               }
             }
           }


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


[10/37] hadoop git commit: HDDS-136:Rename dbPath, containerFilePath. Contributed by Bharat Viswanadham

Posted by bh...@apache.org.
HDDS-136:Rename dbPath,containerFilePath. Contributed by Bharat Viswanadham


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

Branch: refs/heads/trunk
Commit: 7e228e54c56598e263d66e3ef74476e12a3b1f30
Parents: 0e437f9
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Mon Jun 11 13:14:03 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Mon Jun 11 13:14:03 2018 -0700

----------------------------------------------------------------------
 .../common/impl/KeyValueContainerData.java      | 34 ++++++++++----------
 .../container/common/impl/KeyValueYaml.java     |  9 +++---
 .../common/TestKeyValueContainerData.java       |  8 ++---
 .../container/common/impl/TestKeyValueYaml.java | 16 ++++-----
 .../test/resources/additionalfields.container   |  7 ++--
 .../src/test/resources/incorrect.container      |  7 ++--
 6 files changed, 41 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e228e54/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
index 57b5264..0889913 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
@@ -30,11 +30,11 @@ import java.io.IOException;
  */
 public class KeyValueContainerData extends ContainerData {
 
-  // Path to Level DB/RocksDB Store.
-  private String dbPath;
+  // Path to Container metadata Level DB/RocksDB Store and .container file.
+  private String metadataPath;
 
-  // Path to Physical file system where container and checksum are stored.
-  private String containerFilePath;
+  // Path to Physical file system where chunks are stored.
+  private String chunksPath;
 
   //Type of DB used to store key to chunks mapping
   private String containerDBType;
@@ -64,37 +64,37 @@ public class KeyValueContainerData extends ContainerData {
     this.numPendingDeletionBlocks = 0;
   }
   /**
-   * Returns path.
+   * Returns container metadata path.
    *
    * @return - path
    */
-  public String getDbPath() {
-    return dbPath;
+  public String getMetadataPath() {
+    return metadataPath;
   }
 
   /**
-   * Sets path.
+   * Sets container metadata path.
    *
    * @param path - String.
    */
-  public void setDbPath(String path) {
-    this.dbPath = path;
+  public void setMetadataPath(String path) {
+    this.metadataPath = path;
   }
 
   /**
-   * Get container file path.
+   * Get chunks path.
    * @return - Physical path where container file and checksum is stored.
    */
-  public String getContainerFilePath() {
-    return containerFilePath;
+  public String getChunksPath() {
+    return chunksPath;
   }
 
   /**
-   * Set container Path.
-   * @param containerPath - File path.
+   * Set chunks Path.
+   * @param chunkPath - File path.
    */
-  public void setContainerFilePath(String containerPath) {
-    this.containerFilePath = containerPath;
+  public void setChunksPath(String chunkPath) {
+    this.chunksPath = chunkPath;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e228e54/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueYaml.java
index b7ce0d9..d22092c 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueYaml.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueYaml.java
@@ -144,8 +144,8 @@ public final class KeyValueYaml {
           // When a new field needs to be added, it needs to be added here.
           if (name.equals("containerType") || name.equals("containerId") ||
               name.equals("layOutVersion") || name.equals("state") ||
-              name.equals("metadata") || name.equals("dbPath") ||
-              name.equals("containerFilePath") || name.equals(
+              name.equals("metadata") || name.equals("metadataPath") ||
+              name.equals("chunksPath") || name.equals(
                   "containerDBType")) {
             filtered.add(prop);
           }
@@ -186,8 +186,9 @@ public final class KeyValueYaml {
         KeyValueContainerData kvData = new KeyValueContainerData(containerType,
             (long) nodes.get("containerId"), lv);
         kvData.setContainerDBType((String)nodes.get("containerDBType"));
-        kvData.setDbPath((String) nodes.get("dbPath"));
-        kvData.setContainerFilePath((String) nodes.get("containerFilePath"));
+        kvData.setMetadataPath((String) nodes.get(
+            "metadataPath"));
+        kvData.setChunksPath((String) nodes.get("chunksPath"));
         Map<String, String> meta = (Map) nodes.get("metadata");
         meta.forEach((key, val) -> {
           try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e228e54/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
index e057f6f..055110c 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
@@ -92,8 +92,8 @@ public class TestKeyValueContainerData {
 
     kvData.setState(state);
     kvData.setContainerDBType(containerDBType);
-    kvData.setContainerFilePath(path);
-    kvData.setDbPath(path);
+    kvData.setChunksPath(path);
+    kvData.setMetadataPath(path);
     kvData.incrReadBytes(10);
     kvData.incrWriteBytes(10);
     kvData.incrReadCount();
@@ -101,8 +101,8 @@ public class TestKeyValueContainerData {
 
     assertEquals(state, kvData.getState());
     assertEquals(containerDBType, kvData.getContainerDBType());
-    assertEquals(path, kvData.getContainerFilePath());
-    assertEquals(path, kvData.getDbPath());
+    assertEquals(path, kvData.getChunksPath());
+    assertEquals(path, kvData.getMetadataPath());
 
     assertEquals(10, kvData.getReadBytes());
     assertEquals(10, kvData.getWriteBytes());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e228e54/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java
index 06f6f9d..8550c47 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java
@@ -47,8 +47,8 @@ public class TestKeyValueYaml {
     KeyValueContainerData keyValueContainerData = new KeyValueContainerData(
         ContainerProtos.ContainerType.KeyValueContainer, Long.MAX_VALUE);
     keyValueContainerData.setContainerDBType("RocksDB");
-    keyValueContainerData.setDbPath(path);
-    keyValueContainerData.setContainerFilePath(path);
+    keyValueContainerData.setMetadataPath(path);
+    keyValueContainerData.setChunksPath(path);
 
     File containerFile = new File(filePath, containerPath);
 
@@ -65,8 +65,8 @@ public class TestKeyValueYaml {
     assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
         .getContainerType());
     assertEquals("RocksDB", kvData.getContainerDBType());
-    assertEquals(path, kvData.getContainerFilePath());
-    assertEquals(path, kvData.getDbPath());
+    assertEquals(path, kvData.getMetadataPath());
+    assertEquals(path, kvData.getChunksPath());
     assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN, kvData
         .getState());
     assertEquals(1, kvData.getLayOutVersion());
@@ -90,8 +90,8 @@ public class TestKeyValueYaml {
     assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
         .getContainerType());
     assertEquals("RocksDB", kvData.getContainerDBType());
-    assertEquals(path, kvData.getContainerFilePath());
-    assertEquals(path, kvData.getDbPath());
+    assertEquals(path, kvData.getMetadataPath());
+    assertEquals(path, kvData.getChunksPath());
     assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, kvData
         .getState());
     assertEquals(1, kvData.getLayOutVersion());
@@ -143,9 +143,9 @@ public class TestKeyValueYaml {
           .getContainerType());
       assertEquals(9223372036854775807L, kvData.getContainerId());
       assertEquals("/hdds/current/aed-fg4-hji-jkl/containerdir0/1", kvData
-          .getDbPath());
+          .getChunksPath());
       assertEquals("/hdds/current/aed-fg4-hji-jkl/containerdir0/1", kvData
-          .getContainerFilePath());
+          .getMetadataPath());
       assertEquals(1, kvData.getLayOutVersion());
       assertEquals(2, kvData.getMetadata().size());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e228e54/hadoop-hdds/container-service/src/test/resources/additionalfields.container
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/resources/additionalfields.container b/hadoop-hdds/container-service/src/test/resources/additionalfields.container
index d0df0fe..b19ec44 100644
--- a/hadoop-hdds/container-service/src/test/resources/additionalfields.container
+++ b/hadoop-hdds/container-service/src/test/resources/additionalfields.container
@@ -1,9 +1,10 @@
 !<KeyValueContainerData>
 containerDBType: RocksDB
-containerFilePath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
+chunksPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
 containerId: 9223372036854775807
 containerType: KeyValueContainer
-dbPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
+metadataPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
 layOutVersion: 1
 metadata: {OWNER: ozone, VOLUME: hdfs}
-state: CLOSED
\ No newline at end of file
+state: CLOSED
+aclEnabled: true
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e228e54/hadoop-hdds/container-service/src/test/resources/incorrect.container
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/resources/incorrect.container b/hadoop-hdds/container-service/src/test/resources/incorrect.container
index d567023..0d2bfd1 100644
--- a/hadoop-hdds/container-service/src/test/resources/incorrect.container
+++ b/hadoop-hdds/container-service/src/test/resources/incorrect.container
@@ -1,10 +1,9 @@
 !<KeyValueContainerData>
 containerDBType: RocksDB
-containerFilePath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
+chunksPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
 containerId: 9223372036854775807
 containerType: KeyValueContainer
-dbPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
+metadataPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
 layOutVersion: 1
 metadata: {OWNER: ozone, VOLUME: hdfs}
-state: INVALID
-aclEnabled: true
\ No newline at end of file
+state: INVALID
\ No newline at end of file


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


[08/37] hadoop git commit: HDDS-133:Change format of .container files to Yaml. Contributed by Bharat Viswanadham

Posted by bh...@apache.org.
HDDS-133:Change format of .container files to Yaml. Contributed by Bharat Viswanadham


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

Branch: refs/heads/trunk
Commit: 143dd560bf506fafd849aeb47a42becc6c13330d
Parents: 772c953
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Mon Jun 11 09:04:54 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Mon Jun 11 09:04:54 2018 -0700

----------------------------------------------------------------------
 hadoop-hdds/container-service/pom.xml           |   6 +
 .../common/impl/ChunkLayOutVersion.java         |  18 ++
 .../container/common/impl/ContainerData.java    |  28 +-
 .../common/impl/KeyValueContainerData.java      |  19 +-
 .../container/common/impl/KeyValueYaml.java     | 274 +++++++++++++++++++
 .../common/TestKeyValueContainerData.java       |  15 +-
 .../container/common/impl/TestKeyValueYaml.java | 158 +++++++++++
 .../test/resources/additionalfields.container   |   9 +
 .../src/test/resources/incorrect.container      |  10 +
 hadoop-hdds/pom.xml                             |   2 +
 10 files changed, 521 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/143dd560/hadoop-hdds/container-service/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/pom.xml b/hadoop-hdds/container-service/pom.xml
index 542462e..43f400c 100644
--- a/hadoop-hdds/container-service/pom.xml
+++ b/hadoop-hdds/container-service/pom.xml
@@ -53,6 +53,12 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
     </dependency>
 
     <dependency>
+      <groupId>org.yaml</groupId>
+      <artifactId>snakeyaml</artifactId>
+      <version>1.8</version>
+    </dependency>
+
+    <dependency>
       <groupId>io.dropwizard.metrics</groupId>
       <artifactId>metrics-core</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/143dd560/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkLayOutVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkLayOutVersion.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkLayOutVersion.java
index fff68de6..d1b1bd6 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkLayOutVersion.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ChunkLayOutVersion.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.ozone.container.common.impl;
 
 
+import com.google.common.base.Preconditions;
+
 /**
  * Defines layout versions for the Chunks.
  */
@@ -43,6 +45,22 @@ public final class ChunkLayOutVersion {
   }
 
   /**
+   * Return ChunkLayOutVersion object for the chunkVersion.
+   * @param chunkVersion
+   * @return ChunkLayOutVersion
+   */
+  public static ChunkLayOutVersion getChunkLayOutVersion(int chunkVersion) {
+    Preconditions.checkArgument((chunkVersion <= ChunkLayOutVersion
+        .getLatestVersion().getVersion()));
+    for(ChunkLayOutVersion chunkLayOutVersion : CHUNK_LAYOUT_VERSION_INFOS) {
+      if(chunkLayOutVersion.getVersion() == chunkVersion) {
+        return chunkLayOutVersion;
+      }
+    }
+    return null;
+  }
+
+  /**
    * Returns all versions.
    *
    * @return Version info array.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/143dd560/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
index 06aae66..0bd7795 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
@@ -42,7 +42,7 @@ public class ContainerData {
   private final long containerId;
 
   // Layout version of the container data
-  private final ChunkLayOutVersion layOutVersion;
+  private final int layOutVersion;
 
   // Metadata of the container will be a key value pair.
   // This can hold information like volume name, owner etc.,
@@ -67,7 +67,27 @@ public class ContainerData {
   public ContainerData(ContainerType type, long containerId) {
     this.containerType = type;
     this.containerId = containerId;
-    this.layOutVersion = ChunkLayOutVersion.getLatestVersion();
+    this.layOutVersion = ChunkLayOutVersion.getLatestVersion().getVersion();
+    this.metadata = new TreeMap<>();
+    this.state = ContainerLifeCycleState.OPEN;
+    this.readCount = new AtomicLong(0L);
+    this.readBytes =  new AtomicLong(0L);
+    this.writeCount =  new AtomicLong(0L);
+    this.writeBytes =  new AtomicLong(0L);
+    this.bytesUsed = new AtomicLong(0L);
+  }
+
+  /**
+   * Creates a ContainerData Object, which holds metadata of the container.
+   * @param type - ContainerType
+   * @param containerId - ContainerId
+   * @param layOutVersion - Container layOutVersion
+   */
+  public ContainerData(ContainerType type, long containerId, int
+      layOutVersion) {
+    this.containerType = type;
+    this.containerId = containerId;
+    this.layOutVersion = layOutVersion;
     this.metadata = new TreeMap<>();
     this.state = ContainerLifeCycleState.OPEN;
     this.readCount = new AtomicLong(0L);
@@ -113,8 +133,8 @@ public class ContainerData {
    * Returns the layOutVersion of the actual container data format.
    * @return layOutVersion
    */
-  public ChunkLayOutVersion getLayOutVersion() {
-    return layOutVersion;
+  public int getLayOutVersion() {
+    return ChunkLayOutVersion.getChunkLayOutVersion(layOutVersion).getVersion();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/143dd560/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
index 37eaa49..57b5264 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainerData.java
@@ -53,11 +53,22 @@ public class KeyValueContainerData extends ContainerData {
   }
 
   /**
+   * Constructs KeyValueContainerData object.
+   * @param type - containerType
+   * @param id - ContainerId
+   * @param layOutVersion
+   */
+  public KeyValueContainerData(ContainerProtos.ContainerType type, long id,
+                               int layOutVersion) {
+    super(type, id, layOutVersion);
+    this.numPendingDeletionBlocks = 0;
+  }
+  /**
    * Returns path.
    *
    * @return - path
    */
-  public String getDBPath() {
+  public String getDbPath() {
     return dbPath;
   }
 
@@ -66,7 +77,7 @@ public class KeyValueContainerData extends ContainerData {
    *
    * @param path - String.
    */
-  public void setDBPath(String path) {
+  public void setDbPath(String path) {
     this.dbPath = path;
   }
 
@@ -74,7 +85,7 @@ public class KeyValueContainerData extends ContainerData {
    * Get container file path.
    * @return - Physical path where container file and checksum is stored.
    */
-  public String getContainerPath() {
+  public String getContainerFilePath() {
     return containerFilePath;
   }
 
@@ -82,7 +93,7 @@ public class KeyValueContainerData extends ContainerData {
    * Set container Path.
    * @param containerPath - File path.
    */
-  public void setContainerPath(String containerPath) {
+  public void setContainerFilePath(String containerPath) {
     this.containerFilePath = containerPath;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/143dd560/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueYaml.java
new file mode 100644
index 0000000..b7ce0d9
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueYaml.java
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.impl;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.yaml.snakeyaml.Yaml;
+
+
+import java.beans.IntrospectionException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Writer;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.OutputStreamWriter;
+
+import java.io.File;
+
+
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.Map;
+
+import org.yaml.snakeyaml.constructor.AbstractConstruct;
+import org.yaml.snakeyaml.constructor.Constructor;
+import org.yaml.snakeyaml.introspector.BeanAccess;
+import org.yaml.snakeyaml.introspector.Property;
+import org.yaml.snakeyaml.introspector.PropertyUtils;
+import org.yaml.snakeyaml.nodes.MappingNode;
+import org.yaml.snakeyaml.nodes.Node;
+import org.yaml.snakeyaml.nodes.ScalarNode;
+import org.yaml.snakeyaml.nodes.Tag;
+import org.yaml.snakeyaml.representer.Representer;
+
+/**
+ * Class for creating and reading .container files.
+ */
+
+public final class KeyValueYaml {
+
+  private KeyValueYaml() {
+
+  }
+  /**
+   * Creates a .container file in yaml format.
+   *
+   * @param containerFile
+   * @param containerData
+   * @throws IOException
+   */
+  public static void createContainerFile(File containerFile, ContainerData
+      containerData) throws IOException {
+
+    Preconditions.checkNotNull(containerFile, "yamlFile cannot be null");
+    Preconditions.checkNotNull(containerData, "containerData cannot be null");
+
+    PropertyUtils propertyUtils = new PropertyUtils();
+    propertyUtils.setBeanAccess(BeanAccess.FIELD);
+    propertyUtils.setAllowReadOnlyProperties(true);
+
+    Representer representer = new KeyValueContainerDataRepresenter();
+    representer.setPropertyUtils(propertyUtils);
+    representer.addClassTag(org.apache.hadoop.ozone.container.common.impl
+        .KeyValueContainerData.class, new Tag("KeyValueContainerData"));
+
+    Constructor keyValueDataConstructor = new KeyValueDataConstructor();
+
+    Yaml yaml = new Yaml(keyValueDataConstructor, representer);
+
+    Writer writer = new OutputStreamWriter(new FileOutputStream(containerFile),
+        "UTF-8");
+    yaml.dump(containerData, writer);
+    writer.close();
+  }
+
+  /**
+   * Read the yaml file, and return containerData.
+   *
+   * @param containerFile
+   * @throws IOException
+   */
+  public static KeyValueContainerData readContainerFile(File containerFile)
+      throws IOException {
+    Preconditions.checkNotNull(containerFile, "containerFile cannot be null");
+
+    InputStream input = null;
+    KeyValueContainerData keyValueContainerData;
+    try {
+      PropertyUtils propertyUtils = new PropertyUtils();
+      propertyUtils.setBeanAccess(BeanAccess.FIELD);
+      propertyUtils.setAllowReadOnlyProperties(true);
+
+      Representer representer = new KeyValueContainerDataRepresenter();
+      representer.setPropertyUtils(propertyUtils);
+      representer.addClassTag(org.apache.hadoop.ozone.container.common.impl
+          .KeyValueContainerData.class, new Tag("KeyValueContainerData"));
+
+      Constructor keyValueDataConstructor = new KeyValueDataConstructor();
+
+      Yaml yaml = new Yaml(keyValueDataConstructor, representer);
+      yaml.setBeanAccess(BeanAccess.FIELD);
+
+      input = new FileInputStream(containerFile);
+      keyValueContainerData = (KeyValueContainerData)
+          yaml.load(input);
+    } finally {
+      if (input!= null) {
+        input.close();
+      }
+    }
+    return keyValueContainerData;
+  }
+
+  /**
+   * Representer class to define which fields need to be stored in yaml file.
+   */
+  private static class KeyValueContainerDataRepresenter extends Representer {
+    @Override
+    protected Set<Property> getProperties(Class<? extends Object> type)
+        throws IntrospectionException {
+      Set<Property> set = super.getProperties(type);
+      Set<Property> filtered = new TreeSet<Property>();
+      if (type.equals(KeyValueContainerData.class)) {
+        // filter properties
+        for (Property prop : set) {
+          String name = prop.getName();
+          // When a new field needs to be added, it needs to be added here.
+          if (name.equals("containerType") || name.equals("containerId") ||
+              name.equals("layOutVersion") || name.equals("state") ||
+              name.equals("metadata") || name.equals("dbPath") ||
+              name.equals("containerFilePath") || name.equals(
+                  "containerDBType")) {
+            filtered.add(prop);
+          }
+        }
+      }
+      return filtered;
+    }
+  }
+
+  /**
+   * Constructor class for KeyValueData, which will be used by Yaml.
+   */
+  private static class KeyValueDataConstructor extends Constructor {
+    KeyValueDataConstructor() {
+      //Adding our own specific constructors for tags.
+      this.yamlConstructors.put(new Tag("KeyValueContainerData"),
+          new ConstructKeyValueContainerData());
+      this.yamlConstructors.put(Tag.INT, new ConstructLong());
+    }
+
+    private class ConstructKeyValueContainerData extends AbstractConstruct {
+      public Object construct(Node node) {
+        MappingNode mnode = (MappingNode) node;
+        Map<Object, Object> nodes = constructMapping(mnode);
+        String type = (String) nodes.get("containerType");
+
+        ContainerProtos.ContainerType containerType = ContainerProtos
+            .ContainerType.KeyValueContainer;
+        if (type.equals("KeyValueContainer")) {
+          containerType = ContainerProtos.ContainerType.KeyValueContainer;
+        }
+
+        //Needed this, as TAG.INT type is by default converted to Long.
+        long layOutVersion = (long) nodes.get("layOutVersion");
+        int lv = (int) layOutVersion;
+
+        //When a new field is added, it needs to be added here.
+        KeyValueContainerData kvData = new KeyValueContainerData(containerType,
+            (long) nodes.get("containerId"), lv);
+        kvData.setContainerDBType((String)nodes.get("containerDBType"));
+        kvData.setDbPath((String) nodes.get("dbPath"));
+        kvData.setContainerFilePath((String) nodes.get("containerFilePath"));
+        Map<String, String> meta = (Map) nodes.get("metadata");
+        meta.forEach((key, val) -> {
+          try {
+            kvData.addMetadata(key, val);
+          } catch (IOException e) {
+            throw new IllegalStateException("Unexpected " +
+                "Key Value Pair " + "(" + key + "," + val +")in the metadata " +
+                "for containerId " + (long) nodes.get("containerId"));
+          }
+        });
+        String state = (String) nodes.get("state");
+        switch (state) {
+        case "OPEN":
+          kvData.setState(ContainerProtos.ContainerLifeCycleState.OPEN);
+          break;
+        case "CLOSING":
+          kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSING);
+          break;
+        case "CLOSED":
+          kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED);
+          break;
+        default:
+          throw new IllegalStateException("Unexpected " +
+              "ContainerLifeCycleState " + state + " for the containerId " +
+              (long) nodes.get("containerId"));
+        }
+        return kvData;
+      }
+    }
+
+    //Below code is taken from snake yaml, as snakeyaml tries to fit the
+    // number if it fits in integer, otherwise returns long. So, slightly
+    // modified the code to return long in all cases.
+    private class ConstructLong extends AbstractConstruct {
+      public Object construct(Node node) {
+        String value = constructScalar((ScalarNode) node).toString()
+            .replaceAll("_", "");
+        int sign = +1;
+        char first = value.charAt(0);
+        if (first == '-') {
+          sign = -1;
+          value = value.substring(1);
+        } else if (first == '+') {
+          value = value.substring(1);
+        }
+        int base = 10;
+        if ("0".equals(value)) {
+          return Long.valueOf(0);
+        } else if (value.startsWith("0b")) {
+          value = value.substring(2);
+          base = 2;
+        } else if (value.startsWith("0x")) {
+          value = value.substring(2);
+          base = 16;
+        } else if (value.startsWith("0")) {
+          value = value.substring(1);
+          base = 8;
+        } else if (value.indexOf(':') != -1) {
+          String[] digits = value.split(":");
+          int bes = 1;
+          int val = 0;
+          for (int i = 0, j = digits.length; i < j; i++) {
+            val += (Long.parseLong(digits[(j - i) - 1]) * bes);
+            bes *= 60;
+          }
+          return createNumber(sign, String.valueOf(val), 10);
+        } else {
+          return createNumber(sign, value, 10);
+        }
+        return createNumber(sign, value, base);
+      }
+    }
+
+    private Number createNumber(int sign, String number, int radix) {
+      Number result;
+      if (sign < 0) {
+        number = "-" + number;
+      }
+      result = Long.valueOf(number, radix);
+      return result;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/143dd560/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
index 1541921..e057f6f 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
@@ -38,8 +38,6 @@ public class TestKeyValueContainerData {
     long containerId = 1L;
     ContainerProtos.ContainerType containerType = ContainerProtos
         .ContainerType.KeyValueContainer;
-    String path = "/tmp";
-    String containerDBType = "RocksDB";
     int layOutVersion = 1;
     ContainerProtos.ContainerLifeCycleState state = ContainerProtos
         .ContainerLifeCycleState.OPEN;
@@ -57,10 +55,9 @@ public class TestKeyValueContainerData {
 
     KeyValueContainerData kvData = KeyValueContainerData.getFromProtoBuf(
         containerData);
-
     assertEquals(containerType, kvData.getContainerType());
     assertEquals(containerId, kvData.getContainerId());
-    assertEquals(layOutVersion, kvData.getLayOutVersion().getVersion());
+    assertEquals(layOutVersion, kvData.getLayOutVersion());
     assertEquals(state, kvData.getState());
     assertEquals(2, kvData.getMetadata().size());
     assertEquals("ozone", kvData.getMetadata().get("VOLUME"));
@@ -75,11 +72,9 @@ public class TestKeyValueContainerData {
         .ContainerType.KeyValueContainer;
     String path = "/tmp";
     String containerDBType = "RocksDB";
-    int layOutVersion = 1;
     ContainerProtos.ContainerLifeCycleState state = ContainerProtos
         .ContainerLifeCycleState.CLOSED;
     AtomicLong val = new AtomicLong(0);
-    AtomicLong updatedVal = new AtomicLong(100);
 
     KeyValueContainerData kvData = new KeyValueContainerData(containerType,
         containerId);
@@ -97,8 +92,8 @@ public class TestKeyValueContainerData {
 
     kvData.setState(state);
     kvData.setContainerDBType(containerDBType);
-    kvData.setContainerPath(path);
-    kvData.setDBPath(path);
+    kvData.setContainerFilePath(path);
+    kvData.setDbPath(path);
     kvData.incrReadBytes(10);
     kvData.incrWriteBytes(10);
     kvData.incrReadCount();
@@ -106,8 +101,8 @@ public class TestKeyValueContainerData {
 
     assertEquals(state, kvData.getState());
     assertEquals(containerDBType, kvData.getContainerDBType());
-    assertEquals(path, kvData.getContainerPath());
-    assertEquals(path, kvData.getDBPath());
+    assertEquals(path, kvData.getContainerFilePath());
+    assertEquals(path, kvData.getDbPath());
 
     assertEquals(10, kvData.getReadBytes());
     assertEquals(10, kvData.getWriteBytes());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/143dd560/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java
new file mode 100644
index 0000000..06f6f9d
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.impl;
+
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * This class tests create/read .container files.
+ */
+public class TestKeyValueYaml {
+
+  @Test
+  public void testCreateContainerFile() throws IOException {
+    String path = new FileSystemTestHelper().getTestRootDir();
+    String containerPath = "1.container";
+
+    File filePath = new File(new FileSystemTestHelper().getTestRootDir());
+    filePath.mkdirs();
+
+    KeyValueContainerData keyValueContainerData = new KeyValueContainerData(
+        ContainerProtos.ContainerType.KeyValueContainer, Long.MAX_VALUE);
+    keyValueContainerData.setContainerDBType("RocksDB");
+    keyValueContainerData.setDbPath(path);
+    keyValueContainerData.setContainerFilePath(path);
+
+    File containerFile = new File(filePath, containerPath);
+
+    // Create .container file with ContainerData
+    KeyValueYaml.createContainerFile(containerFile, keyValueContainerData);
+
+    //Check .container file exists or not.
+    assertTrue(containerFile.exists());
+
+    // Read from .container file, and verify data.
+    KeyValueContainerData kvData = KeyValueYaml.readContainerFile(
+        containerFile);
+    assertEquals(Long.MAX_VALUE, kvData.getContainerId());
+    assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
+        .getContainerType());
+    assertEquals("RocksDB", kvData.getContainerDBType());
+    assertEquals(path, kvData.getContainerFilePath());
+    assertEquals(path, kvData.getDbPath());
+    assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN, kvData
+        .getState());
+    assertEquals(1, kvData.getLayOutVersion());
+    assertEquals(0, kvData.getMetadata().size());
+
+    // Update ContainerData.
+    kvData.addMetadata("VOLUME", "hdfs");
+    kvData.addMetadata("OWNER", "ozone");
+    kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED);
+
+
+    // Update .container file with new ContainerData.
+    containerFile = new File(filePath, containerPath);
+    KeyValueYaml.createContainerFile(containerFile, kvData);
+
+    // Reading newly updated data from .container file
+    kvData =  KeyValueYaml.readContainerFile(containerFile);
+
+    // verify data.
+    assertEquals(Long.MAX_VALUE, kvData.getContainerId());
+    assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
+        .getContainerType());
+    assertEquals("RocksDB", kvData.getContainerDBType());
+    assertEquals(path, kvData.getContainerFilePath());
+    assertEquals(path, kvData.getDbPath());
+    assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, kvData
+        .getState());
+    assertEquals(1, kvData.getLayOutVersion());
+    assertEquals(2, kvData.getMetadata().size());
+    assertEquals("hdfs", kvData.getMetadata().get("VOLUME"));
+    assertEquals("ozone", kvData.getMetadata().get("OWNER"));
+
+    FileUtil.fullyDelete(filePath);
+
+
+  }
+
+  @Test
+  public void testIncorrectContainerFile() throws IOException{
+    try {
+      String path = "incorrect.container";
+      //Get file from resources folder
+      ClassLoader classLoader = getClass().getClassLoader();
+      File file = new File(classLoader.getResource(path).getFile());
+      KeyValueContainerData kvData = KeyValueYaml.readContainerFile(file);
+      fail("testIncorrectContainerFile failed");
+    } catch (IllegalStateException ex) {
+      GenericTestUtils.assertExceptionContains("Unexpected " +
+          "ContainerLifeCycleState", ex);
+    }
+  }
+
+
+  @Test
+  public void testCheckBackWardCompatabilityOfContainerFile() throws
+      IOException {
+    // This test is for if we upgrade, and then .container files added by new
+    // server will have new fields added to .container file, after a while we
+    // decided to rollback. Then older ozone can read .container files
+    // created or not.
+
+    try {
+      String path = "additionalfields.container";
+      //Get file from resources folder
+      ClassLoader classLoader = getClass().getClassLoader();
+      File file = new File(classLoader.getResource(path).getFile());
+      KeyValueContainerData kvData = KeyValueYaml.readContainerFile(file);
+
+      //Checking the Container file data is consistent or not
+      assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, kvData
+          .getState());
+      assertEquals("RocksDB", kvData.getContainerDBType());
+      assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
+          .getContainerType());
+      assertEquals(9223372036854775807L, kvData.getContainerId());
+      assertEquals("/hdds/current/aed-fg4-hji-jkl/containerdir0/1", kvData
+          .getDbPath());
+      assertEquals("/hdds/current/aed-fg4-hji-jkl/containerdir0/1", kvData
+          .getContainerFilePath());
+      assertEquals(1, kvData.getLayOutVersion());
+      assertEquals(2, kvData.getMetadata().size());
+
+    } catch (Exception ex) {
+      fail("testCheckBackWardCompatabilityOfContainerFile failed");
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/143dd560/hadoop-hdds/container-service/src/test/resources/additionalfields.container
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/resources/additionalfields.container b/hadoop-hdds/container-service/src/test/resources/additionalfields.container
new file mode 100644
index 0000000..d0df0fe
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/resources/additionalfields.container
@@ -0,0 +1,9 @@
+!<KeyValueContainerData>
+containerDBType: RocksDB
+containerFilePath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
+containerId: 9223372036854775807
+containerType: KeyValueContainer
+dbPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
+layOutVersion: 1
+metadata: {OWNER: ozone, VOLUME: hdfs}
+state: CLOSED
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/143dd560/hadoop-hdds/container-service/src/test/resources/incorrect.container
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/resources/incorrect.container b/hadoop-hdds/container-service/src/test/resources/incorrect.container
new file mode 100644
index 0000000..d567023
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/resources/incorrect.container
@@ -0,0 +1,10 @@
+!<KeyValueContainerData>
+containerDBType: RocksDB
+containerFilePath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
+containerId: 9223372036854775807
+containerType: KeyValueContainer
+dbPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
+layOutVersion: 1
+metadata: {OWNER: ozone, VOLUME: hdfs}
+state: INVALID
+aclEnabled: true
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/143dd560/hadoop-hdds/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/pom.xml b/hadoop-hdds/pom.xml
index fab45e2..573803b 100644
--- a/hadoop-hdds/pom.xml
+++ b/hadoop-hdds/pom.xml
@@ -102,6 +102,8 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/main/resources/webapps/static/nvd3-1.8.5.min.css.map</exclude>
             <exclude>src/main/resources/webapps/static/nvd3-1.8.5.min.js</exclude>
             <exclude>src/main/resources/webapps/static/nvd3-1.8.5.min.js.map</exclude>
+            <exclude>src/test/resources/incorrect.container</exclude>
+            <exclude>src/test/resources/additionalfields.container</exclude>
           </excludes>
         </configuration>
       </plugin>


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


[33/37] hadoop git commit: HDDS-237. Add updateDeleteTransactionId. Contributed by Bharat Viswanadham and Lokesh Jain

Posted by bh...@apache.org.
HDDS-237. Add updateDeleteTransactionId. Contributed by Bharat Viswanadham and Lokesh Jain


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

Branch: refs/heads/trunk
Commit: cb9574a337f2012f2ab59c2abf300567562bf3a3
Parents: e899c4c
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Fri Jul 6 16:26:40 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Fri Jul 6 16:26:40 2018 -0700

----------------------------------------------------------------------
 .../container/common/impl/ContainerData.java    | 21 ++++++++++++++++++++
 .../container/common/interfaces/Container.java  |  6 ++++++
 .../container/keyvalue/KeyValueContainer.java   |  5 +++++
 3 files changed, 32 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb9574a3/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
index 5638b60..0d217e4 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
@@ -32,6 +32,8 @@ import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
+import static java.lang.Math.max;
+
 /**
  * ContainerData is the in-memory representation of container metadata and is
  * represented on disk by the .container file.
@@ -70,6 +72,8 @@ public class ContainerData {
 
   private HddsVolume volume;
 
+  private long deleteTransactionId;
+
   /**
    * Number of pending deletion blocks in container.
    */
@@ -110,6 +114,7 @@ public class ContainerData {
     this.keyCount = new AtomicLong(0L);
     this.maxSizeGB = size;
     this.numPendingDeletionBlocks = new AtomicInteger(0);
+    this.deleteTransactionId = 0;
   }
 
   /**
@@ -441,4 +446,20 @@ public class ContainerData {
 
     return builder.build();
   }
+
+  /**
+   * Sets deleteTransactionId to latest delete transactionId for the container.
+   *
+   * @param transactionId latest transactionId of the container.
+   */
+  public void updateDeleteTransactionId(long transactionId) {
+    deleteTransactionId = max(transactionId, deleteTransactionId);
+  }
+
+  /**
+   * Return the latest deleteTransactionId of the container.
+   */
+  public long getDeleteTransactionId() {
+    return deleteTransactionId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb9574a3/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
index f0f1b37..03ed7b1 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
@@ -91,4 +91,10 @@ public interface Container extends RwLock {
    */
   ContainerProtos.ContainerType getContainerType();
 
+  /**
+   * updates the DeleteTransactionId.
+   * @param deleteTransactionId
+   */
+  void updateDeleteTransactionId(long deleteTransactionId);
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb9574a3/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index 95621e5..b07b053 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -415,6 +415,11 @@ public class KeyValueContainer implements Container {
     }
   }
 
+  @Override
+  public void updateDeleteTransactionId(long deleteTransactionId) {
+    containerData.updateDeleteTransactionId(deleteTransactionId);
+  }
+
   /**
    * Acquire read lock.
    */


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


[16/37] hadoop git commit: HDDS-173. Refactor Dispatcher and implement Handler for new ContainerIO design.

Posted by bh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java
new file mode 100644
index 0000000..258d4da
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue.impl;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils;
+import org.apache.hadoop.ozone.container.common.impl.ChunkLayOutVersion;
+import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.security.NoSuchAlgorithmException;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.CONTAINER_INTERNAL_ERROR;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.NO_SUCH_ALGORITHM;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNSUPPORTED_REQUEST;
+
+/**
+ * This class is for performing chunk related operations.
+ */
+public class ChunkManagerImpl implements ChunkManager {
+  static final Logger LOG = LoggerFactory.getLogger(ChunkManagerImpl.class);
+
+  /**
+   * writes a given chunk.
+   *
+   * @param container - Container for the chunk
+   * @param blockID - ID of the block
+   * @param info - ChunkInfo
+   * @param data - data of the chunk
+   * @param stage - Stage of the Chunk operation
+   * @throws StorageContainerException
+   */
+  public void writeChunk(Container container, BlockID blockID, ChunkInfo info,
+                         byte[] data, ContainerProtos.Stage stage)
+      throws StorageContainerException {
+
+    try {
+
+      KeyValueContainerData containerData = (KeyValueContainerData) container
+          .getContainerData();
+
+      File chunkFile = ChunkUtils.validateChunk(containerData, info);
+      File tmpChunkFile = getTmpChunkFile(chunkFile, info);
+
+      LOG.debug("writing chunk:{} chunk stage:{} chunk file:{} tmp chunk file",
+          info.getChunkName(), stage, chunkFile, tmpChunkFile);
+
+      switch (stage) {
+      case WRITE_DATA:
+        // Initially writes to temporary chunk file.
+        ChunkUtils.writeData(tmpChunkFile, info, data);
+        break;
+      case COMMIT_DATA:
+        // commit the data, means move chunk data from temporary chunk file
+        // to actual chunk file.
+        long sizeDiff = tmpChunkFile.length() - chunkFile.length();
+        commitChunk(tmpChunkFile, chunkFile);
+        containerData.incrBytesUsed(sizeDiff);
+        containerData.incrWriteCount();
+        containerData.incrWriteBytes(sizeDiff);
+        break;
+      case COMBINED:
+        // directly write to the chunk file
+        ChunkUtils.writeData(chunkFile, info, data);
+        containerData.incrBytesUsed(info.getLen());
+        containerData.incrWriteCount();
+        containerData.incrWriteBytes(info.getLen());
+        break;
+      default:
+        throw new IOException("Can not identify write operation.");
+      }
+    } catch (StorageContainerException ex) {
+      throw ex;
+    } catch (NoSuchAlgorithmException ex) {
+      LOG.error("write data failed. error: {}", ex);
+      throw new StorageContainerException("Internal error: ", ex,
+          NO_SUCH_ALGORITHM);
+    } catch (ExecutionException  | IOException ex) {
+      LOG.error("write data failed. error: {}", ex);
+      throw new StorageContainerException("Internal error: ", ex,
+          CONTAINER_INTERNAL_ERROR);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("write data failed. error: {}", e);
+      throw new StorageContainerException("Internal error: ", e,
+          CONTAINER_INTERNAL_ERROR);
+    }
+  }
+
+  /**
+   * reads the data defined by a chunk.
+   *
+   * @param container - Container for the chunk
+   * @param blockID - ID of the block.
+   * @param info - ChunkInfo.
+   * @return byte array
+   * @throws StorageContainerException
+   * TODO: Right now we do not support partial reads and writes of chunks.
+   * TODO: Explore if we need to do that for ozone.
+   */
+  public byte[] readChunk(Container container, BlockID blockID, ChunkInfo info)
+      throws StorageContainerException {
+    try {
+      KeyValueContainerData containerData = (KeyValueContainerData) container
+          .getContainerData();
+      ByteBuffer data;
+
+      // Checking here, which layout version the container is, and reading
+      // the chunk file in that format.
+      // In version1, we verify checksum if it is available and return data
+      // of the chunk file.
+      if (containerData.getLayOutVersion() == ChunkLayOutVersion
+          .getLatestVersion().getVersion()) {
+        File chunkFile = ChunkUtils.getChunkFile(containerData, info);
+        data = ChunkUtils.readData(chunkFile, info);
+        containerData.incrReadCount();
+        containerData.incrReadBytes(chunkFile.length());
+        return data.array();
+      }
+    } catch(NoSuchAlgorithmException ex) {
+      LOG.error("read data failed. error: {}", ex);
+      throw new StorageContainerException("Internal error: ",
+          ex, NO_SUCH_ALGORITHM);
+    } catch (ExecutionException ex) {
+      LOG.error("read data failed. error: {}", ex);
+      throw new StorageContainerException("Internal error: ",
+          ex, CONTAINER_INTERNAL_ERROR);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("read data failed. error: {}", e);
+      throw new StorageContainerException("Internal error: ",
+          e, CONTAINER_INTERNAL_ERROR);
+    }
+    return null;
+  }
+
+  /**
+   * Deletes a given chunk.
+   *
+   * @param container - Container for the chunk
+   * @param blockID - ID of the block
+   * @param info - Chunk Info
+   * @throws StorageContainerException
+   */
+  public void deleteChunk(Container container, BlockID blockID, ChunkInfo info)
+      throws StorageContainerException {
+    Preconditions.checkNotNull(blockID, "Block ID cannot be null.");
+    KeyValueContainerData containerData = (KeyValueContainerData) container
+        .getContainerData();
+    // Checking here, which layout version the container is, and performing
+    // deleting chunk operation.
+    // In version1, we have only chunk file.
+    if (containerData.getLayOutVersion() == ChunkLayOutVersion
+        .getLatestVersion().getVersion()) {
+      File chunkFile = ChunkUtils.getChunkFile(containerData, info);
+      if ((info.getOffset() == 0) && (info.getLen() == chunkFile.length())) {
+        FileUtil.fullyDelete(chunkFile);
+        containerData.decrBytesUsed(chunkFile.length());
+      } else {
+        LOG.error("Not Supported Operation. Trying to delete a " +
+            "chunk that is in shared file. chunk info : " + info.toString());
+        throw new StorageContainerException("Not Supported Operation. " +
+            "Trying to delete a chunk that is in shared file. chunk info : "
+            + info.toString(), UNSUPPORTED_REQUEST);
+      }
+    }
+  }
+
+  /**
+   * Shutdown the chunkManager.
+   *
+   * In the chunkManager we haven't acquired any resources, so nothing to do
+   * here.
+   */
+
+  public void shutdown() {
+    //TODO: need to revisit this during integration of container IO.
+  }
+
+  /**
+   * Returns the temporary chunkFile path.
+   * @param chunkFile
+   * @param info
+   * @return temporary chunkFile path
+   * @throws StorageContainerException
+   */
+  private File getTmpChunkFile(File chunkFile, ChunkInfo info)
+      throws StorageContainerException {
+    return new File(chunkFile.getParent(),
+        chunkFile.getName() +
+            OzoneConsts.CONTAINER_CHUNK_NAME_DELIMITER +
+            OzoneConsts.CONTAINER_TEMPORARY_CHUNK_PREFIX);
+  }
+
+  /**
+   * Commit the chunk by renaming the temporary chunk file to chunk file.
+   * @param tmpChunkFile
+   * @param chunkFile
+   * @throws IOException
+   */
+  private void commitChunk(File tmpChunkFile, File chunkFile) throws
+      IOException {
+    Files.move(tmpChunkFile.toPath(), chunkFile.toPath(),
+        StandardCopyOption.REPLACE_EXISTING);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
new file mode 100644
index 0000000..40736e5
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue.impl;
+
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Longs;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
+import org.apache.hadoop.ozone.container.common.helpers.KeyData;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.keyvalue.interfaces.KeyManager;
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
+import org.apache.hadoop.utils.MetadataStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_KEY;
+
+/**
+ * This class is for performing key related operations on the KeyValue
+ * Container.
+ */
+public class KeyManagerImpl implements KeyManager {
+
+  static final Logger LOG = LoggerFactory.getLogger(KeyManagerImpl.class);
+
+  private Configuration config;
+
+  /**
+   * Constructs a key Manager.
+   *
+   * @param conf - Ozone configuration
+   */
+  public KeyManagerImpl(Configuration conf) {
+    Preconditions.checkNotNull(conf, "Config cannot be null");
+    this.config = conf;
+  }
+
+  /**
+   * Puts or overwrites a key.
+   *
+   * @param container - Container for which key need to be added.
+   * @param data     - Key Data.
+   * @throws IOException
+   */
+  public void putKey(Container container, KeyData data) throws IOException {
+    Preconditions.checkNotNull(data, "KeyData cannot be null for put " +
+        "operation.");
+    Preconditions.checkState(data.getContainerID() >= 0, "Container Id " +
+        "cannot be negative");
+    // We are not locking the key manager since LevelDb serializes all actions
+    // against a single DB. We rely on DB level locking to avoid conflicts.
+    MetadataStore db = KeyUtils.getDB((KeyValueContainerData) container
+        .getContainerData(), config);
+
+    // This is a post condition that acts as a hint to the user.
+    // Should never fail.
+    Preconditions.checkNotNull(db, "DB cannot be null here");
+    db.put(Longs.toByteArray(data.getLocalID()), data.getProtoBufMessage()
+        .toByteArray());
+  }
+
+  /**
+   * Gets an existing key.
+   *
+   * @param container - Container from which key need to be get.
+   * @param blockID - BlockID of the key.
+   * @return Key Data.
+   * @throws IOException
+   */
+  public KeyData getKey(Container container, BlockID blockID)
+      throws IOException {
+    Preconditions.checkNotNull(blockID,
+        "BlockID cannot be null in GetKet request");
+    Preconditions.checkNotNull(blockID.getContainerID(),
+        "Container name cannot be null");
+
+    KeyValueContainerData containerData = (KeyValueContainerData) container
+        .getContainerData();
+    MetadataStore db = KeyUtils.getDB(containerData, config);
+    // This is a post condition that acts as a hint to the user.
+    // Should never fail.
+    Preconditions.checkNotNull(db, "DB cannot be null here");
+    byte[] kData = db.get(Longs.toByteArray(blockID.getLocalID()));
+    if (kData == null) {
+      throw new StorageContainerException("Unable to find the key.",
+          NO_SUCH_KEY);
+    }
+    ContainerProtos.KeyData keyData = ContainerProtos.KeyData.parseFrom(kData);
+    return KeyData.getFromProtoBuf(keyData);
+  }
+
+  /**
+   * Deletes an existing Key.
+   *
+   * @param container - Container from which key need to be deleted.
+   * @param blockID - ID of the block.
+   * @throws StorageContainerException
+   */
+  public void deleteKey(Container container, BlockID blockID) throws
+      IOException {
+    Preconditions.checkNotNull(blockID, "block ID cannot be null.");
+    Preconditions.checkState(blockID.getContainerID() >= 0,
+        "Container ID cannot be negative.");
+    Preconditions.checkState(blockID.getLocalID() >= 0,
+        "Local ID cannot be negative.");
+
+    KeyValueContainerData cData = (KeyValueContainerData) container
+        .getContainerData();
+    MetadataStore db = KeyUtils.getDB(cData, config);
+    // This is a post condition that acts as a hint to the user.
+    // Should never fail.
+    Preconditions.checkNotNull(db, "DB cannot be null here");
+    // Note : There is a race condition here, since get and delete
+    // are not atomic. Leaving it here since the impact is refusing
+    // to delete a key which might have just gotten inserted after
+    // the get check.
+    byte[] kKey = Longs.toByteArray(blockID.getLocalID());
+    byte[] kData = db.get(kKey);
+    if (kData == null) {
+      throw new StorageContainerException("Unable to find the key.",
+          NO_SUCH_KEY);
+    }
+    db.delete(kKey);
+  }
+
+  /**
+   * List keys in a container.
+   *
+   * @param container - Container from which keys need to be listed.
+   * @param startLocalID  - Key to start from, 0 to begin.
+   * @param count    - Number of keys to return.
+   * @return List of Keys that match the criteria.
+   */
+  public List<KeyData> listKey(Container container, long startLocalID, int
+      count) throws IOException {
+    Preconditions.checkNotNull(container, "container cannot be null");
+    Preconditions.checkState(startLocalID >= 0, "startLocal ID cannot be " +
+        "negative");
+    Preconditions.checkArgument(count > 0,
+        "Count must be a positive number.");
+    container.readLock();
+    List<KeyData> result = null;
+    KeyValueContainerData cData = (KeyValueContainerData) container
+        .getContainerData();
+    MetadataStore db = KeyUtils.getDB(cData, config);
+    result = new ArrayList<>();
+    byte[] startKeyInBytes = Longs.toByteArray(startLocalID);
+    List<Map.Entry<byte[], byte[]>> range = db.getSequentialRangeKVs(
+        startKeyInBytes, count, null);
+    for (Map.Entry<byte[], byte[]> entry : range) {
+      KeyData value = KeyUtils.getKeyData(entry.getValue());
+      KeyData data = new KeyData(value.getBlockID());
+      result.add(data);
+    }
+    return result;
+  }
+
+  /**
+   * Shutdown KeyValueContainerManager.
+   */
+  public void shutdown() {
+    KeyUtils.shutdownCache(ContainerCache.getInstance(config));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
index ebda97e..7a5d48b 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
@@ -43,11 +43,11 @@ public interface KeyManager {
    * Gets an existing key.
    *
    * @param container - Container from which key need to be get.
-   * @param data - Key Data.
+   * @param blockID - BlockID of the Key.
    * @return Key Data.
    * @throws IOException
    */
-  KeyData getKey(Container container, KeyData data) throws IOException;
+  KeyData getKey(Container container, BlockID blockID) throws IOException;
 
   /**
    * Deletes an existing Key.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
index 055110c..52f291b 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
@@ -19,12 +19,11 @@
 package org.apache.hadoop.ozone.container.common;
 
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 
-import java.io.IOException;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
@@ -33,39 +32,6 @@ import java.util.concurrent.atomic.AtomicLong;
 public class TestKeyValueContainerData {
 
   @Test
-  public void testGetFromProtoBuf() throws IOException {
-
-    long containerId = 1L;
-    ContainerProtos.ContainerType containerType = ContainerProtos
-        .ContainerType.KeyValueContainer;
-    int layOutVersion = 1;
-    ContainerProtos.ContainerLifeCycleState state = ContainerProtos
-        .ContainerLifeCycleState.OPEN;
-
-    ContainerProtos.KeyValue.Builder keyValBuilder =
-        ContainerProtos.KeyValue.newBuilder();
-    ContainerProtos.CreateContainerData containerData = ContainerProtos
-        .CreateContainerData.newBuilder()
-        .setContainerType(containerType)
-        .setContainerId(containerId)
-        .addMetadata(0, keyValBuilder.setKey("VOLUME").setValue("ozone")
-            .build())
-        .addMetadata(1, keyValBuilder.setKey("OWNER").setValue("hdfs")
-            .build()).build();
-
-    KeyValueContainerData kvData = KeyValueContainerData.getFromProtoBuf(
-        containerData);
-    assertEquals(containerType, kvData.getContainerType());
-    assertEquals(containerId, kvData.getContainerId());
-    assertEquals(layOutVersion, kvData.getLayOutVersion());
-    assertEquals(state, kvData.getState());
-    assertEquals(2, kvData.getMetadata().size());
-    assertEquals("ozone", kvData.getMetadata().get("VOLUME"));
-    assertEquals("hdfs", kvData.getMetadata().get("OWNER"));
-
-  }
-
-  @Test
   public void testKeyValueData() {
     long containerId = 1L;
     ContainerProtos.ContainerType containerType = ContainerProtos

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
index 2c9c2c3..5a29e8a 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
@@ -25,6 +25,8 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerExcep
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -37,6 +39,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Class used to test ContainerSet operations.
@@ -59,8 +62,13 @@ public class TestContainerSet {
     //addContainer
     boolean result = containerSet.addContainer(keyValueContainer);
     assertTrue(result);
-    result = containerSet.addContainer(keyValueContainer);
-    assertFalse(result);
+    try {
+      result = containerSet.addContainer(keyValueContainer);
+      fail("Adding same container ID twice should fail.");
+    } catch (StorageContainerException ex) {
+      GenericTestUtils.assertExceptionContains("Container already exists with" +
+          " container Id " + containerId, ex);
+    }
 
     //getContainer
     KeyValueContainer container = (KeyValueContainer) containerSet

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java
index 8550c47..75c0139 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestKeyValueYaml.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.ozone.container.common.impl;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueYaml;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
new file mode 100644
index 0000000..50927d1
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.interfaces;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
+import static org.junit.Assert.fail;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.rules.Timeout;
+import org.mockito.Mockito;
+
+import java.util.UUID;
+
+/**
+ * Tests Handler interface.
+ */
+public class TestHandler {
+  @Rule
+  public TestRule timeout = new Timeout(300000);
+
+  private Configuration conf;
+  private HddsDispatcher dispatcher;
+  private ContainerSet containerSet;
+  private VolumeSet volumeSet;
+  private Handler handler;
+
+  private final static String SCM_ID = UUID.randomUUID().toString();
+  private final static String DATANODE_UUID = UUID.randomUUID().toString();
+
+  @Before
+  public void setup() throws Exception {
+    this.conf = new Configuration();
+    this.containerSet = Mockito.mock(ContainerSet.class);
+    this.volumeSet = Mockito.mock(VolumeSet.class);
+
+    this.dispatcher = new HddsDispatcher(conf, containerSet, volumeSet, SCM_ID);
+  }
+
+  @Test
+  public void testGetKeyValueHandler() throws Exception {
+    Handler kvHandler = dispatcher.getHandlerForContainerType(
+        ContainerProtos.ContainerType.KeyValueContainer);
+
+    Assert.assertTrue("getHandlerForContainerType returned incorrect handler",
+        (kvHandler instanceof KeyValueHandler));
+  }
+
+  @Test
+  public void testGetHandlerForInvalidContainerType() {
+    // When new ContainerProtos.ContainerType are added, increment the code
+    // for invalid enum.
+    ContainerProtos.ContainerType invalidContainerType =
+        ContainerProtos.ContainerType.forNumber(2);
+
+    Assert.assertEquals("New ContainerType detected. Not an invalid " +
+        "containerType", invalidContainerType, null);
+
+    Handler handler = dispatcher.getHandlerForContainerType(
+        invalidContainerType);
+    Assert.assertEquals("Get Handler for Invalid ContainerType should " +
+        "return null.", handler, null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
index ca936c7..4576db6 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
@@ -18,17 +18,16 @@
 
 package org.apache.hadoop.ozone.container.keyvalue;
 
-
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
-import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerImpl;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Before;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
index a6f50c4..722cece 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
@@ -26,9 +26,9 @@ import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
 import org.apache.hadoop.ozone.container.common.volume
     .RoundRobinVolumeChoosingPolicy;
-import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.keyvalue.impl.KeyManagerImpl;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Before;
 import org.junit.Rule;
@@ -110,7 +110,7 @@ public class TestKeyManagerImpl {
 
     //Get Key
     KeyData fromGetKeyData = keyValueContainerManager.getKey(keyValueContainer,
-        keyData);
+        keyData.getBlockID());
 
     assertEquals(keyData.getContainerID(), fromGetKeyData.getContainerID());
     assertEquals(keyData.getLocalID(), fromGetKeyData.getLocalID());
@@ -168,8 +168,7 @@ public class TestKeyManagerImpl {
   @Test
   public void testGetNoSuchKey() throws Exception {
     try {
-      keyData = new KeyData(new BlockID(1L, 2L));
-      keyValueContainerManager.getKey(keyValueContainer, keyData);
+      keyValueContainerManager.getKey(keyValueContainer, new BlockID(1L, 2L));
       fail("testGetNoSuchKey failed");
     } catch (StorageContainerException ex) {
       GenericTestUtils.assertExceptionContains("Unable to find the key.", ex);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
index b24f601..006b82c 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
@@ -22,13 +22,15 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 
 
-import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
-import org.apache.hadoop.ozone.container.common.impl.KeyValueContainerData;
-import org.apache.hadoop.ozone.container.common.impl.KeyValueYaml;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .StorageContainerException;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
-import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.volume
+    .RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 
+import org.apache.hadoop.ozone.container.keyvalue.helpers
+    .KeyValueContainerLocationUtil;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.DiskChecker;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
new file mode 100644
index 0000000..f4dd41c
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.keyvalue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerCommandRequestProto;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.rules.Timeout;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyLong;
+import org.mockito.Mockito;
+import static org.mockito.Mockito.times;
+
+import java.util.UUID;
+
+/**
+ * Unit tests for {@link KeyValueHandler}.
+ */
+public class TestKeyValueHandler {
+
+  @Rule
+  public TestRule timeout = new Timeout(300000);
+
+  private Configuration conf;
+  private HddsDispatcher dispatcher;
+  private ContainerSet containerSet;
+  private VolumeSet volumeSet;
+  private KeyValueHandler handler;
+
+  private final static String SCM_ID = UUID.randomUUID().toString();
+  private final static String DATANODE_UUID = UUID.randomUUID().toString();
+  private int containerID;
+
+  private final String baseDir = MiniDFSCluster.getBaseDirectory();
+  private final String volume = baseDir + "disk1";
+
+  private void setup() throws Exception {
+    this.conf = new Configuration();
+    conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, volume);
+
+    this.containerSet = new ContainerSet();
+    DatanodeDetails datanodeDetails = DatanodeDetails.newBuilder()
+        .setUuid(DATANODE_UUID)
+        .setHostName("localhost")
+        .setIpAddress("127.0.0.1")
+        .build();
+    this.volumeSet = new VolumeSet(datanodeDetails, conf);
+
+    this.dispatcher = new HddsDispatcher(conf, containerSet, volumeSet, SCM_ID);
+    this.handler = (KeyValueHandler) dispatcher.getHandlerForContainerType(
+        ContainerProtos.ContainerType.KeyValueContainer);
+  }
+
+  @Test
+  /**
+   * Test that Handler handles different command types correctly.
+   */
+  public void testHandlerCommandHandling() throws Exception{
+    // Create mock HddsDispatcher and KeyValueHandler.
+    this.handler = Mockito.mock(KeyValueHandler.class);
+    this.dispatcher = Mockito.mock(HddsDispatcher.class);
+    Mockito.when(dispatcher.getHandlerForContainerType(any())).thenReturn
+        (handler);
+    Mockito.when(dispatcher.dispatch(any())).thenCallRealMethod();
+    Mockito.when(dispatcher.getContainer(anyLong())).thenReturn(
+        Mockito.mock(KeyValueContainer.class));
+    Mockito.when(handler.handle(any(), any())).thenCallRealMethod();
+
+    // Test Create Container Request handling
+    ContainerCommandRequestProto createContainerRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.CreateContainer);
+    dispatcher.dispatch(createContainerRequest);
+    Mockito.verify(handler, times(1)).handleCreateContainer(
+        any(ContainerCommandRequestProto.class), any());
+
+    // Test Read Container Request handling
+    ContainerCommandRequestProto readContainerRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.ReadContainer);
+    dispatcher.dispatch(readContainerRequest);
+    Mockito.verify(handler, times(1)).handleReadContainer(
+        any(ContainerCommandRequestProto.class), any());
+
+    // Test Update Container Request handling
+    ContainerCommandRequestProto updateContainerRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.UpdateContainer);
+    dispatcher.dispatch(updateContainerRequest);
+    Mockito.verify(handler, times(1)).handleUpdateContainer(
+        any(ContainerCommandRequestProto.class), any());
+
+    // Test Delete Container Request handling
+    ContainerCommandRequestProto deleteContainerRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.DeleteContainer);
+    dispatcher.dispatch(deleteContainerRequest);
+    Mockito.verify(handler, times(1)).handleDeleteContainer(
+        any(ContainerCommandRequestProto.class), any());
+
+    // Test List Container Request handling
+    ContainerCommandRequestProto listContainerRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.ListContainer);
+    dispatcher.dispatch(listContainerRequest);
+    Mockito.verify(handler, times(1)).handleUnsupportedOp(
+        any(ContainerCommandRequestProto.class));
+
+    // Test Close Container Request handling
+    ContainerCommandRequestProto closeContainerRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.CloseContainer);
+    dispatcher.dispatch(closeContainerRequest);
+    Mockito.verify(handler, times(1)).handleCloseContainer(
+        any(ContainerCommandRequestProto.class), any());
+
+    // Test Put Key Request handling
+    ContainerCommandRequestProto putKeyRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.PutKey);
+    dispatcher.dispatch(putKeyRequest);
+    Mockito.verify(handler, times(1)).handlePutKey(
+        any(ContainerCommandRequestProto.class), any());
+
+    // Test Get Key Request handling
+    ContainerCommandRequestProto getKeyRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.GetKey);
+    dispatcher.dispatch(getKeyRequest);
+    Mockito.verify(handler, times(1)).handleGetKey(
+        any(ContainerCommandRequestProto.class), any());
+
+    // Test Delete Key Request handling
+    ContainerCommandRequestProto deleteKeyRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.DeleteKey);
+    dispatcher.dispatch(deleteKeyRequest);
+    Mockito.verify(handler, times(1)).handleDeleteKey(
+        any(ContainerCommandRequestProto.class), any());
+
+    // Test List Key Request handling
+    ContainerCommandRequestProto listKeyRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.ListKey);
+    dispatcher.dispatch(listKeyRequest);
+    Mockito.verify(handler, times(2)).handleUnsupportedOp(
+        any(ContainerCommandRequestProto.class));
+
+    // Test Read Chunk Request handling
+    ContainerCommandRequestProto readChunkRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.ReadChunk);
+    dispatcher.dispatch(readChunkRequest);
+    Mockito.verify(handler, times(1)).handleReadChunk(
+        any(ContainerCommandRequestProto.class), any());
+
+    // Test Delete Chunk Request handling
+    ContainerCommandRequestProto deleteChunkRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.DeleteChunk);
+    dispatcher.dispatch(deleteChunkRequest);
+    Mockito.verify(handler, times(1)).handleDeleteChunk(
+        any(ContainerCommandRequestProto.class), any());
+
+    // Test Write Chunk Request handling
+    ContainerCommandRequestProto writeChunkRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.WriteChunk);
+    dispatcher.dispatch(writeChunkRequest);
+    Mockito.verify(handler, times(1)).handleWriteChunk(
+        any(ContainerCommandRequestProto.class), any());
+
+    // Test List Chunk Request handling
+    ContainerCommandRequestProto listChunkRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.ListChunk);
+    dispatcher.dispatch(listChunkRequest);
+    Mockito.verify(handler, times(3)).handleUnsupportedOp(
+        any(ContainerCommandRequestProto.class));
+
+    // Test Put Small File Request handling
+    ContainerCommandRequestProto putSmallFileRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.PutSmallFile);
+    dispatcher.dispatch(putSmallFileRequest);
+    Mockito.verify(handler, times(1)).handlePutSmallFile(
+        any(ContainerCommandRequestProto.class), any());
+
+    // Test Get Small File Request handling
+    ContainerCommandRequestProto getSmallFileRequest =
+        getDummyCommandRequestProto(ContainerProtos.Type.GetSmallFile);
+    dispatcher.dispatch(getSmallFileRequest);
+    Mockito.verify(handler, times(1)).handleGetSmallFile(
+        any(ContainerCommandRequestProto.class), any());
+  }
+
+  private ContainerCommandRequestProto getDummyCommandRequestProto
+      (ContainerProtos.Type cmdType) {
+    ContainerCommandRequestProto request =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(cmdType)
+            .setDatanodeUuid(DATANODE_UUID)
+            .build();
+
+    return request;
+  }
+
+  @Test
+  public void testCreateContainer() throws Exception {
+    setup();
+
+    long contId = ++containerID;
+    ContainerProtos.CreateContainerRequestProto createReq =
+        ContainerProtos.CreateContainerRequestProto.newBuilder()
+            .setContainerID(contId)
+            .build();
+
+    ContainerCommandRequestProto request =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.CreateContainer)
+            .setDatanodeUuid(DATANODE_UUID)
+            .setCreateContainer(createReq)
+            .build();
+
+    dispatcher.dispatch(request);
+
+    // Verify that new container is added to containerSet.
+    Container container = containerSet.getContainer(contId);
+    Assert.assertEquals(contId, container.getContainerData().getContainerId());
+    Assert.assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN,
+        container.getContainerState());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java
index e1a2918..38a4769 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java
@@ -281,10 +281,10 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
         LOG.debug("get key accessing {} {}",
             containerID, containerKey);
         groupInputStream.streamOffset[i] = length;
-          ContainerProtos.KeyData containerKeyData = OzoneContainerTranslation
-            .containerKeyDataForRead(blockID);
+        ContainerProtos.DatanodeBlockID datanodeBlockID = blockID
+            .getDatanodeBlockIDProtobuf();
         ContainerProtos.GetKeyResponseProto response = ContainerProtocolCalls
-            .getKey(xceiverClient, containerKeyData, requestId);
+            .getKey(xceiverClient, datanodeBlockID, requestId);
         List<ContainerProtos.ChunkInfo> chunks =
             response.getKeyData().getChunksList();
         for (ContainerProtos.ChunkInfo chunk : chunks) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneContainerTranslation.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneContainerTranslation.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneContainerTranslation.java
deleted file mode 100644
index e74fffd..0000000
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneContainerTranslation.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.client.io;
-
-
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyData;
-import org.apache.hadoop.hdds.client.BlockID;
-
-
-/**
- * This class contains methods that define the translation between the Ozone
- * domain model and the storage container domain model.
- */
-final class OzoneContainerTranslation {
-
-  /**
-   * Creates key data intended for reading a container key.
-   *
-   * @param blockID - ID of the block.
-   * @return KeyData intended for reading the container key
-   */
-  public static KeyData containerKeyDataForRead(BlockID blockID) {
-    return KeyData
-        .newBuilder()
-        .setBlockID(blockID.getDatanodeBlockIDProtobuf())
-        .build();
-  }
-
-  /**
-   * There is no need to instantiate this class.
-   */
-  private OzoneContainerTranslation() {
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
index 7046132..9decdb9 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
@@ -362,10 +362,7 @@ public final class ContainerTestHelper {
     ContainerProtos.CreateContainerRequestProto.Builder createRequest =
         ContainerProtos.CreateContainerRequestProto
             .newBuilder();
-    ContainerProtos.ContainerData.Builder containerData = ContainerProtos
-        .ContainerData.newBuilder();
-    containerData.setContainerID(containerID);
-    createRequest.setContainerData(containerData.build());
+    createRequest.setContainerID(containerID);
 
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
@@ -391,19 +388,16 @@ public final class ContainerTestHelper {
       long containerID, Map<String, String> metaData) throws IOException {
     ContainerProtos.UpdateContainerRequestProto.Builder updateRequestBuilder =
         ContainerProtos.UpdateContainerRequestProto.newBuilder();
-    ContainerProtos.ContainerData.Builder containerData = ContainerProtos
-        .ContainerData.newBuilder();
-    containerData.setContainerID(containerID);
+    updateRequestBuilder.setContainerID(containerID);
     String[] keys = metaData.keySet().toArray(new String[]{});
     for(int i=0; i<keys.length; i++) {
       KeyValue.Builder kvBuilder = KeyValue.newBuilder();
       kvBuilder.setKey(keys[i]);
       kvBuilder.setValue(metaData.get(keys[i]));
-      containerData.addMetadata(i, kvBuilder.build());
+      updateRequestBuilder.addMetadata(kvBuilder.build());
     }
     Pipeline pipeline =
         ContainerTestHelper.createSingleNodePipeline();
-    updateRequestBuilder.setContainerData(containerData.build());
 
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
@@ -478,10 +472,7 @@ public final class ContainerTestHelper {
 
     ContainerProtos.GetKeyRequestProto.Builder getRequest =
         ContainerProtos.GetKeyRequestProto.newBuilder();
-    ContainerProtos.KeyData.Builder keyData = ContainerProtos.KeyData
-        .newBuilder();
-    keyData.setBlockID(blockID);
-    getRequest.setKeyData(keyData);
+    getRequest.setBlockID(blockID);
 
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
@@ -499,13 +490,11 @@ public final class ContainerTestHelper {
    * @param response - Response
    */
   public static void verifyGetKey(ContainerCommandRequestProto request,
-      ContainerCommandResponseProto response) {
+      ContainerCommandResponseProto response, int expectedChunksCount) {
     Assert.assertEquals(request.getTraceID(), response.getTraceID());
     Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
-    ContainerProtos.PutKeyRequestProto putKey = request.getPutKey();
-    ContainerProtos.GetKeyRequestProto getKey = request.getGetKey();
-    Assert.assertEquals(putKey.getKeyData().getChunksCount(),
-        getKey.getKeyData().getChunksCount());
+    Assert.assertEquals(expectedChunksCount,
+        response.getGetKey().getKeyData().getChunksCount());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
index 67a8160..3f02036 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
@@ -158,7 +158,8 @@ public class TestOzoneContainer {
       // Get Key
       request = ContainerTestHelper.getKeyRequest(pipeline, putKeyRequest.getPutKey());
       response = client.sendCommand(request);
-      ContainerTestHelper.verifyGetKey(request, response);
+      int chunksCount = putKeyRequest.getPutKey().getKeyData().getChunksCount();
+      ContainerTestHelper.verifyGetKey(request, response, chunksCount);
 
 
       // Delete Key
@@ -331,7 +332,8 @@ public class TestOzoneContainer {
       request = ContainerTestHelper.getKeyRequest(client.getPipeline(),
           putKeyRequest.getPutKey());
       response = client.sendCommand(request);
-      ContainerTestHelper.verifyGetKey(request, response);
+      int chunksCount = putKeyRequest.getPutKey().getKeyData().getChunksCount();
+      ContainerTestHelper.verifyGetKey(request, response, chunksCount);
 
       // Delete Key must fail on a closed container.
       request =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13579f92/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
index 13b04c3..c14c1b9 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
@@ -174,9 +174,7 @@ public class BenchMarkDatanodeDispatcher {
   private ContainerCommandRequestProto getCreateContainerCommand(long containerID) {
     CreateContainerRequestProto.Builder createRequest =
         CreateContainerRequestProto.newBuilder();
-    createRequest.setContainerData(
-        ContainerData.newBuilder().setContainerID(
-            containerID).build());
+    createRequest.setContainerID(containerID).build();
 
     ContainerCommandRequestProto.Builder request =
         ContainerCommandRequestProto.newBuilder();
@@ -245,10 +243,9 @@ public class BenchMarkDatanodeDispatcher {
     return request.build();
   }
 
-  private ContainerCommandRequestProto getGetKeyCommand(
-      BlockID blockID, String chunkKey) {
+  private ContainerCommandRequestProto getGetKeyCommand(BlockID blockID) {
     GetKeyRequestProto.Builder readKeyRequest = GetKeyRequestProto.newBuilder()
-        .setKeyData(getKeyData(blockID, chunkKey));
+        .setBlockID(blockID.getDatanodeBlockIDProtobuf());
     ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto
         .newBuilder()
         .setCmdType(ContainerProtos.Type.GetKey)
@@ -300,8 +297,7 @@ public class BenchMarkDatanodeDispatcher {
   @Benchmark
   public void getKey(BenchMarkDatanodeDispatcher bmdd) {
     BlockID blockID = getRandomBlockID();
-    String chunkKey = getNewChunkToWrite();
-    bmdd.dispatcher.dispatch(getGetKeyCommand(blockID, chunkKey));
+    bmdd.dispatcher.dispatch(getGetKeyCommand(blockID));
   }
 
   // Chunks writes from benchmark only reaches certain containers


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


[11/37] hadoop git commit: Merge remote-tracking branch 'apache-commit/trunk' into HDDS-48

Posted by bh...@apache.org.
Merge remote-tracking branch 'apache-commit/trunk' into HDDS-48


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

Branch: refs/heads/trunk
Commit: 418cff4820ba73cdbfd09fc5879b8b3aa4e62d5f
Parents: 7e228e5 9119b3c
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Jun 14 09:38:20 2018 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 14 09:38:20 2018 -0700

----------------------------------------------------------------------
 LICENSE.txt                                     |     4 +-
 .../main/resources/checkstyle/suppressions.xml  |     1 +
 .../hadoop-client-minicluster/pom.xml           |   154 +-
 .../src/main/bin/hadoop-functions.sh            |    34 +-
 .../java/org/apache/hadoop/fs/FileSystem.java   |    34 +
 .../hadoop/fs/FileSystemStorageStatistics.java  |     5 +-
 .../java/org/apache/hadoop/fs/FileUtil.java     |    13 +
 .../org/apache/hadoop/fs/FsUrlConnection.java   |     2 +-
 .../org/apache/hadoop/http/HttpServer2.java     |     7 +-
 .../rawcoder/AbstractNativeRawDecoder.java      |    51 +-
 .../rawcoder/AbstractNativeRawEncoder.java      |    49 +-
 .../rawcoder/NativeRSRawDecoder.java            |    19 +-
 .../rawcoder/NativeRSRawEncoder.java            |    19 +-
 .../rawcoder/NativeXORRawDecoder.java           |    19 +-
 .../rawcoder/NativeXORRawEncoder.java           |    19 +-
 .../security/IngressPortBasedResolver.java      |   100 +
 .../hadoop/security/SaslPropertiesResolver.java |    47 +-
 .../hadoop/security/WhitelistBasedResolver.java |    20 +-
 .../main/java/org/apache/hadoop/util/Shell.java |    18 +-
 .../hadoop/util/curator/ZKCuratorManager.java   |    14 +-
 .../hadoop-common/src/site/markdown/Metrics.md  |    12 +
 .../markdown/release/3.0.3/CHANGES.3.0.3.md     |   309 +
 .../release/3.0.3/RELEASENOTES.3.0.3.md         |    31 +
 .../fs/TestFileSystemStorageStatistics.java     |     6 +-
 .../java/org/apache/hadoop/fs/TestFileUtil.java |   206 +
 .../rawcoder/RawErasureCoderBenchmark.java      |     6 +
 .../security/TestIngressPortBasedResolver.java  |    59 +
 .../org/apache/hadoop/test/MetricsAsserts.java  |    22 +-
 .../util/TestCloseableReferenceCount.java       |    91 +
 .../org/apache/hadoop/util/TestDiskChecker.java |     6 +-
 .../hadoop/util/TestIntrusiveCollection.java    |   193 +
 .../hadoop/util/TestLimitInputStream.java       |    74 +
 .../java/org/apache/hadoop/util/TestShell.java  |     8 +
 .../org/apache/hadoop/util/TestStringUtils.java |    27 +
 .../hadoop/util/TestUTF8ByteArrayUtils.java     |    57 +
 .../src/test/scripts/hadoop_stop_daemon.bats    |    24 +-
 .../src/main/resources/kms-default.xml          |    35 +
 .../apache/hadoop/oncrpc/security/Verifier.java |    12 +-
 hadoop-dist/src/main/compose/ozoneperf/.env     |    17 +
 .../src/main/compose/ozoneperf/README.md        |    73 +
 .../src/main/compose/ozoneperf/compose-all.sh   |    18 +
 .../compose/ozoneperf/docker-compose-freon.yaml |    26 +
 .../main/compose/ozoneperf/docker-compose.yaml  |    77 +
 .../src/main/compose/ozoneperf/docker-config    |    37 +
 hadoop-dist/src/main/compose/ozoneperf/init.sh  |    21 +
 .../src/main/compose/ozoneperf/prometheus.yml   |    24 +
 .../hadoop/hdds/scm/XceiverClientGrpc.java      |    23 +
 .../hadoop/hdds/scm/XceiverClientHandler.java   |     2 +-
 .../org/apache/hadoop/hdds/client/BlockID.java  |     2 +-
 .../hadoop/hdds/conf/HddsConfServlet.java       |     5 +-
 .../container/common/helpers/ContainerInfo.java |    27 +-
 .../apache/hadoop/ozone/OzoneConfigKeys.java    |     2 +
 .../org/apache/hadoop/ozone/OzoneConsts.java    |     2 +
 .../apache/hadoop/utils/MetadataKeyFilters.java |   119 +-
 .../main/java/org/apache/ratis/RatisHelper.java |     4 +-
 hadoop-hdds/common/src/main/proto/hdds.proto    |     1 +
 .../apache/hadoop/utils/TestMetadataStore.java  |    78 +-
 .../container/common/helpers/ContainerData.java |    24 +-
 .../common/helpers/ContainerReport.java         |    12 +
 .../common/impl/ContainerManagerImpl.java       |    31 +-
 .../common/impl/ContainerReportManagerImpl.java |    67 -
 .../container/common/impl/KeyManagerImpl.java   |    12 +-
 .../interfaces/ContainerReportManager.java      |    30 -
 .../common/report/ContainerReportPublisher.java |    70 +
 .../common/report/NodeReportPublisher.java      |    40 +
 .../container/common/report/ReportManager.java  |   148 +
 .../common/report/ReportPublisher.java          |    96 +
 .../common/report/ReportPublisherFactory.java   |    71 +
 .../container/common/report/package-info.java   |    80 +
 .../statemachine/DatanodeStateMachine.java      |    27 +-
 .../common/statemachine/StateContext.java       |    59 +-
 .../background/BlockDeletingService.java        |    16 +-
 .../CloseContainerCommandHandler.java           |    21 +-
 .../commandhandler/CommandDispatcher.java       |     4 +
 .../DeleteBlocksCommandHandler.java             |     3 +
 .../states/endpoint/HeartbeatEndpointTask.java  |    30 +-
 .../states/endpoint/RegisterEndpointTask.java   |     2 +-
 .../common/transport/server/XceiverServer.java  |     7 +
 .../transport/server/XceiverServerGrpc.java     |     9 +
 .../transport/server/XceiverServerSpi.java      |     7 +
 .../server/ratis/ContainerStateMachine.java     |     3 +-
 .../server/ratis/XceiverServerRatis.java        |    56 +-
 .../container/common/utils/ContainerCache.java  |     6 +-
 .../container/ozoneimpl/OzoneContainer.java     |    62 +-
 .../commands/CloseContainerCommand.java         |    12 +-
 .../StorageContainerDatanodeProtocol.proto      |     2 +
 .../ozone/container/common/ScmTestMock.java     |    32 +
 .../ozone/container/common/package-info.java    |    22 +
 .../common/report/TestReportManager.java        |    52 +
 .../common/report/TestReportPublisher.java      |   185 +
 .../report/TestReportPublisherFactory.java      |    68 +
 .../container/common/report/package-info.java   |    22 +
 .../hadoop/hdds/scm/block/BlockManagerImpl.java |     9 +-
 .../hadoop/hdds/scm/block/DeletedBlockLog.java  |     3 +-
 .../hdds/scm/block/DeletedBlockLogImpl.java     |     8 +-
 .../container/CloseContainerEventHandler.java   |    84 +
 .../hdds/scm/container/ContainerMapping.java    |    40 +
 .../scm/container/ContainerStateManager.java    |    21 +
 .../hadoop/hdds/scm/container/Mapping.java      |    17 +
 .../scm/container/closer/ContainerCloser.java   |     7 +-
 .../scm/server/SCMDatanodeProtocolServer.java   |   119 +-
 .../SCMDatanodeContainerReportHandler.java      |    76 +
 .../report/SCMDatanodeHeartbeatDispatcher.java  |   189 +
 .../report/SCMDatanodeNodeReportHandler.java    |    43 +
 .../server/report/SCMDatanodeReportHandler.java |    83 +
 .../report/SCMDatanodeReportHandlerFactory.java |    82 +
 .../hdds/scm/server/report/package-info.java    |    57 +
 .../server-scm/src/main/webapps/scm/index.html  |     6 +-
 .../hadoop/hdds/scm/block/TestBlockManager.java |    16 +
 .../TestCloseContainerEventHandler.java         |   177 +
 .../scm/container/TestContainerMapping.java     |     6 +-
 .../container/closer/TestContainerCloser.java   |     3 +-
 .../TestSCMDatanodeContainerReportHandler.java  |    34 +
 .../TestSCMDatanodeHeartbeatDispatcher.java     |   138 +
 .../TestSCMDatanodeNodeReportHandler.java       |    36 +
 .../TestSCMDatanodeReportHandlerFactory.java    |    51 +
 .../hdds/scm/server/report/package-info.java    |    21 +
 .../hdfs/CannotObtainBlockLengthException.java  |    55 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |     6 +
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |    10 +-
 .../org/apache/hadoop/hdfs/ReaderStrategy.java  |     8 +
 .../ha/ConfiguredFailoverProxyProvider.java     |    39 +-
 ...apache.hadoop.security.token.TokenIdentifier |    17 +
 ...rg.apache.hadoop.security.token.TokenRenewer |    16 +
 .../hadoop/fs/TestUrlStreamHandlerFactory.java  |    36 +-
 .../ha/TestConfiguredFailoverProxyProvider.java |   264 +
 .../native/libhdfspp/include/hdfspp/ioservice.h |     1 +
 .../native/libhdfspp/lib/reader/datatransfer.h  |     2 +-
 .../native/libhdfspp/lib/rpc/sasl_protocol.cc   |    15 +-
 .../native/libhdfspp/tests/mock_connection.h    |     2 +-
 .../libhdfspp/tests/remote_block_reader_test.cc |     2 +-
 .../federation/router/ConnectionPool.java       |    12 +-
 .../driver/impl/StateStoreZooKeeperImpl.java    |     6 +-
 .../main/webapps/router/federationhealth.html   |     6 +-
 .../router/TestConnectionManager.java           |    13 +
 .../jdiff/Apache_Hadoop_HDFS_3.0.3.xml          |   322 +
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |     4 +-
 .../BlockPlacementPolicyDefault.java            |    10 +-
 .../server/blockmanagement/DatanodeManager.java |     2 +-
 .../datanode/checker/DatasetVolumeChecker.java  |    13 +-
 .../server/namenode/FSDirErasureCodingOp.java   |    23 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |    10 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |    23 +-
 .../hadoop/hdfs/server/namenode/FSImage.java    |    16 +-
 .../hdfs/server/namenode/FSNamesystem.java      |    35 +-
 .../server/namenode/FSPermissionChecker.java    |     2 +-
 .../hdfs/server/namenode/ha/EditLogTailer.java  |    38 +-
 .../namenode/metrics/NameNodeMetrics.java       |    59 +-
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |     2 +
 .../OfflineImageViewerPB.java                   |     3 +-
 .../offlineImageViewer/WebImageViewer.java      |    17 +
 ...apache.hadoop.security.token.TokenIdentifier |    17 -
 ...rg.apache.hadoop.security.token.TokenRenewer |    16 -
 .../src/main/resources/hdfs-default.xml         |    12 +
 .../src/main/webapps/datanode/datanode.html     |     6 +-
 .../src/main/webapps/hdfs/dfshealth.html        |     6 +-
 .../src/main/webapps/hdfs/dfshealth.js          |     8 +-
 .../src/main/webapps/hdfs/explorer.html         |    10 +-
 .../src/main/webapps/hdfs/explorer.js           |    34 +-
 .../src/main/webapps/journal/index.html         |     6 +-
 .../src/main/webapps/secondary/status.html      |     6 +-
 .../bootstrap-3.0.2/css/bootstrap-editable.css  |   655 -
 .../bootstrap-3.0.2/css/bootstrap.min.css       |     9 -
 .../fonts/glyphicons-halflings-regular.eot      |   Bin 20290 -> 0 bytes
 .../fonts/glyphicons-halflings-regular.svg      |   229 -
 .../fonts/glyphicons-halflings-regular.ttf      |   Bin 41236 -> 0 bytes
 .../fonts/glyphicons-halflings-regular.woff     |   Bin 23292 -> 0 bytes
 .../static/bootstrap-3.0.2/img/clear.png        |   Bin 509 -> 0 bytes
 .../static/bootstrap-3.0.2/img/loading.gif      |   Bin 1849 -> 0 bytes
 .../js/bootstrap-editable.min.js                |     7 -
 .../static/bootstrap-3.0.2/js/bootstrap.min.js  |     9 -
 .../bootstrap-3.3.7/css/bootstrap-editable.css  |   655 +
 .../bootstrap-3.3.7/css/bootstrap-theme.css     |   587 +
 .../bootstrap-3.3.7/css/bootstrap-theme.css.map |     1 +
 .../bootstrap-3.3.7/css/bootstrap-theme.min.css |     6 +
 .../css/bootstrap-theme.min.css.map             |     1 +
 .../static/bootstrap-3.3.7/css/bootstrap.css    |  6757 ++++++++
 .../bootstrap-3.3.7/css/bootstrap.css.map       |     1 +
 .../bootstrap-3.3.7/css/bootstrap.min.css       |     6 +
 .../bootstrap-3.3.7/css/bootstrap.min.css.map   |     1 +
 .../fonts/glyphicons-halflings-regular.eot      |   Bin 0 -> 20127 bytes
 .../fonts/glyphicons-halflings-regular.svg      |   288 +
 .../fonts/glyphicons-halflings-regular.ttf      |   Bin 0 -> 45404 bytes
 .../fonts/glyphicons-halflings-regular.woff     |   Bin 0 -> 23424 bytes
 .../fonts/glyphicons-halflings-regular.woff2    |   Bin 0 -> 18028 bytes
 .../js/bootstrap-editable.min.js                |     7 +
 .../static/bootstrap-3.3.7/js/bootstrap.js      |  2377 +++
 .../static/bootstrap-3.3.7/js/bootstrap.min.js  |     7 +
 .../webapps/static/bootstrap-3.3.7/js/npm.js    |    13 +
 .../src/main/webapps/static/dfs-dust.js         |     2 +-
 .../main/webapps/static/jquery-1.10.2.min.js    |     6 -
 .../src/main/webapps/static/jquery-3.3.1.min.js |     2 +
 .../src/main/webapps/static/moment.min.js       |     6 +-
 .../src/site/markdown/HdfsImageViewer.md        |     3 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |     5 +-
 .../hadoop/hdfs/TestDFSStripedInputStream.java  |     8 +-
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |    16 +
 .../apache/hadoop/hdfs/TestEncryptionZones.java |    29 +
 .../hdfs/TestErasureCodingExerciseAPIs.java     |     2 +-
 .../hadoop/hdfs/TestErasureCodingPolicies.java  |     2 +-
 .../hadoop/hdfs/TestReconstructStripedFile.java |     4 +-
 .../hdfs/server/datanode/TestBlockScanner.java  |     3 +-
 .../checker/TestDatasetVolumeChecker.java       |     3 +
 .../namenode/TestReencryptionHandler.java       |     7 +-
 .../server/namenode/ha/TestEditLogTailer.java   |    13 +-
 .../namenode/metrics/TestNameNodeMetrics.java   |    41 +
 .../TestGetContentSummaryWithSnapshot.java      |    26 +
 .../apache/hadoop/hdfs/tools/TestGetConf.java   |     6 +-
 .../TestOfflineImageViewer.java                 |    20 +
 .../hadoop/tools/TestHdfsConfigFields.java      |     1 +
 .../hadoop-hdfs/src/test/resources/editsStored  |   Bin 7909 -> 7909 bytes
 .../src/test/resources/editsStored.xml          |     2 +-
 .../mapreduce/v2/jobhistory/JHAdminConfig.java  |     9 +-
 .../java/org/apache/hadoop/mapred/Task.java     |    14 +-
 .../hadoop/mapreduce/FileSystemCounter.java     |     1 +
 .../src/main/resources/mapred-default.xml       |     9 +
 .../mapreduce/FileSystemCounter.properties      |     1 +
 .../jobhistory/TestHistoryViewerPrinter.java    |   378 +-
 .../lib/output/TestFileOutputCommitter.java     |    16 +-
 .../mapreduce/v2/hs/HistoryFileManager.java     |     8 +-
 .../mapreduce/v2/hs/webapp/HsJobBlock.java      |    18 +-
 .../mapreduce/v2/hs/webapp/HsJobsBlock.java     |    40 +-
 .../mapreduce/v2/hs/webapp/TestBlocks.java      |    17 +-
 .../mapreduce/v2/hs/webapp/TestHsJobBlock.java  |    20 +-
 .../hadoop/mapred/TestJavaSerialization.java    |    23 +-
 .../TestNativeCollectorOnlyHandler.java         |    11 +-
 .../robotframework/acceptance/ozone-shell.robot |    39 +-
 .../test/robotframework/acceptance/ozone.robot  |     4 +-
 .../hadoop/ozone/client/OzoneClientUtils.java   |    18 +-
 .../hadoop/ozone/client/rest/RestClient.java    |   113 +-
 .../ozone/client/rest/response/KeyInfo.java     |     4 +-
 .../hadoop/ozone/web/response/KeyInfo.java      |     4 +-
 hadoop-ozone/docs/content/CommandShell.md       |   141 +-
 hadoop-ozone/docs/content/GettingStarted.md     |   353 +-
 .../apache/hadoop/ozone/RatisTestHelper.java    |     8 +-
 .../ozone/TestOzoneConfigurationFields.java     |     1 +
 .../TestStorageContainerManagerHelper.java      |     8 +-
 .../ozone/client/rpc/TestOzoneRpcClient.java    |     2 +-
 .../common/TestBlockDeletingService.java        |    29 +-
 .../TestCloseContainerByPipeline.java           |   221 +
 .../TestCloseContainerHandler.java              |     7 +-
 ...TestGenerateOzoneRequiredConfigurations.java |    72 +-
 .../ozone/ksm/TestContainerReportWithKeys.java  |     2 +-
 .../hadoop/ozone/ksm/TestKeySpaceManager.java   |    16 +-
 .../ozone/ksm/TestKsmBlockVersioning.java       |     2 +-
 .../ksm/TestMultipleContainerReadWrite.java     |     2 +-
 .../hadoop/ozone/ozShell/TestOzoneShell.java    |    35 +-
 .../hadoop/ozone/scm/TestAllocateContainer.java |     2 +-
 .../apache/hadoop/ozone/scm/TestSCMMetrics.java |    20 +-
 .../ozone/scm/TestXceiverClientManager.java     |     8 +-
 .../ozone/web/TestOzoneRestWithMiniCluster.java |     2 +-
 .../hadoop/ozone/web/client/TestBuckets.java    |    37 +-
 .../ozone/web/client/TestBucketsRatis.java      |    32 +-
 .../hadoop/ozone/web/client/TestKeys.java       |    37 +-
 .../hadoop/ozone/web/client/TestKeysRatis.java  |     2 +-
 .../hadoop/ozone/web/client/TestVolume.java     |    38 +-
 .../ozone/web/client/TestVolumeRatis.java       |    29 +-
 .../web/storage/DistributedStorageHandler.java  |    21 +-
 .../ozone/ksm/KSMMetadataManagerImpl.java       |     9 +-
 .../hadoop/ozone/web/ozShell/Handler.java       |    49 +-
 .../src/main/webapps/ksm/index.html             |     6 +-
 .../hadoop/ozone/ksm/TestChunkStreams.java      |     2 +-
 hadoop-ozone/pom.xml                            |    14 +-
 .../org/apache/hadoop/ozone/freon/Freon.java    |     6 +-
 .../genesis/BenchMarkContainerStateMap.java     |    44 +-
 .../genesis/BenchMarkDatanodeDispatcher.java    |     2 +-
 .../genesis/BenchMarkMetadataStoreReads.java    |     2 +-
 .../genesis/BenchMarkMetadataStoreWrites.java   |     2 +-
 .../ozone/genesis/BenchMarkRocksDbStore.java    |     2 +-
 .../hadoop/ozone/genesis/GenesisUtil.java       |     2 +-
 hadoop-project-dist/pom.xml                     |     2 +-
 hadoop-project/pom.xml                          |    11 +-
 .../fs/azure/NativeAzureFileSystemHelper.java   |     4 +-
 .../hadoop/fs/azure/ITestContainerChecks.java   |     9 +-
 .../src/main/html/js/thirdparty/jquery.js       | 13607 +++++++++--------
 .../yarn/sls/appmaster/TestAMSimulator.java     |    13 +-
 .../hadoop-yarn/conf/yarn-env.sh                |     1 +
 .../hadoop-yarn/hadoop-yarn-api/pom.xml         |     4 -
 .../protocolrecords/GetApplicationsRequest.java |    22 +-
 .../hadoop/yarn/api/records/Resource.java       |     2 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |     9 +
 .../yarn/util/resource/ResourceUtils.java       |     4 +-
 .../pom.xml                                     |     4 -
 .../distributedshell/ApplicationMaster.java     |    24 +-
 .../applications/distributedshell/Client.java   |    16 +-
 .../hadoop-yarn-services-api/pom.xml            |    16 +
 .../yarn/service/client/ApiServiceClient.java   |    13 +-
 .../client/SystemServiceManagerImpl.java        |    29 +-
 .../hadoop/yarn/service/webapp/ApiServer.java   |     3 +-
 .../hadoop/yarn/service/ServiceClientTest.java  |     4 +-
 .../yarn/service/TestCleanupAfterKill.java      |    94 +
 .../client/TestSystemServiceManagerImpl.java    |    40 +-
 .../src/test/resources/yarn-site.xml            |    19 +
 .../hadoop-yarn-services-core/pom.xml           |     5 -
 .../hadoop/yarn/service/ServiceMaster.java      |    56 +-
 .../yarn/service/api/records/Configuration.java |     2 +-
 .../yarn/service/client/ServiceClient.java      |    51 +-
 .../yarn/service/monitor/probe/Probe.java       |     2 +-
 .../provider/AbstractClientProvider.java        |     2 +-
 .../provider/AbstractProviderService.java       |     2 +-
 .../provider/docker/DockerClientProvider.java   |     2 +-
 .../provider/docker/DockerProviderService.java  |    26 +-
 .../provider/tarball/TarballClientProvider.java |     2 +-
 .../yarn/service/utils/ServiceApiUtil.java      |     2 +-
 .../hadoop/yarn/service/utils/ServiceUtils.java |     4 +-
 .../hadoop/yarn/service/ServiceTestUtils.java   |   135 +
 .../yarn/service/TestYarnNativeServices.java    |   129 -
 .../hadoop-yarn/hadoop-yarn-client/pom.xml      |     4 -
 .../hadoop/yarn/client/api/AppAdminClient.java  |   273 -
 .../hadoop/yarn/client/cli/ClusterCLI.java      |     2 +-
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  |     2 +-
 .../apache/hadoop/yarn/client/cli/NodeCLI.java  |     4 +-
 .../hadoop/yarn/client/cli/RMAdminCLI.java      |     2 +-
 .../apache/hadoop/yarn/client/cli/TopCLI.java   |     6 +-
 .../hadoop/yarn/client/cli/TestLogsCLI.java     |     2 +-
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |     2 +-
 .../hadoop-yarn/hadoop-yarn-common/pom.xml      |     6 +-
 .../impl/pb/GetApplicationsRequestPBImpl.java   |    30 +-
 .../hadoop/yarn/client/api/AppAdminClient.java  |   285 +
 .../yarn/logaggregation/LogCLIHelpers.java      |     2 +-
 .../yarn/logaggregation/LogToolUtils.java       |     2 +-
 .../LogAggregationFileController.java           |     2 +-
 .../LogAggregationFileControllerFactory.java    |     2 +-
 .../LogAggregationIndexedFileController.java    |     2 +-
 .../nodelabels/CommonNodeLabelsManager.java     |     2 +-
 .../hadoop/yarn/nodelabels/RMNodeLabel.java     |     2 +-
 .../org/apache/hadoop/yarn/state/Graph.java     |     2 +-
 .../util/Log4jWarningErrorMetricsAppender.java  |     2 +-
 .../yarn/util/ProcfsBasedProcessTree.java       |     2 +-
 .../org/apache/hadoop/yarn/webapp/Router.java   |     2 +-
 .../org/apache/hadoop/yarn/webapp/WebApps.java  |     2 +-
 .../hadoop/yarn/webapp/hamlet/HamletImpl.java   |     6 +-
 .../hadoop/yarn/webapp/hamlet2/HamletImpl.java  |     6 +-
 .../hadoop/yarn/webapp/view/JQueryUI.java       |     8 +-
 .../hadoop/yarn/webapp/view/TextView.java       |     6 +-
 .../webapps/static/jquery/jquery-1.8.2.min.js   |     2 -
 .../webapps/static/jquery/jquery-3.3.1.min.js   |     2 +
 .../webapps/static/jt/jquery.jstree.js          |    42 +-
 .../src/main/resources/yarn-default.xml         |     9 +
 .../hadoop/yarn/api/BasePBImplRecordsTest.java  |     8 +-
 .../yarn/api/TestGetApplicationsRequest.java    |     7 +-
 .../hadoop/yarn/api/TestPBImplRecords.java      |     4 +-
 .../nodelabels/TestCommonNodeLabelsManager.java |     2 +-
 .../apache/hadoop/yarn/webapp/TestWebApp.java   |     2 +-
 .../hadoop-yarn/hadoop-yarn-registry/pom.xml    |     5 -
 .../client/api/RegistryOperationsFactory.java   |     2 +-
 .../registry/client/binding/JsonSerDeser.java   |     2 +-
 .../registry/client/binding/RegistryUtils.java  |    12 +-
 .../impl/FSRegistryOperationsService.java       |     6 +-
 .../client/impl/zk/RegistrySecurity.java        |     2 +-
 .../hadoop/registry/server/dns/RegistryDNS.java |     9 +-
 .../registry/server/dns/ReverseZoneUtils.java   |     2 +-
 .../integration/SelectByYarnPersistence.java    |     2 +-
 .../server/services/MicroZookeeperService.java  |     2 +-
 .../server/services/RegistryAdminService.java   |     2 +-
 .../hadoop/registry/RegistryTestHelper.java     |     2 +-
 .../registry/server/dns/TestRegistryDNS.java    |     9 +-
 .../ApplicationHistoryClientService.java        |     4 +-
 .../yarn/server/timeline/RollingLevelDB.java    |     2 +-
 .../timeline/RollingLevelDBTimelineStore.java   |     2 +-
 .../webapp/TestAHSWebServices.java              |     2 +-
 .../store/impl/SQLFederationStateStore.java     |     6 +-
 .../impl/ZookeeperFederationStateStore.java     |     8 +-
 .../utils/FederationStateStoreFacade.java       |     4 +-
 .../OpportunisticContainerAllocator.java        |     2 +-
 .../server/uam/UnmanagedApplicationManager.java |     2 +-
 .../util/timeline/TimelineServerUtils.java      |     2 +-
 .../yarn/server/webapp/AppAttemptBlock.java     |     6 +-
 .../hadoop/yarn/server/webapp/AppBlock.java     |     4 +-
 .../hadoop/yarn/server/webapp/AppsBlock.java    |    14 +-
 .../hadoop/yarn/server/webapp/WebServices.java  |     4 +-
 .../hadoop-yarn-server-nodemanager/pom.xml      |    25 +-
 .../nodemanager/DefaultContainerExecutor.java   |     2 +-
 .../WindowsSecureContainerExecutor.java         |     2 +-
 .../containermanager/AuxServices.java           |     6 +-
 .../containermanager/container/Container.java   |     8 +
 .../container/ContainerImpl.java                |     7 +-
 .../privileged/PrivilegedOperationExecutor.java |     2 +-
 .../linux/resources/CGroupsHandlerImpl.java     |     4 +-
 .../linux/resources/DefaultOOMHandler.java      |   249 +-
 .../runtime/DockerLinuxContainerRuntime.java    |    49 +-
 .../runtime/LinuxContainerRuntimeConstants.java |     2 +
 .../executor/ContainerSignalContext.java        |     2 +-
 .../timelineservice/NMTimelinePublisher.java    |    42 +-
 .../nodemanager/webapp/NMWebServices.java       |     2 +-
 .../impl/container-executor.c                   |    58 +-
 .../impl/container-executor.h                   |     4 +
 .../main/native/container-executor/impl/main.c  |    19 +-
 .../container-executor/impl/utils/docker-util.c |     7 +-
 .../test/utils/test_docker_util.cc              |    48 +-
 .../TestLocalDirsHandlerService.java            |     4 +-
 .../containermanager/TestAuxServices.java       |    48 +
 .../launcher/TestContainerLaunch.java           |    60 +-
 .../TestCGroupElasticMemoryController.java      |    76 +-
 .../linux/resources/TestDefaultOOMHandler.java  |   922 +-
 .../runtime/TestDockerContainerRuntime.java     |    34 +-
 .../TestAppLogAggregatorImpl.java               |     2 +-
 .../TestLogAggregationService.java              |     2 +-
 .../TestNonAggregatingLogHandler.java           |     4 +-
 .../nodemanager/webapp/MockContainer.java       |     5 +
 .../conf/capacity-scheduler.xml                 |     2 +
 .../hadoop-yarn-server-resourcemanager/pom.xml  |     4 -
 .../server/resourcemanager/ClientRMService.java |    12 +-
 .../metrics/TimelineServiceV1Publisher.java     |     6 +-
 .../metrics/TimelineServiceV2Publisher.java     |     8 +-
 .../ProportionalCapacityPreemptionPolicy.java   |     2 +-
 .../placement/QueuePlacementRuleUtils.java      |     2 +-
 .../UserGroupMappingPlacementRule.java          |     2 +-
 .../recovery/ZKRMStateStore.java                |     2 +-
 .../reservation/NoOverCommitPolicy.java         |     5 +-
 .../server/resourcemanager/rmapp/RMAppImpl.java |    32 +-
 .../rmapp/attempt/RMAppAttemptMetrics.java      |     2 +-
 .../scheduler/AbstractYarnScheduler.java        |    19 +-
 .../scheduler/SchedulerApplicationAttempt.java  |     6 +-
 .../scheduler/SchedulerUtils.java               |     2 +-
 .../scheduler/capacity/AbstractCSQueue.java     |     2 +-
 .../scheduler/capacity/CapacityScheduler.java   |     4 +-
 .../scheduler/capacity/LeafQueue.java           |     2 +-
 .../scheduler/capacity/ParentQueue.java         |     2 +-
 .../scheduler/capacity/QueueCapacities.java     |     2 +-
 .../allocator/RegularContainerAllocator.java    |     2 +-
 .../capacity/conf/ZKConfigurationStore.java     |     2 +-
 .../constraint/AllocationTagsManager.java       |     2 +-
 .../algorithm/LocalAllocationTagsManager.java   |     2 +-
 .../fair/AllocationFileLoaderService.java       |    23 +-
 .../scheduler/fair/FairScheduler.java           |     8 +-
 .../scheduler/fair/QueueManager.java            |    41 +-
 .../scheduler/fifo/FifoScheduler.java           |     6 +-
 .../security/DelegationTokenRenewer.java        |    10 +-
 .../webapp/FairSchedulerAppsBlock.java          |    10 +-
 .../webapp/RMAppAttemptBlock.java               |     8 +-
 .../resourcemanager/webapp/RMAppBlock.java      |     4 +-
 .../resourcemanager/webapp/RMAppsBlock.java     |    14 +-
 .../webapp/dao/AppAttemptInfo.java              |     2 +-
 .../TestContainerResourceUsage.java             |     2 +-
 .../recovery/TestZKRMStateStore.java            |    29 +-
 .../TestZKRMStateStoreZKClientConnections.java  |    22 +-
 .../TestRMAppAttemptImplDiagnostics.java        |     2 +-
 .../capacity/TestCapacityScheduler.java         |    44 +-
 ...stCapacitySchedulerAutoCreatedQueueBase.java |     4 +-
 .../fair/TestAllocationFileLoaderService.java   |    96 +-
 .../scheduler/fair/TestFairScheduler.java       |    43 +-
 .../scheduler/fifo/TestFifoScheduler.java       |    44 +-
 .../security/TestDelegationTokenRenewer.java    |    24 +
 .../clientrm/FederationClientInterceptor.java   |    58 +-
 .../yarn/server/router/webapp/AppsBlock.java    |     6 +-
 .../webapp/FederationInterceptorREST.java       |    70 +-
 .../pom.xml                                     |     5 -
 .../storage/TimelineSchemaCreator.java          |     2 +-
 .../hadoop-yarn-server-timelineservice/pom.xml  |     5 -
 .../AppLevelTimelineCollectorWithAgg.java       |    14 +-
 .../collector/TimelineCollectorWebService.java  |    19 +-
 .../reader/TimelineReaderManager.java           |    21 +
 .../reader/TimelineReaderUtils.java             |     2 +-
 .../reader/TimelineReaderWebServices.java       |    18 +-
 .../reader/TimelineReaderWebServicesUtils.java  |     2 +-
 .../hadoop-yarn-server-web-proxy/pom.xml        |    13 +
 .../server/webproxy/amfilter/AmIpFilter.java    |    19 +-
 .../webproxy/amfilter/TestSecureAmFilter.java   |   159 +
 .../src/test/resources/krb5.conf                |    33 +
 .../src/site/markdown/CapacityScheduler.md      |     5 +
 .../src/site/markdown/DockerContainers.md       |    48 +-
 .../src/site/markdown/Federation.md             |    19 +-
 .../src/site/markdown/NodeManagerCgroups.md     |     2 +-
 .../src/site/markdown/yarn-service/Examples.md  |    37 +-
 .../markdown/yarn-service/YarnServiceAPI.md     |     2 +-
 .../main/webapp/app/adapters/yarn-app-log.js    |    68 +
 .../src/main/webapp/app/adapters/yarn-log.js    |     6 -
 .../webapp/app/components/deploy-service.js     |     5 +-
 .../webapp/app/controllers/yarn-app/logs.js     |    17 +-
 .../src/main/webapp/app/initializers/loader.js  |     2 +-
 .../src/main/webapp/app/models/yarn-app-log.js  |    25 +
 .../main/webapp/app/models/yarn-servicedef.js   |     8 +-
 .../webapp/app/routes/yarn-node-container.js    |     2 +-
 .../main/webapp/app/serializers/yarn-app-log.js |    38 +
 .../app/templates/components/deploy-service.hbs |    11 +
 .../webapp/app/templates/yarn-container-log.hbs |     2 +-
 .../src/main/webapp/app/utils/converter.js      |    10 +
 .../src/main/webapp/app/utils/info-seeder.js    |     1 +
 .../tests/unit/adapters/yarn-app-log-test.js    |    30 +
 480 files changed, 28447 insertions(+), 9955 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/418cff48/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
----------------------------------------------------------------------
diff --cc hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
index 06db1e1,0000000..d697bbf
mode 100644,000000..100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
@@@ -1,463 -1,0 +1,533 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + * <p>
 + * http://www.apache.org/licenses/LICENSE-2.0
 + * <p>
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.utils;
 +
++import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows;
++
 +import com.google.common.collect.Lists;
 +import org.apache.commons.io.FileUtils;
 +import org.apache.commons.lang3.tuple.ImmutablePair;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 +import org.apache.hadoop.hdfs.DFSUtil;
 +import org.apache.hadoop.hdfs.DFSUtilClient;
 +import org.apache.hadoop.ozone.OzoneConfigKeys;
 +import org.apache.hadoop.test.GenericTestUtils;
 +import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
 +import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
 +import org.junit.After;
 +import org.junit.Assert;
 +import org.junit.Before;
 +import org.junit.Rule;
 +import org.junit.Test;
 +import org.junit.rules.ExpectedException;
 +import org.junit.runner.RunWith;
 +import org.junit.runners.Parameterized;
 +import org.slf4j.event.Level;
 +
 +import java.io.File;
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collection;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.UUID;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import static junit.framework.TestCase.assertTrue;
 +import static org.junit.runners.Parameterized.Parameters;
 +
 +/**
 + * Test class for ozone metadata store.
 + */
 +@RunWith(Parameterized.class)
 +public class TestMetadataStore {
 +
 +  private final String storeImpl;
 +
 +  public TestMetadataStore(String metadataImpl) {
 +    this.storeImpl = metadataImpl;
 +  }
 +
 +  @Parameters
 +  public static Collection<Object[]> data() {
 +    return Arrays.asList(new Object[][] {
 +        {OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB},
 +        {OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_ROCKSDB}
 +    });
 +  }
 +
 +  private MetadataStore store;
 +  private File testDir;
 +  private final static int MAX_GETRANGE_LENGTH = 100;
 +
 +  @Rule
 +  public ExpectedException expectedException = ExpectedException.none();
 +
 +  @Before
 +  public void init() throws IOException {
++    if (OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_ROCKSDB.equals(storeImpl)) {
++      // The initialization of RocksDB fails on Windows
++      assumeNotWindows();
++    }
++
 +    testDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
 +        + "-" + storeImpl.toLowerCase());
 +
 +    Configuration conf = new OzoneConfiguration();
 +    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
 +
 +    store = MetadataStoreBuilder.newBuilder()
 +        .setConf(conf)
 +        .setCreateIfMissing(true)
 +        .setDbFile(testDir)
 +        .build();
 +
 +    // Add 20 entries.
 +    // {a0 : a-value0} to {a9 : a-value9}
 +    // {b0 : b-value0} to {b9 : b-value9}
 +    for (int i=0; i<10; i++) {
 +      store.put(getBytes("a" + i), getBytes("a-value" + i));
 +      store.put(getBytes("b" + i), getBytes("b-value" + i));
 +    }
 +  }
 +
 +  @Test
 +  public void testMetaStoreConfigDifferentFromType() throws IOException {
 +
 +    Configuration conf = new OzoneConfiguration();
 +    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
 +    String dbType;
 +    GenericTestUtils.setLogLevel(MetadataStoreBuilder.LOG, Level.DEBUG);
 +    GenericTestUtils.LogCapturer logCapturer =
 +        GenericTestUtils.LogCapturer.captureLogs(MetadataStoreBuilder.LOG);
 +    if(storeImpl.equals(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB)) {
 +      dbType = "RocksDB";
 +    } else {
 +      dbType = "LevelDB";
 +    }
 +
 +    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
 +        + "-" + dbType.toLowerCase() + "-test");
 +    MetadataStore dbStore = MetadataStoreBuilder.newBuilder().setConf(conf)
 +        .setCreateIfMissing(true).setDbFile(dbDir).setDBType(dbType).build();
 +    assertTrue(logCapturer.getOutput().contains("Using dbType " + dbType + "" +
 +        " for metastore"));
 +    dbStore.close();
 +    dbStore.destroy();
 +    FileUtils.deleteDirectory(dbDir);
 +
 +  }
 +
 +  @Test
 +  public void testdbTypeNotSet() throws IOException {
 +
 +    Configuration conf = new OzoneConfiguration();
 +    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
 +    GenericTestUtils.setLogLevel(MetadataStoreBuilder.LOG, Level.DEBUG);
 +    GenericTestUtils.LogCapturer logCapturer =
 +        GenericTestUtils.LogCapturer.captureLogs(MetadataStoreBuilder.LOG);
 +
 +
 +    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
 +        + "-" + storeImpl.toLowerCase() + "-test");
 +    MetadataStore dbStore = MetadataStoreBuilder.newBuilder().setConf(conf)
 +        .setCreateIfMissing(true).setDbFile(dbDir).build();
 +    assertTrue(logCapturer.getOutput().contains("dbType is null, using dbType" +
 +        " " + storeImpl));
 +    dbStore.close();
 +    dbStore.destroy();
 +    FileUtils.deleteDirectory(dbDir);
 +
 +  }
 +
 +  @After
 +  public void cleanup() throws IOException {
-     store.close();
-     store.destroy();
-     FileUtils.deleteDirectory(testDir);
++    if (store != null) {
++      store.close();
++      store.destroy();
++    }
++    if (testDir != null) {
++      FileUtils.deleteDirectory(testDir);
++    }
 +  }
 +
 +  private byte[] getBytes(String str) {
 +    return str == null ? null :
 +        DFSUtilClient.string2Bytes(str);
 +  }
 +
 +  private String getString(byte[] bytes) {
 +    return bytes == null ? null :
 +        DFSUtilClient.bytes2String(bytes);
 +  }
 +
 +  @Test
 +  public void testGetDelete() throws IOException {
 +    for (int i=0; i<10; i++) {
 +      byte[] va = store.get(getBytes("a" + i));
 +      Assert.assertEquals("a-value" + i, getString(va));
 +
 +      byte[] vb = store.get(getBytes("b" + i));
 +      Assert.assertEquals("b-value" + i, getString(vb));
 +    }
 +
 +    String keyToDel = "del-" + UUID.randomUUID().toString();
 +    store.put(getBytes(keyToDel), getBytes(keyToDel));
 +    Assert.assertEquals(keyToDel, getString(store.get(getBytes(keyToDel))));
 +    store.delete(getBytes(keyToDel));
 +    Assert.assertEquals(null, store.get(getBytes(keyToDel)));
 +  }
 +
 +  @Test
 +  public void testPeekFrom() throws IOException {
 +    // Test peek from an element that has prev as well as next
 +    testPeek("a3", "a2", "a4");
 +
 +    // Test peek from an element that only has prev
 +    testPeek("b9", "b8", null);
 +
 +    // Test peek from an element that only has next
 +    testPeek("a0", null, "a1");
 +  }
 +
 +  private String getExpectedValue(String key) {
 +    if (key == null) {
 +      return null;
 +    }
 +    char[] arr = key.toCharArray();
 +    return new StringBuffer().append(arr[0]).append("-value")
 +        .append(arr[arr.length - 1]).toString();
 +  }
 +
 +  private void testPeek(String peekKey, String prevKey, String nextKey)
 +      throws IOException {
 +    // Look for current
 +    String k = null;
 +    String v = null;
 +    ImmutablePair<byte[], byte[]> current =
 +        store.peekAround(0, getBytes(peekKey));
 +    if (current != null) {
 +      k = getString(current.getKey());
 +      v = getString(current.getValue());
 +    }
 +    Assert.assertEquals(peekKey, k);
 +    Assert.assertEquals(v, getExpectedValue(peekKey));
 +
 +    // Look for prev
 +    k = null;
 +    v = null;
 +    ImmutablePair<byte[], byte[]> prev =
 +        store.peekAround(-1, getBytes(peekKey));
 +    if (prev != null) {
 +      k = getString(prev.getKey());
 +      v = getString(prev.getValue());
 +    }
 +    Assert.assertEquals(prevKey, k);
 +    Assert.assertEquals(v, getExpectedValue(prevKey));
 +
 +    // Look for next
 +    k = null;
 +    v = null;
 +    ImmutablePair<byte[], byte[]> next =
 +        store.peekAround(1, getBytes(peekKey));
 +    if (next != null) {
 +      k = getString(next.getKey());
 +      v = getString(next.getValue());
 +    }
 +    Assert.assertEquals(nextKey, k);
 +    Assert.assertEquals(v, getExpectedValue(nextKey));
 +  }
 +
 +  @Test
 +  public void testIterateKeys() throws IOException {
 +    // iterate keys from b0
 +    ArrayList<String> result = Lists.newArrayList();
 +    store.iterate(getBytes("b0"), (k, v) -> {
 +      // b-value{i}
 +      String value = getString(v);
 +      char num = value.charAt(value.length() - 1);
 +      // each value adds 1
 +      int i = Character.getNumericValue(num) + 1;
 +      value =  value.substring(0, value.length() - 1) + i;
 +      result.add(value);
 +      return true;
 +    });
 +
 +    Assert.assertFalse(result.isEmpty());
 +    for (int i=0; i<result.size(); i++) {
 +      Assert.assertEquals("b-value" + (i+1), result.get(i));
 +    }
 +
 +    // iterate from a non exist key
 +    result.clear();
 +    store.iterate(getBytes("xyz"), (k, v) -> {
 +      result.add(getString(v));
 +      return true;
 +    });
 +    Assert.assertTrue(result.isEmpty());
 +
 +    // iterate from the beginning
 +    result.clear();
 +    store.iterate(null, (k, v) -> {
 +      result.add(getString(v));
 +      return true;
 +    });
 +    Assert.assertEquals(20, result.size());
 +  }
 +
 +  @Test
 +  public void testGetRangeKVs() throws IOException {
 +    List<Map.Entry<byte[], byte[]>> result = null;
 +
 +    // Set empty startKey will return values from beginning.
 +    result = store.getRangeKVs(null, 5);
 +    Assert.assertEquals(5, result.size());
 +    Assert.assertEquals("a-value2", getString(result.get(2).getValue()));
 +
 +    // Empty list if startKey doesn't exist.
 +    result = store.getRangeKVs(getBytes("a12"), 5);
 +    Assert.assertEquals(0, result.size());
 +
 +    // Returns max available entries after a valid startKey.
 +    result = store.getRangeKVs(getBytes("b0"), MAX_GETRANGE_LENGTH);
 +    Assert.assertEquals(10, result.size());
 +    Assert.assertEquals("b0", getString(result.get(0).getKey()));
 +    Assert.assertEquals("b-value0", getString(result.get(0).getValue()));
 +    result = store.getRangeKVs(getBytes("b0"), 5);
 +    Assert.assertEquals(5, result.size());
 +
 +    // Both startKey and count are honored.
 +    result = store.getRangeKVs(getBytes("a9"), 2);
 +    Assert.assertEquals(2, result.size());
 +    Assert.assertEquals("a9", getString(result.get(0).getKey()));
 +    Assert.assertEquals("a-value9", getString(result.get(0).getValue()));
 +    Assert.assertEquals("b0", getString(result.get(1).getKey()));
 +    Assert.assertEquals("b-value0", getString(result.get(1).getValue()));
 +
 +    // Filter keys by prefix.
 +    // It should returns all "b*" entries.
-     MetadataKeyFilter filter1 = new KeyPrefixFilter("b");
++    MetadataKeyFilter filter1 = new KeyPrefixFilter().addFilter("b");
 +    result = store.getRangeKVs(null, 100, filter1);
 +    Assert.assertEquals(10, result.size());
 +    Assert.assertTrue(result.stream().allMatch(entry ->
 +        new String(entry.getKey()).startsWith("b")
 +    ));
 +    Assert.assertEquals(20, filter1.getKeysScannedNum());
 +    Assert.assertEquals(10, filter1.getKeysHintedNum());
 +    result = store.getRangeKVs(null, 3, filter1);
 +    Assert.assertEquals(3, result.size());
 +    result = store.getRangeKVs(getBytes("b3"), 1, filter1);
 +    Assert.assertEquals("b-value3", getString(result.get(0).getValue()));
 +
 +    // Define a customized filter that filters keys by suffix.
 +    // Returns all "*2" entries.
 +    MetadataKeyFilter filter2 = (preKey, currentKey, nextKey)
 +        -> getString(currentKey).endsWith("2");
 +    result = store.getRangeKVs(null, MAX_GETRANGE_LENGTH, filter2);
 +    Assert.assertEquals(2, result.size());
 +    Assert.assertEquals("a2", getString(result.get(0).getKey()));
 +    Assert.assertEquals("b2", getString(result.get(1).getKey()));
 +    result = store.getRangeKVs(null, 1, filter2);
 +    Assert.assertEquals(1, result.size());
 +    Assert.assertEquals("a2", getString(result.get(0).getKey()));
 +
 +    // Apply multiple filters.
 +    result = store.getRangeKVs(null, MAX_GETRANGE_LENGTH, filter1, filter2);
 +    Assert.assertEquals(1, result.size());
 +    Assert.assertEquals("b2", getString(result.get(0).getKey()));
 +    Assert.assertEquals("b-value2", getString(result.get(0).getValue()));
 +
 +    // If filter is null, no effect.
 +    result = store.getRangeKVs(null, 1, null);
 +    Assert.assertEquals(1, result.size());
 +    Assert.assertEquals("a0", getString(result.get(0).getKey()));
 +  }
 +
 +  @Test
 +  public void testGetSequentialRangeKVs() throws IOException {
 +    MetadataKeyFilter suffixFilter = (preKey, currentKey, nextKey)
 +        -> DFSUtil.bytes2String(currentKey).endsWith("2");
 +    // Suppose to return a2 and b2
 +    List<Map.Entry<byte[], byte[]>> result =
 +        store.getRangeKVs(null, MAX_GETRANGE_LENGTH, suffixFilter);
 +    Assert.assertEquals(2, result.size());
 +    Assert.assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
 +    Assert.assertEquals("b2", DFSUtil.bytes2String(result.get(1).getKey()));
 +
 +    // Suppose to return just a2, because when it iterates to a3,
 +    // the filter no long matches and it should stop from there.
 +    result = store.getSequentialRangeKVs(null,
 +        MAX_GETRANGE_LENGTH, suffixFilter);
 +    Assert.assertEquals(1, result.size());
 +    Assert.assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
 +  }
 +
 +  @Test
 +  public void testGetRangeLength() throws IOException {
 +    List<Map.Entry<byte[], byte[]>> result = null;
 +
 +    result = store.getRangeKVs(null, 0);
 +    Assert.assertEquals(0, result.size());
 +
 +    result = store.getRangeKVs(null, 1);
 +    Assert.assertEquals(1, result.size());
 +
 +    // Count less than zero is invalid.
 +    expectedException.expect(IllegalArgumentException.class);
 +    expectedException.expectMessage("Invalid count given");
 +    store.getRangeKVs(null, -1);
 +  }
 +
 +  @Test
 +  public void testInvalidStartKey() throws IOException {
 +    // If startKey is invalid, the returned list should be empty.
 +    List<Map.Entry<byte[], byte[]>> kvs =
 +        store.getRangeKVs(getBytes("unknownKey"), MAX_GETRANGE_LENGTH);
 +    Assert.assertEquals(kvs.size(), 0);
 +  }
 +
 +  @Test
 +  public void testDestroyDB() throws IOException {
 +    // create a new DB to test db destroy
 +    Configuration conf = new OzoneConfiguration();
 +    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
 +
 +    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
 +        + "-" + storeImpl.toLowerCase() + "-toDestroy");
 +    MetadataStore dbStore = MetadataStoreBuilder.newBuilder()
 +        .setConf(conf)
 +        .setCreateIfMissing(true)
 +        .setDbFile(dbDir)
 +        .build();
 +
 +    dbStore.put(getBytes("key1"), getBytes("value1"));
 +    dbStore.put(getBytes("key2"), getBytes("value2"));
 +
 +    Assert.assertFalse(dbStore.isEmpty());
 +    Assert.assertTrue(dbDir.exists());
 +    Assert.assertTrue(dbDir.listFiles().length > 0);
 +
 +    dbStore.destroy();
 +
 +    Assert.assertFalse(dbDir.exists());
 +  }
 +
 +  @Test
 +  public void testBatchWrite() throws IOException {
 +    Configuration conf = new OzoneConfiguration();
 +    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
 +
 +    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
 +        + "-" + storeImpl.toLowerCase() + "-batchWrite");
 +    MetadataStore dbStore = MetadataStoreBuilder.newBuilder()
 +        .setConf(conf)
 +        .setCreateIfMissing(true)
 +        .setDbFile(dbDir)
 +        .build();
 +
 +    List<String> expectedResult = Lists.newArrayList();
 +    for (int i = 0; i<10; i++) {
 +      dbStore.put(getBytes("batch-" + i), getBytes("batch-value-" + i));
 +      expectedResult.add("batch-" + i);
 +    }
 +
 +    BatchOperation batch = new BatchOperation();
 +    batch.delete(getBytes("batch-2"));
 +    batch.delete(getBytes("batch-3"));
 +    batch.delete(getBytes("batch-4"));
 +    batch.put(getBytes("batch-new-2"), getBytes("batch-new-value-2"));
 +
 +    expectedResult.remove("batch-2");
 +    expectedResult.remove("batch-3");
 +    expectedResult.remove("batch-4");
 +    expectedResult.add("batch-new-2");
 +
 +    dbStore.writeBatch(batch);
 +
 +    Iterator<String> it = expectedResult.iterator();
 +    AtomicInteger count = new AtomicInteger(0);
 +    dbStore.iterate(null, (key, value) -> {
 +      count.incrementAndGet();
 +      return it.hasNext() && it.next().equals(getString(key));
 +    });
 +
 +    Assert.assertEquals(8, count.get());
 +  }
++
++  @Test
++  public void testKeyPrefixFilter() throws IOException {
++    List<Map.Entry<byte[], byte[]>> result = null;
++    RuntimeException exception = null;
++
++    try {
++      new KeyPrefixFilter().addFilter("b0", true).addFilter("b");
++    } catch (IllegalArgumentException e) {
++      exception = e;
++    }
++    Assert.assertTrue(
++        exception.getMessage().contains("KeyPrefix: b already rejected"));
++
++    try {
++      new KeyPrefixFilter().addFilter("b0").addFilter("b", true);
++    } catch (IllegalArgumentException e) {
++      exception = e;
++    }
++    Assert.assertTrue(
++        exception.getMessage().contains("KeyPrefix: b already accepted"));
++
++    try {
++      new KeyPrefixFilter().addFilter("b", true).addFilter("b0");
++    } catch (IllegalArgumentException e) {
++      exception = e;
++    }
++    Assert.assertTrue(
++        exception.getMessage().contains("KeyPrefix: b0 already rejected"));
++
++    try {
++      new KeyPrefixFilter().addFilter("b").addFilter("b0", true);
++    } catch (IllegalArgumentException e) {
++      exception = e;
++    }
++    Assert.assertTrue(
++        exception.getMessage().contains("KeyPrefix: b0 already accepted"));
++
++    MetadataKeyFilter filter1 = new KeyPrefixFilter(true)
++            .addFilter("a0")
++            .addFilter("a1")
++            .addFilter("b", true);
++    result = store.getRangeKVs(null, 100, filter1);
++    Assert.assertEquals(2, result.size());
++    Assert.assertTrue(result.stream()
++        .anyMatch(entry -> new String(entry.getKey()).startsWith("a0"))
++        && result.stream()
++        .anyMatch(entry -> new String(entry.getKey()).startsWith("a1")));
++
++    filter1 = new KeyPrefixFilter(true).addFilter("b", true);
++    result = store.getRangeKVs(null, 100, filter1);
++    Assert.assertEquals(0, result.size());
++
++    filter1 = new KeyPrefixFilter().addFilter("b", true);
++    result = store.getRangeKVs(null, 100, filter1);
++    Assert.assertEquals(10, result.size());
++    Assert.assertTrue(result.stream()
++        .allMatch(entry -> new String(entry.getKey()).startsWith("a")));
++  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/418cff48/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
----------------------------------------------------------------------
diff --cc hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
index d9dd360,7f611b9..c63eb73
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
@@@ -121,14 -121,12 +121,15 @@@ public final class ContainerCache exten
     * Returns a DB handle if available, create the handler otherwise.
     *
     * @param containerID - ID of the container.
 +   * @param containerDBType - DB type of the container.
 +   * @param containerDBPath - DB path of the container.
     * @return MetadataStore.
     */
 -  public MetadataStore getDB(long containerID, String containerDBPath)
 +  public MetadataStore getDB(long containerID, String containerDBType, String
 +                             containerDBPath)
        throws IOException {
-     Preconditions.checkState(containerID >= 0, "Container ID cannot be negative.");
+     Preconditions.checkState(containerID >= 0,
+         "Container ID cannot be negative.");
      lock.lock();
      try {
        MetadataStore db = (MetadataStore) this.get(containerID);


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


[32/37] hadoop git commit: HDDS-211. Add a create container Lock. Contributed by Bharat Viswanadham

Posted by bh...@apache.org.
HDDS-211. Add a create container Lock. Contributed by Bharat Viswanadham


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

Branch: refs/heads/trunk
Commit: e899c4cc01df3b657893314adcdbb411dfbf3764
Parents: 7dcf587
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Fri Jul 6 15:02:40 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Fri Jul 6 15:02:40 2018 -0700

----------------------------------------------------------------------
 .../container/common/impl/HddsDispatcher.java   |  4 +++-
 .../container/common/interfaces/Handler.java    |  8 +++-----
 .../container/keyvalue/KeyValueHandler.java     | 20 +++++++++++++++++---
 .../genesis/BenchMarkDatanodeDispatcher.java    |  7 -------
 4 files changed, 23 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e899c4cc/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
index 25700f9..f0c2aa9 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
@@ -119,8 +119,10 @@ public class HddsDispatcher implements ContainerDispatcher {
     responseProto = handler.handle(msg, container);
     if (responseProto != null) {
       metrics.incContainerOpsLatencies(cmdType, System.nanoTime() - startTime);
+      return responseProto;
+    } else {
+      return ContainerUtils.unsupportedRequest(msg);
     }
-    return responseProto;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e899c4cc/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
index 2725f59..15eed4f 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
  * Dispatcher sends ContainerCommandRequests to Handler. Each Container Type
  * should have an implementation for Handler.
  */
-public class Handler {
+public abstract class Handler {
 
   protected final Configuration conf;
   protected final ContainerSet containerSet;
@@ -64,10 +64,8 @@ public class Handler {
     }
   }
 
-  public ContainerCommandResponseProto handle(
-      ContainerCommandRequestProto msg, Container container) {
-    return null;
-  }
+  public abstract ContainerCommandResponseProto handle(
+      ContainerCommandRequestProto msg, Container container);
 
   public void setScmID(String scmId) {
     this.scmID = scmId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e899c4cc/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index 110d0b4..3806ed6 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -62,6 +62,7 @@ import org.apache.hadoop.ozone.container.keyvalue.impl.KeyManagerImpl;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
 import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
 import org.apache.hadoop.ozone.container.keyvalue.interfaces.KeyManager;
+import org.apache.hadoop.util.AutoCloseableLock;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -70,6 +71,7 @@ import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.locks.ReentrantLock;
 
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.CONTAINER_INTERNAL_ERROR;
@@ -102,6 +104,7 @@ public class KeyValueHandler extends Handler {
   private final ChunkManager chunkManager;
   private VolumeChoosingPolicy volumeChoosingPolicy;
   private final int maxContainerSizeGB;
+  private final AutoCloseableLock handlerLock;
 
 
   public KeyValueHandler(Configuration config, ContainerSet contSet,
@@ -115,6 +118,9 @@ public class KeyValueHandler extends Handler {
     maxContainerSizeGB = config.getInt(ScmConfigKeys
             .OZONE_SCM_CONTAINER_SIZE_GB, ScmConfigKeys
         .OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+    // this handler lock is used for synchronizing createContainer Requests,
+    // so using a fair lock here.
+    handlerLock = new AutoCloseableLock(new ReentrantLock(true));
   }
 
   @Override
@@ -159,7 +165,6 @@ public class KeyValueHandler extends Handler {
     case GetSmallFile:
       return handleGetSmallFile(request, kvContainer);
     }
-
     return null;
   }
 
@@ -204,10 +209,19 @@ public class KeyValueHandler extends Handler {
         newContainerData, conf);
 
     try {
-      newContainer.create(volumeSet, volumeChoosingPolicy, scmID);
-      containerSet.addContainer(newContainer);
+      handlerLock.acquire();
+      if (containerSet.getContainer(containerID) == null) {
+        newContainer.create(volumeSet, volumeChoosingPolicy, scmID);
+        containerSet.addContainer(newContainer);
+      } else {
+        throw new StorageContainerException("Container already exists with " +
+            "container Id " + containerID, ContainerProtos.Result
+            .CONTAINER_EXISTS);
+      }
     } catch (StorageContainerException ex) {
       return ContainerUtils.logAndReturnError(LOG, ex, request);
+    } finally {
+      handlerLock.release();
     }
 
     return ContainerUtils.getSuccessResponse(request);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e899c4cc/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
index 331e3ed..93e7ef1 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
@@ -28,7 +28,6 @@ import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 
 import org.apache.hadoop.util.Time;
@@ -42,13 +41,11 @@ import org.openjdk.jmh.annotations.TearDown;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 import java.util.Random;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_ROOT_PREFIX;
 
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
@@ -106,10 +103,6 @@ public class BenchMarkDatanodeDispatcher {
     // data directory
     conf.set("dfs.datanode.data.dir", baseDir + File.separator + "data");
 
-    // metadata directory
-    StorageLocation metadataDir = StorageLocation.parse(
-        baseDir + File.separator + CONTAINER_ROOT_PREFIX);
-
     ContainerSet containerSet = new ContainerSet();
     VolumeSet volumeSet = new VolumeSet(datanodeUuid, conf);
 


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


[12/37] hadoop git commit: HDDS-156. Implement HDDSVolume to manage volume state

Posted by bh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java
new file mode 100644
index 0000000..5889222
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.helpers;
+
+import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
+import org.apache.hadoop.ozone.container.common.DataNodeLayoutVersion;
+import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Time;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Properties;
+import java.util.UUID;
+
+import static org.junit.Assert.*;
+
+/**
+ * This class tests {@link DatanodeVersionFile}.
+ */
+public class TestDatanodeVersionFile {
+
+  private File versionFile;
+  private DatanodeVersionFile dnVersionFile;
+  private Properties properties;
+
+  private String storageID;
+  private String clusterID;
+  private String datanodeUUID;
+  private long cTime;
+  private int lv;
+
+  @Rule
+  public TemporaryFolder folder= new TemporaryFolder();
+
+  @Before
+  public void setup() throws IOException {
+    versionFile = folder.newFile("Version");
+    storageID = UUID.randomUUID().toString();
+    clusterID = UUID.randomUUID().toString();
+    datanodeUUID = UUID.randomUUID().toString();
+    cTime = Time.now();
+    lv = DataNodeLayoutVersion.getLatestVersion().getVersion();
+
+    dnVersionFile = new DatanodeVersionFile(
+        storageID, clusterID, datanodeUUID, cTime, lv);
+
+    dnVersionFile.createVersionFile(versionFile);
+
+    properties = dnVersionFile.readFrom(versionFile);
+  }
+
+  @Test
+  public void testCreateAndReadVersionFile() throws IOException{
+
+    //Check VersionFile exists
+    assertTrue(versionFile.exists());
+
+    assertEquals(storageID, HddsVolumeUtil.getStorageID(
+        properties, versionFile));
+    assertEquals(clusterID, HddsVolumeUtil.getClusterID(
+        properties, versionFile, clusterID));
+    assertEquals(datanodeUUID, HddsVolumeUtil.getDatanodeUUID(
+        properties, versionFile, datanodeUUID));
+    assertEquals(cTime, HddsVolumeUtil.getCreationTime(
+        properties, versionFile));
+    assertEquals(lv, HddsVolumeUtil.getLayOutVersion(
+        properties, versionFile));
+  }
+
+  @Test
+  public void testIncorrectClusterId() throws IOException{
+    try {
+      String randomClusterID = UUID.randomUUID().toString();
+      HddsVolumeUtil.getClusterID(properties, versionFile,
+          randomClusterID);
+      fail("Test failure in testIncorrectClusterId");
+    } catch (InconsistentStorageStateException ex) {
+      GenericTestUtils.assertExceptionContains("Mismatched ClusterIDs", ex);
+    }
+  }
+
+  @Test
+  public void testVerifyCTime() throws IOException{
+    long invalidCTime = -10;
+    dnVersionFile = new DatanodeVersionFile(
+        storageID, clusterID, datanodeUUID, invalidCTime, lv);
+    dnVersionFile.createVersionFile(versionFile);
+    properties = dnVersionFile.readFrom(versionFile);
+
+    try {
+      HddsVolumeUtil.getCreationTime(properties, versionFile);
+      fail("Test failure in testVerifyCTime");
+    } catch (InconsistentStorageStateException ex) {
+      GenericTestUtils.assertExceptionContains("Invalid Creation time in " +
+          "Version File : " + versionFile, ex);
+    }
+  }
+
+  @Test
+  public void testVerifyLayOut() throws IOException{
+    int invalidLayOutVersion = 100;
+    dnVersionFile = new DatanodeVersionFile(
+        storageID, clusterID, datanodeUUID, cTime, invalidLayOutVersion);
+    dnVersionFile.createVersionFile(versionFile);
+    Properties props = dnVersionFile.readFrom(versionFile);
+
+    try {
+      HddsVolumeUtil.getLayOutVersion(props, versionFile);
+      fail("Test failure in testVerifyLayOut");
+    } catch (InconsistentStorageStateException ex) {
+      GenericTestUtils.assertExceptionContains("Invalid layOutVersion.", ex);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestRoundRobinVolumeChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestRoundRobinVolumeChoosingPolicy.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestRoundRobinVolumeChoosingPolicy.java
deleted file mode 100644
index 409db57..0000000
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestRoundRobinVolumeChoosingPolicy.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.impl;
-
-import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Tests {@link RoundRobinVolumeChoosingPolicy}.
- */
-public class TestRoundRobinVolumeChoosingPolicy {
-
-  private RoundRobinVolumeChoosingPolicy policy;
-
-  @Before
-  public void setup() {
-   policy = ReflectionUtils.newInstance(
-       RoundRobinVolumeChoosingPolicy.class, null);
-  }
-
-  @Test
-  public void testRRVolumeChoosingPolicy() throws Exception {
-    final List<VolumeInfo> volumes = new ArrayList<>();
-
-    // First volume, with 100 bytes of space.
-    volumes.add(Mockito.mock(VolumeInfo.class));
-    Mockito.when(volumes.get(0).getAvailable()).thenReturn(100L);
-
-    // Second volume, with 200 bytes of space.
-    volumes.add(Mockito.mock(VolumeInfo.class));
-    Mockito.when(volumes.get(1).getAvailable()).thenReturn(200L);
-
-    // Test two rounds of round-robin choosing
-    Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 0));
-    Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 0));
-    Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 0));
-    Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 0));
-
-    // The first volume has only 100L space, so the policy should
-    // choose the second one in case we ask for more.
-    Assert.assertEquals(volumes.get(1),
-        policy.chooseVolume(volumes, 150));
-
-    // Fail if no volume has enough space available
-    try {
-      policy.chooseVolume(volumes, Long.MAX_VALUE);
-      Assert.fail();
-    } catch (IOException e) {
-      // Passed.
-    }
-  }
-
-  @Test
-  public void testRRPolicyExceptionMessage() throws Exception {
-    final List<VolumeInfo> volumes = new ArrayList<>();
-
-    // First volume, with 100 bytes of space.
-    volumes.add(Mockito.mock(VolumeInfo.class));
-    Mockito.when(volumes.get(0).getAvailable()).thenReturn(100L);
-
-    // Second volume, with 200 bytes of space.
-    volumes.add(Mockito.mock(VolumeInfo.class));
-    Mockito.when(volumes.get(1).getAvailable()).thenReturn(200L);
-
-    int blockSize = 300;
-    try {
-      policy.chooseVolume(volumes, blockSize);
-      Assert.fail("expected to throw DiskOutOfSpaceException");
-    } catch(DiskOutOfSpaceException e) {
-      Assert.assertEquals("Not returnig the expected message",
-          "Out of space: The volume with the most available space (=" + 200
-              + " B) is less than the container size (=" + blockSize + " B).",
-          e.getMessage());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestVolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestVolumeSet.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestVolumeSet.java
deleted file mode 100644
index ceeacff..0000000
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestVolumeSet.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.interfaces;
-
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.ozone.container.common.impl.VolumeInfo;
-import org.apache.hadoop.ozone.container.common.impl.VolumeSet;
-import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Tests {@link VolumeSet} operations.
- */
-public class TestVolumeSet {
-
-  private OzoneConfiguration conf;
-  protected VolumeSet volumeSet;
-  protected final String baseDir = MiniDFSCluster.getBaseDirectory();
-  protected final String volume1 = baseDir + "disk1";
-  protected final String volume2 = baseDir + "disk2";
-  private final List<String> volumes = new ArrayList<>();
-
-  private void initializeVolumeSet() throws Exception {
-    volumeSet = new VolumeSet(conf);
-  }
-
-  @Rule
-  public Timeout testTimeout = new Timeout(300_000);
-
-  @Before
-  public void setup() throws Exception {
-    conf = new OzoneConfiguration();
-    String dataDirKey = volume1 + "," + volume2;
-    volumes.add(volume1);
-    volumes.add(volume2);
-    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataDirKey);
-    initializeVolumeSet();
-  }
-
-  @Test
-  public void testVolumeSetInitialization() throws Exception {
-
-    List<VolumeInfo> volumesList = volumeSet.getVolumesList();
-
-    // VolumeSet initialization should add volume1 and volume2 to VolumeSet
-    assertEquals("VolumeSet intialization is incorrect",
-        volumesList.size(), volumes.size());
-    assertTrue("VolumeSet not initailized correctly",
-        checkVolumeExistsInVolumeSet(volume1));
-    assertTrue("VolumeSet not initailized correctly",
-        checkVolumeExistsInVolumeSet(volume2));
-  }
-
-  @Test
-  public void testAddVolume() throws Exception {
-
-    List<VolumeInfo> volumesList = volumeSet.getVolumesList();
-    assertEquals(2, volumeSet.getVolumesList().size());
-
-    // Add a volume to VolumeSet
-    String volume3 = baseDir + "disk3";
-    volumeSet.addVolume(volume3);
-
-    assertEquals(3, volumeSet.getVolumesList().size());
-    assertTrue("AddVolume did not add requested volume to VolumeSet",
-        checkVolumeExistsInVolumeSet(volume3));
-  }
-
-  @Test
-  public void testFailVolume() throws Exception {
-
-    //Fail a volume
-    volumeSet.failVolume(volume1);
-
-    // Failed volume should not show up in the volumeList
-    assertEquals(1, volumeSet.getVolumesList().size());
-
-    // Failed volume should be added to FailedVolumeList
-    assertEquals("Failed volume not present in FailedVolumeMap",
-        1, volumeSet.getFailedVolumesList().size());
-    assertEquals("Failed Volume list did not match", volume1,
-        volumeSet.getFailedVolumesList().get(0).getRootDir().toString());
-    assertTrue(volumeSet.getFailedVolumesList().get(0).isFailed());
-
-    // Failed volume should not exist in VolumeMap
-    Path volume1Path = new Path(volume1);
-    assertFalse(volumeSet.getVolumeMap().containsKey(volume1Path));
-  }
-
-  @Test
-  public void testRemoveVolume() throws Exception {
-
-    List<VolumeInfo> volumesList = volumeSet.getVolumesList();
-    assertEquals(2, volumeSet.getVolumesList().size());
-
-    // Remove a volume from VolumeSet
-    volumeSet.removeVolume(volume1);
-    assertEquals(1, volumeSet.getVolumesList().size());
-
-    // Attempting to remove a volume which does not exist in VolumeSet should
-    // log a warning.
-    LogCapturer logs = LogCapturer.captureLogs(
-        LogFactory.getLog(VolumeSet.class));
-    volumeSet.removeVolume(volume1);
-    assertEquals(1, volumeSet.getVolumesList().size());
-    String expectedLogMessage = "Volume : " + volume1 + " does not exist in "
-        + "VolumeSet";
-    assertTrue("Log output does not contain expected log message: "
-        + expectedLogMessage, logs.getOutput().contains(expectedLogMessage));
-  }
-
-  private boolean checkVolumeExistsInVolumeSet(String volume) {
-    for (VolumeInfo volumeInfo : volumeSet.getVolumesList()) {
-      if (volumeInfo.getRootDir().toString().equals(volume)) {
-        return true;
-      }
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestHddsVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestHddsVolume.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestHddsVolume.java
new file mode 100644
index 0000000..7755345
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestHddsVolume.java
@@ -0,0 +1,145 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.volume;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.GetSpaceUsed;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.ozone.container.common.helpers.DatanodeVersionFile;
+import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
+import static org.junit.Assert.*;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.util.Properties;
+import java.util.UUID;
+
+/**
+ * Unit tests for {@link HddsVolume}.
+ */
+public class TestHddsVolume {
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  private static final String DATANODE_UUID = UUID.randomUUID().toString();
+  private static final String CLUSTER_ID = UUID.randomUUID().toString();
+  private static final Configuration CONF = new Configuration();
+  private static final String DU_CACHE_FILE = "scmUsed";
+
+  private File rootDir;
+  private HddsVolume volume;
+  private File versionFile;
+
+  @Before
+  public void setup() throws Exception {
+    rootDir = new File(folder.getRoot(), HddsVolume.HDDS_VOLUME_DIR);
+    volume = new HddsVolume.Builder(folder.getRoot().getPath())
+        .datanodeUuid(DATANODE_UUID)
+        .conf(CONF)
+        .build();
+    versionFile = HddsVolumeUtil.getVersionFile(rootDir);
+  }
+
+  @Test
+  public void testHddsVolumeInitialization() throws Exception {
+
+    // The initial state of HddsVolume should be "NOT_FORMATTED" when
+    // clusterID is not specified and the version file should not be written
+    // to disk.
+    assertTrue(volume.getClusterID() == null);
+    assertEquals(volume.getStorageType(), StorageType.DEFAULT);
+    assertEquals(volume.getStorageState(),
+        HddsVolume.VolumeState.NOT_FORMATTED);
+    assertFalse("Version file should not be created when clusterID is not " +
+        "known.", versionFile.exists());
+
+
+    // Format the volume with clusterID.
+    volume.format(CLUSTER_ID);
+
+    // The state of HddsVolume after formatting with clusterID should be
+    // NORMAL and the version file should exist.
+    assertTrue("Volume format should create Version file",
+        versionFile.exists());
+    assertEquals(volume.getClusterID(), CLUSTER_ID);
+    assertEquals(volume.getStorageState(), HddsVolume.VolumeState.NORMAL);
+  }
+
+  @Test
+  public void testReadPropertiesFromVersionFile() throws Exception {
+    volume.format(CLUSTER_ID);
+
+    Properties properties = DatanodeVersionFile.readFrom(versionFile);
+
+    String storageID = HddsVolumeUtil.getStorageID(properties, versionFile);
+    String clusterID = HddsVolumeUtil.getClusterID(
+        properties, versionFile, CLUSTER_ID);
+    String datanodeUuid = HddsVolumeUtil.getDatanodeUUID(
+        properties, versionFile, DATANODE_UUID);
+    long cTime = HddsVolumeUtil.getCreationTime(
+        properties, versionFile);
+    int layoutVersion = HddsVolumeUtil.getLayOutVersion(
+        properties, versionFile);
+
+    assertEquals(volume.getStorageID(), storageID);
+    assertEquals(volume.getClusterID(), clusterID);
+    assertEquals(volume.getDatanodeUuid(), datanodeUuid);
+    assertEquals(volume.getCTime(), cTime);
+    assertEquals(volume.getLayoutVersion(), layoutVersion);
+  }
+
+  @Test
+  public void testShutdown() throws Exception{
+    // Return dummy value > 0 for scmUsage so that scm cache file is written
+    // during shutdown.
+    GetSpaceUsed scmUsageMock = Mockito.mock(GetSpaceUsed.class);
+    volume.setScmUsageForTesting(scmUsageMock);
+    Mockito.when(scmUsageMock.getUsed()).thenReturn(Long.valueOf(100));
+
+    assertTrue("Available volume should be positive",
+        volume.getAvailable() > 0);
+
+    // Shutdown the volume.
+    volume.shutdown();
+
+    // Volume state should be "NON_EXISTENT" when volume is shutdown.
+    assertEquals(volume.getStorageState(),
+        HddsVolume.VolumeState.NON_EXISTENT);
+
+    // Volume should save scmUsed cache file once volume is shutdown
+    File scmUsedFile = new File(folder.getRoot(), DU_CACHE_FILE);
+    System.out.println("scmUsedFile: " + scmUsedFile);
+    assertTrue("scmUsed cache file should be saved on shutdown",
+        scmUsedFile.exists());
+
+    try {
+      // Volume.getAvailable() should fail with NullPointerException as usage
+      // is shutdown.
+      volume.getAvailable();
+      fail("HddsVolume#shutdown test failed");
+    } catch (Exception ex){
+      assertTrue(ex instanceof NullPointerException);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestRoundRobinVolumeChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestRoundRobinVolumeChoosingPolicy.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestRoundRobinVolumeChoosingPolicy.java
new file mode 100644
index 0000000..41610af
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestRoundRobinVolumeChoosingPolicy.java
@@ -0,0 +1,131 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.volume;
+
+import org.apache.hadoop.fs.GetSpaceUsed;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.UUID;
+
+/**
+ * Tests {@link RoundRobinVolumeChoosingPolicy}.
+ */
+public class TestRoundRobinVolumeChoosingPolicy {
+
+  private RoundRobinVolumeChoosingPolicy policy;
+  private List<HddsVolume> volumes;
+
+  private final String baseDir = MiniDFSCluster.getBaseDirectory();
+	private final String volume1 = baseDir + "disk1";
+	private final String volume2 = baseDir + "disk2";
+  private static final String DUMMY_IP_ADDR = "0.0.0.0";
+
+  @Before
+  public void setup() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    String dataDirKey = volume1 + "," + volume2;
+    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataDirKey);
+    policy = ReflectionUtils.newInstance(
+        RoundRobinVolumeChoosingPolicy.class, null);
+    DatanodeDetails datanodeDetails = DatanodeDetails.newBuilder()
+        .setUuid(UUID.randomUUID().toString())
+        .setIpAddress(DUMMY_IP_ADDR)
+        .build();
+    VolumeSet volumeSet = new VolumeSet(datanodeDetails, conf);
+    volumes = volumeSet.getVolumesList();
+  }
+
+  @Test
+  public void testRRVolumeChoosingPolicy() throws Exception {
+    HddsVolume hddsVolume1 = volumes.get(0);
+    HddsVolume hddsVolume2 = volumes.get(1);
+
+    // Set available space in volume1 to 100L
+    setAvailableSpace(hddsVolume1, 100L);
+
+    // Set available space in volume1 to 200L
+    setAvailableSpace(hddsVolume2, 200L);
+
+    Assert.assertEquals(100L, hddsVolume1.getAvailable());
+    Assert.assertEquals(200L, hddsVolume2.getAvailable());
+
+    // Test two rounds of round-robin choosing
+    Assert.assertEquals(hddsVolume1, policy.chooseVolume(volumes, 0));
+    Assert.assertEquals(hddsVolume2, policy.chooseVolume(volumes, 0));
+    Assert.assertEquals(hddsVolume1, policy.chooseVolume(volumes, 0));
+    Assert.assertEquals(hddsVolume2, policy.chooseVolume(volumes, 0));
+
+    // The first volume has only 100L space, so the policy should
+    // choose the second one in case we ask for more.
+    Assert.assertEquals(hddsVolume2,
+        policy.chooseVolume(volumes, 150));
+
+    // Fail if no volume has enough space available
+    try {
+      policy.chooseVolume(volumes, Long.MAX_VALUE);
+      Assert.fail();
+    } catch (IOException e) {
+      // Passed.
+    }
+  }
+
+  @Test
+  public void testRRPolicyExceptionMessage() throws Exception {
+    HddsVolume hddsVolume1 = volumes.get(0);
+    HddsVolume hddsVolume2 = volumes.get(1);
+
+    // Set available space in volume1 to 100L
+    setAvailableSpace(hddsVolume1, 100L);
+
+    // Set available space in volume1 to 200L
+    setAvailableSpace(hddsVolume2, 200L);
+
+    int blockSize = 300;
+    try {
+      policy.chooseVolume(volumes, blockSize);
+      Assert.fail("expected to throw DiskOutOfSpaceException");
+    } catch(DiskOutOfSpaceException e) {
+      Assert.assertEquals("Not returnig the expected message",
+          "Out of space: The volume with the most available space (=" + 200
+              + " B) is less than the container size (=" + blockSize + " B).",
+          e.getMessage());
+    }
+  }
+
+  private void setAvailableSpace(HddsVolume hddsVolume, long availableSpace)
+      throws IOException {
+    GetSpaceUsed scmUsageMock = Mockito.mock(GetSpaceUsed.class);
+    hddsVolume.setScmUsageForTesting(scmUsageMock);
+    // Set used space to capacity -requiredAvailableSpace so that
+    // getAvailable() returns us the specified availableSpace.
+    Mockito.when(scmUsageMock.getUsed()).thenReturn(
+        (hddsVolume.getCapacity() - availableSpace));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSet.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSet.java
new file mode 100644
index 0000000..61383de
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSet.java
@@ -0,0 +1,157 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.volume;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
+import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+/**
+ * Tests {@link VolumeSet} operations.
+ */
+public class TestVolumeSet {
+
+  private OzoneConfiguration conf;
+  private VolumeSet volumeSet;
+  private final String baseDir = MiniDFSCluster.getBaseDirectory();
+  private final String volume1 = baseDir + "disk1";
+  private final String volume2 = baseDir + "disk2";
+  private final List<String> volumes = new ArrayList<>();
+
+  private static final String DUMMY_IP_ADDR = "0.0.0.0";
+
+  private void initializeVolumeSet() throws Exception {
+    DatanodeDetails datanodeDetails = DatanodeDetails.newBuilder()
+        .setUuid(UUID.randomUUID().toString())
+        .setIpAddress(DUMMY_IP_ADDR)
+        .build();
+    volumeSet = new VolumeSet(datanodeDetails, conf);
+  }
+
+  @Rule
+  public Timeout testTimeout = new Timeout(300_000);
+
+  @Before
+  public void setup() throws Exception {
+    conf = new OzoneConfiguration();
+    String dataDirKey = volume1 + "," + volume2;
+    volumes.add(volume1);
+    volumes.add(volume2);
+    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataDirKey);
+    initializeVolumeSet();
+  }
+
+  @Test
+  public void testVolumeSetInitialization() throws Exception {
+
+    List<HddsVolume> volumesList = volumeSet.getVolumesList();
+
+    // VolumeSet initialization should add volume1 and volume2 to VolumeSet
+    assertEquals("VolumeSet intialization is incorrect",
+        volumesList.size(), volumes.size());
+    assertTrue("VolumeSet not initailized correctly",
+        checkVolumeExistsInVolumeSet(volume1));
+    assertTrue("VolumeSet not initailized correctly",
+        checkVolumeExistsInVolumeSet(volume2));
+  }
+
+  @Test
+  public void testAddVolume() throws Exception {
+
+    assertEquals(2, volumeSet.getVolumesList().size());
+
+    // Add a volume to VolumeSet
+    String volume3 = baseDir + "disk3";
+    volumeSet.addVolume(volume3);
+
+    assertEquals(3, volumeSet.getVolumesList().size());
+    assertTrue("AddVolume did not add requested volume to VolumeSet",
+        checkVolumeExistsInVolumeSet(volume3));
+  }
+
+  @Test
+  public void testFailVolume() throws Exception {
+
+    //Fail a volume
+    volumeSet.failVolume(volume1);
+
+    // Failed volume should not show up in the volumeList
+    assertEquals(1, volumeSet.getVolumesList().size());
+
+    // Failed volume should be added to FailedVolumeList
+    assertEquals("Failed volume not present in FailedVolumeMap",
+        1, volumeSet.getFailedVolumesList().size());
+    assertEquals("Failed Volume list did not match",
+        HddsVolumeUtil.getHddsRoot(volume1),
+        volumeSet.getFailedVolumesList().get(0).getHddsRootDir().getPath());
+    assertTrue(volumeSet.getFailedVolumesList().get(0).isFailed());
+
+    // Failed volume should not exist in VolumeMap
+    Path volume1Path = new Path(volume1);
+    assertFalse(volumeSet.getVolumeMap().containsKey(volume1Path));
+  }
+
+  @Test
+  public void testRemoveVolume() throws Exception {
+
+    List<HddsVolume> volumesList = volumeSet.getVolumesList();
+    assertEquals(2, volumeSet.getVolumesList().size());
+
+    // Remove a volume from VolumeSet
+    volumeSet.removeVolume(volume1);
+    assertEquals(1, volumeSet.getVolumesList().size());
+
+    // Attempting to remove a volume which does not exist in VolumeSet should
+    // log a warning.
+    LogCapturer logs = LogCapturer.captureLogs(
+        LogFactory.getLog(VolumeSet.class));
+    volumeSet.removeVolume(volume1);
+    assertEquals(1, volumeSet.getVolumesList().size());
+    String expectedLogMessage = "Volume : " +
+        HddsVolumeUtil.getHddsRoot(volume1) + " does not exist in VolumeSet";
+    assertTrue("Log output does not contain expected log message: "
+        + expectedLogMessage, logs.getOutput().contains(expectedLogMessage));
+  }
+
+  private boolean checkVolumeExistsInVolumeSet(String volume) {
+    for (HddsVolume hddsVolume : volumeSet.getVolumesList()) {
+      if (hddsVolume.getHddsRootDir().getPath().equals(
+          HddsVolumeUtil.getHddsRoot(volume))) {
+        return true;
+      }
+    }
+    return false;
+  }
+}


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


[13/37] hadoop git commit: HDDS-156. Implement HDDSVolume to manage volume state

Posted by bh...@apache.org.
HDDS-156. Implement HDDSVolume to manage volume state


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

Branch: refs/heads/trunk
Commit: 9a5552bf762880c38a233597b7c6e9ea09441108
Parents: 418cff4
Author: Hanisha Koneru <ha...@apache.org>
Authored: Thu Jun 14 13:28:41 2018 -0700
Committer: Hanisha Koneru <ha...@apache.org>
Committed: Thu Jun 14 13:28:41 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/ozone/OzoneConsts.java    |   5 +
 .../container/common/DataNodeLayoutVersion.java |  80 +++++
 .../common/helpers/DatanodeVersionFile.java     |  95 ++++++
 .../impl/RoundRobinVolumeChoosingPolicy.java    |  82 -----
 .../ozone/container/common/impl/VolumeInfo.java | 162 ---------
 .../ozone/container/common/impl/VolumeSet.java  | 251 --------------
 .../container/common/impl/VolumeUsage.java      | 189 -----------
 .../common/interfaces/VolumeChoosingPolicy.java |   4 +-
 .../container/common/utils/HddsVolumeUtil.java  | 163 +++++++++
 .../container/common/volume/HddsVolume.java     | 330 +++++++++++++++++++
 .../volume/RoundRobinVolumeChoosingPolicy.java  |  83 +++++
 .../container/common/volume/VolumeInfo.java     | 132 ++++++++
 .../container/common/volume/VolumeSet.java      | 309 +++++++++++++++++
 .../container/common/volume/VolumeUsage.java    | 198 +++++++++++
 .../container/common/volume/package-info.java   |  21 ++
 .../common/TestDatanodeLayOutVersion.java       |  38 +++
 .../common/helpers/TestDatanodeVersionFile.java | 134 ++++++++
 .../TestRoundRobinVolumeChoosingPolicy.java     | 100 ------
 .../common/interfaces/TestVolumeSet.java        | 149 ---------
 .../container/common/volume/TestHddsVolume.java | 145 ++++++++
 .../TestRoundRobinVolumeChoosingPolicy.java     | 131 ++++++++
 .../container/common/volume/TestVolumeSet.java  | 157 +++++++++
 22 files changed, 2023 insertions(+), 935 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index c40dc8e..36f830b 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -33,6 +33,11 @@ public final class OzoneConsts {
   public static final String OZONE_SIMPLE_ROOT_USER = "root";
   public static final String OZONE_SIMPLE_HDFS_USER = "hdfs";
 
+  public static final String STORAGE_ID = "storageID";
+  public static final String DATANODE_UUID = "datanodeUuid";
+  public static final String CLUSTER_ID = "clusterID";
+  public static final String LAYOUTVERSION = "layOutVersion";
+  public static final String CTIME = "ctime";
   /*
    * BucketName length is used for both buckets and volume lengths
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/DataNodeLayoutVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/DataNodeLayoutVersion.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/DataNodeLayoutVersion.java
new file mode 100644
index 0000000..2d58c39
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/DataNodeLayoutVersion.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.common;
+
+/**
+ * Datanode layout version which describes information about the layout version
+ * on the datanode.
+ */
+public final class DataNodeLayoutVersion {
+
+  // We will just be normal and use positive counting numbers for versions.
+  private final static DataNodeLayoutVersion[] VERSION_INFOS =
+      {new DataNodeLayoutVersion(1, "HDDS Datanode LayOut Version 1")};
+
+  private final String description;
+  private final int version;
+
+  /**
+   * Never created outside this class.
+   *
+   * @param description -- description
+   * @param version     -- version number
+   */
+  private DataNodeLayoutVersion(int version, String description) {
+    this.description = description;
+    this.version = version;
+  }
+
+  /**
+   * Returns all versions.
+   *
+   * @return Version info array.
+   */
+  public static DataNodeLayoutVersion[] getAllVersions() {
+    return VERSION_INFOS.clone();
+  }
+
+  /**
+   * Returns the latest version.
+   *
+   * @return versionInfo
+   */
+  public static DataNodeLayoutVersion getLatestVersion() {
+    return VERSION_INFOS[VERSION_INFOS.length - 1];
+  }
+
+  /**
+   * Return description.
+   *
+   * @return String
+   */
+  public String getDescription() {
+    return description;
+  }
+
+  /**
+   * Return the version.
+   *
+   * @return int.
+   */
+  public int getVersion() {
+    return version;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeVersionFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeVersionFile.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeVersionFile.java
new file mode 100644
index 0000000..4db6d31
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeVersionFile.java
@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.helpers;
+
+import org.apache.hadoop.ozone.OzoneConsts;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.util.Properties;
+
+/**
+ * This is a utility class which helps to create the version file on datanode
+ * and also validate the content of the version file.
+ */
+public class DatanodeVersionFile {
+
+  private final String storageId;
+  private final String clusterId;
+  private final String datanodeUuid;
+  private final long cTime;
+  private final int layOutVersion;
+
+  public DatanodeVersionFile(String storageId, String clusterId,
+      String datanodeUuid, long cTime, int layOutVersion) {
+    this.storageId = storageId;
+    this.clusterId = clusterId;
+    this.datanodeUuid = datanodeUuid;
+    this.cTime = cTime;
+    this.layOutVersion = layOutVersion;
+  }
+
+  private Properties createProperties() {
+    Properties properties = new Properties();
+    properties.setProperty(OzoneConsts.STORAGE_ID, storageId);
+    properties.setProperty(OzoneConsts.CLUSTER_ID, clusterId);
+    properties.setProperty(OzoneConsts.DATANODE_UUID, datanodeUuid);
+    properties.setProperty(OzoneConsts.CTIME, String.valueOf(cTime));
+    properties.setProperty(OzoneConsts.LAYOUTVERSION, String.valueOf(
+        layOutVersion));
+    return properties;
+  }
+
+  /**
+   * Creates a version File in specified path.
+   * @param path
+   * @throws IOException
+   */
+  public void createVersionFile(File path) throws
+      IOException {
+    try (RandomAccessFile file = new RandomAccessFile(path, "rws");
+         FileOutputStream out = new FileOutputStream(file.getFD())) {
+      file.getChannel().truncate(0);
+      Properties properties = createProperties();
+      /*
+       * If server is interrupted before this line,
+       * the version file will remain unchanged.
+       */
+      properties.store(out, null);
+    }
+  }
+
+
+  /**
+   * Creates a property object from the specified file content.
+   * @param  versionFile
+   * @return Properties
+   * @throws IOException
+   */
+  public static Properties readFrom(File versionFile) throws IOException {
+    try (RandomAccessFile file = new RandomAccessFile(versionFile, "rws");
+         FileInputStream in = new FileInputStream(file.getFD())) {
+      Properties props = new Properties();
+      props.load(in);
+      return props;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RoundRobinVolumeChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RoundRobinVolumeChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RoundRobinVolumeChoosingPolicy.java
deleted file mode 100644
index 55b3049..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RoundRobinVolumeChoosingPolicy.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.ozone.container.common.impl;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
-import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Choose volumes in round-robin order.
- * The caller should synchronize access to the list of volumes.
- */
-public class RoundRobinVolumeChoosingPolicy implements VolumeChoosingPolicy {
-
-  public static final Log LOG = LogFactory.getLog(
-		RoundRobinVolumeChoosingPolicy.class);
-
-  // Stores the index of the next volume to be returned.
-  private AtomicInteger nextVolumeIndex = new AtomicInteger(0);
-
-  @Override
-  public VolumeInfo chooseVolume(List<VolumeInfo> volumes,
-      long maxContainerSize) throws IOException {
-
-    // No volumes available to choose from
-    if (volumes.size() < 1) {
-      throw new DiskOutOfSpaceException("No more available volumes");
-    }
-
-    // since volumes could've been removed because of the failure
-    // make sure we are not out of bounds
-    int nextIndex = nextVolumeIndex.get();
-    int currentVolumeIndex = nextIndex < volumes.size() ? nextIndex : 0;
-
-    int startVolumeIndex = currentVolumeIndex;
-    long maxAvailable = 0;
-
-    while (true) {
-      final VolumeInfo volume = volumes.get(currentVolumeIndex);
-      long availableVolumeSize = volume.getAvailable();
-
-      currentVolumeIndex = (currentVolumeIndex + 1) % volumes.size();
-
-      if (availableVolumeSize > maxContainerSize) {
-        nextVolumeIndex.compareAndSet(nextIndex, currentVolumeIndex);
-        return volume;
-      }
-
-      if (availableVolumeSize > maxAvailable) {
-        maxAvailable = availableVolumeSize;
-      }
-
-      if (currentVolumeIndex == startVolumeIndex) {
-        throw new DiskOutOfSpaceException("Out of space: "
-            + "The volume with the most available space (=" + maxAvailable
-            + " B) is less than the container size (=" + maxContainerSize
-            + " B).");
-      }
-
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeInfo.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeInfo.java
deleted file mode 100644
index 3e8dda6..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeInfo.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.impl;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.StorageType;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-
-/**
- * Stores information about a disk/volume.
- */
-public class VolumeInfo {
-
-  private static final Logger LOG = LoggerFactory.getLogger(VolumeInfo.class);
-
-  private final Path rootDir;
-  private final StorageType storageType;
-  private VolumeState state;
-
-  // Space usage calculator
-  private VolumeUsage usage;
-  // Capacity configured. This is useful when we want to
-  // limit the visible capacity for tests. If negative, then we just
-  // query from the filesystem.
-  private long configuredCapacity;
-
-  public static class Builder {
-    private final Configuration conf;
-    private final Path rootDir;
-    private StorageType storageType;
-    private VolumeState state;
-    private long configuredCapacity;
-
-    public Builder(Path rootDir, Configuration conf) {
-      this.rootDir = rootDir;
-      this.conf = conf;
-    }
-
-    public Builder(String rootDirStr, Configuration conf) {
-      this.rootDir = new Path(rootDirStr);
-      this.conf = conf;
-    }
-
-    public Builder storageType(StorageType storageType) {
-      this.storageType = storageType;
-      return this;
-    }
-
-    public Builder volumeState(VolumeState state) {
-      this.state = state;
-      return this;
-    }
-
-    public Builder configuredCapacity(long configuredCapacity) {
-      this.configuredCapacity = configuredCapacity;
-      return this;
-    }
-
-    public VolumeInfo build() throws IOException {
-      return new VolumeInfo(this);
-    }
-  }
-
-  private VolumeInfo(Builder b) throws IOException {
-
-    this.rootDir = b.rootDir;
-    File root = new File(rootDir.toString());
-
-    Boolean succeeded = root.isDirectory() || root.mkdirs();
-
-    if (!succeeded) {
-      LOG.error("Unable to create the volume root dir at : {}", root);
-      throw new IOException("Unable to create the volume root dir at " + root);
-    }
-
-    this.storageType = (b.storageType != null ?
-        b.storageType : StorageType.DEFAULT);
-
-    this.configuredCapacity = (b.configuredCapacity != 0 ?
-        b.configuredCapacity : -1);
-
-    this.state = (b.state != null ? b.state : VolumeState.NOT_FORMATTED);
-
-    this.usage = new VolumeUsage(root, b.conf);
-
-    LOG.info("Creating Volume : " + rootDir + " of storage type : " +
-        storageType + " and capacity : " + configuredCapacity);
-  }
-
-  public long getCapacity() {
-    return configuredCapacity < 0 ? usage.getCapacity() : configuredCapacity;
-  }
-
-  public long getAvailable() throws IOException {
-    return usage.getAvailable();
-  }
-
-  public long getScmUsed() throws IOException {
-    return usage.getScmUsed();
-  }
-
-  void shutdown() {
-    this.state = VolumeState.NON_EXISTENT;
-    shutdownUsageThread();
-  }
-
-  void failVolume() {
-    setState(VolumeState.FAILED);
-    shutdownUsageThread();
-  }
-
-  private void shutdownUsageThread() {
-    if (usage != null) {
-      usage.shutdown();
-    }
-    usage = null;
-  }
-
-  void setState(VolumeState state) {
-    this.state = state;
-  }
-
-  public boolean isFailed() {
-    return (state == VolumeState.FAILED);
-  }
-
-  public Path getRootDir() {
-    return this.rootDir;
-  }
-
-  public StorageType getStorageType() {
-    return this.storageType;
-  }
-
-  public enum VolumeState {
-    NORMAL,
-    FAILED,
-    NON_EXISTENT,
-    NOT_FORMATTED,
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeSet.java
deleted file mode 100644
index c55c84a..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeSet.java
+++ /dev/null
@@ -1,251 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.impl;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.StorageType;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
-import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
-import org.apache.hadoop.ozone.container.common.impl.VolumeInfo.VolumeState;
-import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
-import org.apache.hadoop.util.AutoCloseableLock;
-import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
-import org.apache.hadoop.util.InstrumentedLock;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.EnumMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantLock;
-
-/**
- * VolumeSet to manage volumes in a DataNode.
- */
-public class VolumeSet {
-
-  private static final Logger LOG = LoggerFactory.getLogger(VolumeSet.class);
-
-  private Configuration conf;
-
-  /**
-   * {@link VolumeSet#volumeMap} maintains a map of all active volumes in the
-   * DataNode. Each volume has one-to-one mapping with a volumeInfo object.
-   */
-  private Map<Path, VolumeInfo> volumeMap;
-  /**
-   * {@link VolumeSet#failedVolumeMap} maintains a map of volumes which have
-   * failed. The keys in this map and {@link VolumeSet#volumeMap} are
-   * mutually exclusive.
-   */
-  private Map<Path, VolumeInfo> failedVolumeMap;
-  /**
-   * {@link VolumeSet#volumeStateMap} maintains a list of active volumes per
-   * StorageType.
-   */
-  private EnumMap<StorageType, List<VolumeInfo>> volumeStateMap;
-
-  /**
-   * Lock to synchronize changes to the VolumeSet. Any update to
-   * {@link VolumeSet#volumeMap}, {@link VolumeSet#failedVolumeMap}, or
-   * {@link VolumeSet#volumeStateMap} should be done after acquiring this lock.
-   */
-  private final AutoCloseableLock volumeSetLock;
-
-  public VolumeSet(Configuration conf) throws DiskOutOfSpaceException {
-    this.conf = conf;
-    this.volumeSetLock = new AutoCloseableLock(
-        new InstrumentedLock(getClass().getName(), LOG,
-            new ReentrantLock(true),
-            conf.getTimeDuration(
-                DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
-                DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_DEFAULT,
-                TimeUnit.MILLISECONDS),
-            300));
-
-    initializeVolumeSet();
-  }
-
-  // Add DN volumes configured through ConfigKeys to volumeMap.
-  private void initializeVolumeSet() throws DiskOutOfSpaceException {
-    volumeMap = new ConcurrentHashMap<>();
-    failedVolumeMap = new ConcurrentHashMap<>();
-    volumeStateMap = new EnumMap<>(StorageType.class);
-
-    Collection<String> datanodeDirs = conf.getTrimmedStringCollection(
-        HDDS_DATANODE_DIR_KEY);
-    if (datanodeDirs.isEmpty()) {
-      datanodeDirs = conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY);
-    }
-    if (datanodeDirs.isEmpty()) {
-      throw new IllegalArgumentException("No location configured in either "
-          + HDDS_DATANODE_DIR_KEY + " or " + DFS_DATANODE_DATA_DIR_KEY);
-    }
-
-    for (StorageType storageType : StorageType.values()) {
-      volumeStateMap.put(storageType, new ArrayList<VolumeInfo>());
-    }
-
-    for (String dir : datanodeDirs) {
-      try {
-        VolumeInfo volumeInfo = getVolumeInfo(dir);
-
-        volumeMap.put(volumeInfo.getRootDir(), volumeInfo);
-        volumeStateMap.get(volumeInfo.getStorageType()).add(volumeInfo);
-      } catch (IOException e) {
-        LOG.error("Failed to parse the storage location: " + dir, e);
-      }
-    }
-
-    if (volumeMap.size() == 0) {
-      throw new DiskOutOfSpaceException("No storage location configured");
-    }
-  }
-
-  public void acquireLock() {
-    volumeSetLock.acquire();
-  }
-
-  public void releaseLock() {
-    volumeSetLock.release();
-  }
-
-  private VolumeInfo getVolumeInfo(String rootDir) throws IOException {
-    StorageLocation location = StorageLocation.parse(rootDir);
-    StorageType storageType = location.getStorageType();
-
-    VolumeInfo.Builder volumeBuilder = new VolumeInfo.Builder(rootDir, conf);
-    volumeBuilder.storageType(storageType);
-    return volumeBuilder.build();
-  }
-
-  // Add a volume to VolumeSet
-  public void addVolume(String dataDir) throws IOException {
-    Path dirPath = new Path(dataDir);
-
-    try (AutoCloseableLock lock = volumeSetLock.acquire()) {
-      if (volumeMap.containsKey(dirPath)) {
-        LOG.warn("Volume : {} already exists in VolumeMap", dataDir);
-      } else {
-        if (failedVolumeMap.containsKey(dirPath)) {
-          failedVolumeMap.remove(dirPath);
-        }
-
-        VolumeInfo volumeInfo = getVolumeInfo(dirPath.toString());
-        volumeMap.put(dirPath, volumeInfo);
-        volumeStateMap.get(volumeInfo.getStorageType()).add(volumeInfo);
-
-        LOG.debug("Added Volume : {} to VolumeSet", dataDir);
-      }
-    }
-  }
-
-  // Mark a volume as failed
-  public void failVolume(String dataDir) {
-    Path dirPath = new Path(dataDir);
-
-    try (AutoCloseableLock lock = volumeSetLock.acquire()) {
-      if (volumeMap.containsKey(dirPath)) {
-        VolumeInfo volumeInfo = volumeMap.get(dirPath);
-        volumeInfo.failVolume();
-
-        volumeMap.remove(dirPath);
-        volumeStateMap.get(volumeInfo.getStorageType()).remove(volumeInfo);
-        failedVolumeMap.put(dirPath, volumeInfo);
-
-        LOG.debug("Moving Volume : {} to failed Volumes", dataDir);
-      } else if (failedVolumeMap.containsKey(dirPath)) {
-        LOG.debug("Volume : {} is not active", dataDir);
-      } else {
-        LOG.warn("Volume : {} does not exist in VolumeSet", dataDir);
-      }
-    }
-  }
-
-  // Remove a volume from the VolumeSet completely.
-  public void removeVolume(String dataDir) throws IOException {
-    Path dirPath = new Path(dataDir);
-
-    try (AutoCloseableLock lock = volumeSetLock.acquire()) {
-      if (volumeMap.containsKey(dirPath)) {
-        VolumeInfo volumeInfo = volumeMap.get(dirPath);
-        volumeInfo.shutdown();
-
-        volumeMap.remove(dirPath);
-        volumeStateMap.get(volumeInfo.getStorageType()).remove(volumeInfo);
-
-        LOG.debug("Removed Volume : {} from VolumeSet", dataDir);
-      } else if (failedVolumeMap.containsKey(dirPath)) {
-        VolumeInfo volumeInfo = failedVolumeMap.get(dirPath);
-        volumeInfo.setState(VolumeState.NON_EXISTENT);
-
-        failedVolumeMap.remove(dirPath);
-        LOG.debug("Removed Volume : {} from failed VolumeSet", dataDir);
-      } else {
-        LOG.warn("Volume : {} does not exist in VolumeSet", dataDir);
-      }
-    }
-  }
-
-  public VolumeInfo chooseVolume(long containerSize,
-      VolumeChoosingPolicy choosingPolicy) throws IOException {
-    return choosingPolicy.chooseVolume(getVolumesList(), containerSize);
-  }
-
-  public void shutdown() {
-    for (VolumeInfo volumeInfo : volumeMap.values()) {
-      try {
-        volumeInfo.shutdown();
-      } catch (Exception e) {
-        LOG.error("Failed to shutdown volume : " + volumeInfo.getRootDir(), e);
-      }
-    }
-  }
-
-  @VisibleForTesting
-  public List<VolumeInfo> getVolumesList() {
-    return ImmutableList.copyOf(volumeMap.values());
-  }
-
-  @VisibleForTesting
-  public List<VolumeInfo> getFailedVolumesList() {
-    return ImmutableList.copyOf(failedVolumeMap.values());
-  }
-
-  @VisibleForTesting
-  public Map<Path, VolumeInfo> getVolumeMap() {
-    return ImmutableMap.copyOf(volumeMap);
-  }
-
-  @VisibleForTesting
-  public Map<StorageType, List<VolumeInfo>> getVolumeStateMap() {
-    return ImmutableMap.copyOf(volumeStateMap);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeUsage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeUsage.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeUsage.java
deleted file mode 100644
index bcd78ba..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeUsage.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.common.impl;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CachingGetSpaceUsed;
-import org.apache.hadoop.fs.DF;
-import org.apache.hadoop.fs.GetSpaceUsed;
-import org.apache.hadoop.io.IOUtils;
-import static org.apache.hadoop.util.RunJar.SHUTDOWN_HOOK_PRIORITY;
-import org.apache.hadoop.util.ShutdownHookManager;
-import org.apache.hadoop.util.Time;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.nio.charset.StandardCharsets;
-import java.util.Scanner;
-
-/**
- * Class that wraps the space df of the Datanode Volumes used by SCM
- * containers.
- */
-public class VolumeUsage {
-  private static final Logger LOG = LoggerFactory.getLogger(VolumeUsage.class);
-
-  private final File rootDir;
-  private final DF df;
-  private final File scmUsedFile;
-  private GetSpaceUsed scmUsage;
-  private Runnable shutdownHook;
-
-  private static final String DU_CACHE_FILE = "scmUsed";
-  private volatile boolean scmUsedSaved = false;
-
-  VolumeUsage(File dataLoc, Configuration conf)
-      throws IOException {
-    this.rootDir = dataLoc;
-
-    // SCM used cache file
-    scmUsedFile = new File(rootDir, DU_CACHE_FILE);
-    // get overall disk df
-    this.df = new DF(rootDir, conf);
-
-    startScmUsageThread(conf);
-  }
-
-  void startScmUsageThread(Configuration conf) throws IOException {
-    // get SCM specific df
-    this.scmUsage = new CachingGetSpaceUsed.Builder().setPath(rootDir)
-        .setConf(conf)
-        .setInitialUsed(loadScmUsed())
-        .build();
-
-    // Ensure scm df is saved during shutdown.
-    shutdownHook = () -> {
-      if (!scmUsedSaved) {
-        saveScmUsed();
-      }
-    };
-    ShutdownHookManager.get().addShutdownHook(shutdownHook,
-        SHUTDOWN_HOOK_PRIORITY);
-  }
-
-  long getCapacity() {
-    long capacity = df.getCapacity();
-    return (capacity > 0) ? capacity : 0;
-  }
-
-  /*
-   * Calculate the available space in the volume.
-   */
-  long getAvailable() throws IOException {
-    long remaining = getCapacity() - getScmUsed();
-    long available = df.getAvailable();
-    if (remaining > available) {
-      remaining = available;
-    }
-    return (remaining > 0) ? remaining : 0;
-  }
-
-  long getScmUsed() throws IOException{
-    return scmUsage.getUsed();
-  }
-
-  public void shutdown() {
-    saveScmUsed();
-    scmUsedSaved = true;
-
-    if (shutdownHook != null) {
-      ShutdownHookManager.get().removeShutdownHook(shutdownHook);
-    }
-
-    if (scmUsage instanceof CachingGetSpaceUsed) {
-      IOUtils.cleanupWithLogger(null, ((CachingGetSpaceUsed) scmUsage));
-    }
-  }
-
-  /**
-   * Read in the cached DU value and return it if it is less than 600 seconds
-   * old (DU update interval). Slight imprecision of scmUsed is not critical
-   * and skipping DU can significantly shorten the startup time.
-   * If the cached value is not available or too old, -1 is returned.
-   */
-  long loadScmUsed() {
-    long cachedScmUsed;
-    long mtime;
-    Scanner sc;
-
-    try {
-      sc = new Scanner(scmUsedFile, "UTF-8");
-    } catch (FileNotFoundException fnfe) {
-      return -1;
-    }
-
-    try {
-      // Get the recorded scmUsed from the file.
-      if (sc.hasNextLong()) {
-        cachedScmUsed = sc.nextLong();
-      } else {
-        return -1;
-      }
-      // Get the recorded mtime from the file.
-      if (sc.hasNextLong()) {
-        mtime = sc.nextLong();
-      } else {
-        return -1;
-      }
-
-      // Return the cached value if mtime is okay.
-      if (mtime > 0 && (Time.now() - mtime < 600000L)) {
-        LOG.info("Cached ScmUsed found for {} : {} ", rootDir,
-            cachedScmUsed);
-        return cachedScmUsed;
-      }
-      return -1;
-    } finally {
-      sc.close();
-    }
-  }
-
-  /**
-   * Write the current scmUsed to the cache file.
-   */
-  void saveScmUsed() {
-    if (scmUsedFile.exists() && !scmUsedFile.delete()) {
-      LOG.warn("Failed to delete old scmUsed file in {}.", rootDir);
-    }
-    OutputStreamWriter out = null;
-    try {
-      long used = getScmUsed();
-      if (used > 0) {
-        out = new OutputStreamWriter(new FileOutputStream(scmUsedFile),
-            StandardCharsets.UTF_8);
-        // mtime is written last, so that truncated writes won't be valid.
-        out.write(Long.toString(used) + " " + Long.toString(Time.now()));
-        out.flush();
-        out.close();
-        out = null;
-      }
-    } catch (IOException ioe) {
-      // If write failed, the volume might be bad. Since the cache file is
-      // not critical, log the error and continue.
-      LOG.warn("Failed to write scmUsed to " + scmUsedFile, ioe);
-    } finally {
-      IOUtils.cleanupWithLogger(null, out);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/VolumeChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/VolumeChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/VolumeChoosingPolicy.java
index b8cbcb6..7de0e2a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/VolumeChoosingPolicy.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/VolumeChoosingPolicy.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.ozone.container.common.interfaces;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.ozone.container.common.impl.VolumeInfo;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 
 import java.io.IOException;
 import java.util.List;
@@ -41,6 +41,6 @@ public interface VolumeChoosingPolicy {
    * @return the chosen volume.
    * @throws IOException when disks are unavailable or are full.
    */
-  VolumeInfo chooseVolume(List<VolumeInfo> volumes, long maxContainerSize)
+  HddsVolume chooseVolume(List<HddsVolume> volumes, long maxContainerSize)
       throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java
new file mode 100644
index 0000000..6809d57
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java
@@ -0,0 +1,163 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.utils;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
+import org.apache.hadoop.ozone.container.common.DataNodeLayoutVersion;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.util.Time;
+
+import java.io.File;
+import java.util.Properties;
+import java.util.UUID;
+
+/**
+ * A util class for {@link HddsVolume}.
+ */
+public final class HddsVolumeUtil {
+
+  // Private constructor for Utility class. Unused.
+  private HddsVolumeUtil() {
+  }
+
+  private static final String VERSION_FILE   = "VERSION";
+  private static final String STORAGE_ID_PREFIX = "DS-";
+
+  public static File getVersionFile(File rootDir) {
+    return new File(rootDir, VERSION_FILE);
+  }
+
+  public static String generateUuid() {
+    return STORAGE_ID_PREFIX + UUID.randomUUID();
+  }
+
+  /**
+   * Get hddsRoot from volume root. If volumeRoot points to hddsRoot, it is
+   * returned as is.
+   * For a volumeRoot /data/disk1, the hddsRoot is /data/disk1/hdds.
+   * @param volumeRoot root of the volume.
+   * @return hddsRoot of the volume.
+   */
+  public static String getHddsRoot(String volumeRoot) {
+    if (volumeRoot.endsWith(HddsVolume.HDDS_VOLUME_DIR)) {
+      return volumeRoot;
+    } else {
+      File hddsRoot = new File(volumeRoot, HddsVolume.HDDS_VOLUME_DIR);
+      return hddsRoot.getPath();
+    }
+  }
+
+  /**
+   * Returns storageID if it is valid. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static String getStorageID(Properties props, File versionFile)
+      throws InconsistentStorageStateException {
+    return getProperty(props, OzoneConsts.STORAGE_ID, versionFile);
+  }
+
+  /**
+   * Returns clusterID if it is valid. It should match the clusterID from the
+   * Datanode. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static String getClusterID(Properties props, File versionFile,
+      String clusterID) throws InconsistentStorageStateException {
+    String cid = getProperty(props, OzoneConsts.CLUSTER_ID, versionFile);
+
+    if (clusterID == null) {
+      return cid;
+    }
+    if (!clusterID.equals(cid)) {
+      throw new InconsistentStorageStateException("Mismatched " +
+          "ClusterIDs. Version File : " + versionFile + " has clusterID: " +
+          cid + " and Datanode has clusterID: " + clusterID);
+    }
+    return cid;
+  }
+
+  /**
+   * Returns datanodeUuid if it is valid. It should match the UUID of the
+   * Datanode. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static String getDatanodeUUID(Properties props, File versionFile,
+      String datanodeUuid)
+      throws InconsistentStorageStateException {
+    String datanodeID = getProperty(props, OzoneConsts.DATANODE_UUID,
+        versionFile);
+
+    if (datanodeUuid != null && !datanodeUuid.equals(datanodeID)) {
+      throw new InconsistentStorageStateException("Mismatched " +
+          "DatanodeUUIDs. Version File : " + versionFile + " has datanodeUuid: "
+          + datanodeID + " and Datanode has datanodeUuid: " + datanodeUuid);
+    }
+    return datanodeID;
+  }
+
+  /**
+   * Returns creationTime if it is valid. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static long getCreationTime(Properties props, File versionFile)
+      throws InconsistentStorageStateException {
+    String cTimeStr = getProperty(props, OzoneConsts.CTIME, versionFile);
+
+    long cTime = Long.parseLong(cTimeStr);
+    long currentTime = Time.now();
+    if (cTime > currentTime || cTime < 0) {
+      throw new InconsistentStorageStateException("Invalid Creation time in " +
+          "Version File : " + versionFile + " - " + cTime + ". Current system" +
+          " time is " + currentTime);
+    }
+    return cTime;
+  }
+
+  /**
+   * Returns layOutVersion if it is valid. Throws an exception otherwise.
+   */
+  @VisibleForTesting
+  public static int getLayOutVersion(Properties props, File versionFile) throws
+      InconsistentStorageStateException {
+    String lvStr = getProperty(props, OzoneConsts.LAYOUTVERSION, versionFile);
+
+    int lv = Integer.parseInt(lvStr);
+    if(DataNodeLayoutVersion.getLatestVersion().getVersion() != lv) {
+      throw new InconsistentStorageStateException("Invalid layOutVersion. " +
+          "Version file has layOutVersion as " + lv + " and latest Datanode " +
+          "layOutVersion is " +
+          DataNodeLayoutVersion.getLatestVersion().getVersion());
+    }
+    return lv;
+  }
+
+  private static String getProperty(Properties props, String propName, File
+      versionFile)
+      throws InconsistentStorageStateException {
+    String value = props.getProperty(propName);
+    if (StringUtils.isBlank(value)) {
+      throw new InconsistentStorageStateException("Invalid " + propName +
+          ". Version File : " + versionFile + " has null or empty " + propName);
+    }
+    return value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
new file mode 100644
index 0000000..788e2cf
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
@@ -0,0 +1,330 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.volume;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.GetSpaceUsed;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
+import org.apache.hadoop.ozone.container.common.DataNodeLayoutVersion;
+import org.apache.hadoop.ozone.container.common.helpers.DatanodeVersionFile;
+import org.apache.hadoop.ozone.container.common.impl.ChunkLayOutVersion;
+import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
+
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * HddsVolume represents volume in a datanode. {@link VolumeSet} maitains a
+ * list of HddsVolumes, one for each volume in the Datanode.
+ * {@link VolumeInfo} in encompassed by this class.
+ */
+public final class HddsVolume {
+
+  private static final Logger LOG = LoggerFactory.getLogger(HddsVolume.class);
+
+  public static final String HDDS_VOLUME_DIR = "hdds";
+
+  private final File hddsRootDir;
+  private final VolumeInfo volumeInfo;
+  private VolumeState state;
+
+  // VERSION file properties
+  private String storageID;       // id of the file system
+  private String clusterID;       // id of the cluster
+  private String datanodeUuid;    // id of the DataNode
+  private long cTime;             // creation time of the file system state
+  private int layoutVersion;      // layout version of the storage data
+
+  /**
+   * Builder for HddsVolume.
+   */
+  public static class Builder {
+    private final String volumeRootStr;
+    private Configuration conf;
+    private StorageType storageType;
+    private long configuredCapacity;
+
+    private String datanodeUuid;
+    private String clusterID;
+
+    public Builder(String rootDirStr) {
+      this.volumeRootStr = rootDirStr;
+    }
+
+    public Builder conf(Configuration config) {
+      this.conf = config;
+      return this;
+    }
+
+    public Builder storageType(StorageType st) {
+      this.storageType = st;
+      return this;
+    }
+
+    public Builder configuredCapacity(long capacity) {
+      this.configuredCapacity = capacity;
+      return this;
+    }
+
+    public Builder datanodeUuid(String datanodeUUID) {
+      this.datanodeUuid = datanodeUUID;
+      return this;
+    }
+
+    public Builder clusterID(String cid) {
+      this.clusterID = cid;
+      return this;
+    }
+
+    public HddsVolume build() throws IOException {
+      return new HddsVolume(this);
+    }
+  }
+
+  private HddsVolume(Builder b) throws IOException {
+    Preconditions.checkNotNull(b.volumeRootStr,
+        "Volume root dir cannot be null");
+    Preconditions.checkNotNull(b.datanodeUuid, "DatanodeUUID cannot be null");
+    Preconditions.checkNotNull(b.conf, "Configuration cannot be null");
+
+    StorageLocation location = StorageLocation.parse(b.volumeRootStr);
+    hddsRootDir = new File(location.getUri().getPath(), HDDS_VOLUME_DIR);
+    this.state = VolumeState.NOT_INITIALIZED;
+    this.clusterID = b.clusterID;
+    this.datanodeUuid = b.datanodeUuid;
+
+    VolumeInfo.Builder volumeBuilder =
+        new VolumeInfo.Builder(b.volumeRootStr, b.conf)
+        .storageType(b.storageType)
+        .configuredCapacity(b.configuredCapacity);
+    this.volumeInfo = volumeBuilder.build();
+
+    LOG.info("Creating Volume: " + this.hddsRootDir + " of  storage type : " +
+        b.storageType + " and capacity : " + volumeInfo.getCapacity());
+
+    initialize();
+  }
+
+  /**
+   * Initializes the volume.
+   * Creates the Version file if not present,
+   * otherwise returns with IOException.
+   * @throws IOException
+   */
+  private void initialize() throws IOException {
+    VolumeState intialVolumeState = analyzeVolumeState();
+    switch (intialVolumeState) {
+    case NON_EXISTENT:
+      // Root directory does not exist. Create it.
+      if (!hddsRootDir.mkdir()) {
+        throw new IOException("Cannot create directory " + hddsRootDir);
+      }
+      setState(VolumeState.NOT_FORMATTED);
+      createVersionFile();
+      break;
+    case NOT_FORMATTED:
+      // Version File does not exist. Create it.
+      createVersionFile();
+      break;
+    case NOT_INITIALIZED:
+      // Version File exists. Verify its correctness and update property fields.
+      readVersionFile();
+      setState(VolumeState.NORMAL);
+      break;
+    default:
+      throw new IOException("Unrecognized initial state : " +
+          intialVolumeState + "of volume : " + hddsRootDir);
+    }
+  }
+
+  private VolumeState analyzeVolumeState() {
+    if (!hddsRootDir.exists()) {
+      return VolumeState.NON_EXISTENT;
+    }
+    if (!getVersionFile().exists()) {
+      return VolumeState.NOT_FORMATTED;
+    }
+    return VolumeState.NOT_INITIALIZED;
+  }
+
+  public void format(String cid) throws IOException {
+    Preconditions.checkNotNull(cid, "clusterID cannot be null while " +
+        "formatting Volume");
+    this.clusterID = cid;
+    initialize();
+  }
+
+  /**
+   * Create Version File and write property fields into it.
+   * @throws IOException
+   */
+  private void createVersionFile() throws IOException {
+    this.storageID = HddsVolumeUtil.generateUuid();
+    this.cTime = Time.now();
+    this.layoutVersion = ChunkLayOutVersion.getLatestVersion().getVersion();
+
+    if (this.clusterID == null || datanodeUuid == null) {
+      // HddsDatanodeService does not have the cluster information yet. Wait
+      // for registration with SCM.
+      LOG.debug("ClusterID not available. Cannot format the volume {}",
+          this.hddsRootDir.getPath());
+      setState(VolumeState.NOT_FORMATTED);
+    } else {
+      // Write the version file to disk.
+      writeVersionFile();
+      setState(VolumeState.NORMAL);
+    }
+  }
+
+  private void writeVersionFile() throws IOException {
+    Preconditions.checkNotNull(this.storageID,
+        "StorageID cannot be null in Version File");
+    Preconditions.checkNotNull(this.clusterID,
+        "ClusterID cannot be null in Version File");
+    Preconditions.checkNotNull(this.datanodeUuid,
+        "DatanodeUUID cannot be null in Version File");
+    Preconditions.checkArgument(this.cTime > 0,
+        "Creation Time should be positive");
+    Preconditions.checkArgument(this.layoutVersion ==
+            DataNodeLayoutVersion.getLatestVersion().getVersion(),
+        "Version File should have the latest LayOutVersion");
+
+    File versionFile = getVersionFile();
+    LOG.debug("Writing Version file to disk, {}", versionFile);
+
+    DatanodeVersionFile dnVersionFile = new DatanodeVersionFile(this.storageID,
+        this.clusterID, this.datanodeUuid, this.cTime, this.layoutVersion);
+    dnVersionFile.createVersionFile(versionFile);
+  }
+
+  /**
+   * Read Version File and update property fields.
+   * Get common storage fields.
+   * Should be overloaded if additional fields need to be read.
+   *
+   * @throws IOException on error
+   */
+  private void readVersionFile() throws IOException {
+    File versionFile = getVersionFile();
+    Properties props = DatanodeVersionFile.readFrom(versionFile);
+    if (props.isEmpty()) {
+      throw new InconsistentStorageStateException(
+          "Version file " + versionFile + " is missing");
+    }
+
+    LOG.debug("Reading Version file from disk, {}", versionFile);
+    this.storageID = HddsVolumeUtil.getStorageID(props, versionFile);
+    this.clusterID = HddsVolumeUtil.getClusterID(props, versionFile,
+        this.clusterID);
+    this.datanodeUuid = HddsVolumeUtil.getDatanodeUUID(props, versionFile,
+        this.datanodeUuid);
+    this.cTime = HddsVolumeUtil.getCreationTime(props, versionFile);
+    this.layoutVersion = HddsVolumeUtil.getLayOutVersion(props, versionFile);
+  }
+
+  private File getVersionFile() {
+    return HddsVolumeUtil.getVersionFile(hddsRootDir);
+  }
+
+  public File getHddsRootDir() {
+    return hddsRootDir;
+  }
+
+  public StorageType getStorageType() {
+    return volumeInfo.getStorageType();
+  }
+
+  public String getStorageID() {
+    return storageID;
+  }
+
+  public String getClusterID() {
+    return clusterID;
+  }
+
+  public String getDatanodeUuid() {
+    return datanodeUuid;
+  }
+
+  public long getCTime() {
+    return cTime;
+  }
+
+  public int getLayoutVersion() {
+    return layoutVersion;
+  }
+
+  public VolumeState getStorageState() {
+    return state;
+  }
+
+  public long getCapacity() throws IOException {
+    return volumeInfo.getCapacity();
+  }
+
+  public long getAvailable() throws IOException {
+    return volumeInfo.getAvailable();
+  }
+
+  public void setState(VolumeState state) {
+    this.state = state;
+  }
+
+  public boolean isFailed() {
+    return (state == VolumeState.FAILED);
+  }
+
+  public void failVolume() {
+    setState(VolumeState.FAILED);
+    volumeInfo.shutdownUsageThread();
+  }
+
+  public void shutdown() {
+    this.state = VolumeState.NON_EXISTENT;
+    volumeInfo.shutdownUsageThread();
+  }
+
+  /**
+   * VolumeState represents the different states a HddsVolume can be in.
+   */
+  public enum VolumeState {
+    NORMAL,
+    FAILED,
+    NON_EXISTENT,
+    NOT_FORMATTED,
+    NOT_INITIALIZED
+  }
+
+  /**
+   * Only for testing. Do not use otherwise.
+   */
+  @VisibleForTesting
+  public void setScmUsageForTesting(GetSpaceUsed scmUsageForTest) {
+    volumeInfo.setScmUsageForTesting(scmUsageForTest);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/RoundRobinVolumeChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/RoundRobinVolumeChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/RoundRobinVolumeChoosingPolicy.java
new file mode 100644
index 0000000..75c92ec
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/RoundRobinVolumeChoosingPolicy.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.volume;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
+import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Choose volumes in round-robin order.
+ * The caller should synchronize access to the list of volumes.
+ */
+public class RoundRobinVolumeChoosingPolicy implements VolumeChoosingPolicy {
+
+  public static final Log LOG = LogFactory.getLog(
+      RoundRobinVolumeChoosingPolicy.class);
+
+  // Stores the index of the next volume to be returned.
+  private AtomicInteger nextVolumeIndex = new AtomicInteger(0);
+
+  @Override
+  public HddsVolume chooseVolume(List<HddsVolume> volumes,
+      long maxContainerSize) throws IOException {
+
+    // No volumes available to choose from
+    if (volumes.size() < 1) {
+      throw new DiskOutOfSpaceException("No more available volumes");
+    }
+
+    // since volumes could've been removed because of the failure
+    // make sure we are not out of bounds
+    int nextIndex = nextVolumeIndex.get();
+    int currentVolumeIndex = nextIndex < volumes.size() ? nextIndex : 0;
+
+    int startVolumeIndex = currentVolumeIndex;
+    long maxAvailable = 0;
+
+    while (true) {
+      final HddsVolume volume = volumes.get(currentVolumeIndex);
+      long availableVolumeSize = volume.getAvailable();
+
+      currentVolumeIndex = (currentVolumeIndex + 1) % volumes.size();
+
+      if (availableVolumeSize > maxContainerSize) {
+        nextVolumeIndex.compareAndSet(nextIndex, currentVolumeIndex);
+        return volume;
+      }
+
+      if (availableVolumeSize > maxAvailable) {
+        maxAvailable = availableVolumeSize;
+      }
+
+      if (currentVolumeIndex == startVolumeIndex) {
+        throw new DiskOutOfSpaceException("Out of space: "
+            + "The volume with the most available space (=" + maxAvailable
+            + " B) is less than the container size (=" + maxContainerSize
+            + " B).");
+      }
+
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeInfo.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeInfo.java
new file mode 100644
index 0000000..4b13d45
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeInfo.java
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.volume;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.GetSpaceUsed;
+import org.apache.hadoop.fs.StorageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Stores information about a disk/volume.
+ */
+public class VolumeInfo {
+
+  private static final Logger LOG = LoggerFactory.getLogger(VolumeInfo.class);
+
+  private final String rootDir;
+  private final StorageType storageType;
+
+  // Space usage calculator
+  private VolumeUsage usage;
+  // Capacity configured. This is useful when we want to
+  // limit the visible capacity for tests. If negative, then we just
+  // query from the filesystem.
+  private long configuredCapacity;
+
+  /**
+   * Builder for VolumeInfo.
+   */
+  public static class Builder {
+    private final Configuration conf;
+    private final String rootDir;
+    private StorageType storageType;
+    private long configuredCapacity;
+
+    public Builder(String root, Configuration config) {
+      this.rootDir = root;
+      this.conf = config;
+    }
+
+    public Builder storageType(StorageType st) {
+      this.storageType = st;
+      return this;
+    }
+
+    public Builder configuredCapacity(long capacity) {
+      this.configuredCapacity = capacity;
+      return this;
+    }
+
+    public VolumeInfo build() throws IOException {
+      return new VolumeInfo(this);
+    }
+  }
+
+  private VolumeInfo(Builder b) throws IOException {
+
+    this.rootDir = b.rootDir;
+    File root = new File(this.rootDir);
+
+    Boolean succeeded = root.isDirectory() || root.mkdirs();
+
+    if (!succeeded) {
+      LOG.error("Unable to create the volume root dir at : {}", root);
+      throw new IOException("Unable to create the volume root dir at " + root);
+    }
+
+    this.storageType = (b.storageType != null ?
+        b.storageType : StorageType.DEFAULT);
+
+    this.configuredCapacity = (b.configuredCapacity != 0 ?
+        b.configuredCapacity : -1);
+
+    this.usage = new VolumeUsage(root, b.conf);
+  }
+
+  public long getCapacity() {
+    return configuredCapacity < 0 ? usage.getCapacity() : configuredCapacity;
+  }
+
+  public long getAvailable() throws IOException {
+    return usage.getAvailable();
+  }
+
+  public long getScmUsed() throws IOException {
+    return usage.getScmUsed();
+  }
+
+  protected void shutdownUsageThread() {
+    if (usage != null) {
+      usage.shutdown();
+    }
+    usage = null;
+  }
+
+  public String getRootDir() {
+    return this.rootDir;
+  }
+
+  public StorageType getStorageType() {
+    return this.storageType;
+  }
+
+  /**
+   * Only for testing. Do not use otherwise.
+   */
+  @VisibleForTesting
+  public void setScmUsageForTesting(GetSpaceUsed scmUsageForTest) {
+    usage.setScmUsageForTesting(scmUsageForTest);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java
new file mode 100644
index 0000000..61aca79
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java
@@ -0,0 +1,309 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.volume;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
+import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume.VolumeState;
+import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
+import org.apache.hadoop.util.AutoCloseableLock;
+import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
+import org.apache.hadoop.util.InstrumentedLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * VolumeSet to manage volumes in a DataNode.
+ */
+public class VolumeSet {
+
+  private static final Logger LOG = LoggerFactory.getLogger(VolumeSet.class);
+
+  private Configuration conf;
+
+  /**
+   * {@link VolumeSet#volumeMap} maintains a map of all active volumes in the
+   * DataNode. Each volume has one-to-one mapping with a volumeInfo object.
+   */
+  private Map<String, HddsVolume> volumeMap;
+  /**
+   * {@link VolumeSet#failedVolumeMap} maintains a map of volumes which have
+   * failed. The keys in this map and {@link VolumeSet#volumeMap} are
+   * mutually exclusive.
+   */
+  private Map<String, HddsVolume> failedVolumeMap;
+  /**
+   * {@link VolumeSet#volumeStateMap} maintains a list of active volumes per
+   * StorageType.
+   */
+  private EnumMap<StorageType, List<HddsVolume>> volumeStateMap;
+
+  /**
+   * Lock to synchronize changes to the VolumeSet. Any update to
+   * {@link VolumeSet#volumeMap}, {@link VolumeSet#failedVolumeMap}, or
+   * {@link VolumeSet#volumeStateMap} should be done after acquiring this lock.
+   */
+  private final AutoCloseableLock volumeSetLock;
+
+  private final DatanodeDetails dnDetails;
+  private String datanodeUuid;
+  private String clusterID;
+
+  public VolumeSet(DatanodeDetails datanodeDetails, Configuration conf)
+      throws DiskOutOfSpaceException {
+    this(datanodeDetails, null, conf);
+  }
+
+  public VolumeSet(DatanodeDetails datanodeDetails, String clusterID,
+      Configuration conf)
+      throws DiskOutOfSpaceException {
+    this.dnDetails = datanodeDetails;
+    this.datanodeUuid = datanodeDetails.getUuidString();
+    this.clusterID = clusterID;
+    this.conf = conf;
+    this.volumeSetLock = new AutoCloseableLock(
+        new InstrumentedLock(getClass().getName(), LOG,
+            new ReentrantLock(true),
+            conf.getTimeDuration(
+                DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
+                DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_DEFAULT,
+                TimeUnit.MILLISECONDS),
+            300));
+
+    initializeVolumeSet();
+  }
+
+  // Add DN volumes configured through ConfigKeys to volumeMap.
+  private void initializeVolumeSet() throws DiskOutOfSpaceException {
+    volumeMap = new ConcurrentHashMap<>();
+    failedVolumeMap = new ConcurrentHashMap<>();
+    volumeStateMap = new EnumMap<>(StorageType.class);
+
+    Collection<String> rawLocations = conf.getTrimmedStringCollection(
+        HDDS_DATANODE_DIR_KEY);
+    if (rawLocations.isEmpty()) {
+      rawLocations = conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY);
+    }
+    if (rawLocations.isEmpty()) {
+      throw new IllegalArgumentException("No location configured in either "
+          + HDDS_DATANODE_DIR_KEY + " or " + DFS_DATANODE_DATA_DIR_KEY);
+    }
+
+    for (StorageType storageType : StorageType.values()) {
+      volumeStateMap.put(storageType, new ArrayList<HddsVolume>());
+    }
+
+    for (String locationString : rawLocations) {
+      try {
+        StorageLocation location = StorageLocation.parse(locationString);
+
+        HddsVolume hddsVolume = createVolume(location.getUri().getPath(),
+            location.getStorageType());
+
+        checkAndSetClusterID(hddsVolume.getClusterID());
+
+        volumeMap.put(hddsVolume.getHddsRootDir().getPath(), hddsVolume);
+        volumeStateMap.get(hddsVolume.getStorageType()).add(hddsVolume);
+        LOG.info("Added Volume : {} to VolumeSet",
+            hddsVolume.getHddsRootDir().getPath());
+      } catch (IOException e) {
+        LOG.error("Failed to parse the storage location: " + locationString, e);
+      }
+    }
+
+    if (volumeMap.size() == 0) {
+      throw new DiskOutOfSpaceException("No storage location configured");
+    }
+  }
+
+  /**
+   * If Version file exists and the {@link VolumeSet#clusterID} is not set yet,
+   * assign it the value from Version file. Otherwise, check that the given
+   * id matches with the id from version file.
+   * @param idFromVersionFile value of the property from Version file
+   * @throws InconsistentStorageStateException
+   */
+  private void checkAndSetClusterID(String idFromVersionFile)
+      throws InconsistentStorageStateException {
+    // If the clusterID is null (not set), assign it the value
+    // from version file.
+    if (this.clusterID == null) {
+      this.clusterID = idFromVersionFile;
+      return;
+    }
+
+    // If the clusterID is already set, it should match with the value from the
+    // version file.
+    if (!idFromVersionFile.equals(this.clusterID)) {
+      throw new InconsistentStorageStateException(
+          "Mismatched ClusterIDs. VolumeSet has: " + this.clusterID +
+              ", and version file has: " + idFromVersionFile);
+    }
+  }
+
+  public void acquireLock() {
+    volumeSetLock.acquire();
+  }
+
+  public void releaseLock() {
+    volumeSetLock.release();
+  }
+
+  private HddsVolume createVolume(String locationString,
+      StorageType storageType) throws IOException {
+    HddsVolume.Builder volumeBuilder = new HddsVolume.Builder(locationString)
+        .conf(conf)
+        .datanodeUuid(datanodeUuid)
+        .clusterID(clusterID)
+        .storageType(storageType);
+    return volumeBuilder.build();
+  }
+
+
+  // Add a volume to VolumeSet
+  public void addVolume(String dataDir) throws IOException {
+    addVolume(dataDir, StorageType.DEFAULT);
+  }
+
+  // Add a volume to VolumeSet
+  public void addVolume(String volumeRoot, StorageType storageType)
+      throws IOException {
+    String hddsRoot = HddsVolumeUtil.getHddsRoot(volumeRoot);
+
+    try (AutoCloseableLock lock = volumeSetLock.acquire()) {
+      if (volumeMap.containsKey(hddsRoot)) {
+        LOG.warn("Volume : {} already exists in VolumeMap", hddsRoot);
+      } else {
+        if (failedVolumeMap.containsKey(hddsRoot)) {
+          failedVolumeMap.remove(hddsRoot);
+        }
+
+        HddsVolume hddsVolume = createVolume(volumeRoot, storageType);
+        volumeMap.put(hddsVolume.getHddsRootDir().getPath(), hddsVolume);
+        volumeStateMap.get(hddsVolume.getStorageType()).add(hddsVolume);
+
+        LOG.info("Added Volume : {} to VolumeSet",
+            hddsVolume.getHddsRootDir().getPath());
+      }
+    }
+  }
+
+  // Mark a volume as failed
+  public void failVolume(String dataDir) {
+    String hddsRoot = HddsVolumeUtil.getHddsRoot(dataDir);
+
+    try (AutoCloseableLock lock = volumeSetLock.acquire()) {
+      if (volumeMap.containsKey(hddsRoot)) {
+        HddsVolume hddsVolume = volumeMap.get(hddsRoot);
+        hddsVolume.failVolume();
+
+        volumeMap.remove(hddsRoot);
+        volumeStateMap.get(hddsVolume.getStorageType()).remove(hddsVolume);
+        failedVolumeMap.put(hddsRoot, hddsVolume);
+
+        LOG.info("Moving Volume : {} to failed Volumes", hddsRoot);
+      } else if (failedVolumeMap.containsKey(hddsRoot)) {
+        LOG.info("Volume : {} is not active", hddsRoot);
+      } else {
+        LOG.warn("Volume : {} does not exist in VolumeSet", hddsRoot);
+      }
+    }
+  }
+
+  // Remove a volume from the VolumeSet completely.
+  public void removeVolume(String dataDir) throws IOException {
+    String hddsRoot = HddsVolumeUtil.getHddsRoot(dataDir);
+
+    try (AutoCloseableLock lock = volumeSetLock.acquire()) {
+      if (volumeMap.containsKey(hddsRoot)) {
+        HddsVolume hddsVolume = volumeMap.get(hddsRoot);
+        hddsVolume.shutdown();
+
+        volumeMap.remove(hddsRoot);
+        volumeStateMap.get(hddsVolume.getStorageType()).remove(hddsVolume);
+
+        LOG.info("Removed Volume : {} from VolumeSet", hddsRoot);
+      } else if (failedVolumeMap.containsKey(hddsRoot)) {
+        HddsVolume hddsVolume = failedVolumeMap.get(hddsRoot);
+        hddsVolume.setState(VolumeState.NON_EXISTENT);
+
+        failedVolumeMap.remove(hddsRoot);
+        LOG.info("Removed Volume : {} from failed VolumeSet", hddsRoot);
+      } else {
+        LOG.warn("Volume : {} does not exist in VolumeSet", hddsRoot);
+      }
+    }
+  }
+
+  public HddsVolume chooseVolume(long containerSize,
+      VolumeChoosingPolicy choosingPolicy) throws IOException {
+    return choosingPolicy.chooseVolume(getVolumesList(), containerSize);
+  }
+
+  public void shutdown() {
+    for (HddsVolume hddsVolume : volumeMap.values()) {
+      try {
+        hddsVolume.shutdown();
+      } catch (Exception ex) {
+        LOG.error("Failed to shutdown volume : " + hddsVolume.getHddsRootDir(),
+            ex);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  public List<HddsVolume> getVolumesList() {
+    return ImmutableList.copyOf(volumeMap.values());
+  }
+
+  @VisibleForTesting
+  public List<HddsVolume> getFailedVolumesList() {
+    return ImmutableList.copyOf(failedVolumeMap.values());
+  }
+
+  @VisibleForTesting
+  public Map<String, HddsVolume> getVolumeMap() {
+    return ImmutableMap.copyOf(volumeMap);
+  }
+
+  @VisibleForTesting
+  public Map<StorageType, List<HddsVolume>> getVolumeStateMap() {
+    return ImmutableMap.copyOf(volumeStateMap);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeUsage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeUsage.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeUsage.java
new file mode 100644
index 0000000..e10d1d4
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeUsage.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.volume;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CachingGetSpaceUsed;
+import org.apache.hadoop.fs.DF;
+import org.apache.hadoop.fs.GetSpaceUsed;
+import org.apache.hadoop.io.IOUtils;
+import static org.apache.hadoop.util.RunJar.SHUTDOWN_HOOK_PRIORITY;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Scanner;
+
+/**
+ * Class that wraps the space df of the Datanode Volumes used by SCM
+ * containers.
+ */
+public class VolumeUsage {
+  private static final Logger LOG = LoggerFactory.getLogger(VolumeUsage.class);
+
+  private final File rootDir;
+  private final DF df;
+  private final File scmUsedFile;
+  private GetSpaceUsed scmUsage;
+  private Runnable shutdownHook;
+
+  private static final String DU_CACHE_FILE = "scmUsed";
+  private volatile boolean scmUsedSaved = false;
+
+  VolumeUsage(File dataLoc, Configuration conf)
+      throws IOException {
+    this.rootDir = dataLoc;
+
+    // SCM used cache file
+    scmUsedFile = new File(rootDir, DU_CACHE_FILE);
+    // get overall disk df
+    this.df = new DF(rootDir, conf);
+
+    startScmUsageThread(conf);
+  }
+
+  void startScmUsageThread(Configuration conf) throws IOException {
+    // get SCM specific df
+    this.scmUsage = new CachingGetSpaceUsed.Builder().setPath(rootDir)
+        .setConf(conf)
+        .setInitialUsed(loadScmUsed())
+        .build();
+
+    // Ensure scm df is saved during shutdown.
+    shutdownHook = () -> {
+      if (!scmUsedSaved) {
+        saveScmUsed();
+      }
+    };
+    ShutdownHookManager.get().addShutdownHook(shutdownHook,
+        SHUTDOWN_HOOK_PRIORITY);
+  }
+
+  long getCapacity() {
+    long capacity = df.getCapacity();
+    return (capacity > 0) ? capacity : 0;
+  }
+
+  /*
+   * Calculate the available space in the volume.
+   */
+  long getAvailable() throws IOException {
+    long remaining = getCapacity() - getScmUsed();
+    long available = df.getAvailable();
+    if (remaining > available) {
+      remaining = available;
+    }
+    return (remaining > 0) ? remaining : 0;
+  }
+
+  long getScmUsed() throws IOException{
+    return scmUsage.getUsed();
+  }
+
+  public void shutdown() {
+    saveScmUsed();
+    scmUsedSaved = true;
+
+    if (shutdownHook != null) {
+      ShutdownHookManager.get().removeShutdownHook(shutdownHook);
+    }
+
+    if (scmUsage instanceof CachingGetSpaceUsed) {
+      IOUtils.cleanupWithLogger(null, ((CachingGetSpaceUsed) scmUsage));
+    }
+  }
+
+  /**
+   * Read in the cached DU value and return it if it is less than 600 seconds
+   * old (DU update interval). Slight imprecision of scmUsed is not critical
+   * and skipping DU can significantly shorten the startup time.
+   * If the cached value is not available or too old, -1 is returned.
+   */
+  long loadScmUsed() {
+    long cachedScmUsed;
+    long mtime;
+    Scanner sc;
+
+    try {
+      sc = new Scanner(scmUsedFile, "UTF-8");
+    } catch (FileNotFoundException fnfe) {
+      return -1;
+    }
+
+    try {
+      // Get the recorded scmUsed from the file.
+      if (sc.hasNextLong()) {
+        cachedScmUsed = sc.nextLong();
+      } else {
+        return -1;
+      }
+      // Get the recorded mtime from the file.
+      if (sc.hasNextLong()) {
+        mtime = sc.nextLong();
+      } else {
+        return -1;
+      }
+
+      // Return the cached value if mtime is okay.
+      if (mtime > 0 && (Time.now() - mtime < 600000L)) {
+        LOG.info("Cached ScmUsed found for {} : {} ", rootDir,
+            cachedScmUsed);
+        return cachedScmUsed;
+      }
+      return -1;
+    } finally {
+      sc.close();
+    }
+  }
+
+  /**
+   * Write the current scmUsed to the cache file.
+   */
+  void saveScmUsed() {
+    if (scmUsedFile.exists() && !scmUsedFile.delete()) {
+      LOG.warn("Failed to delete old scmUsed file in {}.", rootDir);
+    }
+    OutputStreamWriter out = null;
+    try {
+      long used = getScmUsed();
+      if (used > 0) {
+        out = new OutputStreamWriter(new FileOutputStream(scmUsedFile),
+            StandardCharsets.UTF_8);
+        // mtime is written last, so that truncated writes won't be valid.
+        out.write(Long.toString(used) + " " + Long.toString(Time.now()));
+        out.flush();
+        out.close();
+        out = null;
+      }
+    } catch (IOException ioe) {
+      // If write failed, the volume might be bad. Since the cache file is
+      // not critical, log the error and continue.
+      LOG.warn("Failed to write scmUsed to " + scmUsedFile, ioe);
+    } finally {
+      IOUtils.cleanupWithLogger(null, out);
+    }
+  }
+
+  /**
+   * Only for testing. Do not use otherwise.
+   */
+  @VisibleForTesting
+  public void setScmUsageForTesting(GetSpaceUsed scmUsageForTest) {
+    this.scmUsage = scmUsageForTest;
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a5552bf/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeLayOutVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeLayOutVersion.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeLayOutVersion.java
new file mode 100644
index 0000000..5cabef2
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeLayOutVersion.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.container.common;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * This class tests DatanodeLayOutVersion.
+ */
+public class TestDatanodeLayOutVersion {
+
+  @Test
+  public void testDatanodeLayOutVersion() {
+    // Check Latest Version and description
+    Assert.assertEquals(1, DataNodeLayoutVersion.getLatestVersion()
+        .getVersion());
+    Assert.assertEquals("HDDS Datanode LayOut Version 1", DataNodeLayoutVersion
+        .getLatestVersion().getDescription());
+    Assert.assertEquals(DataNodeLayoutVersion.getAllVersions().length,
+        DataNodeLayoutVersion.getAllVersions().length);
+  }
+}


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


[27/37] hadoop git commit: HDDS-182:CleanUp Reimplemented classes. Contributed by Hansiha Koneru

Posted by bh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
index 8d01c80..724a682 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
@@ -29,14 +29,17 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
 import org.apache.hadoop.ozone.container.testutils.BlockDeletingServiceTestImpl;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
-import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
-import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
 import org.apache.hadoop.ozone.container.common.impl.RandomContainerDeletionChoosingPolicy;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
-import org.apache.hadoop.ozone.container.common.statemachine.background.BlockDeletingService;
+import org.apache.hadoop.ozone.container.keyvalue.statemachine.background
+    .BlockDeletingService;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
@@ -44,6 +47,7 @@ import org.apache.hadoop.utils.BackgroundService;
 import org.apache.hadoop.utils.MetadataKeyFilters;
 import org.apache.hadoop.utils.MetadataStore;
 import org.junit.Assert;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.BeforeClass;
 import org.junit.Before;
@@ -70,6 +74,8 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys
 /**
  * Tests to test block deleting service.
  */
+// TODO: Fix BlockDeletingService to work with new StorageLayer
+@Ignore
 public class TestBlockDeletingService {
 
   private static final Logger LOG =
@@ -101,36 +107,22 @@ public class TestBlockDeletingService {
     FileUtils.deleteDirectory(testRoot);
   }
 
-  private ContainerManager createContainerManager(Configuration conf)
-      throws Exception {
-    // use random container choosing policy for testing
-    conf.set(ScmConfigKeys.OZONE_SCM_CONTAINER_DELETION_CHOOSING_POLICY,
-        RandomContainerDeletionChoosingPolicy.class.getName());
-    conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
-        containersDir.getAbsolutePath());
-    if (containersDir.exists()) {
-      FileUtils.deleteDirectory(containersDir);
-    }
-    ContainerManager containerManager = new ContainerManagerImpl();
-    List<StorageLocation> pathLists = new LinkedList<>();
-    pathLists.add(StorageLocation.parse(containersDir.getAbsolutePath()));
-    containerManager.init(conf, pathLists, TestUtils.getDatanodeDetails());
-    return containerManager;
-  }
-
   /**
    * A helper method to create some blocks and put them under deletion
    * state for testing. This method directly updates container.db and
    * creates some fake chunk files for testing.
    */
-  private void createToDeleteBlocks(ContainerManager mgr,
+  private void createToDeleteBlocks(ContainerSet containerSet,
       Configuration conf, int numOfContainers, int numOfBlocksPerContainer,
       int numOfChunksPerBlock, File chunkDir) throws IOException {
     for (int x = 0; x < numOfContainers; x++) {
       long containerID = ContainerTestHelper.getTestContainerID();
-      ContainerData data = new ContainerData(containerID, conf);
-      mgr.createContainer(data);
-      data = mgr.readContainer(containerID);
+      KeyValueContainerData data = new KeyValueContainerData(containerID,
+          ContainerTestHelper.CONTAINER_MAX_SIZE_GB);
+      Container container = new KeyValueContainer(data, conf);
+      containerSet.addContainer(container);
+      data = (KeyValueContainerData) containerSet.getContainer(
+          containerID).getContainerData();
       MetadataStore metadata = KeyUtils.getDB(data, conf);
       for (int j = 0; j<numOfBlocksPerContainer; j++) {
         BlockID blockID =
@@ -198,29 +190,28 @@ public class TestBlockDeletingService {
     Configuration conf = new OzoneConfiguration();
     conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
     conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
-    ContainerManager containerManager = createContainerManager(conf);
-    createToDeleteBlocks(containerManager, conf, 1, 3, 1, chunksDir);
+    ContainerSet containerSet = new ContainerSet();
+    createToDeleteBlocks(containerSet, conf, 1, 3, 1, chunksDir);
 
     BlockDeletingServiceTestImpl svc =
-        new BlockDeletingServiceTestImpl(containerManager, 1000, conf);
+        new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
     svc.start();
     GenericTestUtils.waitFor(() -> svc.isStarted(), 100, 3000);
 
     // Ensure 1 container was created
     List<ContainerData> containerData = Lists.newArrayList();
-    containerManager.listContainer(0L, 1, containerData);
+    containerSet.listContainer(0L, 1, containerData);
     Assert.assertEquals(1, containerData.size());
 
-    MetadataStore meta = KeyUtils.getDB(containerData.get(0), conf);
-    Map<Long, ContainerData> containerMap =
-        ((ContainerManagerImpl) containerManager).getContainerMap();
-    long transactionId =
-        containerMap.get(containerData.get(0).getContainerID())
-            .getDeleteTransactionId();
+    MetadataStore meta = KeyUtils.getDB(
+        (KeyValueContainerData) containerData.get(0), conf);
+    Map<Long, Container> containerMap = containerSet.getContainerMap();
+
 
     // Number of deleted blocks in container should be equal to 0 before
     // block delete
-    Assert.assertEquals(0, transactionId);
+    // TODO : Implement deleteTransactionID in ContainerData.
+//    Assert.assertEquals(0, transactionId);
 
     // Ensure there are 3 blocks under deletion and 0 deleted blocks
     Assert.assertEquals(3, getUnderDeletionBlocksCount(meta));
@@ -240,7 +231,6 @@ public class TestBlockDeletingService {
     Assert.assertEquals(3, getDeletedBlocksCount(meta));
 
     svc.shutdown();
-    shutdownContainerMangaer(containerManager);
   }
 
   @Test
@@ -250,12 +240,12 @@ public class TestBlockDeletingService {
         TimeUnit.MILLISECONDS);
     conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
     conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 10);
-    ContainerManager containerManager = createContainerManager(conf);
+    ContainerSet containerSet = new ContainerSet();
     // Create 1 container with 100 blocks
-    createToDeleteBlocks(containerManager, conf, 1, 100, 1, chunksDir);
+    createToDeleteBlocks(containerSet, conf, 1, 100, 1, chunksDir);
 
     BlockDeletingServiceTestImpl service =
-        new BlockDeletingServiceTestImpl(containerManager, 1000, conf);
+        new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
     service.start();
     GenericTestUtils.waitFor(() -> service.isStarted(), 100, 3000);
 
@@ -269,7 +259,6 @@ public class TestBlockDeletingService {
     // Shutdown service and verify all threads are stopped
     service.shutdown();
     GenericTestUtils.waitFor(() -> service.getThreadCount() == 0, 100, 1000);
-    shutdownContainerMangaer(containerManager);
   }
 
   @Test
@@ -277,14 +266,13 @@ public class TestBlockDeletingService {
     Configuration conf = new OzoneConfiguration();
     conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
     conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
-    ContainerManager containerManager = createContainerManager(conf);
-    createToDeleteBlocks(containerManager, conf, 1, 3, 1, chunksDir);
+    ContainerSet containerSet = new ContainerSet();
+    createToDeleteBlocks(containerSet, conf, 1, 3, 1, chunksDir);
 
     // set timeout value as 1ns to trigger timeout behavior
     long timeout  = 1;
-    BlockDeletingService svc = new BlockDeletingService(containerManager,
-        TimeUnit.MILLISECONDS.toNanos(1000), timeout, TimeUnit.NANOSECONDS,
-        conf);
+    BlockDeletingService svc =
+        new BlockDeletingService(containerSet, 1000, timeout, conf);
     svc.start();
 
     LogCapturer log = LogCapturer.captureLogs(BackgroundService.LOG);
@@ -303,16 +291,15 @@ public class TestBlockDeletingService {
 
     // test for normal case that doesn't have timeout limitation
     timeout  = 0;
-    createToDeleteBlocks(containerManager, conf, 1, 3, 1, chunksDir);
-    svc = new BlockDeletingService(containerManager,
-        TimeUnit.MILLISECONDS.toNanos(1000), timeout, TimeUnit.NANOSECONDS,
-        conf);
+    createToDeleteBlocks(containerSet, conf, 1, 3, 1, chunksDir);
+    svc =  new BlockDeletingService(containerSet, 1000, timeout, conf);
     svc.start();
 
     // get container meta data
     List<ContainerData> containerData = Lists.newArrayList();
-    containerManager.listContainer(0L, 1, containerData);
-    MetadataStore meta = KeyUtils.getDB(containerData.get(0), conf);
+    containerSet.listContainer(0L, 1, containerData);
+    MetadataStore meta = KeyUtils.getDB(
+        (KeyValueContainerData) containerData.get(0), conf);
 
     LogCapturer newLog = LogCapturer.captureLogs(BackgroundService.LOG);
     GenericTestUtils.waitFor(() -> {
@@ -331,7 +318,6 @@ public class TestBlockDeletingService {
     Assert.assertTrue(!newLog.getOutput().contains(
         "Background task executes timed out, retrying in next interval"));
     svc.shutdown();
-    shutdownContainerMangaer(containerManager);
   }
 
   @Test(timeout = 30000)
@@ -349,11 +335,11 @@ public class TestBlockDeletingService {
     // Process 1 container per interval
     conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 1);
     conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 1);
-    ContainerManager containerManager = createContainerManager(conf);
-    createToDeleteBlocks(containerManager, conf, 2, 1, 10, chunksDir);
+    ContainerSet containerSet = new ContainerSet();
+    createToDeleteBlocks(containerSet, conf, 2, 1, 10, chunksDir);
 
     BlockDeletingServiceTestImpl service =
-        new BlockDeletingServiceTestImpl(containerManager, 1000, conf);
+        new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
     service.start();
 
     try {
@@ -363,7 +349,6 @@ public class TestBlockDeletingService {
       Assert.assertEquals(10, chunksDir.listFiles().length);
     } finally {
       service.shutdown();
-      shutdownContainerMangaer(containerManager);
     }
   }
 
@@ -383,14 +368,14 @@ public class TestBlockDeletingService {
     Configuration conf = new OzoneConfiguration();
     conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
     conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
-    ContainerManager containerManager = createContainerManager(conf);
-    createToDeleteBlocks(containerManager, conf, 5, 3, 1, chunksDir);
+    ContainerSet containerSet = new ContainerSet();
+    createToDeleteBlocks(containerSet, conf, 5, 3, 1, chunksDir);
 
     // Make sure chunks are created
     Assert.assertEquals(15, chunksDir.listFiles().length);
 
     BlockDeletingServiceTestImpl service =
-        new BlockDeletingServiceTestImpl(containerManager, 1000, conf);
+        new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
     service.start();
 
     try {
@@ -407,17 +392,6 @@ public class TestBlockDeletingService {
       Assert.assertEquals(0, chunksDir.listFiles().length);
     } finally {
       service.shutdown();
-      shutdownContainerMangaer(containerManager);
-    }
-  }
-
-  private void shutdownContainerMangaer(ContainerManager mgr)
-      throws IOException {
-    mgr.writeLock();
-    try {
-      mgr.shutdown();
-    } finally {
-      mgr.writeUnlock();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDeletionChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDeletionChoosingPolicy.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDeletionChoosingPolicy.java
index 4344419..c161551 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDeletionChoosingPolicy.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDeletionChoosingPolicy.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.ozone.container.common.impl;
 
-import static org.apache.hadoop.ozone.container.ContainerTestHelper.createSingleNodePipeline;
-
 import java.io.File;
 import java.io.IOException;
 import java.util.HashMap;
@@ -36,23 +34,25 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
-import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.utils.MetadataStore;
-import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
 /**
  * The class for testing container deletion choosing policy.
  */
+@Ignore
 public class TestContainerDeletionChoosingPolicy {
   private static String path;
-  private static ContainerManagerImpl containerManager;
+  private static ContainerSet containerSet;
   private static OzoneConfiguration conf;
 
   @Before
@@ -65,18 +65,6 @@ public class TestContainerDeletionChoosingPolicy {
     conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
   }
 
-  @After
-  public void shutdown() throws IOException {
-    FileUtils.deleteDirectory(new File(path));
-
-    containerManager.writeLock();
-    try{
-      containerManager.shutdown();
-    } finally {
-      containerManager.writeUnlock();
-    }
-  }
-
   @Test
   public void testRandomChoosingPolicy() throws IOException {
     File containerDir = new File(path);
@@ -89,25 +77,26 @@ public class TestContainerDeletionChoosingPolicy {
         RandomContainerDeletionChoosingPolicy.class.getName());
     List<StorageLocation> pathLists = new LinkedList<>();
     pathLists.add(StorageLocation.parse(containerDir.getAbsolutePath()));
-    containerManager = new ContainerManagerImpl();
-    containerManager.init(conf, pathLists, TestUtils.getDatanodeDetails());
+    containerSet = new ContainerSet();
 
     int numContainers = 10;
     for (int i = 0; i < numContainers; i++) {
-      ContainerData data = new ContainerData(new Long(i), conf);
-      containerManager.createContainer(data);
+      KeyValueContainerData data = new KeyValueContainerData(new Long(i),
+          ContainerTestHelper.CONTAINER_MAX_SIZE_GB);
+      KeyValueContainer container = new KeyValueContainer(data, conf);
+      containerSet.addContainer(container);
       Assert.assertTrue(
-          containerManager.getContainerMap().containsKey(data.getContainerID()));
+          containerSet.getContainerMap().containsKey(data.getContainerID()));
     }
 
-    List<ContainerData> result0 = containerManager
+    List<ContainerData> result0 = containerSet
         .chooseContainerForBlockDeletion(5);
     Assert.assertEquals(5, result0.size());
 
     // test random choosing
-    List<ContainerData> result1 = containerManager
+    List<ContainerData> result1 = containerSet
         .chooseContainerForBlockDeletion(numContainers);
-    List<ContainerData> result2 = containerManager
+    List<ContainerData> result2 = containerSet
         .chooseContainerForBlockDeletion(numContainers);
 
     boolean hasShuffled = false;
@@ -133,9 +122,8 @@ public class TestContainerDeletionChoosingPolicy {
         TopNOrderedContainerDeletionChoosingPolicy.class.getName());
     List<StorageLocation> pathLists = new LinkedList<>();
     pathLists.add(StorageLocation.parse(containerDir.getAbsolutePath()));
-    containerManager = new ContainerManagerImpl();
+    containerSet = new ContainerSet();
     DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
-    containerManager.init(conf, pathLists, datanodeDetails);
 
     int numContainers = 10;
     Random random = new Random();
@@ -143,10 +131,12 @@ public class TestContainerDeletionChoosingPolicy {
     // create [numContainers + 1] containers
     for (int i = 0; i <= numContainers; i++) {
       long containerId = RandomUtils.nextLong();
-      ContainerData data = new ContainerData(containerId, conf);
-      containerManager.createContainer(data);
+      KeyValueContainerData data = new KeyValueContainerData(new Long(i),
+          ContainerTestHelper.CONTAINER_MAX_SIZE_GB);
+      KeyValueContainer container = new KeyValueContainer(data, conf);
+      containerSet.addContainer(container);
       Assert.assertTrue(
-          containerManager.getContainerMap().containsKey(containerId));
+          containerSet.getContainerMap().containsKey(containerId));
 
       // don't create deletion blocks in the last container.
       if (i == numContainers) {
@@ -167,16 +157,11 @@ public class TestContainerDeletionChoosingPolicy {
       }
     }
 
-    containerManager.writeLock();
-    containerManager.shutdown();
-    containerManager.writeUnlock();
-    containerManager.init(conf, pathLists, datanodeDetails);
-
-    List<ContainerData> result0 = containerManager
+    List<ContainerData> result0 = containerSet
         .chooseContainerForBlockDeletion(5);
     Assert.assertEquals(5, result0.size());
 
-    List<ContainerData> result1 = containerManager
+    List<ContainerData> result1 = containerSet
         .chooseContainerForBlockDeletion(numContainers + 1);
     // the empty deletion blocks container should not be chosen
     Assert.assertEquals(numContainers, result1.size());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
index 4975fd3..e634dd8 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
@@ -17,26 +17,38 @@
 
 package org.apache.hadoop.ozone.container.common.impl;
 
+import com.google.common.collect.Maps;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.RandomUtils;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.hdds.scm.TestUtils;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.volume
+    .RoundRobinVolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
+import org.apache.hadoop.ozone.container.keyvalue.helpers
+    .KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerImpl;
+import org.apache.hadoop.ozone.container.keyvalue.impl.KeyManagerImpl;
+import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
+import org.apache.hadoop.ozone.container.keyvalue.interfaces.KeyManager;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
-import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.utils.MetadataStore;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -51,7 +63,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.IOException;
 import java.nio.file.DirectoryStream;
 import java.nio.file.Files;
@@ -65,11 +76,10 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.ArrayList;
+import java.util.UUID;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_ROOT_PREFIX;
-import static org.apache.hadoop.ozone.container.ContainerTestHelper
-    .createSingleNodePipeline;
 import static org.apache.hadoop.ozone.container.ContainerTestHelper.getChunk;
 import static org.apache.hadoop.ozone.container.ContainerTestHelper.getData;
 import static org.apache.hadoop.ozone.container.ContainerTestHelper
@@ -95,40 +105,42 @@ public class TestContainerPersistence {
 
   private static Logger log =
       LoggerFactory.getLogger(TestContainerPersistence.class);
+  private static String hddsPath;
   private static String path;
-  private static ContainerManagerImpl containerManager;
-  private static ChunkManagerImpl chunkManager;
-  private static KeyManagerImpl keyManager;
   private static OzoneConfiguration conf;
   private static List<StorageLocation> pathLists = new LinkedList<>();
   private Long  containerID = 8888L;;
+  private static final String datanodeUuid = UUID.randomUUID().toString();
+  private static final String scmId = UUID.randomUUID().toString();
+
+  private static ContainerSet containerSet;
+  private static VolumeSet volumeSet;
+  private static VolumeChoosingPolicy volumeChoosingPolicy;
+  private static KeyManager keyManager;
+  private static ChunkManager chunkManager;
 
   @BeforeClass
   public static void init() throws Throwable {
     conf = new OzoneConfiguration();
-    path = GenericTestUtils
+    hddsPath = GenericTestUtils
         .getTempPath(TestContainerPersistence.class.getSimpleName());
-    path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
+    path = hddsPath + conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
         OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
     conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
+    conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, hddsPath);
 
     File containerDir = new File(path);
     if (containerDir.exists()) {
       FileUtils.deleteDirectory(new File(path));
     }
     Assert.assertTrue(containerDir.mkdirs());
-
-    containerManager = new ContainerManagerImpl();
-    chunkManager = new ChunkManagerImpl(containerManager);
-    containerManager.setChunkManager(chunkManager);
-    keyManager = new KeyManagerImpl(containerManager, conf);
-    containerManager.setKeyManager(keyManager);
-
+    volumeChoosingPolicy = new RoundRobinVolumeChoosingPolicy();
   }
 
   @AfterClass
   public static void shutdown() throws IOException {
     FileUtils.deleteDirectory(new File(path));
+    FileUtils.deleteDirectory(new File(hddsPath));
   }
 
   @Before
@@ -140,7 +152,10 @@ public class TestContainerPersistence {
         Paths.get(path).resolve(CONTAINER_ROOT_PREFIX).toString());
 
     pathLists.clear();
-    containerManager.getContainerMap().clear();
+    containerSet = new ContainerSet();
+    volumeSet = new VolumeSet(datanodeUuid, conf);
+    keyManager = new KeyManagerImpl(conf);
+    chunkManager = new ChunkManagerImpl();
 
     if (!new File(loc.getNormalizedUri()).mkdirs()) {
       throw new IOException("unable to create paths. " +
@@ -152,26 +167,18 @@ public class TestContainerPersistence {
       StorageLocation location = StorageLocation.parse(dir);
       FileUtils.forceMkdir(new File(location.getNormalizedUri()));
     }
-
-    containerManager.init(conf, pathLists, TestUtils.getDatanodeDetails());
-  }
+ }
 
   @After
   public void cleanupDir() throws IOException {
-    // Shutdown containerManager
-    containerManager.writeLock();
-    try {
-      containerManager.shutdown();
-    } finally {
-      containerManager.writeUnlock();
-    }
-
     // Clean up SCM metadata
     log.info("Deleting {}", path);
     FileUtils.deleteDirectory(new File(path));
+    log.info("Deleting {}", hddsPath);
+    FileUtils.deleteDirectory(new File(hddsPath));
 
     // Clean up SCM datanode container metadata/data
-    for (String dir : conf.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
+    for (String dir : conf.getStrings(ScmConfigKeys.HDDS_DATANODE_DIR_KEY)) {
       StorageLocation location = StorageLocation.parse(dir);
       FileUtils.deleteDirectory(new File(location.getNormalizedUri()));
     }
@@ -181,32 +188,39 @@ public class TestContainerPersistence {
     return ContainerTestHelper.getTestContainerID();
   }
 
+  private Container addContainer(ContainerSet containerSet, long containerID)
+      throws IOException {
+    KeyValueContainerData data = new KeyValueContainerData(containerID,
+        ContainerTestHelper.CONTAINER_MAX_SIZE_GB);
+    data.addMetadata("VOLUME", "shire");
+    data.addMetadata("owner)", "bilbo");
+    KeyValueContainer container = new KeyValueContainer(data, conf);
+    container.create(volumeSet, volumeChoosingPolicy, scmId);
+    containerSet.addContainer(container);
+    return  container;
+  }
+
   @Test
   public void testCreateContainer() throws Exception {
     long testContainerID = getTestContainerID();
-    ContainerData data = new ContainerData(testContainerID, conf);
-    data.addMetadata("VOLUME", "shire");
-    data.addMetadata("owner)", "bilbo");
-    containerManager.createContainer(data);
-    Assert.assertTrue(containerManager.getContainerMap()
+    addContainer(containerSet, testContainerID);
+    Assert.assertTrue(containerSet.getContainerMap()
         .containsKey(testContainerID));
-    ContainerData cData = containerManager
-        .getContainerMap().get(testContainerID);
+    KeyValueContainerData kvData =
+        (KeyValueContainerData) containerSet.getContainer(testContainerID)
+        .getContainerData();
 
-    Assert.assertNotNull(cData);
-    Assert.assertNotNull(cData.getContainerPath());
-    Assert.assertNotNull(cData.getDBPath());
+    Assert.assertNotNull(kvData);
+    Assert.assertTrue(new File(kvData.getMetadataPath()).exists());
+    Assert.assertTrue(new File(kvData.getChunksPath()).exists());
+    Assert.assertTrue(kvData.getDbFile().exists());
 
-
-    Assert.assertTrue(new File(cData.getContainerPath())
-        .exists());
-
-    Path meta = Paths.get(cData.getDBPath()).getParent();
+    Path meta = kvData.getDbFile().toPath().getParent();
     Assert.assertTrue(meta != null && Files.exists(meta));
 
     MetadataStore store = null;
     try {
-      store = KeyUtils.getDB(cData, conf);
+      store = KeyUtils.getDB(kvData, conf);
       Assert.assertNotNull(store);
     } finally {
       if (store != null) {
@@ -219,12 +233,9 @@ public class TestContainerPersistence {
   public void testCreateDuplicateContainer() throws Exception {
     long testContainerID = getTestContainerID();
 
-    ContainerData data = new ContainerData(testContainerID, conf);
-    data.addMetadata("VOLUME", "shire");
-    data.addMetadata("owner)", "bilbo");
-    containerManager.createContainer(data);
+    Container container = addContainer(containerSet, testContainerID);
     try {
-      containerManager.createContainer(data);
+      containerSet.addContainer(container);
       fail("Expected Exception not thrown.");
     } catch (IOException ex) {
       Assert.assertNotNull(ex);
@@ -237,54 +248,40 @@ public class TestContainerPersistence {
     Thread.sleep(100);
     long testContainerID2 = getTestContainerID();
 
-    ContainerData data = new ContainerData(testContainerID1, conf);
-    data.addMetadata("VOLUME", "shire");
-    data.addMetadata("owner)", "bilbo");
-    containerManager.createContainer(data);
-    containerManager.closeContainer(testContainerID1);
+    Container container1 = addContainer(containerSet, testContainerID1);
+    container1.close();
 
-    data = new ContainerData(testContainerID2, conf);
-    data.addMetadata("VOLUME", "shire");
-    data.addMetadata("owner)", "bilbo");
-    containerManager.createContainer(data);
-    containerManager.closeContainer(testContainerID2);
+    Container container2 = addContainer(containerSet, testContainerID2);
 
-    Assert.assertTrue(containerManager.getContainerMap()
+    Assert.assertTrue(containerSet.getContainerMap()
         .containsKey(testContainerID1));
-    Assert.assertTrue(containerManager.getContainerMap()
+    Assert.assertTrue(containerSet.getContainerMap()
         .containsKey(testContainerID2));
 
-    containerManager.deleteContainer(testContainerID1, false);
-    Assert.assertFalse(containerManager.getContainerMap()
+    container1.delete(false);
+    containerSet.removeContainer(testContainerID1);
+    Assert.assertFalse(containerSet.getContainerMap()
         .containsKey(testContainerID1));
 
-    // Let us make sure that we are able to re-use a container name after
-    // delete.
-
-    data = new ContainerData(testContainerID1, conf);
-    data.addMetadata("VOLUME", "shire");
-    data.addMetadata("owner)", "bilbo");
-    containerManager.createContainer(data);
-    containerManager.closeContainer(testContainerID1);
-
-    // Assert we still have both containers.
-    Assert.assertTrue(containerManager.getContainerMap()
-        .containsKey(testContainerID1));
-    Assert.assertTrue(containerManager.getContainerMap()
-        .containsKey(testContainerID2));
-
-    // Add some key to a container and then delete.
-    // Delete should fail because the container is no longer empty.
+    // Adding key to a deleted container should fail.
+    exception.expect(StorageContainerException.class);
+    exception.expectMessage("Error opening DB.");
     BlockID blockID1 = ContainerTestHelper.getTestBlockID(testContainerID1);
-    KeyData someKey = new KeyData(blockID1);
-    someKey.setChunks(new LinkedList<ContainerProtos.ChunkInfo>());
-    keyManager.putKey(someKey);
+    KeyData someKey1 = new KeyData(blockID1);
+    someKey1.setChunks(new LinkedList<ContainerProtos.ChunkInfo>());
+    keyManager.putKey(container1, someKey1);
+
+    // Deleting a non-empty container should fail.
+    BlockID blockID2 = ContainerTestHelper.getTestBlockID(testContainerID2);
+    KeyData someKey2 = new KeyData(blockID2);
+    someKey2.setChunks(new LinkedList<ContainerProtos.ChunkInfo>());
+    keyManager.putKey(container2, someKey2);
 
     exception.expect(StorageContainerException.class);
     exception.expectMessage(
         "Container cannot be deleted because it is not empty.");
-    containerManager.deleteContainer(testContainerID1, false);
-    Assert.assertTrue(containerManager.getContainerMap()
+    container2.delete(false);
+    Assert.assertTrue(containerSet.getContainerMap()
         .containsKey(testContainerID1));
   }
 
@@ -295,21 +292,21 @@ public class TestContainerPersistence {
 
     for (int i = 0; i < count; i++) {
       long testContainerID = getTestContainerID();
-      ContainerData data = new ContainerData(testContainerID, conf);
-      containerManager.createContainer(data);
+      Container container = addContainer(containerSet, testContainerID);
 
       // Close a bunch of containers.
-      // Put closed container names to a list.
       if (i%3 == 0) {
-        containerManager.closeContainer(testContainerID);
-        containerIDs.add(testContainerID);
+        container.close();
       }
+      containerIDs.add(testContainerID);
     }
 
-    // The container report only returns reports of closed containers.
-    List<ContainerData> reports = containerManager.getClosedContainerReports();
-    Assert.assertEquals(4, reports.size());
-    for(ContainerData report : reports) {
+    // ContainerSet#getContainerReport currently returns all containers (open
+    // and closed) reports.
+    List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports =
+        containerSet.getContainerReport().getReportsList();
+    Assert.assertEquals(10, reports.size());
+    for(StorageContainerDatanodeProtocolProtos.ContainerInfo   report : reports) {
       long actualContainerID = report.getContainerID();
       Assert.assertTrue(containerIDs.remove(actualContainerID));
     }
@@ -324,24 +321,21 @@ public class TestContainerPersistence {
    */
   @Test
   public void testListContainer() throws IOException {
-    final int count = 50;
+    final int count = 10;
     final int step = 5;
 
     Map<Long, ContainerData> testMap = new HashMap<>();
     for (int x = 0; x < count; x++) {
       long testContainerID = getTestContainerID();
-      ContainerData data = new ContainerData(testContainerID, conf);
-      data.addMetadata("VOLUME", "shire");
-      data.addMetadata("owner)", "bilbo");
-      containerManager.createContainer(data);
-      testMap.put(testContainerID, data);
+      Container container = addContainer(containerSet, testContainerID);
+      testMap.put(testContainerID, container.getContainerData());
     }
 
     int counter = 0;
     long prevKey = 0;
     List<ContainerData> results = new LinkedList<>();
     while (counter < count) {
-      containerManager.listContainer(prevKey, step, results);
+      containerSet.listContainer(prevKey, step, results);
       for (int y = 0; y < results.size(); y++) {
         testMap.remove(results.get(y).getContainerID());
       }
@@ -350,7 +344,7 @@ public class TestContainerPersistence {
 
       //Assert that container is returning results in a sorted fashion.
       Assert.assertTrue(prevKey < nextKey);
-      prevKey = nextKey;
+      prevKey = nextKey + 1;
       results.clear();
     }
     // Assert that we listed all the keys that we had put into
@@ -358,22 +352,18 @@ public class TestContainerPersistence {
     Assert.assertTrue(testMap.isEmpty());
   }
 
-  private ChunkInfo writeChunkHelper(BlockID blockID,
-      Pipeline pipeline) throws IOException,
-      NoSuchAlgorithmException {
+  private ChunkInfo writeChunkHelper(BlockID blockID)
+      throws IOException, NoSuchAlgorithmException {
     final int datalen = 1024;
     long testContainerID = blockID.getContainerID();
-    ContainerData cData = new ContainerData(testContainerID, conf);
-    cData.addMetadata("VOLUME", "shire");
-    cData.addMetadata("owner", "bilbo");
-    if(!containerManager.getContainerMap()
-        .containsKey(testContainerID)) {
-      containerManager.createContainer(cData);
+    Container container = containerSet.getContainer(testContainerID);
+    if (container == null) {
+      container = addContainer(containerSet, testContainerID);
     }
     ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
     byte[] data = getData(datalen);
     setDataChecksum(info, data);
-    chunkManager.writeChunk(blockID, info, data, COMBINED);
+    chunkManager.writeChunk(container, blockID, info, data, COMBINED);
     return info;
 
   }
@@ -389,8 +379,7 @@ public class TestContainerPersistence {
       NoSuchAlgorithmException {
     BlockID blockID = ContainerTestHelper.
         getTestBlockID(getTestContainerID());
-    Pipeline pipeline = createSingleNodePipeline();
-    writeChunkHelper(blockID, pipeline);
+    writeChunkHelper(blockID);
   }
 
   /**
@@ -407,27 +396,22 @@ public class TestContainerPersistence {
     final int chunkCount = 1024;
 
     long testContainerID = getTestContainerID();
-    Map<String, ChunkInfo> fileHashMap = new HashMap<>();
-
-    ContainerData cData = new ContainerData(testContainerID, conf);
-    cData.addMetadata("VOLUME", "shire");
-    cData.addMetadata("owner)", "bilbo");
-    containerManager.createContainer(cData);
-    BlockID blockID = ContainerTestHelper.
-        getTestBlockID(testContainerID);
+    Container container = addContainer(containerSet, testContainerID);
 
+    BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
+    Map<String, ChunkInfo> fileHashMap = new HashMap<>();
     for (int x = 0; x < chunkCount; x++) {
       ChunkInfo info = getChunk(blockID.getLocalID(), x, 0, datalen);
       byte[] data = getData(datalen);
       setDataChecksum(info, data);
-      chunkManager.writeChunk(blockID, info, data, COMBINED);
+      chunkManager.writeChunk(container, blockID, info, data, COMBINED);
       String fileName = String.format("%s.data.%d", blockID.getLocalID(), x);
       fileHashMap.put(fileName, info);
     }
 
-    ContainerData cNewData = containerManager.readContainer(testContainerID);
+    ContainerData cNewData = container.getContainerData();
     Assert.assertNotNull(cNewData);
-    Path dataDir = ContainerUtils.getDataDirectory(cNewData);
+    Path dataDir = Paths.get(cNewData.getDataPath());
 
     String globFormat = String.format("%s.data.*", blockID.getLocalID());
     MessageDigest sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
@@ -451,7 +435,7 @@ public class TestContainerPersistence {
       for (int x = 0; x < chunkCount; x++) {
         String fileName = String.format("%s.data.%d", blockID.getLocalID(), x);
         ChunkInfo info = fileHashMap.get(fileName);
-        byte[] data = chunkManager.readChunk(blockID, info);
+        byte[] data = chunkManager.readChunk(container, blockID, info);
         sha.update(data);
         Assert.assertEquals(Hex.encodeHexString(sha.digest()),
             info.getChecksum());
@@ -472,23 +456,19 @@ public class TestContainerPersistence {
     final int length = datalen/2;
 
     long testContainerID = getTestContainerID();
-    BlockID blockID = ContainerTestHelper.
-        getTestBlockID(testContainerID);
+    Container container = addContainer(containerSet, testContainerID);
 
-    ContainerData cData = new ContainerData(testContainerID, conf);
-    cData.addMetadata("VOLUME", "shire");
-    cData.addMetadata("owner)", "bilbo");
-    containerManager.createContainer(cData);
+    BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
     ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
     byte[] data = getData(datalen);
     setDataChecksum(info, data);
-    chunkManager.writeChunk(blockID, info, data, COMBINED);
+    chunkManager.writeChunk(container, blockID, info, data, COMBINED);
 
-    byte[] readData = chunkManager.readChunk(blockID, info);
+    byte[] readData = chunkManager.readChunk(container, blockID, info);
     assertTrue(Arrays.equals(data, readData));
 
     ChunkInfo info2 = getChunk(blockID.getLocalID(), 0, start, length);
-    byte[] readData2 = chunkManager.readChunk(blockID, info2);
+    byte[] readData2 = chunkManager.readChunk(container, blockID, info2);
     assertEquals(length, readData2.length);
     assertTrue(Arrays.equals(
         Arrays.copyOfRange(data, start, start + length), readData2));
@@ -507,31 +487,29 @@ public class TestContainerPersistence {
     final int datalen = 1024;
 
     long testContainerID = getTestContainerID();
-    BlockID blockID = ContainerTestHelper.
-        getTestBlockID(testContainerID);
+    Container container = addContainer(containerSet, testContainerID);
 
-    ContainerData cData = new ContainerData(testContainerID, conf);
-    cData.addMetadata("VOLUME", "shire");
-    cData.addMetadata("owner)", "bilbo");
-    containerManager.createContainer(cData);
+    BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
     ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
     byte[] data = getData(datalen);
     setDataChecksum(info, data);
-    chunkManager.writeChunk(blockID, info, data, COMBINED);
+    chunkManager.writeChunk(container, blockID, info, data, COMBINED);
     try {
-      chunkManager.writeChunk(blockID, info, data, COMBINED);
-    } catch (IOException ex) {
-      Assert.assertTrue(ex.getCause().getMessage().contains(
+      chunkManager.writeChunk(container, blockID, info, data, COMBINED);
+    } catch (StorageContainerException ex) {
+      Assert.assertTrue(ex.getMessage().contains(
           "Rejecting write chunk request. OverWrite flag required"));
+      Assert.assertEquals(ex.getResult(),
+          ContainerProtos.Result.OVERWRITE_FLAG_REQUIRED);
     }
 
     // With the overwrite flag it should work now.
     info.addMetadata(OzoneConsts.CHUNK_OVERWRITE, "true");
-    chunkManager.writeChunk(blockID, info, data, COMBINED);
-    long bytesUsed = containerManager.getBytesUsed(testContainerID);
+    chunkManager.writeChunk(container, blockID, info, data, COMBINED);
+    long bytesUsed = container.getContainerData().getBytesUsed();
     Assert.assertEquals(datalen, bytesUsed);
 
-    long bytesWrite = containerManager.getWriteBytes(testContainerID);
+    long bytesWrite = container.getContainerData().getWriteBytes();
     Assert.assertEquals(datalen * 2, bytesWrite);
   }
 
@@ -549,13 +527,9 @@ public class TestContainerPersistence {
     final int chunkCount = 1024;
 
     long testContainerID = getTestContainerID();
-    BlockID blockID = ContainerTestHelper.
-        getTestBlockID(testContainerID);
+    Container container = addContainer(containerSet, testContainerID);
 
-    ContainerData cData = new ContainerData(testContainerID, conf);
-    cData.addMetadata("VOLUME", "shire");
-    cData.addMetadata("owner)", "bilbo");
-    containerManager.createContainer(cData);
+    BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
     MessageDigest oldSha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
     for (int x = 0; x < chunkCount; x++) {
       // we are writing to the same chunk file but at different offsets.
@@ -564,12 +538,12 @@ public class TestContainerPersistence {
       byte[] data = getData(datalen);
       oldSha.update(data);
       setDataChecksum(info, data);
-      chunkManager.writeChunk(blockID, info, data, COMBINED);
+      chunkManager.writeChunk(container, blockID, info, data, COMBINED);
     }
 
     // Request to read the whole data in a single go.
     ChunkInfo largeChunk = getChunk(blockID.getLocalID(), 0, 0, datalen * chunkCount);
-    byte[] newdata = chunkManager.readChunk(blockID, largeChunk);
+    byte[] newdata = chunkManager.readChunk(container, blockID, largeChunk);
     MessageDigest newSha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
     newSha.update(newdata);
     Assert.assertEquals(Hex.encodeHexString(oldSha.digest()),
@@ -587,21 +561,17 @@ public class TestContainerPersistence {
       NoSuchAlgorithmException {
     final int datalen = 1024;
     long testContainerID = getTestContainerID();
-    BlockID blockID = ContainerTestHelper.
-        getTestBlockID(testContainerID);
+    Container container = addContainer(containerSet, testContainerID);
 
-    ContainerData cData = new ContainerData(testContainerID, conf);
-    cData.addMetadata("VOLUME", "shire");
-    cData.addMetadata("owner)", "bilbo");
-    containerManager.createContainer(cData);
+    BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
     ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
     byte[] data = getData(datalen);
     setDataChecksum(info, data);
-    chunkManager.writeChunk(blockID, info, data, COMBINED);
-    chunkManager.deleteChunk(blockID, info);
+    chunkManager.writeChunk(container, blockID, info, data, COMBINED);
+    chunkManager.deleteChunk(container, blockID, info);
     exception.expect(StorageContainerException.class);
     exception.expectMessage("Unable to find the chunk file.");
-    chunkManager.readChunk(blockID, info);
+    chunkManager.readChunk(container, blockID, info);
   }
 
   /**
@@ -613,16 +583,16 @@ public class TestContainerPersistence {
   @Test
   public void testPutKey() throws IOException, NoSuchAlgorithmException {
     long testContainerID = getTestContainerID();
-    BlockID blockID = ContainerTestHelper.
-        getTestBlockID(testContainerID);
-    Pipeline pipeline = createSingleNodePipeline();
-    ChunkInfo info = writeChunkHelper(blockID, pipeline);
+    Container container = addContainer(containerSet, testContainerID);
+
+    BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
+    ChunkInfo info = writeChunkHelper(blockID);
     KeyData keyData = new KeyData(blockID);
     List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
     chunkList.add(info.getProtoBufMessage());
     keyData.setChunks(chunkList);
-    keyManager.putKey(keyData);
-    KeyData readKeyData = keyManager.getKey(keyData);
+    keyManager.putKey(container, keyData);
+    KeyData readKeyData = keyManager.getKey(container, keyData.getBlockID());
     ChunkInfo readChunk =
         ChunkInfo.getFromProtoBuf(readKeyData.getChunks().get(0));
     Assert.assertEquals(info.getChecksum(), readChunk.getChecksum());
@@ -641,11 +611,10 @@ public class TestContainerPersistence {
     final int datalen = 1024;
     long totalSize = 0L;
     long testContainerID = getTestContainerID();
-    BlockID blockID = ContainerTestHelper.
-        getTestBlockID(testContainerID);
-    Pipeline pipeline = createSingleNodePipeline();
+    Container container = addContainer(containerSet, testContainerID);
+    BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
     List<ChunkInfo> chunkList = new LinkedList<>();
-    ChunkInfo info = writeChunkHelper(blockID, pipeline);
+    ChunkInfo info = writeChunkHelper(blockID);
     totalSize += datalen;
     chunkList.add(info);
     for (int x = 1; x < chunkCount; x++) {
@@ -653,18 +622,18 @@ public class TestContainerPersistence {
       info = getChunk(blockID.getLocalID(), x, x * datalen, datalen);
       byte[] data = getData(datalen);
       setDataChecksum(info, data);
-      chunkManager.writeChunk(blockID, info, data, COMBINED);
-      totalSize += datalen * (x + 1);
+      chunkManager.writeChunk(container, blockID, info, data, COMBINED);
+      totalSize += datalen;
       chunkList.add(info);
     }
 
-    long bytesUsed = containerManager.getBytesUsed(testContainerID);
+    long bytesUsed = container.getContainerData().getBytesUsed();
     Assert.assertEquals(totalSize, bytesUsed);
-    long writeBytes = containerManager.getWriteBytes(testContainerID);
+    long writeBytes = container.getContainerData().getWriteBytes();
     Assert.assertEquals(chunkCount * datalen, writeBytes);
-    long readCount = containerManager.getReadCount(testContainerID);
+    long readCount = container.getContainerData().getReadCount();
     Assert.assertEquals(0, readCount);
-    long writeCount = containerManager.getWriteCount(testContainerID);
+    long writeCount = container.getContainerData().getWriteCount();
     Assert.assertEquals(chunkCount, writeCount);
 
     KeyData keyData = new KeyData(blockID);
@@ -673,8 +642,8 @@ public class TestContainerPersistence {
       chunkProtoList.add(i.getProtoBufMessage());
     }
     keyData.setChunks(chunkProtoList);
-    keyManager.putKey(keyData);
-    KeyData readKeyData = keyManager.getKey(keyData);
+    keyManager.putKey(container, keyData);
+    KeyData readKeyData = keyManager.getKey(container, keyData.getBlockID());
     ChunkInfo lastChunk = chunkList.get(chunkList.size() - 1);
     ChunkInfo readChunk =
         ChunkInfo.getFromProtoBuf(readKeyData.getChunks().get(readKeyData
@@ -691,18 +660,18 @@ public class TestContainerPersistence {
   @Test
   public void testDeleteKey() throws IOException, NoSuchAlgorithmException {
     long testContainerID = getTestContainerID();
+    Container container = addContainer(containerSet, testContainerID);
     BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
-    Pipeline pipeline = createSingleNodePipeline();
-    ChunkInfo info = writeChunkHelper(blockID, pipeline);
+    ChunkInfo info = writeChunkHelper(blockID);
     KeyData keyData = new KeyData(blockID);
     List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
     chunkList.add(info.getProtoBufMessage());
     keyData.setChunks(chunkList);
-    keyManager.putKey(keyData);
-    keyManager.deleteKey(blockID);
+    keyManager.putKey(container, keyData);
+    keyManager.deleteKey(container, blockID);
     exception.expect(StorageContainerException.class);
     exception.expectMessage("Unable to find the key.");
-    keyManager.getKey(keyData);
+    keyManager.getKey(container, keyData.getBlockID());
   }
 
   /**
@@ -715,19 +684,18 @@ public class TestContainerPersistence {
   public void testDeleteKeyTwice() throws IOException,
       NoSuchAlgorithmException {
     long testContainerID = getTestContainerID();
-    BlockID blockID = ContainerTestHelper.
-        getTestBlockID(testContainerID);
-    Pipeline pipeline = createSingleNodePipeline();
-    ChunkInfo info = writeChunkHelper(blockID, pipeline);
+    Container container = addContainer(containerSet, testContainerID);
+    BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
+    ChunkInfo info = writeChunkHelper(blockID);
     KeyData keyData = new KeyData(blockID);
     List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
     chunkList.add(info.getProtoBufMessage());
     keyData.setChunks(chunkList);
-    keyManager.putKey(keyData);
-    keyManager.deleteKey(blockID);
+    keyManager.putKey(container, keyData);
+    keyManager.deleteKey(container, blockID);
     exception.expect(StorageContainerException.class);
     exception.expectMessage("Unable to find the key.");
-    keyManager.deleteKey(blockID);
+    keyManager.deleteKey(container, blockID);
   }
 
   /**
@@ -738,88 +706,86 @@ public class TestContainerPersistence {
    */
   @Test
   public void testUpdateContainer() throws IOException {
-    long testContainerID = ContainerTestHelper.
-        getTestContainerID();
-    ContainerData data = new ContainerData(testContainerID, conf);
-    data.addMetadata("VOLUME", "shire");
-    data.addMetadata("owner", "bilbo");
+    long testContainerID = ContainerTestHelper.getTestContainerID();
+    Container container = addContainer(containerSet, testContainerID);
 
-    containerManager.createContainer(data);
-
-    File orgContainerFile = containerManager.getContainerFile(data);
+    File orgContainerFile = KeyValueContainerLocationUtil.getContainerFile(
+        new File(container.getContainerData().getMetadataPath()),
+        String.valueOf(testContainerID));
     Assert.assertTrue(orgContainerFile.exists());
 
-    ContainerData newData = new ContainerData(testContainerID, conf);
-    newData.addMetadata("VOLUME", "shire_new");
-    newData.addMetadata("owner", "bilbo_new");
+    Map<String, String> newMetadata = Maps.newHashMap();
+    newMetadata.put("VOLUME", "shire_new");
+    newMetadata.put("owner", "bilbo_new");
 
-    containerManager.updateContainer(testContainerID, newData, false);
+    container.update(newMetadata, false);
 
-    Assert.assertEquals(1, containerManager.getContainerMap().size());
-    Assert.assertTrue(containerManager.getContainerMap()
+    Assert.assertEquals(1, containerSet.getContainerMap().size());
+    Assert.assertTrue(containerSet.getContainerMap()
         .containsKey(testContainerID));
 
     // Verify in-memory map
-    ContainerData actualNewData = containerManager.getContainerMap()
-        .get(testContainerID);
+    ContainerData actualNewData =
+        containerSet.getContainer(testContainerID).getContainerData();
     Assert.assertEquals("shire_new",
-        actualNewData.getAllMetadata().get("VOLUME"));
+        actualNewData.getMetadata().get("VOLUME"));
     Assert.assertEquals("bilbo_new",
-        actualNewData.getAllMetadata().get("owner"));
+        actualNewData.getMetadata().get("owner"));
 
     // Verify container data on disk
-    File newContainerFile = containerManager.getContainerFile(actualNewData);
+    File newContainerFile = KeyValueContainerLocationUtil.getContainerFile(
+        new File(actualNewData.getMetadataPath()),
+        String.valueOf(testContainerID));
     Assert.assertTrue("Container file should exist.",
         newContainerFile.exists());
     Assert.assertEquals("Container file should be in same location.",
         orgContainerFile.getAbsolutePath(),
         newContainerFile.getAbsolutePath());
 
-    try (FileInputStream newIn = new FileInputStream(newContainerFile)) {
-      ContainerProtos.ContainerData actualContainerDataProto =
-          ContainerProtos.ContainerData.parseDelimitedFrom(newIn);
-      ContainerData actualContainerData = ContainerData
-          .getFromProtBuf(actualContainerDataProto, conf);
-      Assert.assertEquals("shire_new",
-          actualContainerData.getAllMetadata().get("VOLUME"));
-      Assert.assertEquals("bilbo_new",
-          actualContainerData.getAllMetadata().get("owner"));
-    }
+    ContainerData actualContainerData =  ContainerDataYaml.readContainerFile(
+        newContainerFile);
+    Assert.assertEquals("shire_new",
+        actualContainerData.getMetadata().get("VOLUME"));
+    Assert.assertEquals("bilbo_new",
+        actualContainerData.getMetadata().get("owner"));
+
 
     // Test force update flag.
-    // Delete container file then try to update without force update flag.
-    FileUtil.fullyDelete(newContainerFile);
+    // Close the container and then try to update without force update flag.
+    container.close();
     try {
-      containerManager.updateContainer(testContainerID, newData, false);
+      container.update(newMetadata, false);
     } catch (StorageContainerException ex) {
-      Assert.assertEquals("Container file not exists or "
-          + "corrupted. ID: " + testContainerID, ex.getMessage());
+      Assert.assertEquals("Updating a closed container without force option " +
+          "is not allowed. ContainerID: " + testContainerID, ex.getMessage());
     }
 
     // Update with force flag, it should be success.
-    newData = new ContainerData(testContainerID, conf);
-    newData.addMetadata("VOLUME", "shire_new_1");
-    newData.addMetadata("owner", "bilbo_new_1");
-    containerManager.updateContainer(testContainerID, newData, true);
+    newMetadata.put("VOLUME", "shire_new_1");
+    newMetadata.put("owner", "bilbo_new_1");
+    container.update(newMetadata, true);
 
     // Verify in-memory map
-    actualNewData = containerManager.getContainerMap()
-        .get(testContainerID);
+    actualNewData =
+        containerSet.getContainer(testContainerID).getContainerData();
     Assert.assertEquals("shire_new_1",
-        actualNewData.getAllMetadata().get("VOLUME"));
+        actualNewData.getMetadata().get("VOLUME"));
     Assert.assertEquals("bilbo_new_1",
-        actualNewData.getAllMetadata().get("owner"));
+        actualNewData.getMetadata().get("owner"));
 
     // Update a non-existing container
     exception.expect(StorageContainerException.class);
-    exception.expectMessage("Container doesn't exist.");
-    containerManager.updateContainer(RandomUtils.nextLong(),
-        newData, false);
+    exception.expectMessage("Container is an Inconsistent state, missing " +
+        "required files(.container, .chksm).");
+    Container nonExistentContainer = new KeyValueContainer(
+        new KeyValueContainerData(RandomUtils.nextLong(),
+            ContainerTestHelper.CONTAINER_MAX_SIZE_GB), conf);
+    nonExistentContainer.update(newMetadata, false);
   }
 
-  private KeyData writeKeyHelper(Pipeline pipeline, BlockID blockID)
+  private KeyData writeKeyHelper(BlockID blockID)
       throws IOException, NoSuchAlgorithmException {
-    ChunkInfo info = writeChunkHelper(blockID, pipeline);
+    ChunkInfo info = writeChunkHelper(blockID);
     KeyData keyData = new KeyData(blockID);
     List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
     chunkList.add(info.getProtoBufMessage());
@@ -829,20 +795,18 @@ public class TestContainerPersistence {
 
   @Test
   public void testListKey() throws Exception {
-
     long testContainerID = getTestContainerID();
-    Pipeline pipeline = createSingleNodePipeline();
+    Container container = addContainer(containerSet, testContainerID);
     List<BlockID> expectedKeys = new ArrayList<>();
     for (int i = 0; i < 10; i++) {
-      BlockID blockID = new BlockID(
-          testContainerID, i);
+      BlockID blockID = new BlockID(testContainerID, i);
       expectedKeys.add(blockID);
-      KeyData kd = writeKeyHelper(pipeline, blockID);
-      keyManager.putKey(kd);
+      KeyData kd = writeKeyHelper(blockID);
+      keyManager.putKey(container, kd);
     }
 
     // List all keys
-    List<KeyData> result = keyManager.listKey(testContainerID, 0, 100);
+    List<KeyData> result = keyManager.listKey(container, 0, 100);
     Assert.assertEquals(10, result.size());
 
     int index = 0;
@@ -855,7 +819,7 @@ public class TestContainerPersistence {
 
     // List key with startKey filter
     long k6 = expectedKeys.get(6).getLocalID();
-    result = keyManager.listKey(testContainerID, k6, 100);
+    result = keyManager.listKey(container, k6, 100);
 
     Assert.assertEquals(4, result.size());
     for (int i = 6; i < 10; i++) {
@@ -866,6 +830,6 @@ public class TestContainerPersistence {
     // Count must be >0
     exception.expect(IllegalArgumentException.class);
     exception.expectMessage("Count must be a positive number.");
-    keyManager.listKey(testContainerID, 0, -1);
+    keyManager.listKey(container, 0, -1);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
index ef4b423..b1c2065 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
@@ -74,7 +74,8 @@ public class TestContainerMetrics {
 
       DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
       conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, path);
-      VolumeSet volumeSet = new VolumeSet(datanodeDetails, conf);
+      VolumeSet volumeSet = new VolumeSet(
+          datanodeDetails.getUuidString(), conf);
       ContainerSet containerSet = new ContainerSet();
       HddsDispatcher dispatcher = new HddsDispatcher(conf, containerSet,
           volumeSet);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
index bd9259d..3605677 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
@@ -18,7 +18,10 @@
 
 package org.apache.hadoop.ozone.container.server;
 
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
 import org.apache.hadoop.ozone.container.common.interfaces.Handler;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.ratis.shaded.io.netty.channel.embedded.EmbeddedChannel;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
@@ -32,9 +35,7 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.RatisTestHelper;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
-import org.apache.hadoop.ozone.container.common.impl.Dispatcher;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
 import org.apache.hadoop.ozone.container.common.transport.server.XceiverServer;
 import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerHandler;
 import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi;
@@ -204,7 +205,6 @@ public class TestContainerServer {
   public void testClientServerWithContainerDispatcher() throws Exception {
     XceiverServer server = null;
     XceiverClient client = null;
-    String containerName = OzoneUtils.getRequestID();
 
     try {
       Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline();
@@ -213,8 +213,8 @@ public class TestContainerServer {
           pipeline.getLeader()
               .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
 
-      Dispatcher dispatcher =
-              new Dispatcher(mock(ContainerManager.class), conf);
+      HddsDispatcher dispatcher = new HddsDispatcher(
+          conf, mock(ContainerSet.class), mock(VolumeSet.class));
       dispatcher.init();
       DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
       server = new XceiverServer(datanodeDetails, conf, dispatcher);
@@ -229,10 +229,6 @@ public class TestContainerServer {
       ContainerCommandResponseProto response = client.sendCommand(request);
       Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
       Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
-      Assert.assertTrue(dispatcher.
-                          getContainerMetrics().
-                            getContainerOpsMetrics(
-                              ContainerProtos.Type.CreateContainer)== 1);
     } finally {
       if (client != null) {
         client.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java
index 732221a..12d444a 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java
@@ -24,8 +24,6 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
-import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.cli.ResultCode;
 import org.apache.hadoop.hdds.scm.cli.SCMCLI;
@@ -35,9 +33,14 @@ import org.apache.hadoop.hdds.scm.client.ScmClient;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
+
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
@@ -56,9 +59,11 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
+
 /**
  * This class tests the CLI of SCM.
  */
+@Ignore ("Needs to be fixed for new SCM and Storage design")
 public class TestSCMCli {
   private static SCMCLI cli;
 
@@ -161,11 +166,12 @@ public class TestSCMCli {
         .createContainer(xceiverClientManager.getType(),
             HddsProtos.ReplicationFactor.ONE, containerOwner);
 
-    ContainerData cdata = ContainerData
-        .getFromProtBuf(containerOperationClient.readContainer(
-            container.getContainerID(), container.getPipeline()), conf);
-    KeyUtils.getDB(cdata, conf).put(Longs.toByteArray(container.getContainerID()),
-        "someKey".getBytes());
+    KeyValueContainerData kvData = KeyValueContainerData
+        .getFromProtoBuf(containerOperationClient.readContainer(
+            container.getContainerID(), container.getPipeline()));
+    KeyUtils.getDB(kvData, conf)
+        .put(Longs.toByteArray(container.getContainerID()),
+            "someKey".getBytes());
     Assert.assertTrue(containerExist(container.getContainerID()));
 
     // Gracefully delete a container should fail because it is open.
@@ -272,10 +278,10 @@ public class TestSCMCli {
     ContainerInfo container = containerOperationClient
         .createContainer(xceiverClientManager.getType(),
             HddsProtos.ReplicationFactor.ONE, containerOwner);
-    ContainerData data = ContainerData
-        .getFromProtBuf(containerOperationClient.
+    KeyValueContainerData data = KeyValueContainerData
+        .getFromProtoBuf(containerOperationClient.
             readContainer(container.getContainerID(),
-                container.getPipeline()), conf);
+                container.getPipeline()));
 
     info = new String[] { "-container", "-info", "-c",
         Long.toString(container.getContainerID()) };
@@ -287,7 +293,7 @@ public class TestSCMCli {
     String openStatus = data.isOpen() ? "OPEN" : "CLOSED";
     String expected =
         String.format(formatStr, container.getContainerID(), openStatus,
-        data.getDBPath(), data.getContainerPath(), "",
+        data.getDbFile().getPath(), data.getContainerPath(), "",
         datanodeDetails.getHostName(), datanodeDetails.getHostName());
     assertEquals(expected, out.toString());
 
@@ -297,9 +303,9 @@ public class TestSCMCli {
     container = containerOperationClient
         .createContainer(xceiverClientManager.getType(),
             HddsProtos.ReplicationFactor.ONE, containerOwner);
-    data = ContainerData
-        .getFromProtBuf(containerOperationClient.readContainer(
-            container.getContainerID(), container.getPipeline()), conf);
+    data = KeyValueContainerData
+        .getFromProtoBuf(containerOperationClient.readContainer(
+            container.getContainerID(), container.getPipeline()));
     KeyUtils.getDB(data, conf)
         .put(containerID.getBytes(), "someKey".getBytes());
 
@@ -310,7 +316,7 @@ public class TestSCMCli {
 
     openStatus = data.isOpen() ? "OPEN" : "CLOSED";
     expected = String.format(formatStr, container.getContainerID(), openStatus,
-        data.getDBPath(), data.getContainerPath(), "",
+        data.getDbFile().getPath(), data.getContainerPath(), "",
         datanodeDetails.getHostName(), datanodeDetails.getHostName());
     assertEquals(expected, out.toString());
 
@@ -325,14 +331,14 @@ public class TestSCMCli {
         Long.toString(container.getContainerID()) };
     exitCode = runCommandAndGetOutput(info, out, null);
     assertEquals(ResultCode.SUCCESS, exitCode);
-    data = ContainerData
-        .getFromProtBuf(containerOperationClient.readContainer(
-            container.getContainerID(), container.getPipeline()), conf);
+    data = KeyValueContainerData
+        .getFromProtoBuf(containerOperationClient.readContainer(
+            container.getContainerID(), container.getPipeline()));
 
     openStatus = data.isOpen() ? "OPEN" : "CLOSED";
     expected = String
         .format(formatStr, container.getContainerID(), openStatus,
-            data.getDBPath(), data.getContainerPath(), "",
+            data.getDbFile().getPath(), data.getContainerPath(), "",
             datanodeDetails.getHostName(), datanodeDetails.getHostName());
     assertEquals(expected, out.toString());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c2351e8/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
index 1e73165..331e3ed 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkDatanodeDispatcher.java
@@ -18,6 +18,9 @@
 package org.apache.hadoop.ozone.genesis;
 
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.ratis.shaded.com.google.protobuf.ByteString;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.io.FileUtils;
@@ -27,11 +30,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
-import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
-import org.apache.hadoop.ozone.container.common.impl.Dispatcher;
-import org.apache.hadoop.ozone.container.common.impl.KeyManagerImpl;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
 
 import org.apache.hadoop.util.Time;
 import org.openjdk.jmh.annotations.Benchmark;
@@ -65,8 +63,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .PutKeyRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .GetKeyRequestProto;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .ContainerData;
 
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
@@ -77,8 +73,8 @@ public class BenchMarkDatanodeDispatcher {
 
   private String baseDir;
   private String datanodeUuid;
-  private Dispatcher dispatcher;
   private Pipeline pipeline;
+  private HddsDispatcher dispatcher;
   private ByteString data;
   private Random random;
   private AtomicInteger containerCount;
@@ -104,7 +100,6 @@ public class BenchMarkDatanodeDispatcher {
     data = ByteString.copyFromUtf8(RandomStringUtils.randomAscii(1048576));
     random  = new Random();
     Configuration conf = new OzoneConfiguration();
-    ContainerManager manager = new ContainerManagerImpl();
     baseDir = System.getProperty("java.io.tmpdir") + File.separator +
         datanodeUuid;
 
@@ -113,15 +108,12 @@ public class BenchMarkDatanodeDispatcher {
 
     // metadata directory
     StorageLocation metadataDir = StorageLocation.parse(
-        baseDir+ File.separator + CONTAINER_ROOT_PREFIX);
-    List<StorageLocation> locations = Arrays.asList(metadataDir);
+        baseDir + File.separator + CONTAINER_ROOT_PREFIX);
 
-    manager
-        .init(conf, locations, GenesisUtil.createDatanodeDetails(datanodeUuid));
-    manager.setChunkManager(new ChunkManagerImpl(manager));
-    manager.setKeyManager(new KeyManagerImpl(manager, conf));
+    ContainerSet containerSet = new ContainerSet();
+    VolumeSet volumeSet = new VolumeSet(datanodeUuid, conf);
 
-    dispatcher = new Dispatcher(manager, conf);
+    dispatcher = new HddsDispatcher(conf, containerSet, volumeSet);
     dispatcher.init();
 
     containerCount = new AtomicInteger();


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


[07/37] hadoop git commit: HDDS-140. Add DU usage to VolumeInfo.

Posted by bh...@apache.org.
HDDS-140. Add DU usage to VolumeInfo.


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

Branch: refs/heads/trunk
Commit: 772c95395b59dc9db61fa8f3721a6c81f97a1f4d
Parents: 977c8cd
Author: Hanisha Koneru <ha...@apache.org>
Authored: Wed Jun 6 15:36:36 2018 -0700
Committer: Hanisha Koneru <ha...@apache.org>
Committed: Wed Jun 6 16:21:21 2018 -0700

----------------------------------------------------------------------
 .../impl/RoundRobinVolumeChoosingPolicy.java    |   2 +-
 .../ozone/container/common/impl/VolumeInfo.java |  59 ++++--
 .../ozone/container/common/impl/VolumeSet.java  | 109 +++++------
 .../container/common/impl/VolumeUsage.java      | 189 +++++++++++++++++++
 .../common/interfaces/TestVolumeSet.java        |  33 ++--
 .../fsdataset/VolumeChoosingPolicy.java         |   2 +-
 6 files changed, 316 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/772c9539/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RoundRobinVolumeChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RoundRobinVolumeChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RoundRobinVolumeChoosingPolicy.java
index 0a20bf2..55b3049 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RoundRobinVolumeChoosingPolicy.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RoundRobinVolumeChoosingPolicy.java
@@ -28,7 +28,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 /**
  * Choose volumes in round-robin order.
- * Use fine-grained locks to synchronize volume choosing.
+ * The caller should synchronize access to the list of volumes.
  */
 public class RoundRobinVolumeChoosingPolicy implements VolumeChoosingPolicy {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/772c9539/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeInfo.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeInfo.java
index 1b5a7aa..3e8dda6 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeInfo.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeInfo.java
@@ -18,13 +18,14 @@
 
 package org.apache.hadoop.ozone.container.common.impl;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
 import java.io.IOException;
-import java.util.concurrent.atomic.AtomicLong;
 
 /**
  * Stores information about a disk/volume.
@@ -36,24 +37,29 @@ public class VolumeInfo {
   private final Path rootDir;
   private final StorageType storageType;
   private VolumeState state;
+
+  // Space usage calculator
+  private VolumeUsage usage;
   // Capacity configured. This is useful when we want to
   // limit the visible capacity for tests. If negative, then we just
   // query from the filesystem.
   private long configuredCapacity;
-  private volatile AtomicLong scmUsed = new AtomicLong(0);
 
   public static class Builder {
+    private final Configuration conf;
     private final Path rootDir;
     private StorageType storageType;
     private VolumeState state;
     private long configuredCapacity;
 
-    public Builder(Path rootDir) {
+    public Builder(Path rootDir, Configuration conf) {
       this.rootDir = rootDir;
+      this.conf = conf;
     }
 
-    public Builder(String rootDirStr) {
+    public Builder(String rootDirStr, Configuration conf) {
       this.rootDir = new Path(rootDirStr);
+      this.conf = conf;
     }
 
     public Builder storageType(StorageType storageType) {
@@ -76,9 +82,17 @@ public class VolumeInfo {
     }
   }
 
-  private VolumeInfo(Builder b) {
+  private VolumeInfo(Builder b) throws IOException {
 
     this.rootDir = b.rootDir;
+    File root = new File(rootDir.toString());
+
+    Boolean succeeded = root.isDirectory() || root.mkdirs();
+
+    if (!succeeded) {
+      LOG.error("Unable to create the volume root dir at : {}", root);
+      throw new IOException("Unable to create the volume root dir at " + root);
+    }
 
     this.storageType = (b.storageType != null ?
         b.storageType : StorageType.DEFAULT);
@@ -88,19 +102,42 @@ public class VolumeInfo {
 
     this.state = (b.state != null ? b.state : VolumeState.NOT_FORMATTED);
 
-    LOG.info("Creating Volume : " + rootDir + " of  storage type : " +
+    this.usage = new VolumeUsage(root, b.conf);
+
+    LOG.info("Creating Volume : " + rootDir + " of storage type : " +
         storageType + " and capacity : " + configuredCapacity);
   }
 
-  public void addSpaceUsed(long spaceUsed) {
-    this.scmUsed.getAndAdd(spaceUsed);
+  public long getCapacity() {
+    return configuredCapacity < 0 ? usage.getCapacity() : configuredCapacity;
+  }
+
+  public long getAvailable() throws IOException {
+    return usage.getAvailable();
   }
 
-  public long getAvailable() {
-    return configuredCapacity - scmUsed.get();
+  public long getScmUsed() throws IOException {
+    return usage.getScmUsed();
+  }
+
+  void shutdown() {
+    this.state = VolumeState.NON_EXISTENT;
+    shutdownUsageThread();
+  }
+
+  void failVolume() {
+    setState(VolumeState.FAILED);
+    shutdownUsageThread();
+  }
+
+  private void shutdownUsageThread() {
+    if (usage != null) {
+      usage.shutdown();
+    }
+    usage = null;
   }
 
-  public void setState(VolumeState state) {
+  void setState(VolumeState state) {
     this.state = state;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/772c9539/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeSet.java
index 27fd657..c55c84a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeSet.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeSet.java
@@ -40,7 +40,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.EnumMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -55,32 +54,28 @@ public class VolumeSet {
   private static final Logger LOG = LoggerFactory.getLogger(VolumeSet.class);
 
   private Configuration conf;
+
   /**
-   * {@link VolumeSet#volumeList} maintains a list of active volumes in the
+   * {@link VolumeSet#volumeMap} maintains a map of all active volumes in the
    * DataNode. Each volume has one-to-one mapping with a volumeInfo object.
    */
-  private List<VolumeInfo> volumeList;
+  private Map<Path, VolumeInfo> volumeMap;
   /**
-   * {@link VolumeSet#failedVolumeList} maintains a list of volumes which have
-   * failed. This list is mutually exclusive to {@link VolumeSet#volumeList}.
+   * {@link VolumeSet#failedVolumeMap} maintains a map of volumes which have
+   * failed. The keys in this map and {@link VolumeSet#volumeMap} are
+   * mutually exclusive.
    */
-  private List<VolumeInfo> failedVolumeList;
+  private Map<Path, VolumeInfo> failedVolumeMap;
   /**
-   * {@link VolumeSet#volumeMap} maintains a map of all volumes in the
-   * DataNode irrespective of their state.
-   */
-  private Map<Path, VolumeInfo> volumeMap;
-  /**
-   * {@link VolumeSet#volumeStateMap} maintains a list of volumes per
+   * {@link VolumeSet#volumeStateMap} maintains a list of active volumes per
    * StorageType.
    */
   private EnumMap<StorageType, List<VolumeInfo>> volumeStateMap;
 
   /**
    * Lock to synchronize changes to the VolumeSet. Any update to
-   * {@link VolumeSet#volumeList}, {@link VolumeSet#failedVolumeList},
-   * {@link VolumeSet#volumeMap} or {@link VolumeSet#volumeStateMap} should
-   * be done after acquiring this lock.
+   * {@link VolumeSet#volumeMap}, {@link VolumeSet#failedVolumeMap}, or
+   * {@link VolumeSet#volumeStateMap} should be done after acquiring this lock.
    */
   private final AutoCloseableLock volumeSetLock;
 
@@ -100,9 +95,8 @@ public class VolumeSet {
 
   // Add DN volumes configured through ConfigKeys to volumeMap.
   private void initializeVolumeSet() throws DiskOutOfSpaceException {
-    volumeList = new ArrayList<>();
-    failedVolumeList = new ArrayList<>();
     volumeMap = new ConcurrentHashMap<>();
+    failedVolumeMap = new ConcurrentHashMap<>();
     volumeStateMap = new EnumMap<>(StorageType.class);
 
     Collection<String> datanodeDirs = conf.getTrimmedStringCollection(
@@ -123,7 +117,6 @@ public class VolumeSet {
       try {
         VolumeInfo volumeInfo = getVolumeInfo(dir);
 
-        volumeList.add(volumeInfo);
         volumeMap.put(volumeInfo.getRootDir(), volumeInfo);
         volumeStateMap.get(volumeInfo.getStorageType()).add(volumeInfo);
       } catch (IOException e) {
@@ -131,7 +124,7 @@ public class VolumeSet {
       }
     }
 
-    if (volumeList.size() == 0) {
+    if (volumeMap.size() == 0) {
       throw new DiskOutOfSpaceException("No storage location configured");
     }
   }
@@ -148,7 +141,7 @@ public class VolumeSet {
     StorageLocation location = StorageLocation.parse(rootDir);
     StorageType storageType = location.getStorageType();
 
-    VolumeInfo.Builder volumeBuilder = new VolumeInfo.Builder(rootDir);
+    VolumeInfo.Builder volumeBuilder = new VolumeInfo.Builder(rootDir, conf);
     volumeBuilder.storageType(storageType);
     return volumeBuilder.build();
   }
@@ -159,21 +152,17 @@ public class VolumeSet {
 
     try (AutoCloseableLock lock = volumeSetLock.acquire()) {
       if (volumeMap.containsKey(dirPath)) {
-        VolumeInfo volumeInfo = volumeMap.get(dirPath);
-        if (volumeInfo.isFailed()) {
-          volumeInfo.setState(VolumeState.NORMAL);
-          failedVolumeList.remove(volumeInfo);
-          volumeList.add(volumeInfo);
-        } else {
-          LOG.warn("Volume : " + volumeInfo.getRootDir() + " already " +
-              "exists in VolumeMap");
-        }
+        LOG.warn("Volume : {} already exists in VolumeMap", dataDir);
       } else {
-        VolumeInfo volumeInfo = getVolumeInfo(dataDir);
+        if (failedVolumeMap.containsKey(dirPath)) {
+          failedVolumeMap.remove(dirPath);
+        }
 
-        volumeList.add(volumeInfo);
-        volumeMap.put(volumeInfo.getRootDir(), volumeInfo);
+        VolumeInfo volumeInfo = getVolumeInfo(dirPath.toString());
+        volumeMap.put(dirPath, volumeInfo);
         volumeStateMap.get(volumeInfo.getStorageType()).add(volumeInfo);
+
+        LOG.debug("Added Volume : {} to VolumeSet", dataDir);
       }
     }
   }
@@ -185,13 +174,17 @@ public class VolumeSet {
     try (AutoCloseableLock lock = volumeSetLock.acquire()) {
       if (volumeMap.containsKey(dirPath)) {
         VolumeInfo volumeInfo = volumeMap.get(dirPath);
-        if (!volumeInfo.isFailed()) {
-          volumeInfo.setState(VolumeState.FAILED);
-          volumeList.remove(volumeInfo);
-          failedVolumeList.add(volumeInfo);
-        }
+        volumeInfo.failVolume();
+
+        volumeMap.remove(dirPath);
+        volumeStateMap.get(volumeInfo.getStorageType()).remove(volumeInfo);
+        failedVolumeMap.put(dirPath, volumeInfo);
+
+        LOG.debug("Moving Volume : {} to failed Volumes", dataDir);
+      } else if (failedVolumeMap.containsKey(dirPath)) {
+        LOG.debug("Volume : {} is not active", dataDir);
       } else {
-        LOG.warn("Volume : " + dataDir + " does not exist in VolumeMap");
+        LOG.warn("Volume : {} does not exist in VolumeSet", dataDir);
       }
     }
   }
@@ -203,39 +196,47 @@ public class VolumeSet {
     try (AutoCloseableLock lock = volumeSetLock.acquire()) {
       if (volumeMap.containsKey(dirPath)) {
         VolumeInfo volumeInfo = volumeMap.get(dirPath);
-        if (!volumeInfo.isFailed()) {
-          volumeList.remove(volumeInfo);
-        } else {
-          failedVolumeList.remove(volumeInfo);
-        }
+        volumeInfo.shutdown();
+
         volumeMap.remove(dirPath);
         volumeStateMap.get(volumeInfo.getStorageType()).remove(volumeInfo);
+
+        LOG.debug("Removed Volume : {} from VolumeSet", dataDir);
+      } else if (failedVolumeMap.containsKey(dirPath)) {
+        VolumeInfo volumeInfo = failedVolumeMap.get(dirPath);
+        volumeInfo.setState(VolumeState.NON_EXISTENT);
+
+        failedVolumeMap.remove(dirPath);
+        LOG.debug("Removed Volume : {} from failed VolumeSet", dataDir);
       } else {
-        LOG.warn("Volume: " + dataDir + " does not exist in " + "volumeMap.");
+        LOG.warn("Volume : {} does not exist in VolumeSet", dataDir);
       }
     }
   }
 
-  /**
-   * Return an iterator over {@link VolumeSet#volumeList}.
-   */
-  public Iterator<VolumeInfo> getIterator() {
-    return volumeList.iterator();
-  }
-
   public VolumeInfo chooseVolume(long containerSize,
       VolumeChoosingPolicy choosingPolicy) throws IOException {
-    return choosingPolicy.chooseVolume(volumeList, containerSize);
+    return choosingPolicy.chooseVolume(getVolumesList(), containerSize);
+  }
+
+  public void shutdown() {
+    for (VolumeInfo volumeInfo : volumeMap.values()) {
+      try {
+        volumeInfo.shutdown();
+      } catch (Exception e) {
+        LOG.error("Failed to shutdown volume : " + volumeInfo.getRootDir(), e);
+      }
+    }
   }
 
   @VisibleForTesting
   public List<VolumeInfo> getVolumesList() {
-    return ImmutableList.copyOf(volumeList);
+    return ImmutableList.copyOf(volumeMap.values());
   }
 
   @VisibleForTesting
   public List<VolumeInfo> getFailedVolumesList() {
-    return ImmutableList.copyOf(failedVolumeList);
+    return ImmutableList.copyOf(failedVolumeMap.values());
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/772c9539/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeUsage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeUsage.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeUsage.java
new file mode 100644
index 0000000..bcd78ba
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/VolumeUsage.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.impl;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CachingGetSpaceUsed;
+import org.apache.hadoop.fs.DF;
+import org.apache.hadoop.fs.GetSpaceUsed;
+import org.apache.hadoop.io.IOUtils;
+import static org.apache.hadoop.util.RunJar.SHUTDOWN_HOOK_PRIORITY;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Scanner;
+
+/**
+ * Class that wraps the space df of the Datanode Volumes used by SCM
+ * containers.
+ */
+public class VolumeUsage {
+  private static final Logger LOG = LoggerFactory.getLogger(VolumeUsage.class);
+
+  private final File rootDir;
+  private final DF df;
+  private final File scmUsedFile;
+  private GetSpaceUsed scmUsage;
+  private Runnable shutdownHook;
+
+  private static final String DU_CACHE_FILE = "scmUsed";
+  private volatile boolean scmUsedSaved = false;
+
+  VolumeUsage(File dataLoc, Configuration conf)
+      throws IOException {
+    this.rootDir = dataLoc;
+
+    // SCM used cache file
+    scmUsedFile = new File(rootDir, DU_CACHE_FILE);
+    // get overall disk df
+    this.df = new DF(rootDir, conf);
+
+    startScmUsageThread(conf);
+  }
+
+  void startScmUsageThread(Configuration conf) throws IOException {
+    // get SCM specific df
+    this.scmUsage = new CachingGetSpaceUsed.Builder().setPath(rootDir)
+        .setConf(conf)
+        .setInitialUsed(loadScmUsed())
+        .build();
+
+    // Ensure scm df is saved during shutdown.
+    shutdownHook = () -> {
+      if (!scmUsedSaved) {
+        saveScmUsed();
+      }
+    };
+    ShutdownHookManager.get().addShutdownHook(shutdownHook,
+        SHUTDOWN_HOOK_PRIORITY);
+  }
+
+  long getCapacity() {
+    long capacity = df.getCapacity();
+    return (capacity > 0) ? capacity : 0;
+  }
+
+  /*
+   * Calculate the available space in the volume.
+   */
+  long getAvailable() throws IOException {
+    long remaining = getCapacity() - getScmUsed();
+    long available = df.getAvailable();
+    if (remaining > available) {
+      remaining = available;
+    }
+    return (remaining > 0) ? remaining : 0;
+  }
+
+  long getScmUsed() throws IOException{
+    return scmUsage.getUsed();
+  }
+
+  public void shutdown() {
+    saveScmUsed();
+    scmUsedSaved = true;
+
+    if (shutdownHook != null) {
+      ShutdownHookManager.get().removeShutdownHook(shutdownHook);
+    }
+
+    if (scmUsage instanceof CachingGetSpaceUsed) {
+      IOUtils.cleanupWithLogger(null, ((CachingGetSpaceUsed) scmUsage));
+    }
+  }
+
+  /**
+   * Read in the cached DU value and return it if it is less than 600 seconds
+   * old (DU update interval). Slight imprecision of scmUsed is not critical
+   * and skipping DU can significantly shorten the startup time.
+   * If the cached value is not available or too old, -1 is returned.
+   */
+  long loadScmUsed() {
+    long cachedScmUsed;
+    long mtime;
+    Scanner sc;
+
+    try {
+      sc = new Scanner(scmUsedFile, "UTF-8");
+    } catch (FileNotFoundException fnfe) {
+      return -1;
+    }
+
+    try {
+      // Get the recorded scmUsed from the file.
+      if (sc.hasNextLong()) {
+        cachedScmUsed = sc.nextLong();
+      } else {
+        return -1;
+      }
+      // Get the recorded mtime from the file.
+      if (sc.hasNextLong()) {
+        mtime = sc.nextLong();
+      } else {
+        return -1;
+      }
+
+      // Return the cached value if mtime is okay.
+      if (mtime > 0 && (Time.now() - mtime < 600000L)) {
+        LOG.info("Cached ScmUsed found for {} : {} ", rootDir,
+            cachedScmUsed);
+        return cachedScmUsed;
+      }
+      return -1;
+    } finally {
+      sc.close();
+    }
+  }
+
+  /**
+   * Write the current scmUsed to the cache file.
+   */
+  void saveScmUsed() {
+    if (scmUsedFile.exists() && !scmUsedFile.delete()) {
+      LOG.warn("Failed to delete old scmUsed file in {}.", rootDir);
+    }
+    OutputStreamWriter out = null;
+    try {
+      long used = getScmUsed();
+      if (used > 0) {
+        out = new OutputStreamWriter(new FileOutputStream(scmUsedFile),
+            StandardCharsets.UTF_8);
+        // mtime is written last, so that truncated writes won't be valid.
+        out.write(Long.toString(used) + " " + Long.toString(Time.now()));
+        out.flush();
+        out.close();
+        out = null;
+      }
+    } catch (IOException ioe) {
+      // If write failed, the volume might be bad. Since the cache file is
+      // not critical, log the error and continue.
+      LOG.warn("Failed to write scmUsed to " + scmUsedFile, ioe);
+    } finally {
+      IOUtils.cleanupWithLogger(null, out);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/772c9539/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestVolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestVolumeSet.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestVolumeSet.java
index 5a1bc79..ceeacff 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestVolumeSet.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestVolumeSet.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.ozone.container.common.impl.VolumeInfo;
 import org.apache.hadoop.ozone.container.common.impl.VolumeSet;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import org.junit.Before;
 import org.junit.Rule;
@@ -73,8 +74,10 @@ public class TestVolumeSet {
     // VolumeSet initialization should add volume1 and volume2 to VolumeSet
     assertEquals("VolumeSet intialization is incorrect",
         volumesList.size(), volumes.size());
-    assertEquals(volume1, volumesList.get(0).getRootDir().toString());
-    assertEquals(volume2, volumesList.get(1).getRootDir().toString());
+    assertTrue("VolumeSet not initailized correctly",
+        checkVolumeExistsInVolumeSet(volume1));
+    assertTrue("VolumeSet not initailized correctly",
+        checkVolumeExistsInVolumeSet(volume2));
   }
 
   @Test
@@ -88,9 +91,8 @@ public class TestVolumeSet {
     volumeSet.addVolume(volume3);
 
     assertEquals(3, volumeSet.getVolumesList().size());
-    assertEquals("AddVolume did not add requested volume to VolumeSet",
-        volume3,
-        volumeSet.getVolumesList().get(2).getRootDir().toString());
+    assertTrue("AddVolume did not add requested volume to VolumeSet",
+        checkVolumeExistsInVolumeSet(volume3));
   }
 
   @Test
@@ -103,15 +105,15 @@ public class TestVolumeSet {
     assertEquals(1, volumeSet.getVolumesList().size());
 
     // Failed volume should be added to FailedVolumeList
-    assertEquals("Failed volume not present in FailedVolumeList",
+    assertEquals("Failed volume not present in FailedVolumeMap",
         1, volumeSet.getFailedVolumesList().size());
     assertEquals("Failed Volume list did not match", volume1,
         volumeSet.getFailedVolumesList().get(0).getRootDir().toString());
+    assertTrue(volumeSet.getFailedVolumesList().get(0).isFailed());
 
-    // Failed volume should exist in VolumeMap with isFailed flag set to true
+    // Failed volume should not exist in VolumeMap
     Path volume1Path = new Path(volume1);
-    assertTrue(volumeSet.getVolumeMap().containsKey(volume1Path));
-    assertTrue(volumeSet.getVolumeMap().get(volume1Path).isFailed());
+    assertFalse(volumeSet.getVolumeMap().containsKey(volume1Path));
   }
 
   @Test
@@ -130,9 +132,18 @@ public class TestVolumeSet {
         LogFactory.getLog(VolumeSet.class));
     volumeSet.removeVolume(volume1);
     assertEquals(1, volumeSet.getVolumesList().size());
-    String expectedLogMessage = "Volume: " + volume1 + " does not exist in "
-        + "volumeMap.";
+    String expectedLogMessage = "Volume : " + volume1 + " does not exist in "
+        + "VolumeSet";
     assertTrue("Log output does not contain expected log message: "
         + expectedLogMessage, logs.getOutput().contains(expectedLogMessage));
   }
+
+  private boolean checkVolumeExistsInVolumeSet(String volume) {
+    for (VolumeInfo volumeInfo : volumeSet.getVolumesList()) {
+      if (volumeInfo.getRootDir().toString().equals(volume)) {
+        return true;
+      }
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/772c9539/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/VolumeChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/VolumeChoosingPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/VolumeChoosingPolicy.java
index 8cbc058..e9fa37b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/VolumeChoosingPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/VolumeChoosingPolicy.java
@@ -32,7 +32,7 @@ public interface VolumeChoosingPolicy<V extends FsVolumeSpi> {
    * Choose a volume to place a replica,
    * given a list of volumes and the replica size sought for storage.
    * 
-   * The implementations of this interface must be thread-safe.
+   * The caller should synchronize access to the list of volumes.
    * 
    * @param volumes - a list of available volumes.
    * @param replicaSize - the size of the replica for which a volume is sought.


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


[22/37] hadoop git commit: Merge branch 'trunk' into HDDS-48

Posted by bh...@apache.org.
Merge branch 'trunk' into HDDS-48


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

Branch: refs/heads/trunk
Commit: 84ac6bb1b1974ed7746c55ec79486525b1cca19b
Parents: c104525 2911943
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Thu Jun 28 14:57:23 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Thu Jun 28 14:57:23 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/ha/ActiveStandbyElector.java  |   5 +-
 .../apache/hadoop/ha/ZKFailoverController.java  |   2 +-
 .../AbstractPreemptableResourceCalculator.java  |  21 +-
 .../CapacitySchedulerPreemptionContext.java     |   2 +
 .../CapacitySchedulerPreemptionUtils.java       |  23 +-
 .../capacity/FifoCandidatesSelector.java        |  45 ++--
 .../capacity/IntraQueueCandidatesSelector.java  |   9 +-
 .../capacity/PreemptableResourceCalculator.java |   7 +-
 .../capacity/PreemptionCandidatesSelector.java  |  11 +
 .../ProportionalCapacityPreemptionPolicy.java   | 129 +++++++---
 ...QueuePriorityContainerCandidateSelector.java |  16 +-
 .../ReservedContainerCandidatesSelector.java    |  16 +-
 .../monitor/capacity/TempQueuePerPartition.java |   8 +-
 .../CapacitySchedulerConfiguration.java         |  17 ++
 .../resourcemanager/TestRMEmbeddedElector.java  |  22 ++
 .../TestPreemptionForQueueWithPriorities.java   |  58 +++++
 ...apacityPreemptionPolicyPreemptToBalance.java | 254 +++++++++++++++++++
 ...TestCapacitySchedulerSurgicalPreemption.java | 111 ++++++++
 18 files changed, 664 insertions(+), 92 deletions(-)
----------------------------------------------------------------------



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


[09/37] hadoop git commit: Revert "Create Version File in Datanode. Contributed by Bharat Viswanadham."

Posted by bh...@apache.org.
Revert "Create Version File in Datanode. Contributed by Bharat Viswanadham."

This reverts commit f26d3466d79125123cba00ab81481655d7bfe3c1.


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

Branch: refs/heads/trunk
Commit: 0e437f9b174f3b1eaf41b63ae707dd76379b8e8b
Parents: 143dd56
Author: Hanisha Koneru <ha...@apache.org>
Authored: Mon Jun 11 12:15:39 2018 -0700
Committer: Hanisha Koneru <ha...@apache.org>
Committed: Mon Jun 11 12:15:39 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/ozone/OzoneConsts.java    |   2 -
 .../org/apache/hadoop/ozone/common/Storage.java |   6 +-
 .../container/common/DataNodeLayoutVersion.java |  80 ---------
 .../common/helpers/DatanodeVersionFile.java     | 172 -------------------
 .../states/datanode/RunningDatanodeState.java   |   3 +-
 .../states/endpoint/VersionEndpointTask.java    |  71 +-------
 .../container/ozoneimpl/OzoneContainer.java     |   8 +-
 .../hadoop/ozone/protocol/VersionResponse.java  |   4 -
 .../ozone/container/common/ScmTestMock.java     |  24 ---
 .../common/TestDatanodeLayOutVersion.java       |  38 ----
 .../common/TestDatanodeStateMachine.java        |   3 +-
 .../common/helpers/TestDatanodeVersionFile.java | 120 -------------
 .../hadoop/hdds/scm/node/SCMNodeManager.java    |   2 -
 .../ozone/container/common/TestEndPoint.java    | 169 +-----------------
 14 files changed, 14 insertions(+), 688 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e437f9b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index ce1a733..451a08f 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -29,8 +29,6 @@ public final class OzoneConsts {
 
   public static final String STORAGE_DIR = "scm";
   public static final String SCM_ID = "scmUuid";
-  public static final String LAYOUTVERSION = "layOutVersion";
-  public static final String CTIME = "ctime";
 
   public static final String OZONE_SIMPLE_ROOT_USER = "root";
   public static final String OZONE_SIMPLE_HDFS_USER = "hdfs";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e437f9b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
index 35ddc71..fb30d92 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
@@ -45,10 +45,8 @@ import java.util.Properties;
 public abstract class Storage {
   private static final Logger LOG = LoggerFactory.getLogger(Storage.class);
 
-  public static final String STORAGE_DIR_CURRENT = "current";
-  public static final String STORAGE_FILE_VERSION = "VERSION";
-  public static final String STORAGE_DIR_HDDS = "hdds";
-
+  protected static final String STORAGE_DIR_CURRENT = "current";
+  protected static final String STORAGE_FILE_VERSION = "VERSION";
 
   private final NodeType nodeType;
   private final File root;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e437f9b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/DataNodeLayoutVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/DataNodeLayoutVersion.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/DataNodeLayoutVersion.java
deleted file mode 100644
index 2d58c39..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/DataNodeLayoutVersion.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.ozone.container.common;
-
-/**
- * Datanode layout version which describes information about the layout version
- * on the datanode.
- */
-public final class DataNodeLayoutVersion {
-
-  // We will just be normal and use positive counting numbers for versions.
-  private final static DataNodeLayoutVersion[] VERSION_INFOS =
-      {new DataNodeLayoutVersion(1, "HDDS Datanode LayOut Version 1")};
-
-  private final String description;
-  private final int version;
-
-  /**
-   * Never created outside this class.
-   *
-   * @param description -- description
-   * @param version     -- version number
-   */
-  private DataNodeLayoutVersion(int version, String description) {
-    this.description = description;
-    this.version = version;
-  }
-
-  /**
-   * Returns all versions.
-   *
-   * @return Version info array.
-   */
-  public static DataNodeLayoutVersion[] getAllVersions() {
-    return VERSION_INFOS.clone();
-  }
-
-  /**
-   * Returns the latest version.
-   *
-   * @return versionInfo
-   */
-  public static DataNodeLayoutVersion getLatestVersion() {
-    return VERSION_INFOS[VERSION_INFOS.length - 1];
-  }
-
-  /**
-   * Return description.
-   *
-   * @return String
-   */
-  public String getDescription() {
-    return description;
-  }
-
-  /**
-   * Return the version.
-   *
-   * @return int.
-   */
-  public int getVersion() {
-    return version;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e437f9b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeVersionFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeVersionFile.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeVersionFile.java
deleted file mode 100644
index 0010d7e..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeVersionFile.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.ozone.container.common.helpers;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
-import org.apache.hadoop.ozone.common.Storage;
-import org.apache.hadoop.ozone.container.common.DataNodeLayoutVersion;
-
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.util.Properties;
-
-/**
- * This is a utility class which helps to create the version file on datanode
- * and also validate the content of the version file.
- */
-public class DatanodeVersionFile {
-
-  private final String scmUuid;
-  private final long cTime;
-  private final int layOutVersion;
-
-  public DatanodeVersionFile(String scmUuid, long cTime, int layOutVersion) {
-    this.scmUuid = scmUuid;
-    this.cTime = cTime;
-    this.layOutVersion = layOutVersion;
-  }
-
-  private Properties createProperties() {
-    Properties properties = new Properties();
-    properties.setProperty(OzoneConsts.SCM_ID, scmUuid);
-    properties.setProperty(OzoneConsts.CTIME, String.valueOf(cTime));
-    properties.setProperty(OzoneConsts.LAYOUTVERSION, String.valueOf(
-        layOutVersion));
-    return properties;
-  }
-
-  /**
-   * Creates a version File in specified path.
-   * @param path
-   * @throws IOException
-   */
-  public void createVersionFile(File path) throws
-      IOException {
-    try (RandomAccessFile file = new RandomAccessFile(path, "rws");
-         FileOutputStream out = new FileOutputStream(file.getFD())) {
-      file.getChannel().truncate(0);
-      Properties properties = createProperties();
-      /*
-       * If server is interrupted before this line,
-       * the version file will remain unchanged.
-       */
-      properties.store(out, null);
-      /*
-       * Now the new fields are flushed to the head of the file, but file
-       * length can still be larger then required and therefore the file can
-       * contain whole or corrupted fields from its old contents in the end.
-       * If server is interrupted here and restarted later these extra fields
-       * either should not effect server behavior or should be handled
-       * by the server correctly.
-       */
-      file.getChannel().truncate(file.getChannel().size());
-    }
-  }
-
-
-  /**
-   * Creates a property object from the specified file content.
-   * @param  versionFile
-   * @return Properties
-   * @throws IOException
-   */
-  public static Properties readFrom(File versionFile) throws IOException {
-    try (RandomAccessFile file = new RandomAccessFile(versionFile, "rws");
-         FileInputStream in = new FileInputStream(file.getFD())) {
-      Properties props = new Properties();
-      props.load(in);
-      return props;
-    }
-  }
-
-  /**
-   * Verifies scmUuid is valid or not.
-   * @param scmIdVersionFile
-   * @param scmId
-   * @throws InconsistentStorageStateException
-   */
-  @VisibleForTesting
-  public static void verifyScmUuid(String scmIdVersionFile, String scmId) throws
-      InconsistentStorageStateException {
-    Preconditions.checkState(StringUtils.isNotBlank(scmIdVersionFile),
-        "Invalid scmUuid from Version File.");
-    Preconditions.checkState(StringUtils.isNotBlank(scmId),
-        "Invalid scmUuid from SCM version request response");
-    if(!scmIdVersionFile.equals(scmId)) {
-      throw new InconsistentStorageStateException("MisMatch of ScmUuid " +
-          "scmUuid from version File is: " + scmIdVersionFile + "SCM " +
-          "version response scmUuid is" + scmId);
-    }
-  }
-
-  /**
-   * Verifies creationTime is valid or not.
-   * @param creationTime
-   */
-  @VisibleForTesting
-  public static void verifyCreationTime(String creationTime) {
-    Preconditions.checkState(!StringUtils.isBlank(creationTime),
-        "Invalid creation Time.");
-  }
-
-  /**
-   * Verifies layOutVersion is valid or not.
-   * @param lv
-   * @throws InconsistentStorageStateException
-   */
-  @VisibleForTesting
-  public static void verifyLayOutVersion(String lv) throws
-      InconsistentStorageStateException {
-    Preconditions.checkState(!StringUtils.isBlank(lv),
-        "Invalid layOutVersion.");
-    int version = Integer.parseInt(lv);
-    if(DataNodeLayoutVersion.getLatestVersion().getVersion() != version) {
-      throw new InconsistentStorageStateException("Incorrect layOutVersion");
-    }
-  }
-
-  /**
-   * Returns the versionFile path for the StorageLocation.
-   * @param location
-   * @param scmUuid
-   * @return versionFile - File
-   */
-  @VisibleForTesting
-  public static File getVersionFile(StorageLocation location, String scmUuid) {
-    if (location != null) {
-      String path = location.getUri().getPath();
-      File parentPath = new File(path + File.separator + Storage
-          .STORAGE_DIR_HDDS + File.separator +  scmUuid + File.separator +
-          Storage.STORAGE_DIR_CURRENT + File.separator);
-      File versionFile = new File(parentPath, Storage.STORAGE_FILE_VERSION);
-      return versionFile;
-    } else {
-      return null;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e437f9b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
index 6e30ebc..3e11d12 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
@@ -95,8 +95,7 @@ public class RunningDatanodeState implements DatanodeState {
       getEndPointTask(EndpointStateMachine endpoint) {
     switch (endpoint.getState()) {
     case GETVERSION:
-      return new VersionEndpointTask(endpoint, conf, context.getParent().
-          getContainer());
+      return new VersionEndpointTask(endpoint, conf);
     case REGISTER:
       return  RegisterEndpointTask.newBuilder()
           .setConfig(conf)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e437f9b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
index e03a438..b048ee5 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
@@ -16,30 +16,14 @@
  */
 package org.apache.hadoop.ozone.container.common.states.endpoint;
 
-import com.google.common.base.Preconditions;
-import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
-import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
-import org.apache.hadoop.ozone.common.Storage;
-import org.apache.hadoop.ozone.container.common.DataNodeLayoutVersion;
-import org.apache.hadoop.ozone.container.common.helpers.DatanodeVersionFile;
 import org.apache.hadoop.ozone.container.common.statemachine
     .EndpointStateMachine;
-import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.ozone.protocol.VersionResponse;
-import org.apache.hadoop.util.Time;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-
-import java.io.File;
 import java.io.IOException;
-import java.util.List;
-import java.util.Properties;
 import java.util.concurrent.Callable;
 
 /**
@@ -49,15 +33,11 @@ public class VersionEndpointTask implements
     Callable<EndpointStateMachine.EndPointStates> {
   private final EndpointStateMachine rpcEndPoint;
   private final Configuration configuration;
-  private final OzoneContainer datanodeContainerManager;
-  static final Logger LOG =
-      LoggerFactory.getLogger(VersionEndpointTask.class);
 
   public VersionEndpointTask(EndpointStateMachine rpcEndPoint,
-                             Configuration conf, OzoneContainer container) {
+      Configuration conf) {
     this.rpcEndPoint = rpcEndPoint;
     this.configuration = conf;
-    this.datanodeContainerManager = container;
   }
 
   /**
@@ -69,56 +49,15 @@ public class VersionEndpointTask implements
   @Override
   public EndpointStateMachine.EndPointStates call() throws Exception {
     rpcEndPoint.lock();
-    try {
+    try{
       SCMVersionResponseProto versionResponse =
           rpcEndPoint.getEndPoint().getVersion(null);
-      VersionResponse response = VersionResponse.getFromProtobuf(
-          versionResponse);
-      String scmUuid = response.getValue(OzoneConsts.SCM_ID);
-      Preconditions.checkState(!StringUtils.isBlank(scmUuid),
-          "Invalid SCM UuiD in the response.");
-
-      rpcEndPoint.setVersion(response);
-      LOG.debug("scmUuid is {}", scmUuid);
-
-      List<StorageLocation> locations = datanodeContainerManager.getLocations();
+      rpcEndPoint.setVersion(VersionResponse.getFromProtobuf(versionResponse));
 
-      for (StorageLocation location : locations) {
-        String path = location.getUri().getPath();
-        File parentPath = new File(path + File.separator + Storage
-            .STORAGE_DIR_HDDS + File.separator + scmUuid + File.separator +
-            Storage.STORAGE_DIR_CURRENT);
-        File versionFile = DatanodeVersionFile.getVersionFile(location,
-            scmUuid);
-        if (!parentPath.exists() && !parentPath.mkdirs()) {
-          LOG.error("Directory doesn't exist and cannot be created. Path: {}",
-              parentPath.toString());
-          rpcEndPoint.setState(EndpointStateMachine.EndPointStates.SHUTDOWN);
-          throw new IllegalArgumentException("Directory doesn't exist and " +
-              "cannot be created. " + parentPath.toString());
-        } else {
-          if (versionFile.exists()) {
-            Properties properties = DatanodeVersionFile.readFrom(versionFile);
-            DatanodeVersionFile.verifyScmUuid(properties.getProperty(
-                OzoneConsts.SCM_ID), scmUuid);
-            DatanodeVersionFile.verifyCreationTime(properties.getProperty(
-                OzoneConsts.CTIME));
-            DatanodeVersionFile.verifyLayOutVersion(properties.getProperty(
-                OzoneConsts.LAYOUTVERSION));
-          } else {
-            DatanodeVersionFile dnVersionFile = new DatanodeVersionFile(scmUuid,
-                Time.now(), DataNodeLayoutVersion.getLatestVersion()
-                .getVersion());
-            dnVersionFile.createVersionFile(versionFile);
-          }
-        }
-      }
-      EndpointStateMachine.EndPointStates nextState = rpcEndPoint.getState().
-          getNextState();
+      EndpointStateMachine.EndPointStates nextState =
+          rpcEndPoint.getState().getNextState();
       rpcEndPoint.setState(nextState);
       rpcEndPoint.zeroMissedCount();
-    } catch (InconsistentStorageStateException ex) {
-      throw ex;
     } catch (IOException ex) {
       rpcEndPoint.logIfNeeded(ex);
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e437f9b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
index 69bdf32..b357fef 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
@@ -51,7 +51,6 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.nio.file.Paths;
-import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
@@ -83,7 +82,6 @@ public class OzoneContainer {
   private final ChunkManager chunkManager;
   private final KeyManager keyManager;
   private final BlockDeletingService blockDeletingService;
-  private final List<StorageLocation> locations;
 
   /**
    * Creates a network endpoint and enables Ozone container.
@@ -95,7 +93,7 @@ public class OzoneContainer {
       DatanodeDetails datanodeDetails, Configuration ozoneConfig)
       throws IOException {
     this.ozoneConfig = ozoneConfig;
-    locations = new LinkedList<>();
+    List<StorageLocation> locations = new LinkedList<>();
     String[] paths = ozoneConfig.getStrings(
         OzoneConfigKeys.OZONE_METADATA_DIRS);
     if (paths != null && paths.length > 0) {
@@ -139,10 +137,6 @@ public class OzoneContainer {
     };
   }
 
-  public List<StorageLocation> getLocations() {
-    return Collections.unmodifiableList(this.locations);
-  }
-
   /**
    * Starts serving requests to ozone container.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e437f9b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/VersionResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/VersionResponse.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/VersionResponse.java
index c26fbfa..83acf5b 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/VersionResponse.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/VersionResponse.java
@@ -88,10 +88,6 @@ public class VersionResponse {
     values.put(key, value);
   }
 
-  public String getValue(String key) {
-    return this.values.get(key);
-  }
-
   /**
    * Return a protobuf message.
    * @return SCMVersionResponseProto.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e437f9b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
index cd940b5..0ee6321 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
@@ -57,28 +57,6 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
   private Map<DatanodeDetails, Map<String, ContainerInfo>> nodeContainers =
       new HashMap();
   private Map<DatanodeDetails, NodeReportProto> nodeReports = new HashMap<>();
-  private UUID scmUuid;
-
-  public ScmTestMock() {
-    scmUuid = UUID.randomUUID();
-  }
-
-  /**
-   * Return scmUuid.
-   * @return UUID
-   */
-  public UUID getScmUuid() {
-    return scmUuid;
-  }
-
-  /**
-   * set scmUuid.
-   * @param id
-   */
-  public void setSCMUuid(UUID id) {
-    this.scmUuid = id;
-  }
-
   /**
    * Returns the number of heartbeats made to this class.
    *
@@ -169,11 +147,9 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
     rpcCount.incrementAndGet();
     sleepIfNeeded();
     VersionInfo versionInfo = VersionInfo.getLatestVersion();
-
     return VersionResponse.newBuilder()
         .setVersion(versionInfo.getVersion())
         .addValue(VersionInfo.DESCRIPTION_KEY, versionInfo.getDescription())
-        .addValue("scmUuid", scmUuid.toString())
         .build().getProtobufMessage();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e437f9b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeLayOutVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeLayOutVersion.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeLayOutVersion.java
deleted file mode 100644
index 5cabef2..0000000
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeLayOutVersion.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.ozone.container.common;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * This class tests DatanodeLayOutVersion.
- */
-public class TestDatanodeLayOutVersion {
-
-  @Test
-  public void testDatanodeLayOutVersion() {
-    // Check Latest Version and description
-    Assert.assertEquals(1, DataNodeLayoutVersion.getLatestVersion()
-        .getVersion());
-    Assert.assertEquals("HDDS Datanode LayOut Version 1", DataNodeLayoutVersion
-        .getLatestVersion().getDescription());
-    Assert.assertEquals(DataNodeLayoutVersion.getAllVersions().length,
-        DataNodeLayoutVersion.getAllVersions().length);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e437f9b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
index 79841bd..ece7545 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.ozone.container.common;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ipc.RPC;
@@ -146,7 +145,7 @@ public class TestDatanodeStateMachine {
     } catch (Exception e) {
       //ignore all execption from the shutdown
     } finally {
-      FileUtil.fullyDelete(testRoot);
+      testRoot.delete();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e437f9b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java
deleted file mode 100644
index e798a7d..0000000
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestDatanodeVersionFile.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.ozone.container.common.helpers;
-
-import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
-import org.apache.hadoop.ozone.container.common.DataNodeLayoutVersion;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.util.Time;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Properties;
-import java.util.UUID;
-
-import static org.junit.Assert.*;
-
-/**
- * This class tests DatanodeVersionFile.
- */
-public class TestDatanodeVersionFile {
-
-  @Rule
-  public TemporaryFolder folder= new TemporaryFolder();
-
-  @Test
-  public void testCreateAndReadVersionFile() throws IOException{
-    File versionFile = folder.newFile("Version");
-    String uuid = UUID.randomUUID().toString();
-    long now = Time.now();
-    int lv = DataNodeLayoutVersion.getLatestVersion().getVersion();
-
-    DatanodeVersionFile dnVersionFile = new DatanodeVersionFile(uuid, now, lv);
-
-    dnVersionFile.createVersionFile(versionFile);
-
-    //Check VersionFile exists
-    assertTrue(versionFile.exists());
-
-    Properties properties = dnVersionFile.readFrom(versionFile);
-
-    assertEquals(uuid, properties.getProperty(OzoneConsts.SCM_ID));
-    assertEquals(String.valueOf(now), properties.get(OzoneConsts.CTIME));
-    assertEquals(String.valueOf(lv), properties.get(OzoneConsts.LAYOUTVERSION));
-
-    DatanodeVersionFile.verifyCreationTime(String.valueOf(properties.get(
-        OzoneConsts.CTIME)));
-    DatanodeVersionFile.verifyLayOutVersion(String.valueOf(properties
-        .getProperty(OzoneConsts.LAYOUTVERSION)));
-    DatanodeVersionFile.verifyScmUuid(uuid, String.valueOf(properties
-        .getProperty(OzoneConsts.SCM_ID)));
-
-
-  }
-
-  @Test
-  public void testVerifyUuid() throws IOException{
-    String uuid = UUID.randomUUID().toString();
-    try {
-      DatanodeVersionFile.verifyScmUuid(uuid, uuid);
-      DatanodeVersionFile.verifyScmUuid(uuid, UUID.randomUUID().toString());
-      fail("Test failure in testVerifyUuid");
-    } catch (InconsistentStorageStateException ex) {
-      GenericTestUtils.assertExceptionContains("MisMatch of ScmUuid", ex);
-    }
-  }
-
-  @Test
-  public void testVerifyCTime() throws IOException{
-    try {
-      DatanodeVersionFile.verifyCreationTime(String.valueOf(Time.now()));
-      DatanodeVersionFile.verifyCreationTime(null);
-      fail("Test failure in testVerifyCTime");
-    } catch (IllegalStateException ex) {
-      GenericTestUtils.assertExceptionContains("Invalid creation Time.", ex);
-    }
-  }
-
-  @Test
-  public void testVerifyLayOut() throws IOException{
-    String lv = String.valueOf(DataNodeLayoutVersion.getLatestVersion()
-        .getVersion());
-    try {
-      DatanodeVersionFile.verifyLayOutVersion(lv);
-      DatanodeVersionFile.verifyLayOutVersion(null);
-      fail("Test failure in testVerifyLayOut");
-    } catch (IllegalStateException ex) {
-      GenericTestUtils.assertExceptionContains("Invalid layOutVersion.", ex);
-    }
-  }
-
-  @Test
-  public void testGetVersionFile() throws IOException {
-    StorageLocation location = StorageLocation.parse("/tmp/disk1");
-    String scmId = UUID.randomUUID().toString();
-    assertEquals(new File("/tmp/disk1/hdds/" + scmId + "/current/VERSION"),
-        DatanodeVersionFile.getVersionFile(location, scmId));
-    assertEquals(null, DatanodeVersionFile.getVersionFile(null, scmId));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e437f9b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
index 016e1e1..b339fb7 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.metrics2.util.MBeans;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.protocol.StorageContainerNodeProtocol;
 import org.apache.hadoop.ozone.protocol.VersionResponse;
 import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
@@ -704,7 +703,6 @@ public class SCMNodeManager
   public VersionResponse getVersion(SCMVersionRequestProto versionRequest) {
     return VersionResponse.newBuilder()
         .setVersion(this.version.getVersion())
-        .addValue(OzoneConsts.SCM_ID, scmManager.getScmStorage().getScmId())
         .build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e437f9b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
index 9d2e61a..34779da 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.ozone.container.common;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.scm.TestUtils;
@@ -34,7 +33,6 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
-import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdds.protocol.proto
@@ -43,7 +41,6 @@ import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerReport;
-import org.apache.hadoop.ozone.container.common.helpers.DatanodeVersionFile;
 import org.apache.hadoop.ozone.container.common.statemachine
     .DatanodeStateMachine;
 import org.apache.hadoop.ozone.container.common.statemachine
@@ -56,22 +53,17 @@ import org.apache.hadoop.ozone.container.common.states.endpoint
 import org.apache.hadoop.ozone.container.common.states.endpoint
     .VersionEndpointTask;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
-import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.Time;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-
-import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 
 import java.io.File;
 import java.net.InetSocketAddress;
-import java.util.ArrayList;
 import java.util.List;
-import java.util.Properties;
 import java.util.UUID;
 
 import static org.apache.hadoop.hdds.scm.TestUtils.getDatanodeDetails;
@@ -124,11 +116,6 @@ public class TestEndPoint {
           responseProto.getKeys(0).getKey());
       Assert.assertEquals(VersionInfo.getLatestVersion().getDescription(),
           responseProto.getKeys(0).getValue());
-      Assert.assertEquals("scmUuid", responseProto.getKeys(
-          1).getKey());
-      Assert.assertEquals(scmServerImpl.getScmUuid().toString(),
-          responseProto.getKeys(1).getValue());
-
     }
   }
 
@@ -139,20 +126,11 @@ public class TestEndPoint {
    */
   public void testGetVersionTask() throws Exception {
     Configuration conf = SCMTestUtils.getConf();
-    String path = new FileSystemTestHelper().getTestRootDir();
-    conf.set(DFS_DATANODE_DATA_DIR_KEY, path);
     try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
         serverAddress, 1000)) {
       rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
-      OzoneContainer ozoneContainer = mock(OzoneContainer.class);
-      List<StorageLocation> pathList = new ArrayList<>();
-      for (String dir : conf.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
-        StorageLocation location = StorageLocation.parse(dir);
-        pathList.add(location);
-      }
-      when(ozoneContainer.getLocations()).thenReturn(pathList);
       VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
-          conf, ozoneContainer);
+          conf);
       EndpointStateMachine.EndPointStates newState = versionTask.call();
 
       // if version call worked the endpoint should automatically move to the
@@ -162,132 +140,10 @@ public class TestEndPoint {
 
       // Now rpcEndpoint should remember the version it got from SCM
       Assert.assertNotNull(rpcEndPoint.getVersion());
-      FileUtil.fullyDelete(new File(path));
     }
   }
 
   @Test
-  public void testVersionCheckFail() throws Exception {
-    Configuration conf = SCMTestUtils.getConf();
-    String path = new FileSystemTestHelper().getTestRootDir();
-    conf.set(DFS_DATANODE_DATA_DIR_KEY, path);
-    try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
-        serverAddress, 1000)) {
-      rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
-      OzoneContainer ozoneContainer = mock(OzoneContainer.class);
-      List<StorageLocation> pathList = new ArrayList<>();
-      for (String dir : conf.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
-        StorageLocation location = StorageLocation.parse(dir);
-        pathList.add(location);
-      }
-      when(ozoneContainer.getLocations()).thenReturn(pathList);
-      VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
-          conf, ozoneContainer);
-      EndpointStateMachine.EndPointStates newState = versionTask.call();
-
-      // if version call worked the endpoint should automatically move to the
-      // next state.
-      Assert.assertEquals(EndpointStateMachine.EndPointStates.REGISTER,
-          newState);
-
-      // Now rpcEndpoint should remember the version it got from SCM
-      Assert.assertNotNull(rpcEndPoint.getVersion());
-
-      // Now call again version task with an incorrect layout version.
-      // This will fail with Incorrect layOutVersion error.
-      DatanodeVersionFile datanodeVersionFile = new DatanodeVersionFile(
-          scmServerImpl.getScmUuid().toString(), Time.now(), 2);
-      datanodeVersionFile.createVersionFile(DatanodeVersionFile
-          .getVersionFile(pathList.get(0), scmServerImpl.getScmUuid()
-              .toString()));
-      rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
-      versionTask.call();
-      fail("Test fail");
-    } catch(Throwable t) {
-      GenericTestUtils.assertExceptionContains("Incorrect layOutVersion", t);
-      FileUtil.fullyDelete(new File(path));
-    }
-  }
-
-
-  @Test
-  public void testVersionCheckSuccess() throws Exception {
-    Configuration conf = SCMTestUtils.getConf();
-    String path = new FileSystemTestHelper().getTestRootDir();
-    conf.set(DFS_DATANODE_DATA_DIR_KEY, path);
-    try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
-        serverAddress, 1000)) {
-      rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
-      OzoneContainer ozoneContainer = mock(OzoneContainer.class);
-      List<StorageLocation> pathList = new ArrayList<>();
-      for (String dir : conf.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
-        StorageLocation location = StorageLocation.parse(dir);
-        pathList.add(location);
-      }
-      when(ozoneContainer.getLocations()).thenReturn(pathList);
-      VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
-          conf, ozoneContainer);
-      EndpointStateMachine.EndPointStates newState = versionTask.call();
-
-      // if version call worked the endpoint should automatically move to the
-      // next state.
-      Assert.assertEquals(EndpointStateMachine.EndPointStates.REGISTER,
-          newState);
-
-      // Now rpcEndpoint should remember the version it got from SCM
-      Assert.assertNotNull(rpcEndPoint.getVersion());
-
-      // Now call again Version Task, this time version check should succeed.
-      rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
-      newState = versionTask.call();
-      Assert.assertEquals(EndpointStateMachine.EndPointStates.REGISTER,
-          newState);
-      FileUtil.fullyDelete(new File(path));
-    }
-  }
-
-  @Test
-  public void testVersionCheckFile() throws Exception {
-    Configuration conf = SCMTestUtils.getConf();
-    FileUtil.fullyDelete(new File("/tmp/hadoop"));
-    conf.set(DFS_DATANODE_DATA_DIR_KEY, "/tmp/hadoop");
-    try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
-        serverAddress, 1000)) {
-      rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
-      OzoneContainer ozoneContainer = mock(OzoneContainer.class);
-      List<StorageLocation> pathList = new ArrayList<>();
-      String dir = conf.get(DFS_DATANODE_DATA_DIR_KEY);
-      StorageLocation location = StorageLocation.parse(dir);
-      pathList.add(location);
-
-      when(ozoneContainer.getLocations()).thenReturn(pathList);
-      VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
-          conf, ozoneContainer);
-      EndpointStateMachine.EndPointStates newState = versionTask.call();
-
-      // if version call worked the endpoint should automatically move to the
-      // next state.
-      Assert.assertEquals(EndpointStateMachine.EndPointStates.REGISTER,
-          newState);
-
-      // Now rpcEndpoint should remember the version it got from SCM
-      Assert.assertNotNull(rpcEndPoint.getVersion());
-
-      // Check Version File created or not and content is expected or not.
-      File versionFile = DatanodeVersionFile.getVersionFile(pathList.get(0),
-          scmServerImpl.getScmUuid().toString());
-      Assert.assertTrue(versionFile.exists());
-
-      Properties props = DatanodeVersionFile.readFrom(versionFile);
-      DatanodeVersionFile.verifyCreationTime(props.getProperty(OzoneConsts
-          .CTIME));
-      DatanodeVersionFile.verifyScmUuid(scmServerImpl.getScmUuid().toString(),
-          props.getProperty(OzoneConsts.SCM_ID));
-      DatanodeVersionFile.verifyLayOutVersion(props.getProperty(OzoneConsts
-          .LAYOUTVERSION));
-    }
-  }
-  @Test
   /**
    * This test makes a call to end point where there is no SCM server. We
    * expect that versionTask should be able to handle it.
@@ -296,20 +152,11 @@ public class TestEndPoint {
     Configuration conf = SCMTestUtils.getConf();
     InetSocketAddress nonExistentServerAddress = SCMTestUtils
         .getReuseableAddress();
-    FileUtil.fullyDelete(new File("/tmp/hadoop"));
-    conf.set(DFS_DATANODE_DATA_DIR_KEY, "/tmp/hadoop");
     try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
         nonExistentServerAddress, 1000)) {
       rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
-      OzoneContainer ozoneContainer = mock(OzoneContainer.class);
-      List<StorageLocation> pathList = new ArrayList<>();
-      for (String dir : conf.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
-        StorageLocation location = StorageLocation.parse(dir);
-        pathList.add(location);
-      }
-      when(ozoneContainer.getLocations()).thenReturn(pathList);
       VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
-          conf, ozoneContainer);
+          conf);
       EndpointStateMachine.EndPointStates newState = versionTask.call();
 
       // This version call did NOT work, so endpoint should remain in the same
@@ -329,20 +176,12 @@ public class TestEndPoint {
     final long rpcTimeout = 1000;
     final long tolerance = 100;
     Configuration conf = SCMTestUtils.getConf();
-    FileUtil.fullyDelete(new File("/tmp/hadoop"));
-    conf.set(DFS_DATANODE_DATA_DIR_KEY, "/tmp/hadoop");
+
     try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
         serverAddress, (int) rpcTimeout)) {
       rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
-      OzoneContainer ozoneContainer = mock(OzoneContainer.class);
-      List<StorageLocation> pathList = new ArrayList<>();
-      for (String dir : conf.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
-        StorageLocation location = StorageLocation.parse(dir);
-        pathList.add(location);
-      }
-      when(ozoneContainer.getLocations()).thenReturn(pathList);
       VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
-          conf, ozoneContainer);
+          conf);
 
       scmServerImpl.setRpcResponseDelay(1500);
       long start = Time.monotonicNow();


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


[20/37] hadoop git commit: HDDS-183:Integrate Volumeset, ContainerSet and HddsDispatcher. Contributed by Bharat Viswanadham

Posted by bh...@apache.org.
HDDS-183:Integrate Volumeset, ContainerSet and HddsDispatcher. Contributed by Bharat Viswanadham


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

Branch: refs/heads/trunk
Commit: 52d1d9603ecc03dbe3ef5fafa60377ef461ecca3
Parents: 13579f9
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Thu Jun 28 14:07:52 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Thu Jun 28 14:07:52 2018 -0700

----------------------------------------------------------------------
 .../main/proto/DatanodeContainerProtocol.proto  |   1 +
 .../container/common/impl/ContainerData.java    |   1 +
 .../common/impl/ContainerDataYaml.java          | 276 +++++++++++++
 .../ozone/container/common/impl/Dispatcher.java |  11 +
 .../container/common/impl/HddsDispatcher.java   |  28 +-
 .../common/interfaces/ContainerDispatcher.java  |  14 +
 .../container/common/interfaces/Handler.java    |  15 +-
 .../statemachine/DatanodeStateMachine.java      |   4 +-
 .../background/BlockDeletingService.java        |   1 +
 .../DeleteBlocksCommandHandler.java             |  56 ++-
 .../states/datanode/RunningDatanodeState.java   |   3 +-
 .../states/endpoint/VersionEndpointTask.java    |  34 +-
 .../container/common/volume/HddsVolume.java     |   6 +
 .../container/common/volume/VolumeSet.java      |  48 +++
 .../container/keyvalue/KeyValueContainer.java   |  52 +--
 .../keyvalue/KeyValueContainerData.java         |  22 +-
 .../container/keyvalue/KeyValueHandler.java     |  22 +-
 .../ozone/container/keyvalue/KeyValueYaml.java  | 272 -------------
 .../keyvalue/helpers/KeyValueContainerUtil.java | 134 +++++++
 .../container/ozoneimpl/ContainerReader.java    | 157 ++++++++
 .../container/ozoneimpl/OzoneContainer.java     | 396 ++++++++-----------
 .../hadoop/ozone/protocol/VersionResponse.java  |   4 +
 .../ozone/container/common/SCMTestUtils.java    |  13 +-
 .../ozone/container/common/ScmTestMock.java     |   4 +
 .../common/TestKeyValueContainerData.java       |   3 +-
 .../common/impl/TestContainerDataYaml.java      | 163 ++++++++
 .../container/common/impl/TestContainerSet.java |   6 +-
 .../container/common/impl/TestKeyValueYaml.java | 160 --------
 .../common/interfaces/TestHandler.java          |   8 +-
 .../keyvalue/TestChunkManagerImpl.java          |   3 +-
 .../container/keyvalue/TestKeyManagerImpl.java  |   3 +-
 .../keyvalue/TestKeyValueContainer.java         |  10 +-
 .../container/keyvalue/TestKeyValueHandler.java |   8 +-
 .../container/ozoneimpl/TestOzoneContainer.java | 108 +++++
 .../hadoop/hdds/scm/node/SCMNodeManager.java    |   4 +
 .../ozone/container/common/TestEndPoint.java    |  20 +-
 .../TestStorageContainerManagerHelper.java      |  11 +-
 .../TestCloseContainerByPipeline.java           |   6 +-
 .../TestCloseContainerHandler.java              |   6 +-
 .../container/ozoneimpl/TestOzoneContainer.java |  13 +-
 .../container/server/TestContainerServer.java   |  10 +
 .../ozone/ksm/TestContainerReportWithKeys.java  |  14 +-
 .../hadoop/ozone/web/client/TestKeys.java       |  22 +-
 43 files changed, 1329 insertions(+), 823 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
index d29e479..ff1582e 100644
--- a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
+++ b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
@@ -137,6 +137,7 @@ enum Result {
   CONTAINER_METADATA_ERROR = 31;
   CONTAINER_FILES_CREATE_ERROR = 32;
   CONTAINER_CHECKSUM_ERROR = 33;
+  UNKNOWN_CONTAINER_TYPE = 34;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
index 0bd7795..b11b66c 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.ozone.container.common.impl;
 
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
     ContainerType;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
new file mode 100644
index 0000000..6b8e6ee
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.impl;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.yaml.snakeyaml.Yaml;
+
+import java.beans.IntrospectionException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Writer;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.OutputStreamWriter;
+import java.io.File;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.Map;
+
+import org.yaml.snakeyaml.constructor.AbstractConstruct;
+import org.yaml.snakeyaml.constructor.Constructor;
+import org.yaml.snakeyaml.introspector.BeanAccess;
+import org.yaml.snakeyaml.introspector.Property;
+import org.yaml.snakeyaml.introspector.PropertyUtils;
+import org.yaml.snakeyaml.nodes.MappingNode;
+import org.yaml.snakeyaml.nodes.Node;
+import org.yaml.snakeyaml.nodes.ScalarNode;
+import org.yaml.snakeyaml.nodes.Tag;
+import org.yaml.snakeyaml.representer.Representer;
+
+import static org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData.YAML_FIELDS;
+import static org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData.YAML_TAG;
+
+/**
+ * Class for creating and reading .container files.
+ */
+
+public final class ContainerDataYaml {
+
+  private ContainerDataYaml() {
+
+  }
+  /**
+   * Creates a .container file in yaml format.
+   *
+   * @param containerFile
+   * @param containerData
+   * @throws IOException
+   */
+  public static void createContainerFile(ContainerProtos.ContainerType
+                                             containerType, File containerFile,
+                                         ContainerData containerData) throws
+      IOException {
+
+    Preconditions.checkNotNull(containerFile, "yamlFile cannot be null");
+    Preconditions.checkNotNull(containerData, "containerData cannot be null");
+    Preconditions.checkNotNull(containerType, "containerType cannot be null");
+
+    PropertyUtils propertyUtils = new PropertyUtils();
+    propertyUtils.setBeanAccess(BeanAccess.FIELD);
+    propertyUtils.setAllowReadOnlyProperties(true);
+
+    switch(containerType) {
+    case KeyValueContainer:
+      Representer representer = new ContainerDataRepresenter();
+      representer.setPropertyUtils(propertyUtils);
+      representer.addClassTag(KeyValueContainerData.class,
+          KeyValueContainerData.YAML_TAG);
+
+      Constructor keyValueDataConstructor = new ContainerDataConstructor();
+
+      Yaml yaml = new Yaml(keyValueDataConstructor, representer);
+      Writer writer = new OutputStreamWriter(new FileOutputStream(
+          containerFile), "UTF-8");
+      yaml.dump(containerData, writer);
+      writer.close();
+      break;
+    default:
+      throw new StorageContainerException("Unrecognized container Type " +
+          "format " + containerType, ContainerProtos.Result
+          .UNKNOWN_CONTAINER_TYPE);
+    }
+  }
+
+  /**
+   * Read the yaml file, and return containerData.
+   *
+   * @param containerFile
+   * @throws IOException
+   */
+  public static ContainerData readContainerFile(File containerFile)
+      throws IOException {
+    Preconditions.checkNotNull(containerFile, "containerFile cannot be null");
+
+    InputStream input = null;
+    ContainerData containerData;
+    try {
+      PropertyUtils propertyUtils = new PropertyUtils();
+      propertyUtils.setBeanAccess(BeanAccess.FIELD);
+      propertyUtils.setAllowReadOnlyProperties(true);
+
+      Representer representer = new ContainerDataRepresenter();
+      representer.setPropertyUtils(propertyUtils);
+
+      Constructor containerDataConstructor = new ContainerDataConstructor();
+
+      Yaml yaml = new Yaml(containerDataConstructor, representer);
+      yaml.setBeanAccess(BeanAccess.FIELD);
+
+      input = new FileInputStream(containerFile);
+      containerData = (ContainerData)
+          yaml.load(input);
+    } finally {
+      if (input!= null) {
+        input.close();
+      }
+    }
+    return containerData;
+  }
+
+  /**
+   * Representer class to define which fields need to be stored in yaml file.
+   */
+  private static class ContainerDataRepresenter extends Representer {
+    @Override
+    protected Set<Property> getProperties(Class<? extends Object> type)
+        throws IntrospectionException {
+      Set<Property> set = super.getProperties(type);
+      Set<Property> filtered = new TreeSet<Property>();
+
+      // When a new Container type is added, we need to add what fields need
+      // to be filtered here
+      if (type.equals(KeyValueContainerData.class)) {
+        // filter properties
+        for (Property prop : set) {
+          String name = prop.getName();
+          if (YAML_FIELDS.contains(name)) {
+            filtered.add(prop);
+          }
+        }
+      }
+      return filtered;
+    }
+  }
+
+  /**
+   * Constructor class for KeyValueData, which will be used by Yaml.
+   */
+  private static class ContainerDataConstructor extends Constructor {
+    ContainerDataConstructor() {
+      //Adding our own specific constructors for tags.
+      // When a new Container type is added, we need to add yamlConstructor
+      // for that
+      this.yamlConstructors.put(YAML_TAG, new ConstructKeyValueContainerData());
+      this.yamlConstructors.put(Tag.INT, new ConstructLong());
+    }
+
+    private class ConstructKeyValueContainerData extends AbstractConstruct {
+      public Object construct(Node node) {
+        MappingNode mnode = (MappingNode) node;
+        Map<Object, Object> nodes = constructMapping(mnode);
+
+        //Needed this, as TAG.INT type is by default converted to Long.
+        long layOutVersion = (long) nodes.get("layOutVersion");
+        int lv = (int) layOutVersion;
+
+        //When a new field is added, it needs to be added here.
+        KeyValueContainerData kvData = new KeyValueContainerData((long) nodes
+            .get("containerId"), lv);
+        kvData.setContainerDBType((String)nodes.get("containerDBType"));
+        kvData.setMetadataPath((String) nodes.get(
+            "metadataPath"));
+        kvData.setChunksPath((String) nodes.get("chunksPath"));
+        Map<String, String> meta = (Map) nodes.get("metadata");
+        meta.forEach((key, val) -> {
+          try {
+            kvData.addMetadata(key, val);
+          } catch (IOException e) {
+            throw new IllegalStateException("Unexpected " +
+                "Key Value Pair " + "(" + key + "," + val +")in the metadata " +
+                "for containerId " + (long) nodes.get("containerId"));
+          }
+        });
+        String state = (String) nodes.get("state");
+        switch (state) {
+        case "OPEN":
+          kvData.setState(ContainerProtos.ContainerLifeCycleState.OPEN);
+          break;
+        case "CLOSING":
+          kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSING);
+          break;
+        case "CLOSED":
+          kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED);
+          break;
+        default:
+          throw new IllegalStateException("Unexpected " +
+              "ContainerLifeCycleState " + state + " for the containerId " +
+              (long) nodes.get("containerId"));
+        }
+        return kvData;
+      }
+    }
+
+    //Below code is taken from snake yaml, as snakeyaml tries to fit the
+    // number if it fits in integer, otherwise returns long. So, slightly
+    // modified the code to return long in all cases.
+    private class ConstructLong extends AbstractConstruct {
+      public Object construct(Node node) {
+        String value = constructScalar((ScalarNode) node).toString()
+            .replaceAll("_", "");
+        int sign = +1;
+        char first = value.charAt(0);
+        if (first == '-') {
+          sign = -1;
+          value = value.substring(1);
+        } else if (first == '+') {
+          value = value.substring(1);
+        }
+        int base = 10;
+        if ("0".equals(value)) {
+          return Long.valueOf(0);
+        } else if (value.startsWith("0b")) {
+          value = value.substring(2);
+          base = 2;
+        } else if (value.startsWith("0x")) {
+          value = value.substring(2);
+          base = 16;
+        } else if (value.startsWith("0")) {
+          value = value.substring(1);
+          base = 8;
+        } else if (value.indexOf(':') != -1) {
+          String[] digits = value.split(":");
+          int bes = 1;
+          int val = 0;
+          for (int i = 0, j = digits.length; i < j; i++) {
+            val += (Long.parseLong(digits[(j - i) - 1]) * bes);
+            bes *= 60;
+          }
+          return createNumber(sign, String.valueOf(val), 10);
+        } else {
+          return createNumber(sign, value, 10);
+        }
+        return createNumber(sign, value, base);
+      }
+    }
+
+    private Number createNumber(int sign, String number, int radix) {
+      Number result;
+      if (sign < 0) {
+        number = "-" + number;
+      }
+      result = Long.valueOf(number, radix);
+      return result;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
index b5fb08d..c485caf 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/Dispatcher.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.container.common.impl;
 
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.container.common.interfaces.Handler;
 import org.apache.ratis.shaded.com.google.protobuf.ByteString;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.scm.container.common.helpers
@@ -91,6 +92,16 @@ public class Dispatcher implements ContainerDispatcher {
   }
 
   @Override
+  public Handler getHandler(ContainerProtos.ContainerType containerType) {
+    return null;
+  }
+
+  @Override
+  public void setScmId(String scmId) {
+    // DO nothing, this will be removed when cleanup.
+  }
+
+  @Override
   public ContainerCommandResponseProto dispatch(
       ContainerCommandRequestProto msg) {
     LOG.trace("Command {}, trace ID: {} ", msg.getCmdType().toString(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
index e73b761..cbb48ec 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
@@ -52,24 +52,23 @@ public class HddsDispatcher implements ContainerDispatcher {
   private final Configuration conf;
   private final ContainerSet containerSet;
   private final VolumeSet volumeSet;
-  private final String scmID;
+  private String scmID;
 
   /**
    * Constructs an OzoneContainer that receives calls from
    * XceiverServerHandler.
    */
   public HddsDispatcher(Configuration config, ContainerSet contSet,
-      VolumeSet volumes, String scmId) {
-    // TODO: Pass ContainerSet, VolumeSet and scmID, intialize metrics
+      VolumeSet volumes) {
+    //TODO: initialize metrics
     this.conf = config;
     this.containerSet = contSet;
     this.volumeSet = volumes;
-    this.scmID = scmId;
     this.handlers = Maps.newHashMap();
     for (ContainerType containerType : ContainerType.values()) {
       handlers.put(containerType,
           Handler.getHandlerForContainerType(
-              containerType, conf, containerSet, volumeSet, scmID));
+              containerType, conf, containerSet, volumeSet));
     }
   }
 
@@ -103,7 +102,7 @@ public class HddsDispatcher implements ContainerDispatcher {
       return ContainerUtils.logAndReturnError(LOG, ex, msg);
     }
 
-    Handler handler = getHandlerForContainerType(containerType);
+    Handler handler = getHandler(containerType);
     if (handler == null) {
       StorageContainerException ex = new StorageContainerException("Invalid " +
           "ContainerType " + containerType,
@@ -113,9 +112,20 @@ public class HddsDispatcher implements ContainerDispatcher {
     return handler.handle(msg, container);
   }
 
-  @VisibleForTesting
-  public Handler getHandlerForContainerType(ContainerType type) {
-    return handlers.get(type);
+  @Override
+  public Handler getHandler(ContainerProtos.ContainerType containerType) {
+    return handlers.get(containerType);
+  }
+
+  @Override
+  public void setScmId(String scmId) {
+    Preconditions.checkNotNull(scmId, "scmId Cannot be null");
+    if (this.scmID == null) {
+      this.scmID = scmId;
+      for (Map.Entry<ContainerType, Handler> handlerMap : handlers.entrySet()) {
+        handlerMap.getValue().setScmID(scmID);
+      }
+    }
   }
 
   private long getContainerID(ContainerCommandRequestProto request)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java
index 7e12614..18644bb 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.container.common.interfaces;
 
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
@@ -48,4 +49,17 @@ public interface ContainerDispatcher {
    * Shutdown Dispatcher services.
    */
   void shutdown();
+
+  /**
+   * Returns the handler for the specified containerType.
+   * @param containerType
+   * @return
+   */
+  Handler getHandler(ContainerProtos.ContainerType containerType);
+
+  /**
+   * If scmId is not set, this will set scmId, otherwise it is a no-op.
+   * @param scmId
+   */
+  void setScmId(String scmId);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
index d08ad74..8069d71 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
 
-import java.io.IOException;
 
 /**
  * Dispatcher sends ContainerCommandRequests to Handler. Each Container Type
@@ -42,22 +41,20 @@ public class Handler {
   protected final Configuration conf;
   protected final ContainerSet containerSet;
   protected final VolumeSet volumeSet;
-  protected final String scmID;
+  protected String scmID;
 
   protected Handler(Configuration config, ContainerSet contSet,
-      VolumeSet volumeSet, String scmID) {
+      VolumeSet volumeSet) {
     conf = config;
     containerSet = contSet;
     this.volumeSet = volumeSet;
-    this.scmID = scmID;
   }
 
   public static Handler getHandlerForContainerType(ContainerType containerType,
-      Configuration config, ContainerSet contSet, VolumeSet volumeSet,
-      String scmID) {
+      Configuration config, ContainerSet contSet, VolumeSet volumeSet) {
     switch (containerType) {
     case KeyValueContainer:
-      return KeyValueHandler.getInstance(config, contSet, volumeSet, scmID);
+      return KeyValueHandler.getInstance(config, contSet, volumeSet);
     default:
       throw new IllegalArgumentException("Handler for ContainerType: " +
         containerType + "doesn't exist.");
@@ -68,4 +65,8 @@ public class Handler {
       ContainerCommandRequestProto msg, Container container) {
     return null;
   }
+
+  public void setScmID(String scmId) {
+    this.scmID = scmId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
index dc4e673..b6a9bb9 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
@@ -93,8 +93,8 @@ public class DatanodeStateMachine implements Closeable {
      // trick.
     commandDispatcher = CommandDispatcher.newBuilder()
         .addHandler(new CloseContainerCommandHandler())
-        .addHandler(new DeleteBlocksCommandHandler(
-            container.getContainerManager(), conf))
+        .addHandler(new DeleteBlocksCommandHandler(container.getContainerSet(),
+            conf))
         .setConnectionManager(connectionManager)
         .setContainer(container)
         .setContext(context)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java
index 63f57b4..50dea0a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/background/BlockDeletingService.java
@@ -97,6 +97,7 @@ public class BlockDeletingService extends BackgroundService{
         OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL_DEFAULT);
   }
 
+
   @Override
   public BackgroundTaskQueue getTasks() {
     BackgroundTaskQueue queue = new BackgroundTaskQueue();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
index f954d98..4fc1cd9 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
@@ -18,8 +18,10 @@ package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
 
 import com.google.common.primitives.Longs;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
@@ -29,11 +31,13 @@ import org.apache.hadoop.hdds.protocol.proto
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 import org.apache.hadoop.ozone.container.common.helpers
     .DeletedContainerBlocksSummary;
-import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.statemachine
     .EndpointStateMachine;
 import org.apache.hadoop.ozone.container.common.statemachine
@@ -51,6 +55,8 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.util.List;
 
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_NOT_FOUND;
+
 /**
  * Handle block deletion commands.
  */
@@ -59,14 +65,14 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
   private static final Logger LOG =
       LoggerFactory.getLogger(DeleteBlocksCommandHandler.class);
 
-  private ContainerManager containerManager;
-  private Configuration conf;
+  private final ContainerSet containerSet;
+  private final Configuration conf;
   private int invocationCount;
   private long totalTime;
 
-  public DeleteBlocksCommandHandler(ContainerManager containerManager,
+  public DeleteBlocksCommandHandler(ContainerSet cset,
       Configuration conf) {
-    this.containerManager = containerManager;
+    this.containerSet = cset;
     this.conf = conf;
   }
 
@@ -105,8 +111,24 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
           DeleteBlockTransactionResult.newBuilder();
       txResultBuilder.setTxID(entry.getTxID());
       try {
-        deleteContainerBlocks(entry, conf);
-        txResultBuilder.setSuccess(true);
+        long containerId = entry.getContainerID();
+        Container cont = containerSet.getContainer(containerId);
+        if(cont == null) {
+          throw new StorageContainerException("Unable to find the container "
+              + containerId, CONTAINER_NOT_FOUND);
+        }
+        ContainerProtos.ContainerType containerType = cont.getContainerType();
+        switch (containerType) {
+        case KeyValueContainer:
+          KeyValueContainerData containerData = (KeyValueContainerData)
+              cont.getContainerData();
+          deleteKeyValueContainerBlocks(containerData, entry);
+          txResultBuilder.setSuccess(true);
+          break;
+        default:
+          LOG.error("Delete Blocks Command Handler is not implemented for " +
+              "containerType {}", containerType);
+        }
       } catch (IOException e) {
         LOG.warn("Failed to delete blocks for container={}, TXID={}",
             entry.getContainerID(), entry.getTxID(), e);
@@ -145,21 +167,21 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
    * Move a bunch of blocks from a container to deleting state.
    * This is a meta update, the actual deletes happen in async mode.
    *
+   * @param containerData - KeyValueContainerData
    * @param delTX a block deletion transaction.
-   * @param config configuration.
    * @throws IOException if I/O error occurs.
    */
-  private void deleteContainerBlocks(DeletedBlocksTransaction delTX,
-      Configuration config) throws IOException {
+  private void deleteKeyValueContainerBlocks(
+      KeyValueContainerData containerData, DeletedBlocksTransaction delTX)
+      throws IOException {
     long containerId = delTX.getContainerID();
-    ContainerData containerInfo = containerManager.readContainer(containerId);
     if (LOG.isDebugEnabled()) {
       LOG.debug("Processing Container : {}, DB path : {}", containerId,
-          containerInfo.getDBPath());
+          containerData.getMetadataPath());
     }
 
     int newDeletionBlocks = 0;
-    MetadataStore containerDB = KeyUtils.getDB(containerInfo, config);
+    MetadataStore containerDB = KeyUtils.getDB(containerData, conf);
     for (Long blk : delTX.getLocalIDList()) {
       BatchOperation batch = new BatchOperation();
       byte[] blkBytes = Longs.toByteArray(blk);
@@ -187,12 +209,12 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
                 + " container {}, skip deleting it.", blk, containerId);
       }
       containerDB.put(DFSUtil.string2Bytes(
-          OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX + delTX.getContainerID()),
+          OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX + containerId),
           Longs.toByteArray(delTX.getTxID()));
     }
 
     // update pending deletion blocks count in in-memory container status
-    containerManager.incrPendingDeletionBlocks(newDeletionBlocks, containerId);
+    containerData.incrPendingDeletionBlocks(newDeletionBlocks);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
index 3e11d12..1758c03 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
@@ -95,7 +95,8 @@ public class RunningDatanodeState implements DatanodeState {
       getEndPointTask(EndpointStateMachine endpoint) {
     switch (endpoint.getState()) {
     case GETVERSION:
-      return new VersionEndpointTask(endpoint, conf);
+      return new VersionEndpointTask(endpoint, conf, context.getParent()
+          .getContainer());
     case REGISTER:
       return  RegisterEndpointTask.newBuilder()
           .setConfig(conf)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
index b048ee5..e4cb4d5 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/VersionEndpointTask.java
@@ -16,14 +16,22 @@
  */
 package org.apache.hadoop.ozone.container.common.states.endpoint;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.statemachine
     .EndpointStateMachine;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.ozone.protocol.VersionResponse;
 
 import java.io.IOException;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.Callable;
 
 /**
@@ -33,11 +41,13 @@ public class VersionEndpointTask implements
     Callable<EndpointStateMachine.EndPointStates> {
   private final EndpointStateMachine rpcEndPoint;
   private final Configuration configuration;
+  private final OzoneContainer ozoneContainer;
 
   public VersionEndpointTask(EndpointStateMachine rpcEndPoint,
-      Configuration conf) {
+                             Configuration conf, OzoneContainer container) {
     this.rpcEndPoint = rpcEndPoint;
     this.configuration = conf;
+    this.ozoneContainer = container;
   }
 
   /**
@@ -52,7 +62,27 @@ public class VersionEndpointTask implements
     try{
       SCMVersionResponseProto versionResponse =
           rpcEndPoint.getEndPoint().getVersion(null);
-      rpcEndPoint.setVersion(VersionResponse.getFromProtobuf(versionResponse));
+      VersionResponse response = VersionResponse.getFromProtobuf(
+          versionResponse);
+      rpcEndPoint.setVersion(response);
+      VolumeSet volumeSet = ozoneContainer.getVolumeSet();
+      Map<String, HddsVolume> volumeMap = volumeSet.getVolumeMap();
+      List<HddsProtos.KeyValue> keyValues =  versionResponse.getKeysList();
+
+      String scmId = response.getValue(OzoneConsts.SCM_ID);
+      String clusterId = response.getValue(OzoneConsts.CLUSTER_ID);
+
+      Preconditions.checkNotNull(scmId, "Reply from SCM: scmId cannot be " +
+          "null");
+      Preconditions.checkNotNull(scmId, "Reply from SCM: clusterId cannot be" +
+          " null");
+
+      // If version file does not exist create version file and also set scmId
+      for (Map.Entry<String, HddsVolume> entry : volumeMap.entrySet()) {
+        HddsVolume hddsVolume = entry.getValue();
+        hddsVolume.format(clusterId);
+        ozoneContainer.getDispatcher().setScmId(scmId);
+      }
 
       EndpointStateMachine.EndPointStates nextState =
           rpcEndPoint.getState().getNextState();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
index 788e2cf..9006133 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
@@ -130,6 +130,10 @@ public final class HddsVolume {
     initialize();
   }
 
+  public VolumeInfo getVolumeInfo() {
+    return volumeInfo;
+  }
+
   /**
    * Initializes the volume.
    * Creates the Version file if not present,
@@ -327,4 +331,6 @@ public final class HddsVolume {
   public void setScmUsageForTesting(GetSpaceUsed scmUsageForTest) {
     volumeInfo.setScmUsageForTesting(scmUsageForTest);
   }
+
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java
index 9e052b0..e35becd 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java
@@ -27,8 +27,13 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
+import org.apache.hadoop.ozone.container.common.impl.StorageLocationReport;
 import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume.VolumeState;
 import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
@@ -309,4 +314,47 @@ public class VolumeSet {
   public Map<StorageType, List<HddsVolume>> getVolumeStateMap() {
     return ImmutableMap.copyOf(volumeStateMap);
   }
+
+  public StorageContainerDatanodeProtocolProtos.NodeReportProto getNodeReport()
+      throws IOException {
+    boolean failed;
+    StorageLocationReport[] reports =
+        new StorageLocationReport[volumeMap.size()];
+    int counter = 0;
+    for (Map.Entry<String, HddsVolume> entry : volumeMap.entrySet()) {
+      HddsVolume hddsVolume = entry.getValue();
+      VolumeInfo volumeInfo = hddsVolume.getVolumeInfo();
+      long scmUsed = 0;
+      long remaining = 0;
+      failed = false;
+      try {
+        scmUsed = volumeInfo.getScmUsed();
+        remaining = volumeInfo.getAvailable();
+      } catch (IOException ex) {
+        LOG.warn("Failed to get scmUsed and remaining for container " +
+            "storage location {}", volumeInfo.getRootDir());
+        // reset scmUsed and remaining if df/du failed.
+        scmUsed = 0;
+        remaining = 0;
+        failed = true;
+      }
+
+      StorageLocationReport.Builder builder =
+          StorageLocationReport.newBuilder();
+      builder.setStorageLocation(volumeInfo.getRootDir())
+          .setId(hddsVolume.getStorageID())
+          .setFailed(failed)
+          .setCapacity(hddsVolume.getCapacity())
+          .setRemaining(remaining)
+          .setScmUsed(scmUsed)
+          .setStorageType(hddsVolume.getStorageType());
+      StorageLocationReport r = builder.build();
+      reports[counter++] = r;
+    }
+    NodeReportProto.Builder nrb = NodeReportProto.newBuilder();
+    for (int i = 0; i < reports.length; i++) {
+      nrb.addStorageReport(reports[i].getProtoBufMessage());
+    }
+    return nrb.build();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index a1cbb4e..553e3f5 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.ozone.container.keyvalue;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileUtil;
@@ -33,6 +32,7 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
@@ -47,22 +47,17 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
 
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
 import java.util.Map;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.CONTAINER_ALREADY_EXISTS;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.CONTAINER_CHECKSUM_ERROR;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.CONTAINER_METADATA_ERROR;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.CONTAINER_INTERNAL_ERROR;
@@ -75,8 +70,6 @@ import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.INVALID_CONTAINER_STATE;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.NO_SUCH_ALGORITHM;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.UNSUPPORTED_REQUEST;
 
 /**
@@ -198,10 +191,12 @@ public class KeyValueContainer implements Container {
     try {
       tempContainerFile = createTempFile(containerFile);
       tempCheckSumFile = createTempFile(containerCheckSumFile);
-      KeyValueYaml.createContainerFile(tempContainerFile, containerData);
+      ContainerDataYaml.createContainerFile(ContainerProtos.ContainerType
+              .KeyValueContainer, tempContainerFile, containerData);
 
       //Compute Checksum for container file
-      String checksum = computeCheckSum(tempContainerFile);
+      String checksum = KeyValueContainerUtil.computeCheckSum(containerId,
+          tempContainerFile);
       containerCheckSumStream = new FileOutputStream(tempCheckSumFile);
       writer = new OutputStreamWriter(containerCheckSumStream, "UTF-8");
       writer.write(checksum);
@@ -308,43 +303,6 @@ public class KeyValueContainer implements Container {
   }
 
 
-  /**
-   * Compute checksum of the .container file.
-   * @param containerFile
-   * @throws StorageContainerException
-   */
-  private String computeCheckSum(File containerFile) throws
-      StorageContainerException {
-
-    MessageDigest sha;
-    FileInputStream containerFileStream = null;
-    try {
-      sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
-    } catch (NoSuchAlgorithmException e) {
-      throw new StorageContainerException("Unable to create Message Digest,"
-          + " usually this is a java configuration issue.",
-          NO_SUCH_ALGORITHM);
-    }
-
-    try {
-      containerFileStream = new FileInputStream(containerFile);
-      byte[] byteArray = new byte[1024];
-      int bytesCount = 0;
-
-      while ((bytesCount = containerFileStream.read(byteArray)) != -1) {
-        sha.update(byteArray, 0, bytesCount);
-      }
-      String checksum = DigestUtils.sha256Hex(sha.digest());
-      return checksum;
-    } catch (IOException ex) {
-      throw new StorageContainerException("Error during update of " +
-          "check sum file. Container Name: " + containerData.getContainerId(),
-          ex, CONTAINER_CHECKSUM_ERROR);
-    } finally {
-      IOUtils.closeStream(containerFileStream);
-    }
-  }
-
   @Override
   public void delete(boolean forceDelete)
       throws StorageContainerException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
index 8da4084..3b24468 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
@@ -18,12 +18,14 @@
 
 package org.apache.hadoop.ozone.container.keyvalue;
 
+import com.google.common.collect.Lists;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.yaml.snakeyaml.nodes.Tag;
 
 
 import java.io.File;
-import java.io.IOException;
+import java.util.List;
 import java.util.Map;
 
 /**
@@ -33,6 +35,14 @@ import java.util.Map;
  */
 public class KeyValueContainerData extends ContainerData {
 
+  // Yaml Tag used for KeyValueContainerData.
+  public static final Tag YAML_TAG = new Tag("KeyValueContainerData");
+
+  // Fields need to be stored in .container file.
+  public static final List<String> YAML_FIELDS = Lists.newArrayList(
+      "containerType", "containerId", "layOutVersion", "state", "metadata",
+      "metadataPath", "chunksPath", "containerDBType");
+
   // Path to Container metadata Level DB/RocksDB Store and .container file.
   private String metadataPath;
 
@@ -49,23 +59,21 @@ public class KeyValueContainerData extends ContainerData {
 
   /**
    * Constructs KeyValueContainerData object.
-   * @param type - containerType
    * @param id - ContainerId
    */
-  public KeyValueContainerData(ContainerProtos.ContainerType type, long id) {
-    super(type, id);
+  public KeyValueContainerData(long id) {
+    super(ContainerProtos.ContainerType.KeyValueContainer, id);
     this.numPendingDeletionBlocks = 0;
   }
 
   /**
    * Constructs KeyValueContainerData object.
-   * @param type - containerType
    * @param id - ContainerId
    * @param layOutVersion
    */
-  public KeyValueContainerData(ContainerProtos.ContainerType type, long id,
+  public KeyValueContainerData(long id,
                                int layOutVersion) {
-    super(type, id, layOutVersion);
+    super(ContainerProtos.ContainerType.KeyValueContainer, id, layOutVersion);
     this.numPendingDeletionBlocks = 0;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index d9ee7fd..ffe0f21 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.container.keyvalue;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
 import com.sun.jersey.spi.resource.Singleton;
@@ -93,16 +94,16 @@ public class KeyValueHandler extends Handler {
   // TODO : Add metrics and populate it.
 
   public static KeyValueHandler getInstance(Configuration config,
-      ContainerSet contSet, VolumeSet volSet, String scmID) {
+      ContainerSet contSet, VolumeSet volSet) {
     if (INSTANCE == null) {
-      INSTANCE = new KeyValueHandler(config, contSet, volSet, scmID);
+      INSTANCE = new KeyValueHandler(config, contSet, volSet);
     }
     return INSTANCE;
   }
 
   private KeyValueHandler(Configuration config, ContainerSet contSet,
-      VolumeSet volSet, String scmID) {
-    super(config, contSet, volSet, scmID);
+      VolumeSet volSet) {
+    super(config, contSet, volSet);
     containerType = ContainerType.KeyValueContainer;
     keyManager = new KeyManagerImpl(config);
     chunkManager = new ChunkManagerImpl();
@@ -156,6 +157,16 @@ public class KeyValueHandler extends Handler {
     return null;
   }
 
+  @VisibleForTesting
+  public ChunkManager getChunkManager() {
+    return this.chunkManager;
+  }
+
+  @VisibleForTesting
+  public KeyManager getKeyManager() {
+    return this.keyManager;
+  }
+
   /**
    * Handles Create Container Request. If successful, adds the container to
    * ContainerSet.
@@ -180,7 +191,7 @@ public class KeyValueHandler extends Handler {
     }
 
     KeyValueContainerData newContainerData = new KeyValueContainerData(
-        containerType, containerID);
+        containerID);
     // TODO: Add support to add metadataList to ContainerData. Add metadata
     // to container during creation.
     KeyValueContainer newContainer = new KeyValueContainer(
@@ -262,7 +273,6 @@ public class KeyValueHandler extends Handler {
 
     boolean forceDelete = request.getDeleteContainer().getForceDelete();
     kvContainer.writeLock();
-
     try {
       // Check if container is open
       if (kvContainer.getContainerData().isOpen()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueYaml.java
deleted file mode 100644
index 64f7152..0000000
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueYaml.java
+++ /dev/null
@@ -1,272 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.hadoop.ozone.container.keyvalue;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.ozone.container.common.impl.ContainerData;
-import org.yaml.snakeyaml.Yaml;
-
-import java.beans.IntrospectionException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.Writer;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.OutputStreamWriter;
-import java.io.File;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.Map;
-
-import org.yaml.snakeyaml.constructor.AbstractConstruct;
-import org.yaml.snakeyaml.constructor.Constructor;
-import org.yaml.snakeyaml.introspector.BeanAccess;
-import org.yaml.snakeyaml.introspector.Property;
-import org.yaml.snakeyaml.introspector.PropertyUtils;
-import org.yaml.snakeyaml.nodes.MappingNode;
-import org.yaml.snakeyaml.nodes.Node;
-import org.yaml.snakeyaml.nodes.ScalarNode;
-import org.yaml.snakeyaml.nodes.Tag;
-import org.yaml.snakeyaml.representer.Representer;
-
-/**
- * Class for creating and reading .container files.
- */
-
-public final class KeyValueYaml {
-
-  private KeyValueYaml() {
-
-  }
-  /**
-   * Creates a .container file in yaml format.
-   *
-   * @param containerFile
-   * @param containerData
-   * @throws IOException
-   */
-  public static void createContainerFile(File containerFile, ContainerData
-      containerData) throws IOException {
-
-    Preconditions.checkNotNull(containerFile, "yamlFile cannot be null");
-    Preconditions.checkNotNull(containerData, "containerData cannot be null");
-
-    PropertyUtils propertyUtils = new PropertyUtils();
-    propertyUtils.setBeanAccess(BeanAccess.FIELD);
-    propertyUtils.setAllowReadOnlyProperties(true);
-
-    Representer representer = new KeyValueContainerDataRepresenter();
-    representer.setPropertyUtils(propertyUtils);
-    representer.addClassTag(
-        KeyValueContainerData.class, new Tag("KeyValueContainerData"));
-
-    Constructor keyValueDataConstructor = new KeyValueDataConstructor();
-
-    Yaml yaml = new Yaml(keyValueDataConstructor, representer);
-
-    Writer writer = new OutputStreamWriter(new FileOutputStream(containerFile),
-        "UTF-8");
-    yaml.dump(containerData, writer);
-    writer.close();
-  }
-
-  /**
-   * Read the yaml file, and return containerData.
-   *
-   * @param containerFile
-   * @throws IOException
-   */
-  public static KeyValueContainerData readContainerFile(File containerFile)
-      throws IOException {
-    Preconditions.checkNotNull(containerFile, "containerFile cannot be null");
-
-    InputStream input = null;
-    KeyValueContainerData keyValueContainerData;
-    try {
-      PropertyUtils propertyUtils = new PropertyUtils();
-      propertyUtils.setBeanAccess(BeanAccess.FIELD);
-      propertyUtils.setAllowReadOnlyProperties(true);
-
-      Representer representer = new KeyValueContainerDataRepresenter();
-      representer.setPropertyUtils(propertyUtils);
-      representer.addClassTag(
-          KeyValueContainerData.class, new Tag("KeyValueContainerData"));
-
-      Constructor keyValueDataConstructor = new KeyValueDataConstructor();
-
-      Yaml yaml = new Yaml(keyValueDataConstructor, representer);
-      yaml.setBeanAccess(BeanAccess.FIELD);
-
-      input = new FileInputStream(containerFile);
-      keyValueContainerData = (KeyValueContainerData)
-          yaml.load(input);
-    } finally {
-      if (input!= null) {
-        input.close();
-      }
-    }
-    return keyValueContainerData;
-  }
-
-  /**
-   * Representer class to define which fields need to be stored in yaml file.
-   */
-  private static class KeyValueContainerDataRepresenter extends Representer {
-    @Override
-    protected Set<Property> getProperties(Class<? extends Object> type)
-        throws IntrospectionException {
-      Set<Property> set = super.getProperties(type);
-      Set<Property> filtered = new TreeSet<Property>();
-      if (type.equals(KeyValueContainerData.class)) {
-        // filter properties
-        for (Property prop : set) {
-          String name = prop.getName();
-          // When a new field needs to be added, it needs to be added here.
-          if (name.equals("containerType") || name.equals("containerId") ||
-              name.equals("layOutVersion") || name.equals("state") ||
-              name.equals("metadata") || name.equals("metadataPath") ||
-              name.equals("chunksPath") || name.equals(
-                  "containerDBType")) {
-            filtered.add(prop);
-          }
-        }
-      }
-      return filtered;
-    }
-  }
-
-  /**
-   * Constructor class for KeyValueData, which will be used by Yaml.
-   */
-  private static class KeyValueDataConstructor extends Constructor {
-    KeyValueDataConstructor() {
-      //Adding our own specific constructors for tags.
-      this.yamlConstructors.put(new Tag("KeyValueContainerData"),
-          new ConstructKeyValueContainerData());
-      this.yamlConstructors.put(Tag.INT, new ConstructLong());
-    }
-
-    private class ConstructKeyValueContainerData extends AbstractConstruct {
-      public Object construct(Node node) {
-        MappingNode mnode = (MappingNode) node;
-        Map<Object, Object> nodes = constructMapping(mnode);
-        String type = (String) nodes.get("containerType");
-
-        ContainerProtos.ContainerType containerType = ContainerProtos
-            .ContainerType.KeyValueContainer;
-        if (type.equals("KeyValueContainer")) {
-          containerType = ContainerProtos.ContainerType.KeyValueContainer;
-        }
-
-        //Needed this, as TAG.INT type is by default converted to Long.
-        long layOutVersion = (long) nodes.get("layOutVersion");
-        int lv = (int) layOutVersion;
-
-        //When a new field is added, it needs to be added here.
-        KeyValueContainerData kvData = new KeyValueContainerData(containerType,
-            (long) nodes.get("containerId"), lv);
-        kvData.setContainerDBType((String)nodes.get("containerDBType"));
-        kvData.setMetadataPath((String) nodes.get(
-            "metadataPath"));
-        kvData.setChunksPath((String) nodes.get("chunksPath"));
-        Map<String, String> meta = (Map) nodes.get("metadata");
-        meta.forEach((key, val) -> {
-          try {
-            kvData.addMetadata(key, val);
-          } catch (IOException e) {
-            throw new IllegalStateException("Unexpected " +
-                "Key Value Pair " + "(" + key + "," + val +")in the metadata " +
-                "for containerId " + (long) nodes.get("containerId"));
-          }
-        });
-        String state = (String) nodes.get("state");
-        switch (state) {
-        case "OPEN":
-          kvData.setState(ContainerProtos.ContainerLifeCycleState.OPEN);
-          break;
-        case "CLOSING":
-          kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSING);
-          break;
-        case "CLOSED":
-          kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED);
-          break;
-        default:
-          throw new IllegalStateException("Unexpected " +
-              "ContainerLifeCycleState " + state + " for the containerId " +
-              (long) nodes.get("containerId"));
-        }
-        return kvData;
-      }
-    }
-
-    //Below code is taken from snake yaml, as snakeyaml tries to fit the
-    // number if it fits in integer, otherwise returns long. So, slightly
-    // modified the code to return long in all cases.
-    private class ConstructLong extends AbstractConstruct {
-      public Object construct(Node node) {
-        String value = constructScalar((ScalarNode) node).toString()
-            .replaceAll("_", "");
-        int sign = +1;
-        char first = value.charAt(0);
-        if (first == '-') {
-          sign = -1;
-          value = value.substring(1);
-        } else if (first == '+') {
-          value = value.substring(1);
-        }
-        int base = 10;
-        if ("0".equals(value)) {
-          return Long.valueOf(0);
-        } else if (value.startsWith("0b")) {
-          value = value.substring(2);
-          base = 2;
-        } else if (value.startsWith("0x")) {
-          value = value.substring(2);
-          base = 16;
-        } else if (value.startsWith("0")) {
-          value = value.substring(1);
-          base = 8;
-        } else if (value.indexOf(':') != -1) {
-          String[] digits = value.split(":");
-          int bes = 1;
-          int val = 0;
-          for (int i = 0, j = digits.length; i < j; i++) {
-            val += (Long.parseLong(digits[(j - i) - 1]) * bes);
-            bes *= 60;
-          }
-          return createNumber(sign, String.valueOf(val), 10);
-        } else {
-          return createNumber(sign, value, 10);
-        }
-        return createNumber(sign, value, base);
-      }
-    }
-
-    private Number createNumber(int sign, String number, int radix) {
-      Number result;
-      if (sign < 0) {
-        number = "-" + number;
-      }
-      result = Long.valueOf(number, radix);
-      return result;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
index b868f1d..029e94d 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
@@ -18,9 +18,11 @@
 package org.apache.hadoop.ozone.container.keyvalue.helpers;
 
 import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
@@ -28,15 +30,29 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandResponseProto;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.helpers.KeyData;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.utils.MetadataKeyFilters;
 import org.apache.hadoop.utils.MetadataStore;
 import org.apache.hadoop.utils.MetadataStoreBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.*;
 
 /**
  * Class which defines utility methods for KeyValueContainer.
@@ -170,4 +186,122 @@ public final class KeyValueContainerUtil {
     builder.setReadContainer(response);
     return builder.build();
   }
+
+  /**
+   * Compute checksum of the .container file.
+   * @param containerId
+   * @param containerFile
+   * @throws StorageContainerException
+   */
+  public static String computeCheckSum(long containerId, File
+      containerFile) throws StorageContainerException {
+    Preconditions.checkNotNull(containerFile, "containerFile cannot be null");
+    MessageDigest sha;
+    FileInputStream containerFileStream = null;
+    try {
+      sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
+    } catch (NoSuchAlgorithmException e) {
+      throw new StorageContainerException("Unable to create Message Digest, " +
+          "usually this is a java configuration issue.", NO_SUCH_ALGORITHM);
+    }
+    try {
+      containerFileStream = new FileInputStream(containerFile);
+      byte[] byteArray = new byte[1024];
+      int bytesCount = 0;
+      while ((bytesCount = containerFileStream.read(byteArray)) != -1) {
+        sha.update(byteArray, 0, bytesCount);
+      }
+      String checksum = DigestUtils.sha256Hex(sha.digest());
+      return checksum;
+    } catch (IOException ex) {
+      throw new StorageContainerException("Error during computing checksum: " +
+          "for container " + containerId, ex, CONTAINER_CHECKSUM_ERROR);
+    } finally {
+      IOUtils.closeStream(containerFileStream);
+    }
+  }
+
+  /**
+   * Verify checksum of the container.
+   * @param containerId
+   * @param checksumFile
+   * @param checksum
+   * @throws StorageContainerException
+   */
+  public static void verifyCheckSum(long containerId, File checksumFile,
+                                    String checksum)
+      throws StorageContainerException {
+    try {
+      Preconditions.checkNotNull(checksum);
+      Preconditions.checkNotNull(checksumFile);
+      Path path = Paths.get(checksumFile.getAbsolutePath());
+      List<String> fileCheckSum = Files.readAllLines(path);
+      Preconditions.checkState(fileCheckSum.size() == 1, "checksum " +
+          "should be 32 byte string");
+      if (!checksum.equals(fileCheckSum.get(0))) {
+        LOG.error("Checksum mismatch for the container {}", containerId);
+        throw new StorageContainerException("Checksum mismatch for " +
+            "the container " + containerId, CHECKSUM_MISMATCH);
+      }
+    } catch (StorageContainerException ex) {
+      throw ex;
+    } catch (IOException ex) {
+      LOG.error("Error during verify checksum for container {}", containerId);
+      throw new StorageContainerException("Error during verify checksum" +
+          " for container " + containerId, IO_EXCEPTION);
+    }
+  }
+
+  /**
+   * Parse KeyValueContainerData and verify checksum.
+   * @param containerData
+   * @param containerFile
+   * @param checksumFile
+   * @param dbFile
+   * @param config
+   * @throws IOException
+   */
+  public static void parseKeyValueContainerData(
+      KeyValueContainerData containerData, File containerFile, File
+      checksumFile, File dbFile, OzoneConfiguration config) throws IOException {
+
+    Preconditions.checkNotNull(containerData, "containerData cannot be null");
+    Preconditions.checkNotNull(containerFile, "containerFile cannot be null");
+    Preconditions.checkNotNull(checksumFile, "checksumFile cannot be null");
+    Preconditions.checkNotNull(dbFile, "dbFile cannot be null");
+    Preconditions.checkNotNull(config, "ozone config cannot be null");
+
+    long containerId = containerData.getContainerId();
+    String containerName = String.valueOf(containerId);
+    File metadataPath = new File(containerData.getMetadataPath());
+
+    Preconditions.checkNotNull(containerName, "container Name cannot be " +
+        "null");
+    Preconditions.checkNotNull(metadataPath, "metadata path cannot be " +
+        "null");
+
+    // Verify Checksum
+    String checksum = KeyValueContainerUtil.computeCheckSum(
+        containerData.getContainerId(), containerFile);
+    KeyValueContainerUtil.verifyCheckSum(containerId, checksumFile, checksum);
+
+    containerData.setDbFile(dbFile);
+
+    MetadataStore metadata = KeyUtils.getDB(containerData, config);
+    long bytesUsed = 0;
+    List<Map.Entry<byte[], byte[]>> liveKeys = metadata
+        .getRangeKVs(null, Integer.MAX_VALUE,
+            MetadataKeyFilters.getNormalKeyFilter());
+    bytesUsed = liveKeys.parallelStream().mapToLong(e-> {
+      KeyData keyData;
+      try {
+        keyData = KeyUtils.getKeyData(e.getValue());
+        return keyData.getSize();
+      } catch (IOException ex) {
+        return 0L;
+      }
+    }).sum();
+    containerData.setBytesUsed(bytesUsed);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52d1d960/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
new file mode 100644
index 0000000..68823bc
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.ozoneimpl;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.common.Storage;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+
+
+/**
+ * Class used to read .container files from Volume and build container map.
+ */
+public class ContainerReader implements Runnable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ContainerReader.class);
+  private File hddsVolumeDir;
+  private final ContainerSet containerSet;
+  private final OzoneConfiguration config;
+
+  ContainerReader(File volumeRoot, ContainerSet cset, OzoneConfiguration conf) {
+    Preconditions.checkNotNull(volumeRoot);
+    this.hddsVolumeDir = volumeRoot;
+    this.containerSet = cset;
+    this.config = conf;
+  }
+
+  @Override
+  public void run() {
+    try {
+      readVolume(hddsVolumeDir);
+    } catch (RuntimeException ex) {
+      LOG.info("Caught an Run time exception during reading container files" +
+          " from Volume {}", hddsVolumeDir);
+    }
+  }
+
+  public void readVolume(File hddsVolumeRootDir) {
+    Preconditions.checkNotNull(hddsVolumeRootDir, "hddsVolumeRootDir" +
+        "cannot be null");
+
+
+    /**
+     *
+     * layout of the container directory on the disk.
+     * /hdds/<<scmUuid>>/current/<<containerdir>>/</containerID>/metadata
+     * /<<containerID>>.container
+     * /hdds/<<scmUuid>>/current/<<containerdir>>/<<containerID>>/metadata
+     * /<<containerID>>.checksum
+     * /hdds/<<scmUuid>>/current/<<containerdir>>/<<containerID>>/metadata
+     * /<<containerID>>.db
+     * /hdds/<<scmUuid>>/current/<<containerdir>>/<<containerID>>/chunks
+     * /<<chunkFile>>
+     *
+     **/
+
+    //filtering scm directory
+    File[] scmDir = hddsVolumeRootDir.listFiles(new FileFilter() {
+      @Override
+      public boolean accept(File pathname) {
+        return pathname.isDirectory();
+      }
+    });
+
+    for (File scmLoc : scmDir) {
+      File currentDir = null;
+      currentDir = new File(scmLoc, Storage.STORAGE_DIR_CURRENT);
+      File[] containerTopDirs = currentDir.listFiles();
+      if (containerTopDirs != null) {
+        for (File containerTopDir : containerTopDirs) {
+          if (containerTopDir.isDirectory()) {
+            File[] containerDirs = containerTopDir.listFiles();
+            for (File containerDir : containerDirs) {
+              File metadataPath = new File(containerDir + File.separator +
+                  OzoneConsts.CONTAINER_META_PATH);
+              String containerName = containerDir.getName();
+              if (metadataPath.exists()) {
+                File containerFile = KeyValueContainerLocationUtil
+                    .getContainerFile(metadataPath, containerName);
+                File checksumFile = KeyValueContainerLocationUtil
+                    .getContainerCheckSumFile(metadataPath, containerName);
+                File dbFile = KeyValueContainerLocationUtil
+                    .getContainerDBFile(metadataPath, containerName);
+                if (containerFile.exists() && checksumFile.exists() &&
+                    dbFile.exists()) {
+                  verifyContainerFile(containerFile, checksumFile, dbFile);
+                } else {
+                  LOG.error("Missing container metadata files for Container: " +
+                      "{}", containerName);
+                }
+              } else {
+                LOG.error("Missing container metadata directory for " +
+                    "Container: {}", containerName);
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+
+  private void verifyContainerFile(File containerFile, File checksumFile,
+                                   File dbFile) {
+    try {
+      ContainerData containerData =  ContainerDataYaml.readContainerFile(
+          containerFile);
+
+      switch (containerData.getContainerType()) {
+      case KeyValueContainer:
+        KeyValueContainerData keyValueContainerData = (KeyValueContainerData)
+            containerData;
+        KeyValueContainerUtil.parseKeyValueContainerData(keyValueContainerData,
+            containerFile, checksumFile, dbFile, config);
+        KeyValueContainer keyValueContainer = new KeyValueContainer(
+            keyValueContainerData, config);
+        containerSet.addContainer(keyValueContainer);
+        break;
+      default:
+        LOG.error("Unrecognized ContainerType {} format during verify " +
+            "ContainerFile", containerData.getContainerType());
+      }
+    } catch (IOException ex) {
+      LOG.error("Error during reading container file {}", containerFile);
+    }
+  }
+
+}


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


[31/37] hadoop git commit: HDDS-204. Modify Integration tests for new ContainerIO classes. Contributed by Bharat Viswanadham

Posted by bh...@apache.org.
HDDS-204. Modify Integration tests for new ContainerIO classes. Contributed by Bharat Viswanadham


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

Branch: refs/heads/trunk
Commit: 7dcf5878a40bcaff815527af85bfd1302bae6d32
Parents: 2c2351e
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Fri Jul 6 11:01:33 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Fri Jul 6 11:01:33 2018 -0700

----------------------------------------------------------------------
 .../container/keyvalue/KeyValueHandler.java     |  8 +++
 .../keyvalue/helpers/KeyValueContainerUtil.java | 10 ----
 .../container/ozoneimpl/ContainerReader.java    | 18 +++++--
 .../container/ozoneimpl/OzoneContainer.java     |  2 +-
 .../container/keyvalue/TestKeyValueHandler.java | 51 +-------------------
 ...TestGenerateOzoneRequiredConfigurations.java |  5 +-
 .../ozone/scm/TestContainerSmallFile.java       |  2 +-
 .../hadoop/ozone/web/client/TestKeys.java       |  2 +
 8 files changed, 31 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7dcf5878/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index b2c82f0..110d0b4 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -286,6 +286,14 @@ public class KeyValueHandler extends Handler {
         throw new StorageContainerException(
             "Deletion of Open Container is not allowed.",
             DELETE_ON_OPEN_CONTAINER);
+      } else if (!forceDelete && kvContainer.getContainerData().getKeyCount()
+          > 0) {
+        // If the container is not empty and cannot be deleted forcibly,
+        // then throw a SCE to stop deleting.
+        kvContainer.writeUnlock();
+        throw new StorageContainerException(
+            "Container cannot be deleted because it is not empty.",
+            ContainerProtos.Result.ERROR_CONTAINER_NOT_EMPTY);
       } else {
         containerSet.removeContainer(
             kvContainer.getContainerData().getContainerID());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7dcf5878/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
index 3c3c9cb..1856111 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
@@ -130,16 +130,6 @@ public final class KeyValueContainerUtil {
         .getMetadataPath());
     File chunksPath = new File(containerData.getChunksPath());
 
-    MetadataStore db = KeyUtils.getDB(containerData, conf);
-
-    // If the container is not empty and cannot be deleted forcibly,
-    // then throw a SCE to stop deleting.
-    if(!forceDelete && !db.isEmpty()) {
-      throw new StorageContainerException(
-          "Container cannot be deleted because it is not empty.",
-          ContainerProtos.Result.ERROR_CONTAINER_NOT_EMPTY);
-    }
-
     // Close the DB connection and remove the DB handler from cache
     KeyUtils.removeDB(containerData, conf);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7dcf5878/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
index 50a2d08..b90efdc 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.Storage;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
@@ -44,13 +45,16 @@ public class ContainerReader implements Runnable {
 
   private static final Logger LOG = LoggerFactory.getLogger(
       ContainerReader.class);
-  private File hddsVolumeDir;
+  private HddsVolume hddsVolume;
   private final ContainerSet containerSet;
   private final OzoneConfiguration config;
+  private final File hddsVolumeDir;
 
-  ContainerReader(File volumeRoot, ContainerSet cset, OzoneConfiguration conf) {
-    Preconditions.checkNotNull(volumeRoot);
-    this.hddsVolumeDir = volumeRoot;
+  ContainerReader(HddsVolume volume, ContainerSet cset, OzoneConfiguration
+      conf) {
+    Preconditions.checkNotNull(volume);
+    this.hddsVolume = volume;
+    this.hddsVolumeDir = hddsVolume.getHddsRootDir();
     this.containerSet = cset;
     this.config = conf;
   }
@@ -92,6 +96,11 @@ public class ContainerReader implements Runnable {
       }
     });
 
+    if (scmDir == null) {
+      LOG.error("Volume {} is empty with out metadata and chunks",
+          hddsVolumeRootDir);
+      return;
+    }
     for (File scmLoc : scmDir) {
       File currentDir = null;
       currentDir = new File(scmLoc, Storage.STORAGE_DIR_CURRENT);
@@ -137,6 +146,7 @@ public class ContainerReader implements Runnable {
       case KeyValueContainer:
         KeyValueContainerData keyValueContainerData = (KeyValueContainerData)
             containerData;
+        containerData.setVolume(hddsVolume);
         File dbFile = KeyValueContainerLocationUtil
             .getContainerDBFile(new File(containerFile.getParent()),
                 containerName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7dcf5878/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
index 667ea5c..8c3a0a2 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
@@ -106,7 +106,7 @@ public class OzoneContainer {
     while (volumeSetIterator.hasNext()) {
       HddsVolume volume = volumeSetIterator.next();
       File hddsVolumeRootDir = volume.getHddsRootDir();
-      Thread thread = new Thread(new ContainerReader(hddsVolumeRootDir,
+      Thread thread = new Thread(new ContainerReader(volume,
           containerSet, config));
       thread.start();
       volumeThreads.add(thread);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7dcf5878/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
index a997145..42ec54f 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
@@ -18,21 +18,13 @@
 
 package org.apache.hadoop.ozone.container.keyvalue;
 
-import com.google.common.base.Supplier;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
-import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
-import org.apache.hadoop.ozone.container.common.interfaces.Container;
-import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestRule;
@@ -55,31 +47,14 @@ public class TestKeyValueHandler {
   @Rule
   public TestRule timeout = new Timeout(300000);
 
-  private Configuration conf;
   private HddsDispatcher dispatcher;
-  private ContainerSet containerSet;
-  private VolumeSet volumeSet;
   private KeyValueHandler handler;
 
-  private final static String SCM_ID = UUID.randomUUID().toString();
   private final static String DATANODE_UUID = UUID.randomUUID().toString();
-  private int containerID;
 
   private final String baseDir = MiniDFSCluster.getBaseDirectory();
   private final String volume = baseDir + "disk1";
 
-  private void setup() throws Exception {
-    this.conf = new Configuration();
-    conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, volume);
-
-    this.containerSet = new ContainerSet();
-    this.volumeSet = new VolumeSet(DATANODE_UUID, conf);
-
-    this.dispatcher = new HddsDispatcher(conf, containerSet, volumeSet);
-    this.handler = (KeyValueHandler) dispatcher.getHandler(
-        ContainerProtos.ContainerType.KeyValueContainer);
-    dispatcher.setScmId(UUID.randomUUID().toString());
-  }
 
   @Test
   /**
@@ -222,29 +197,5 @@ public class TestKeyValueHandler {
     return request;
   }
 
-  @Test
-  public void testCreateContainer() throws Exception {
-    setup();
-
-    long contId = ++containerID;
-    ContainerProtos.CreateContainerRequestProto createReq =
-        ContainerProtos.CreateContainerRequestProto.newBuilder()
-            .setContainerID(contId)
-            .build();
-
-    ContainerCommandRequestProto request =
-        ContainerProtos.ContainerCommandRequestProto.newBuilder()
-            .setCmdType(ContainerProtos.Type.CreateContainer)
-            .setDatanodeUuid(DATANODE_UUID)
-            .setCreateContainer(createReq)
-            .build();
-
-    dispatcher.dispatch(request);
 
-    // Verify that new container is added to containerSet.
-    Container container = containerSet.getContainer(contId);
-    Assert.assertEquals(contId, container.getContainerData().getContainerID());
-    Assert.assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN,
-        container.getContainerState());
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7dcf5878/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/genconf/TestGenerateOzoneRequiredConfigurations.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/genconf/TestGenerateOzoneRequiredConfigurations.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/genconf/TestGenerateOzoneRequiredConfigurations.java
index c2f5eb7..75c15a4 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/genconf/TestGenerateOzoneRequiredConfigurations.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/genconf/TestGenerateOzoneRequiredConfigurations.java
@@ -107,7 +107,9 @@ public class TestGenerateOzoneRequiredConfigurations {
    */
   @Test
   public void generateConfigurationsFailure() throws Exception {
-    String[] args = new String[]{"-output", "/"};
+    File tempPath = getRandomTempDir();
+    tempPath.setReadOnly();
+    String[] args = new String[]{"-output", tempPath.getAbsolutePath()};
     GenerateOzoneRequiredConfigurations.main(args);
 
     Assert.assertEquals("Path is valid",
@@ -118,6 +120,7 @@ public class TestGenerateOzoneRequiredConfigurations {
 
     Assert.assertEquals("Config file not generated",
         1, GenerateOzoneRequiredConfigurations.generateConfigurations(args[1]));
+    tempPath.setWritable(true);
   }
 
   private File getRandomTempDir() throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7dcf5878/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java
index ce1fe46..5c62803 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java
@@ -143,7 +143,7 @@ public class TestContainerSmallFile {
 
 
     thrown.expect(StorageContainerException.class);
-    thrown.expectMessage("Unable to find the container");
+    thrown.expectMessage("ContainerID 8888 does not exist");
 
     // Try to read a invalid key
     ContainerProtos.GetSmallFileResponseProto response =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7dcf5878/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
index cda54cb..2f592c1 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
@@ -67,6 +67,7 @@ import org.apache.log4j.Logger;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
@@ -662,6 +663,7 @@ public class TestKeys {
   }
 
   @Test
+  @Ignore("Needs to be fixed for new SCM and Storage design")
   public void testDeleteKey() throws Exception {
     KeySpaceManager ksm = ozoneCluster.getKeySpaceManager();
     // To avoid interference from other test cases,


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


[23/37] hadoop git commit: HDDS-169:Add Volume IO Stats. Contributed by Bharat Viswanadham

Posted by bh...@apache.org.
HDDS-169:Add Volume IO Stats. Contributed by Bharat Viswanadham


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

Branch: refs/heads/trunk
Commit: 23ebe0f4f0073816b0639bed5800e77b08b63681
Parents: 84ac6bb
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Thu Jun 28 17:10:10 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Thu Jun 28 17:10:10 2018 -0700

----------------------------------------------------------------------
 .../container/common/impl/ContainerData.java    |  20 +++
 .../container/common/volume/HddsVolume.java     |   6 +
 .../container/common/volume/VolumeIOStats.java  | 139 +++++++++++++++++++
 .../container/keyvalue/KeyValueContainer.java   |   1 +
 .../container/keyvalue/helpers/ChunkUtils.java  |  30 +++-
 .../keyvalue/impl/ChunkManagerImpl.java         |  26 ++--
 .../keyvalue/TestChunkManagerImpl.java          | 111 +++++++++++----
 .../container/keyvalue/TestKeyManagerImpl.java  |  29 ++--
 8 files changed, 308 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/23ebe0f4/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
index b11b66c..872d958 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
     ContainerType;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
     ContainerLifeCycleState;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 
 import java.io.IOException;
 import java.util.Collections;
@@ -59,6 +60,8 @@ public class ContainerData {
   private final AtomicLong writeCount;
   private final AtomicLong bytesUsed;
 
+  private HddsVolume volume;
+
 
   /**
    * Creates a ContainerData Object, which holds metadata of the container.
@@ -289,5 +292,22 @@ public class ContainerData {
     return this.bytesUsed.addAndGet(-1L * reclaimed);
   }
 
+  /**
+   * Set the Volume for the Container.
+   * This should be called only from the createContainer.
+   * @param hddsVolume
+   */
+  public void setVolume(HddsVolume hddsVolume) {
+    this.volume = hddsVolume;
+  }
+
+  /**
+   * Returns the volume of the Container.
+   * @return HddsVolume
+   */
+  public HddsVolume getVolume() {
+    return volume;
+  }
+
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23ebe0f4/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
index 9006133..1e71494 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java
@@ -52,6 +52,7 @@ public final class HddsVolume {
   private final File hddsRootDir;
   private final VolumeInfo volumeInfo;
   private VolumeState state;
+  private final VolumeIOStats volumeIOStats;
 
   // VERSION file properties
   private String storageID;       // id of the file system
@@ -117,6 +118,7 @@ public final class HddsVolume {
     this.state = VolumeState.NOT_INITIALIZED;
     this.clusterID = b.clusterID;
     this.datanodeUuid = b.datanodeUuid;
+    this.volumeIOStats = new VolumeIOStats();
 
     VolumeInfo.Builder volumeBuilder =
         new VolumeInfo.Builder(b.volumeRootStr, b.conf)
@@ -303,6 +305,10 @@ public final class HddsVolume {
     return (state == VolumeState.FAILED);
   }
 
+  public VolumeIOStats getVolumeIOStats() {
+    return volumeIOStats;
+  }
+
   public void failVolume() {
     setState(VolumeState.FAILED);
     volumeInfo.shutdownUsageThread();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23ebe0f4/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeIOStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeIOStats.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeIOStats.java
new file mode 100644
index 0000000..9e2eb22
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeIOStats.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.volume;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This class is used to track Volume IO stats for each HDDS Volume.
+ */
+public class VolumeIOStats {
+
+  private final AtomicLong readBytes;
+  private final AtomicLong readOpCount;
+  private final AtomicLong writeBytes;
+  private final AtomicLong writeOpCount;
+  private final AtomicLong readTime;
+  private final AtomicLong writeTime;
+
+  public VolumeIOStats() {
+    readBytes = new AtomicLong(0);
+    readOpCount = new AtomicLong(0);
+    writeBytes = new AtomicLong(0);
+    writeOpCount = new AtomicLong(0);
+    readTime = new AtomicLong(0);
+    writeTime = new AtomicLong(0);
+  }
+
+  /**
+   * Increment number of bytes read from the volume.
+   * @param bytesRead
+   */
+  public void incReadBytes(long bytesRead) {
+    readBytes.addAndGet(bytesRead);
+  }
+
+  /**
+   * Increment the read operations performed on the volume.
+   */
+  public void incReadOpCount() {
+    readOpCount.incrementAndGet();
+  }
+
+  /**
+   * Increment number of bytes written on to the volume.
+   * @param bytesWritten
+   */
+  public void incWriteBytes(long bytesWritten) {
+    writeBytes.addAndGet(bytesWritten);
+  }
+
+  /**
+   * Increment the write operations performed on the volume.
+   */
+  public void incWriteOpCount() {
+    writeOpCount.incrementAndGet();
+  }
+
+  /**
+   * Increment the time taken by read operation on the volume.
+   * @param time
+   */
+  public void incReadTime(long time) {
+    readTime.addAndGet(time);
+  }
+
+  /**
+   * Increment the time taken by write operation on the volume.
+   * @param time
+   */
+  public void incWriteTime(long time) {
+    writeTime.addAndGet(time);
+  }
+
+  /**
+   * Returns total number of bytes read from the volume.
+   * @return long
+   */
+  public long getReadBytes() {
+    return readBytes.get();
+  }
+
+  /**
+   * Returns total number of bytes written to the volume.
+   * @return long
+   */
+  public long getWriteBytes() {
+    return writeBytes.get();
+  }
+
+  /**
+   * Returns total number of read operations performed on the volume.
+   * @return long
+   */
+  public long getReadOpCount() {
+    return readOpCount.get();
+  }
+
+  /**
+   * Returns total number of write operations performed on the volume.
+   * @return long
+   */
+  public long getWriteOpCount() {
+    return writeOpCount.get();
+  }
+
+  /**
+   * Returns total read operations time on the volume.
+   * @return long
+   */
+  public long getReadTime() {
+    return readTime.get();
+  }
+
+  /**
+   * Returns total write operations time on the volume.
+   * @return long
+   */
+  public long getWriteTime() {
+    return writeTime.get();
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23ebe0f4/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index 553e3f5..72d50d6 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -145,6 +145,7 @@ public class KeyValueContainer implements Container {
       containerData.setChunksPath(chunksPath.getPath());
       containerData.setContainerDBType(impl);
       containerData.setDbFile(dbFile);
+      containerData.setVolume(containerVolume);
 
       // Create .container file and .chksm file
       createContainerFile(containerFile, containerCheckSumFile);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23ebe0f4/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
index 872d84d..3529af8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
@@ -36,6 +36,8 @@ import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.hadoop.ozone.container.common.volume.VolumeIOStats;
+import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -67,10 +69,11 @@ public final class ChunkUtils {
    * @param chunkFile - File to write data to.
    * @param chunkInfo - Data stream to write.
    * @param data - The data buffer.
+   * @param volumeIOStats
    * @throws StorageContainerException
    */
   public static void writeData(File chunkFile, ChunkInfo chunkInfo,
-                               byte[] data) throws
+                               byte[] data, VolumeIOStats volumeIOStats) throws
       StorageContainerException, ExecutionException, InterruptedException,
       NoSuchAlgorithmException {
 
@@ -87,6 +90,12 @@ public final class ChunkUtils {
     FileLock lock = null;
 
     try {
+      if (chunkInfo.getChecksum() != null &&
+          !chunkInfo.getChecksum().isEmpty()) {
+        verifyChecksum(chunkInfo, data, log);
+      }
+
+      long writeTimeStart = Time.monotonicNow();
       file =
           AsynchronousFileChannel.open(chunkFile.toPath(),
               StandardOpenOption.CREATE,
@@ -94,11 +103,11 @@ public final class ChunkUtils {
               StandardOpenOption.SPARSE,
               StandardOpenOption.SYNC);
       lock = file.lock().get();
-      if (chunkInfo.getChecksum() != null &&
-          !chunkInfo.getChecksum().isEmpty()) {
-        verifyChecksum(chunkInfo, data, log);
-      }
       int size = file.write(ByteBuffer.wrap(data), chunkInfo.getOffset()).get();
+      // Increment volumeIO stats here.
+      volumeIOStats.incWriteTime(Time.monotonicNow() - writeTimeStart);
+      volumeIOStats.incWriteOpCount();
+      volumeIOStats.incWriteBytes(size);
       if (size != data.length) {
         log.error("Invalid write size found. Size:{}  Expected: {} ", size,
             data.length);
@@ -136,12 +145,15 @@ public final class ChunkUtils {
    *
    * @param chunkFile - file where data lives.
    * @param data - chunk definition.
+   * @param volumeIOStats
    * @return ByteBuffer
    * @throws StorageContainerException
    * @throws ExecutionException
    * @throws InterruptedException
    */
-  public static ByteBuffer readData(File chunkFile, ChunkInfo data) throws
+  public static ByteBuffer readData(File chunkFile, ChunkInfo data,
+                                    VolumeIOStats volumeIOStats)
+      throws
       StorageContainerException, ExecutionException, InterruptedException,
       NoSuchAlgorithmException {
     Logger log = LoggerFactory.getLogger(ChunkManagerImpl.class);
@@ -157,6 +169,7 @@ public final class ChunkUtils {
     AsynchronousFileChannel file = null;
     FileLock lock = null;
     try {
+      long readStartTime = Time.monotonicNow();
       file =
           AsynchronousFileChannel.open(chunkFile.toPath(),
               StandardOpenOption.READ);
@@ -165,10 +178,13 @@ public final class ChunkUtils {
       ByteBuffer buf = ByteBuffer.allocate((int) data.getLen());
       file.read(buf, data.getOffset()).get();
 
+      // Increment volumeIO stats here.
+      volumeIOStats.incReadTime(Time.monotonicNow() - readStartTime);
+      volumeIOStats.incReadOpCount();
+      volumeIOStats.incReadBytes(data.getLen());
       if (data.getChecksum() != null && !data.getChecksum().isEmpty()) {
         verifyChecksum(data, buf.array(), log);
       }
-
       return buf;
     } catch (IOException e) {
       throw new StorageContainerException(e, IO_EXCEPTION);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23ebe0f4/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java
index 258d4da..c3160a8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerImpl.java
@@ -26,6 +26,8 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerExcep
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.VolumeIOStats;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils;
 import org.apache.hadoop.ozone.container.common.impl.ChunkLayOutVersion;
 import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
@@ -64,13 +66,15 @@ public class ChunkManagerImpl implements ChunkManager {
    * @throws StorageContainerException
    */
   public void writeChunk(Container container, BlockID blockID, ChunkInfo info,
-                         byte[] data, ContainerProtos.Stage stage)
+      byte[] data, ContainerProtos.Stage stage)
       throws StorageContainerException {
 
     try {
 
       KeyValueContainerData containerData = (KeyValueContainerData) container
           .getContainerData();
+      HddsVolume volume = containerData.getVolume();
+      VolumeIOStats volumeIOStats = volume.getVolumeIOStats();
 
       File chunkFile = ChunkUtils.validateChunk(containerData, info);
       File tmpChunkFile = getTmpChunkFile(chunkFile, info);
@@ -81,20 +85,23 @@ public class ChunkManagerImpl implements ChunkManager {
       switch (stage) {
       case WRITE_DATA:
         // Initially writes to temporary chunk file.
-        ChunkUtils.writeData(tmpChunkFile, info, data);
+        ChunkUtils.writeData(tmpChunkFile, info, data, volumeIOStats);
+        // No need to increment container stats here, as still data is not
+        // committed here.
         break;
       case COMMIT_DATA:
         // commit the data, means move chunk data from temporary chunk file
         // to actual chunk file.
-        long sizeDiff = tmpChunkFile.length() - chunkFile.length();
         commitChunk(tmpChunkFile, chunkFile);
-        containerData.incrBytesUsed(sizeDiff);
+        // Increment container stats here, as we commit the data.
+        containerData.incrBytesUsed(info.getLen());
         containerData.incrWriteCount();
-        containerData.incrWriteBytes(sizeDiff);
+        containerData.incrWriteBytes(info.getLen());
         break;
       case COMBINED:
         // directly write to the chunk file
-        ChunkUtils.writeData(chunkFile, info, data);
+        ChunkUtils.writeData(chunkFile, info, data, volumeIOStats);
+        // Increment container stats here, as we directly write to chunk file.
         containerData.incrBytesUsed(info.getLen());
         containerData.incrWriteCount();
         containerData.incrWriteBytes(info.getLen());
@@ -137,6 +144,8 @@ public class ChunkManagerImpl implements ChunkManager {
       KeyValueContainerData containerData = (KeyValueContainerData) container
           .getContainerData();
       ByteBuffer data;
+      HddsVolume volume = containerData.getVolume();
+      VolumeIOStats volumeIOStats = volume.getVolumeIOStats();
 
       // Checking here, which layout version the container is, and reading
       // the chunk file in that format.
@@ -145,9 +154,10 @@ public class ChunkManagerImpl implements ChunkManager {
       if (containerData.getLayOutVersion() == ChunkLayOutVersion
           .getLatestVersion().getVersion()) {
         File chunkFile = ChunkUtils.getChunkFile(containerData, info);
-        data = ChunkUtils.readData(chunkFile, info);
+        data = ChunkUtils.readData(chunkFile, info, volumeIOStats);
         containerData.incrReadCount();
-        containerData.incrReadBytes(chunkFile.length());
+        long length = chunkFile.length();
+        containerData.incrReadBytes(length);
         return data.array();
       }
     } catch(NoSuchAlgorithmException ex) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23ebe0f4/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
index 272bdb9..6becf39 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestChunkManagerImpl.java
@@ -22,10 +22,12 @@ import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
-import org.apache.hadoop.ozone.container.common.helpers.KeyData;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.volume.VolumeIOStats;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerImpl;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -37,13 +39,9 @@ import org.mockito.Mockito;
 
 import java.io.File;
 import java.util.Arrays;
-import java.util.LinkedList;
-import java.util.List;
 import java.util.UUID;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 import static org.mockito.ArgumentMatchers.anyList;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.Mockito.mock;
@@ -57,21 +55,21 @@ public class TestChunkManagerImpl {
   private String scmId = UUID.randomUUID().toString();
   private VolumeSet volumeSet;
   private RoundRobinVolumeChoosingPolicy volumeChoosingPolicy;
+  private HddsVolume hddsVolume;
   private KeyValueContainerData keyValueContainerData;
   private KeyValueContainer keyValueContainer;
-  private KeyData keyData;
   private BlockID blockID;
   private ChunkManagerImpl chunkManager;
   private ChunkInfo chunkInfo;
   private byte[] data;
+
   @Rule
   public TemporaryFolder folder = new TemporaryFolder();
 
   @Before
   public void setUp() throws Exception {
     config = new OzoneConfiguration();
-
-    HddsVolume hddsVolume = new HddsVolume.Builder(folder.getRoot()
+    hddsVolume = new HddsVolume.Builder(folder.getRoot()
         .getAbsolutePath()).conf(config).datanodeUuid(UUID.randomUUID()
         .toString()).build();
 
@@ -83,22 +81,15 @@ public class TestChunkManagerImpl {
 
     keyValueContainerData = new KeyValueContainerData(1L);
 
-    keyValueContainer = new KeyValueContainer(
-        keyValueContainerData, config);
+    keyValueContainer = new KeyValueContainer(keyValueContainerData, config);
 
     keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
 
     data = "testing write chunks".getBytes();
     // Creating KeyData
     blockID = new BlockID(1L, 1L);
-    keyData = new KeyData(blockID);
-    keyData.addMetadata("VOLUME", "ozone");
-    keyData.addMetadata("OWNER", "hdfs");
-    List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
     chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
         .getLocalID(), 0), 0, data.length);
-    chunkList.add(chunkInfo.getProtoBufMessage());
-    keyData.setChunks(chunkList);
 
     // Create a ChunkManager object.
     chunkManager = new ChunkManagerImpl();
@@ -113,16 +104,38 @@ public class TestChunkManagerImpl {
     assertTrue(chunksPath.exists());
     // Initially chunks folder should be empty.
     assertTrue(chunksPath.listFiles().length == 0);
+
+    // As no chunks are written to the volume writeBytes should be 0
+    checkWriteIOStats(0, 0);
     chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
         ContainerProtos.Stage.WRITE_DATA);
     // Now a chunk file is being written with Stage WRITE_DATA, so it should
     // create a temporary chunk file.
     assertTrue(chunksPath.listFiles().length == 1);
+
+    File chunkFile = ChunkUtils.getChunkFile(keyValueContainerData, chunkInfo);
+    File tempChunkFile = new File(chunkFile.getParent(),
+        chunkFile.getName() +
+            OzoneConsts.CONTAINER_CHUNK_NAME_DELIMITER +
+            OzoneConsts.CONTAINER_TEMPORARY_CHUNK_PREFIX);
+
+    // As chunk write stage is WRITE_DATA, temp chunk file will be created.
+    assertTrue(tempChunkFile.exists());
+
+    checkWriteIOStats(data.length, 1);
+
     chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
         ContainerProtos.Stage.COMMIT_DATA);
+
+    checkWriteIOStats(data.length, 1);
+
     // Old temp file should have been renamed to chunk file.
     assertTrue(chunksPath.listFiles().length == 1);
 
+    // As commit happened, chunk file should exist.
+    assertTrue(chunkFile.exists());
+    assertFalse(tempChunkFile.exists());
+
   }
 
   @Test
@@ -131,13 +144,12 @@ public class TestChunkManagerImpl {
       long randomLength = 200L;
       chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
           .getLocalID(), 0), 0, randomLength);
-      List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
-      chunkList.add(chunkInfo.getProtoBufMessage());
-      keyData.setChunks(chunkList);
       chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
           ContainerProtos.Stage.WRITE_DATA);
       fail("testWriteChunkIncorrectLength failed");
     } catch (StorageContainerException ex) {
+      // As we got an exception, writeBytes should be 0.
+      checkWriteIOStats(0, 0);
       GenericTestUtils.assertExceptionContains("data array does not match " +
           "the length ", ex);
       assertEquals(ContainerProtos.Result.INVALID_WRITE_SIZE, ex.getResult());
@@ -152,21 +164,29 @@ public class TestChunkManagerImpl {
     assertTrue(chunksPath.exists());
     // Initially chunks folder should be empty.
     assertTrue(chunksPath.listFiles().length == 0);
+    checkWriteIOStats(0, 0);
     chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
         ContainerProtos.Stage.COMBINED);
-    // Now a chunk file is being written with Stage WRITE_DATA, so it should
-    // create a temporary chunk file.
+    // Now a chunk file is being written with Stage COMBINED_DATA, so it should
+    // create a chunk file.
     assertTrue(chunksPath.listFiles().length == 1);
+    File chunkFile = ChunkUtils.getChunkFile(keyValueContainerData, chunkInfo);
+    assertTrue(chunkFile.exists());
+    checkWriteIOStats(data.length, 1);
   }
 
   @Test
   public void testReadChunk() throws Exception {
+    checkWriteIOStats(0, 0);
     chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
         ContainerProtos.Stage.COMBINED);
+    checkWriteIOStats(data.length, 1);
+    checkReadIOStats(0, 0);
     byte[] expectedData = chunkManager.readChunk(keyValueContainer, blockID,
         chunkInfo);
     assertEquals(expectedData.length, data.length);
     assertTrue(Arrays.equals(expectedData, data));
+    checkReadIOStats(data.length, 1);
   }
 
   @Test
@@ -187,9 +207,6 @@ public class TestChunkManagerImpl {
       long randomLength = 200L;
       chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
           .getLocalID(), 0), 0, randomLength);
-      List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
-      chunkList.add(chunkInfo.getProtoBufMessage());
-      keyData.setChunks(chunkList);
       chunkManager.deleteChunk(keyValueContainer, blockID, chunkInfo);
       fail("testDeleteChunkUnsupportedRequest");
     } catch (StorageContainerException ex) {
@@ -205,9 +222,6 @@ public class TestChunkManagerImpl {
           .getLocalID(), 0), 0, data.length);
       //Setting checksum to some value.
       chunkInfo.setChecksum("some garbage");
-      List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
-      chunkList.add(chunkInfo.getProtoBufMessage());
-      keyData.setChunks(chunkList);
       chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
           ContainerProtos.Stage.COMBINED);
       fail("testWriteChunkChecksumMismatch failed");
@@ -231,5 +245,46 @@ public class TestChunkManagerImpl {
     }
   }
 
+  @Test
+  public void testWriteAndReadChunkMultipleTimes() throws Exception {
+    for (int i=0; i<100; i++) {
+      chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
+          .getLocalID(), i), 0, data.length);
+      chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
+          ContainerProtos.Stage.COMBINED);
+    }
+    checkWriteIOStats(data.length*100, 100);
+    assertTrue(hddsVolume.getVolumeIOStats().getWriteTime() > 0);
+
+    for (int i=0; i<100; i++) {
+      chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
+          .getLocalID(), i), 0, data.length);
+      chunkManager.readChunk(keyValueContainer, blockID, chunkInfo);
+    }
+    checkReadIOStats(data.length*100, 100);
+    assertTrue(hddsVolume.getVolumeIOStats().getReadTime() > 0);
+  }
+
 
+  /**
+   * Check WriteIO stats.
+   * @param length
+   * @param opCount
+   */
+  private void checkWriteIOStats(long length, long opCount) {
+    VolumeIOStats volumeIOStats = hddsVolume.getVolumeIOStats();
+    assertEquals(length, volumeIOStats.getWriteBytes());
+    assertEquals(opCount, volumeIOStats.getWriteOpCount());
+  }
+
+  /**
+   * Check ReadIO stats.
+   * @param length
+   * @param opCount
+   */
+  private void checkReadIOStats(long length, long opCount) {
+    VolumeIOStats volumeIOStats = hddsVolume.getVolumeIOStats();
+    assertEquals(length, volumeIOStats.getReadBytes());
+    assertEquals(opCount, volumeIOStats.getReadOpCount());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23ebe0f4/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
index fa7c66d..62d9382 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyManagerImpl.java
@@ -58,7 +58,7 @@ public class TestKeyManagerImpl {
   private KeyValueContainerData keyValueContainerData;
   private KeyValueContainer keyValueContainer;
   private KeyData keyData;
-  private KeyManagerImpl keyValueContainerManager;
+  private KeyManagerImpl keyManager;
   private BlockID blockID;
 
   @Rule
@@ -98,17 +98,17 @@ public class TestKeyManagerImpl {
     keyData.setChunks(chunkList);
 
     // Create KeyValueContainerManager
-    keyValueContainerManager = new KeyManagerImpl(config);
+    keyManager = new KeyManagerImpl(config);
 
   }
 
   @Test
   public void testPutAndGetKey() throws Exception {
     //Put Key
-    keyValueContainerManager.putKey(keyValueContainer, keyData);
+    keyManager.putKey(keyValueContainer, keyData);
 
     //Get Key
-    KeyData fromGetKeyData = keyValueContainerManager.getKey(keyValueContainer,
+    KeyData fromGetKeyData = keyManager.getKey(keyValueContainer,
         keyData.getBlockID());
 
     assertEquals(keyData.getContainerID(), fromGetKeyData.getContainerID());
@@ -124,9 +124,15 @@ public class TestKeyManagerImpl {
   public void testDeleteKey() throws Exception {
     try {
       //Put Key
-      keyValueContainerManager.putKey(keyValueContainer, keyData);
+      keyManager.putKey(keyValueContainer, keyData);
       //Delete Key
-      keyValueContainerManager.deleteKey(keyValueContainer, blockID);
+      keyManager.deleteKey(keyValueContainer, blockID);
+      try {
+        keyManager.getKey(keyValueContainer, blockID);
+        fail("testDeleteKey");
+      } catch (StorageContainerException ex) {
+        GenericTestUtils.assertExceptionContains("Unable to find the key", ex);
+      }
     } catch (IOException ex) {
       fail("testDeleteKey failed");
     }
@@ -135,8 +141,8 @@ public class TestKeyManagerImpl {
   @Test
   public void testListKey() throws Exception {
     try {
-      keyValueContainerManager.putKey(keyValueContainer, keyData);
-      List<KeyData> listKeyData = keyValueContainerManager.listKey(
+      keyManager.putKey(keyValueContainer, keyData);
+      List<KeyData> listKeyData = keyManager.listKey(
           keyValueContainer, 1, 10);
       assertNotNull(listKeyData);
       assertTrue(listKeyData.size() == 1);
@@ -151,10 +157,10 @@ public class TestKeyManagerImpl {
             .getLocalID(), 0), 0, 1024);
         chunkList.add(info.getProtoBufMessage());
         keyData.setChunks(chunkList);
-        keyValueContainerManager.putKey(keyValueContainer, keyData);
+        keyManager.putKey(keyValueContainer, keyData);
       }
 
-      listKeyData = keyValueContainerManager.listKey(
+      listKeyData = keyManager.listKey(
           keyValueContainer, 1, 10);
       assertNotNull(listKeyData);
       assertTrue(listKeyData.size() == 10);
@@ -167,7 +173,8 @@ public class TestKeyManagerImpl {
   @Test
   public void testGetNoSuchKey() throws Exception {
     try {
-      keyValueContainerManager.getKey(keyValueContainer, new BlockID(1L, 2L));
+      keyData = new KeyData(new BlockID(1L, 2L));
+      keyManager.getKey(keyValueContainer, new BlockID(1L, 2L));
       fail("testGetNoSuchKey failed");
     } catch (StorageContainerException ex) {
       GenericTestUtils.assertExceptionContains("Unable to find the key.", ex);


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


[06/37] hadoop git commit: HDDS-123:ContainerSet class to manage ContainerMap. Contributed by Bharat Viswanadham

Posted by bh...@apache.org.
HDDS-123:ContainerSet class to manage ContainerMap. Contributed by Bharat Viswanadham


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

Branch: refs/heads/trunk
Commit: 977c8cd1665b42ba45e9d21655262ef78714b716
Parents: f26d346
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Wed Jun 6 13:59:20 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Wed Jun 6 13:59:20 2018 -0700

----------------------------------------------------------------------
 .../container/common/impl/ContainerData.java    |  38 +++
 .../container/common/impl/ContainerSet.java     | 239 +++++++++++++++++++
 .../common/impl/KeyValueContainer.java          |   8 +-
 .../container/common/impl/TestContainerSet.java | 169 +++++++++++++
 4 files changed, 452 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/977c8cd1/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
index a4b2130..06aae66 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
@@ -56,6 +56,7 @@ public class ContainerData {
   private final AtomicLong writeBytes;
   private final AtomicLong readCount;
   private final AtomicLong writeCount;
+  private final AtomicLong bytesUsed;
 
 
   /**
@@ -73,6 +74,7 @@ public class ContainerData {
     this.readBytes =  new AtomicLong(0L);
     this.writeCount =  new AtomicLong(0L);
     this.writeBytes =  new AtomicLong(0L);
+    this.bytesUsed = new AtomicLong(0L);
   }
 
   /**
@@ -231,4 +233,40 @@ public class ContainerData {
     this.writeCount.incrementAndGet();
   }
 
+  /**
+   * Sets the number of bytes used by the container.
+   * @param used
+   */
+  public void setBytesUsed(long used) {
+    this.bytesUsed.set(used);
+  }
+
+  /**
+   * Get the number of bytes used by the container.
+   * @return the number of bytes used by the container.
+   */
+  public long getBytesUsed() {
+    return bytesUsed.get();
+  }
+
+  /**
+   * Increase the number of bytes used by the container.
+   * @param used number of bytes used by the container.
+   * @return the current number of bytes used by the container afert increase.
+   */
+  public long incrBytesUsed(long used) {
+    return this.bytesUsed.addAndGet(used);
+  }
+
+
+  /**
+   * Decrease the number of bytes used by the container.
+   * @param reclaimed the number of bytes reclaimed from the container.
+   * @return the current number of bytes used by the container after decrease.
+   */
+  public long decrBytesUsed(long reclaimed) {
+    return this.bytesUsed.addAndGet(-1L * reclaimed);
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/977c8cd1/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
new file mode 100644
index 0000000..79f038f
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
@@ -0,0 +1,239 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerInfo;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .StorageContainerException;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.INVALID_CONTAINER_STATE;
+
+/**
+ * Class that manages Containers created on the datanode.
+ */
+public class ContainerSet {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ContainerSet.class);
+
+  private final ConcurrentSkipListMap<Long, Container> containerMap = new
+      ConcurrentSkipListMap<>();
+
+  /**
+   * Add Container to container map.
+   * @param container
+   * @return If container is added to containerMap returns true, otherwise
+   * false
+   */
+  public boolean addContainer(Container container) throws
+      StorageContainerException {
+    Preconditions.checkNotNull(container, "container cannot be null");
+
+    long containerId = container.getContainerData().getContainerId();
+    if(containerMap.putIfAbsent(containerId, container) == null) {
+      LOG.debug("Container with container Id {} is added to containerMap",
+          containerId);
+      return true;
+    } else {
+      LOG.debug("Container already exists with container Id {}", containerId);
+      return false;
+    }
+  }
+
+  /**
+   * Returns the Container with specified containerId.
+   * @param containerId
+   * @return Container
+   */
+  public Container getContainer(long containerId) {
+    Preconditions.checkState(containerId >= 0,
+        "Container Id cannot be negative.");
+    return containerMap.get(containerId);
+  }
+
+  /**
+   * Removes the Container matching with specified containerId.
+   * @param containerId
+   * @return If container is removed from containerMap returns true, otherwise
+   * false
+   */
+  public boolean removeContainer(long containerId) {
+    Preconditions.checkState(containerId >= 0,
+        "Container Id cannot be negative.");
+    Container removed = containerMap.remove(containerId);
+    if(removed == null) {
+      LOG.debug("Container with containerId {} is not present in " +
+          "containerMap", containerId);
+      return false;
+    } else {
+      LOG.debug("Container with containerId {} is removed from containerMap",
+          containerId);
+      return true;
+    }
+  }
+
+  /**
+   * Return number of containers in container map.
+   * @return container count
+   */
+  @VisibleForTesting
+  public int containerCount() {
+    return containerMap.size();
+  }
+
+  /**
+   * Return an container Iterator over {@link ContainerSet#containerMap}.
+   * @return Iterator<Container>
+   */
+  public Iterator<Container> getContainerIterator() {
+    return containerMap.values().iterator();
+  }
+
+  /**
+   * Return an containerMap iterator over {@link ContainerSet#containerMap}.
+   * @return containerMap Iterator
+   */
+  public Iterator<Map.Entry<Long, Container>> getContainerMapIterator() {
+    return containerMap.entrySet().iterator();
+  }
+
+
+  /**
+   * A simple interface for container Iterations.
+   * <p/>
+   * This call make no guarantees about consistency of the data between
+   * different list calls. It just returns the best known data at that point of
+   * time. It is possible that using this iteration you can miss certain
+   * container from the listing.
+   *
+   * @param startContainerId -  Return containers with Id >= startContainerId.
+   * @param count - how many to return
+   * @param data - Actual containerData
+   * @throws StorageContainerException
+   */
+  public void listContainer(long startContainerId, long count,
+                            List<ContainerData> data) throws
+      StorageContainerException {
+    Preconditions.checkNotNull(data,
+        "Internal assertion: data cannot be null");
+    Preconditions.checkState(startContainerId >= 0,
+        "Start container Id cannot be negative");
+    Preconditions.checkState(count > 0,
+        "max number of containers returned " +
+            "must be positive");
+    LOG.debug("listContainer returns containerData starting from {} of count " +
+        "{}", startContainerId, count);
+    ConcurrentNavigableMap<Long, Container> map;
+    if (startContainerId == 0) {
+      map = containerMap.tailMap(containerMap.firstKey(), true);
+    } else {
+      map = containerMap.tailMap(startContainerId, true);
+    }
+    int currentCount = 0;
+    for (Container entry : map.values()) {
+      if (currentCount < count) {
+        data.add(entry.getContainerData());
+        currentCount++;
+      } else {
+        return;
+      }
+    }
+  }
+
+  /**
+   * Get container report.
+   *
+   * @return The container report.
+   * @throws IOException
+   */
+  public ContainerReportsProto getContainerReport() throws IOException {
+    LOG.debug("Starting container report iteration.");
+
+    // No need for locking since containerMap is a ConcurrentSkipListMap
+    // And we can never get the exact state since close might happen
+    // after we iterate a point.
+    List<Container> containers = containerMap.values().stream().collect(
+        Collectors.toList());
+
+    ContainerReportsProto.Builder crBuilder =
+        ContainerReportsProto.newBuilder();
+
+
+    for (Container container: containers) {
+      long containerId = container.getContainerData().getContainerId();
+      ContainerInfo.Builder ciBuilder = ContainerInfo.newBuilder();
+      ContainerData containerData = container.getContainerData();
+      ciBuilder.setContainerID(containerId)
+          .setReadCount(containerData.getReadCount())
+          .setWriteCount(containerData.getWriteCount())
+          .setReadBytes(containerData.getReadBytes())
+          .setWriteBytes(containerData.getWriteBytes())
+          .setUsed(containerData.getBytesUsed())
+          .setState(getState(containerData));
+
+      crBuilder.addReports(ciBuilder.build());
+    }
+
+    return crBuilder.build();
+  }
+
+  /**
+   * Returns LifeCycle State of the container.
+   * @param containerData - ContainerData
+   * @return LifeCycle State of the container
+   * @throws StorageContainerException
+   */
+  private HddsProtos.LifeCycleState getState(ContainerData containerData)
+      throws StorageContainerException {
+    HddsProtos.LifeCycleState state;
+    switch (containerData.getState()) {
+    case OPEN:
+      state = HddsProtos.LifeCycleState.OPEN;
+      break;
+    case CLOSING:
+      state = HddsProtos.LifeCycleState.CLOSING;
+      break;
+    case CLOSED:
+      state = HddsProtos.LifeCycleState.CLOSED;
+      break;
+    default:
+      throw new StorageContainerException("Invalid Container state found: " +
+          containerData.getContainerId(), INVALID_CONTAINER_STATE);
+    }
+    return state;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/977c8cd1/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java
index 956840b..a35845d 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/KeyValueContainer.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.container.common.impl;
 
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 
 
@@ -40,12 +41,15 @@ public class KeyValueContainer implements Container {
   private KeyValueContainerData containerData;
 
   public KeyValueContainer(KeyValueContainerData containerData) {
+    Preconditions.checkNotNull(containerData, "KeyValueContainerData cannot " +
+        "be null");
     this.containerData = containerData;
   }
 
   @Override
   public void create(ContainerData cData) throws StorageContainerException {
 
+
   }
 
   @Override
@@ -61,8 +65,8 @@ public class KeyValueContainer implements Container {
   }
 
   @Override
-  public ContainerData getContainerData() throws StorageContainerException {
-    return null;
+  public ContainerData getContainerData()  {
+    return containerData;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/977c8cd1/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
new file mode 100644
index 0000000..235a32f
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerSet.java
@@ -0,0 +1,169 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.ozone.container.common.impl;
+
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Class used to test ContainerSet operations.
+ */
+public class TestContainerSet {
+
+  @Test
+  public void testAddGetRemoveContainer() throws StorageContainerException {
+    ContainerSet containerSet = new ContainerSet();
+    long containerId = 100L;
+    ContainerProtos.ContainerLifeCycleState state = ContainerProtos
+        .ContainerLifeCycleState.CLOSED;
+
+    KeyValueContainerData kvData = new KeyValueContainerData(
+        ContainerProtos.ContainerType.KeyValueContainer, containerId);
+    kvData.setState(state);
+    KeyValueContainer keyValueContainer = new KeyValueContainer(kvData);
+
+    //addContainer
+    boolean result = containerSet.addContainer(keyValueContainer);
+    assertTrue(result);
+    result = containerSet.addContainer(keyValueContainer);
+    assertFalse(result);
+
+    //getContainer
+    KeyValueContainer container = (KeyValueContainer) containerSet
+        .getContainer(containerId);
+    KeyValueContainerData keyValueContainerData = (KeyValueContainerData)
+        container.getContainerData();
+    assertEquals(containerId, keyValueContainerData.getContainerId());
+    assertEquals(state, keyValueContainerData.getState());
+    assertNull(containerSet.getContainer(1000L));
+
+    //removeContainer
+    assertTrue(containerSet.removeContainer(containerId));
+    assertFalse(containerSet.removeContainer(1000L));
+  }
+
+  @Test
+  public void testIteratorsAndCount() throws StorageContainerException {
+
+    ContainerSet containerSet = createContainerSet();
+
+    assertEquals(10, containerSet.containerCount());
+
+    // Using containerIterator.
+    Iterator<Container> containerIterator = containerSet.getContainerIterator();
+
+    int count = 0;
+    while(containerIterator.hasNext()) {
+      Container kv = containerIterator.next();
+      ContainerData containerData = kv.getContainerData();
+      long containerId = containerData.getContainerId();
+      if (containerId%2 == 0) {
+        assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED,
+            containerData.getState());
+      } else {
+        assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN,
+            containerData.getState());
+      }
+      count++;
+    }
+    assertEquals(10, count);
+
+    //Using containerMapIterator.
+    Iterator<Map.Entry<Long, Container>> containerMapIterator = containerSet
+        .getContainerMapIterator();
+
+    count = 0;
+    while (containerMapIterator.hasNext()) {
+      Container kv = containerMapIterator.next().getValue();
+      ContainerData containerData = kv.getContainerData();
+      long containerId = containerData.getContainerId();
+      if (containerId%2 == 0) {
+        assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED,
+            containerData.getState());
+      } else {
+        assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN,
+            containerData.getState());
+      }
+      count++;
+    }
+    assertEquals(10, count);
+
+  }
+
+
+  @Test
+  public void testGetContainerReport() throws IOException {
+
+    ContainerSet containerSet = createContainerSet();
+
+    ContainerReportsProto containerReportsRequestProto = containerSet
+        .getContainerReport();
+
+    assertEquals(10, containerReportsRequestProto.getReportsList().size());
+  }
+
+
+
+  @Test
+  public void testListContainer() throws StorageContainerException {
+    ContainerSet containerSet = createContainerSet();
+
+    List<ContainerData> result = new ArrayList<>();
+    containerSet.listContainer(2, 5, result);
+
+    assertEquals(5, result.size());
+
+    for(ContainerData containerData : result) {
+      assertTrue(containerData.getContainerId() >=2 && containerData
+          .getContainerId()<=6);
+    }
+  }
+
+  private ContainerSet createContainerSet() throws StorageContainerException {
+    ContainerSet containerSet = new ContainerSet();
+    for (int i=0; i<10; i++) {
+      KeyValueContainerData kvData = new KeyValueContainerData(
+          ContainerProtos.ContainerType.KeyValueContainer, i);
+      if (i%2 == 0) {
+        kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED);
+      } else {
+        kvData.setState(ContainerProtos.ContainerLifeCycleState.OPEN);
+      }
+      KeyValueContainer kv = new KeyValueContainer(kvData);
+      containerSet.addContainer(kv);
+    }
+    return containerSet;
+  }
+
+}


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


[04/37] hadoop git commit: Merge remote-tracking branch 'apache-commit/trunk' into HDDS-48

Posted by bh...@apache.org.
Merge remote-tracking branch 'apache-commit/trunk' into HDDS-48


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

Branch: refs/heads/trunk
Commit: 79b298111fb9d1bd10251fd30a95173fa9ccb398
Parents: 5977718 a58acd9
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu May 31 11:45:01 2018 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu May 31 11:45:01 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/util/StringUtils.java     |   2 +-
 .../org/apache/hadoop/util/TestStringUtils.java |   2 +-
 .../StorageContainerDatanodeProtocol.proto      |  28 ++-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  11 +-
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  |  59 +++---
 .../apache/hadoop/hdfs/web/TestWebHdfsUrl.java  |  55 ++++++
 .../v2/app/job/impl/TaskAttemptImpl.java        |   2 +-
 .../mapreduce/v2/app/webapp/AppController.java  |   2 +-
 .../mapreduce/v2/app/webapp/TaskPage.java       |  10 +-
 .../mapreduce/v2/app/webapp/TasksBlock.java     |   6 +-
 .../v2/app/webapp/TestAppController.java        |   2 +-
 .../mapreduce/checkpoint/RandomNameCNS.java     |   2 +-
 .../HumanReadableHistoryViewerPrinter.java      |   2 +-
 .../lib/output/PathOutputCommitterFactory.java  |   2 +-
 .../hadoop/mapreduce/security/TokenCache.java   |   2 +-
 .../org/apache/hadoop/mapreduce/tools/CLI.java  |   2 +-
 .../lib/partition/TestRehashPartitioner.java    |   2 +-
 .../mapreduce/v2/hs/webapp/HsJobsBlock.java     |   8 +-
 .../mapreduce/v2/hs/webapp/HsTaskPage.java      |  10 +-
 .../org/apache/hadoop/mapred/ClientCache.java   |   2 +-
 .../hadoop/mapred/ClientServiceDelegate.java    |   2 +-
 .../org/apache/hadoop/mapred/NotRunningJob.java |  12 +-
 .../org/apache/hadoop/mapred/YARNRunner.java    |   2 +-
 .../apache/hadoop/mapred/TestJobCounters.java   |   2 +-
 .../mapreduce/lib/db/TestDBOutputFormat.java    |   2 +-
 .../hadoop-mapreduce-client/pom.xml             |   5 -
 .../server/dns/BaseServiceRecordProcessor.java  |  20 ++
 .../dns/ContainerServiceRecordProcessor.java    |   3 +-
 .../registry/server/dns/TestRegistryDNS.java    |  62 ++++--
 .../SingleConstraintAppPlacementAllocator.java  | 187 ++++++++++---------
 ...estSchedulingRequestContainerAllocation.java |  84 +++++++++
 .../markdown/yarn-service/ServiceDiscovery.md   |  14 +-
 pom.xml                                         |  41 ++--
 33 files changed, 437 insertions(+), 210 deletions(-)
----------------------------------------------------------------------



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


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

Posted by bh...@apache.org.
HDDS-92. Use DBType during parsing datanode .container files. Contributed by Bharat Viswanadham.


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

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

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee1e0e20/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/TestMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/TestMetadataStore.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/TestMetadataStore.java
deleted file mode 100644
index 6b26b60..0000000
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/TestMetadataStore.java
+++ /dev/null
@@ -1,414 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.ozone;
-
-import com.google.common.collect.Lists;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.tuple.ImmutablePair;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.DFSUtilClient;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.utils.BatchOperation;
-import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
-import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
-import org.apache.hadoop.utils.MetadataStore;
-import org.apache.hadoop.utils.MetadataStoreBuilder;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.junit.runners.Parameterized.Parameters;
-
-/**
- * Test class for ozone metadata store.
- */
-@RunWith(Parameterized.class)
-public class TestMetadataStore {
-
-  private final String storeImpl;
-
-  public TestMetadataStore(String metadataImpl) {
-    this.storeImpl = metadataImpl;
-  }
-
-  @Parameters
-  public static Collection<Object[]> data() {
-    return Arrays.asList(new Object[][] {
-        {OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB},
-        {OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_ROCKSDB}
-    });
-  }
-
-  private MetadataStore store;
-  private File testDir;
-  private final static int MAX_GETRANGE_LENGTH = 100;
-
-  @Rule
-  public ExpectedException expectedException = ExpectedException.none();
-
-  @Before
-  public void init() throws IOException {
-    testDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
-        + "-" + storeImpl.toLowerCase());
-
-    Configuration conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
-
-    store = MetadataStoreBuilder.newBuilder()
-        .setConf(conf)
-        .setCreateIfMissing(true)
-        .setDbFile(testDir)
-        .build();
-
-    // Add 20 entries.
-    // {a0 : a-value0} to {a9 : a-value9}
-    // {b0 : b-value0} to {b9 : b-value9}
-    for (int i=0; i<10; i++) {
-      store.put(getBytes("a" + i), getBytes("a-value" + i));
-      store.put(getBytes("b" + i), getBytes("b-value" + i));
-    }
-  }
-
-  @After
-  public void cleanup() throws IOException {
-    store.close();
-    store.destroy();
-    FileUtils.deleteDirectory(testDir);
-  }
-
-  private byte[] getBytes(String str) {
-    return str == null ? null :
-        DFSUtilClient.string2Bytes(str);
-  }
-
-  private String getString(byte[] bytes) {
-    return bytes == null ? null :
-        DFSUtilClient.bytes2String(bytes);
-  }
-
-  @Test
-  public void testGetDelete() throws IOException {
-    for (int i=0; i<10; i++) {
-      byte[] va = store.get(getBytes("a" + i));
-      Assert.assertEquals("a-value" + i, getString(va));
-
-      byte[] vb = store.get(getBytes("b" + i));
-      Assert.assertEquals("b-value" + i, getString(vb));
-    }
-
-    String keyToDel = "del-" + UUID.randomUUID().toString();
-    store.put(getBytes(keyToDel), getBytes(keyToDel));
-    Assert.assertEquals(keyToDel, getString(store.get(getBytes(keyToDel))));
-    store.delete(getBytes(keyToDel));
-    Assert.assertEquals(null, store.get(getBytes(keyToDel)));
-  }
-
-  @Test
-  public void testPeekFrom() throws IOException {
-    // Test peek from an element that has prev as well as next
-    testPeek("a3", "a2", "a4");
-
-    // Test peek from an element that only has prev
-    testPeek("b9", "b8", null);
-
-    // Test peek from an element that only has next
-    testPeek("a0", null, "a1");
-  }
-
-  private String getExpectedValue(String key) {
-    if (key == null) {
-      return null;
-    }
-    char[] arr = key.toCharArray();
-    return new StringBuffer().append(arr[0]).append("-value")
-        .append(arr[arr.length - 1]).toString();
-  }
-
-  private void testPeek(String peekKey, String prevKey, String nextKey)
-      throws IOException {
-    // Look for current
-    String k = null;
-    String v = null;
-    ImmutablePair<byte[], byte[]> current =
-        store.peekAround(0, getBytes(peekKey));
-    if (current != null) {
-      k = getString(current.getKey());
-      v = getString(current.getValue());
-    }
-    Assert.assertEquals(peekKey, k);
-    Assert.assertEquals(v, getExpectedValue(peekKey));
-
-    // Look for prev
-    k = null;
-    v = null;
-    ImmutablePair<byte[], byte[]> prev =
-        store.peekAround(-1, getBytes(peekKey));
-    if (prev != null) {
-      k = getString(prev.getKey());
-      v = getString(prev.getValue());
-    }
-    Assert.assertEquals(prevKey, k);
-    Assert.assertEquals(v, getExpectedValue(prevKey));
-
-    // Look for next
-    k = null;
-    v = null;
-    ImmutablePair<byte[], byte[]> next =
-        store.peekAround(1, getBytes(peekKey));
-    if (next != null) {
-      k = getString(next.getKey());
-      v = getString(next.getValue());
-    }
-    Assert.assertEquals(nextKey, k);
-    Assert.assertEquals(v, getExpectedValue(nextKey));
-  }
-
-  @Test
-  public void testIterateKeys() throws IOException {
-    // iterate keys from b0
-    ArrayList<String> result = Lists.newArrayList();
-    store.iterate(getBytes("b0"), (k, v) -> {
-      // b-value{i}
-      String value = getString(v);
-      char num = value.charAt(value.length() - 1);
-      // each value adds 1
-      int i = Character.getNumericValue(num) + 1;
-      value =  value.substring(0, value.length() - 1) + i;
-      result.add(value);
-      return true;
-    });
-
-    Assert.assertFalse(result.isEmpty());
-    for (int i=0; i<result.size(); i++) {
-      Assert.assertEquals("b-value" + (i+1), result.get(i));
-    }
-
-    // iterate from a non exist key
-    result.clear();
-    store.iterate(getBytes("xyz"), (k, v) -> {
-      result.add(getString(v));
-      return true;
-    });
-    Assert.assertTrue(result.isEmpty());
-
-    // iterate from the beginning
-    result.clear();
-    store.iterate(null, (k, v) -> {
-      result.add(getString(v));
-      return true;
-    });
-    Assert.assertEquals(20, result.size());
-  }
-
-  @Test
-  public void testGetRangeKVs() throws IOException {
-    List<Map.Entry<byte[], byte[]>> result = null;
-
-    // Set empty startKey will return values from beginning.
-    result = store.getRangeKVs(null, 5);
-    Assert.assertEquals(5, result.size());
-    Assert.assertEquals("a-value2", getString(result.get(2).getValue()));
-
-    // Empty list if startKey doesn't exist.
-    result = store.getRangeKVs(getBytes("a12"), 5);
-    Assert.assertEquals(0, result.size());
-
-    // Returns max available entries after a valid startKey.
-    result = store.getRangeKVs(getBytes("b0"), MAX_GETRANGE_LENGTH);
-    Assert.assertEquals(10, result.size());
-    Assert.assertEquals("b0", getString(result.get(0).getKey()));
-    Assert.assertEquals("b-value0", getString(result.get(0).getValue()));
-    result = store.getRangeKVs(getBytes("b0"), 5);
-    Assert.assertEquals(5, result.size());
-
-    // Both startKey and count are honored.
-    result = store.getRangeKVs(getBytes("a9"), 2);
-    Assert.assertEquals(2, result.size());
-    Assert.assertEquals("a9", getString(result.get(0).getKey()));
-    Assert.assertEquals("a-value9", getString(result.get(0).getValue()));
-    Assert.assertEquals("b0", getString(result.get(1).getKey()));
-    Assert.assertEquals("b-value0", getString(result.get(1).getValue()));
-
-    // Filter keys by prefix.
-    // It should returns all "b*" entries.
-    MetadataKeyFilter filter1 = new KeyPrefixFilter("b");
-    result = store.getRangeKVs(null, 100, filter1);
-    Assert.assertEquals(10, result.size());
-    Assert.assertTrue(result.stream().allMatch(entry ->
-        new String(entry.getKey()).startsWith("b")
-    ));
-    Assert.assertEquals(20, filter1.getKeysScannedNum());
-    Assert.assertEquals(10, filter1.getKeysHintedNum());
-    result = store.getRangeKVs(null, 3, filter1);
-    Assert.assertEquals(3, result.size());
-    result = store.getRangeKVs(getBytes("b3"), 1, filter1);
-    Assert.assertEquals("b-value3", getString(result.get(0).getValue()));
-
-    // Define a customized filter that filters keys by suffix.
-    // Returns all "*2" entries.
-    MetadataKeyFilter filter2 = (preKey, currentKey, nextKey)
-        -> getString(currentKey).endsWith("2");
-    result = store.getRangeKVs(null, MAX_GETRANGE_LENGTH, filter2);
-    Assert.assertEquals(2, result.size());
-    Assert.assertEquals("a2", getString(result.get(0).getKey()));
-    Assert.assertEquals("b2", getString(result.get(1).getKey()));
-    result = store.getRangeKVs(null, 1, filter2);
-    Assert.assertEquals(1, result.size());
-    Assert.assertEquals("a2", getString(result.get(0).getKey()));
-
-    // Apply multiple filters.
-    result = store.getRangeKVs(null, MAX_GETRANGE_LENGTH, filter1, filter2);
-    Assert.assertEquals(1, result.size());
-    Assert.assertEquals("b2", getString(result.get(0).getKey()));
-    Assert.assertEquals("b-value2", getString(result.get(0).getValue()));
-
-    // If filter is null, no effect.
-    result = store.getRangeKVs(null, 1, null);
-    Assert.assertEquals(1, result.size());
-    Assert.assertEquals("a0", getString(result.get(0).getKey()));
-  }
-
-  @Test
-  public void testGetSequentialRangeKVs() throws IOException {
-    MetadataKeyFilter suffixFilter = (preKey, currentKey, nextKey)
-        -> DFSUtil.bytes2String(currentKey).endsWith("2");
-    // Suppose to return a2 and b2
-    List<Map.Entry<byte[], byte[]>> result =
-        store.getRangeKVs(null, MAX_GETRANGE_LENGTH, suffixFilter);
-    Assert.assertEquals(2, result.size());
-    Assert.assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
-    Assert.assertEquals("b2", DFSUtil.bytes2String(result.get(1).getKey()));
-
-    // Suppose to return just a2, because when it iterates to a3,
-    // the filter no long matches and it should stop from there.
-    result = store.getSequentialRangeKVs(null,
-        MAX_GETRANGE_LENGTH, suffixFilter);
-    Assert.assertEquals(1, result.size());
-    Assert.assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
-  }
-
-  @Test
-  public void testGetRangeLength() throws IOException {
-    List<Map.Entry<byte[], byte[]>> result = null;
-
-    result = store.getRangeKVs(null, 0);
-    Assert.assertEquals(0, result.size());
-
-    result = store.getRangeKVs(null, 1);
-    Assert.assertEquals(1, result.size());
-
-    // Count less than zero is invalid.
-    expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage("Invalid count given");
-    store.getRangeKVs(null, -1);
-  }
-
-  @Test
-  public void testInvalidStartKey() throws IOException {
-    // If startKey is invalid, the returned list should be empty.
-    List<Map.Entry<byte[], byte[]>> kvs =
-        store.getRangeKVs(getBytes("unknownKey"), MAX_GETRANGE_LENGTH);
-    Assert.assertEquals(kvs.size(), 0);
-  }
-
-  @Test
-  public void testDestroyDB() throws IOException {
-    // create a new DB to test db destroy
-    Configuration conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
-
-    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
-        + "-" + storeImpl.toLowerCase() + "-toDestroy");
-    MetadataStore dbStore = MetadataStoreBuilder.newBuilder()
-        .setConf(conf)
-        .setCreateIfMissing(true)
-        .setDbFile(dbDir)
-        .build();
-
-    dbStore.put(getBytes("key1"), getBytes("value1"));
-    dbStore.put(getBytes("key2"), getBytes("value2"));
-
-    Assert.assertFalse(dbStore.isEmpty());
-    Assert.assertTrue(dbDir.exists());
-    Assert.assertTrue(dbDir.listFiles().length > 0);
-
-    dbStore.destroy();
-
-    Assert.assertFalse(dbDir.exists());
-  }
-
-  @Test
-  public void testBatchWrite() throws IOException {
-    Configuration conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
-
-    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
-        + "-" + storeImpl.toLowerCase() + "-batchWrite");
-    MetadataStore dbStore = MetadataStoreBuilder.newBuilder()
-        .setConf(conf)
-        .setCreateIfMissing(true)
-        .setDbFile(dbDir)
-        .build();
-
-    List<String> expectedResult = Lists.newArrayList();
-    for (int i = 0; i<10; i++) {
-      dbStore.put(getBytes("batch-" + i), getBytes("batch-value-" + i));
-      expectedResult.add("batch-" + i);
-    }
-
-    BatchOperation batch = new BatchOperation();
-    batch.delete(getBytes("batch-2"));
-    batch.delete(getBytes("batch-3"));
-    batch.delete(getBytes("batch-4"));
-    batch.put(getBytes("batch-new-2"), getBytes("batch-new-value-2"));
-
-    expectedResult.remove("batch-2");
-    expectedResult.remove("batch-3");
-    expectedResult.remove("batch-4");
-    expectedResult.add("batch-new-2");
-
-    dbStore.writeBatch(batch);
-
-    Iterator<String> it = expectedResult.iterator();
-    AtomicInteger count = new AtomicInteger(0);
-    dbStore.iterate(null, (key, value) -> {
-      count.incrementAndGet();
-      return it.hasNext() && it.next().equals(getString(key));
-    });
-
-    Assert.assertEquals(8, count.get());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee1e0e20/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
new file mode 100644
index 0000000..06db1e1
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
@@ -0,0 +1,463 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.utils;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
+import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.event.Level;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Test class for ozone metadata store.
+ */
+@RunWith(Parameterized.class)
+public class TestMetadataStore {
+
+  private final String storeImpl;
+
+  public TestMetadataStore(String metadataImpl) {
+    this.storeImpl = metadataImpl;
+  }
+
+  @Parameters
+  public static Collection<Object[]> data() {
+    return Arrays.asList(new Object[][] {
+        {OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB},
+        {OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_ROCKSDB}
+    });
+  }
+
+  private MetadataStore store;
+  private File testDir;
+  private final static int MAX_GETRANGE_LENGTH = 100;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  @Before
+  public void init() throws IOException {
+    testDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
+        + "-" + storeImpl.toLowerCase());
+
+    Configuration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
+
+    store = MetadataStoreBuilder.newBuilder()
+        .setConf(conf)
+        .setCreateIfMissing(true)
+        .setDbFile(testDir)
+        .build();
+
+    // Add 20 entries.
+    // {a0 : a-value0} to {a9 : a-value9}
+    // {b0 : b-value0} to {b9 : b-value9}
+    for (int i=0; i<10; i++) {
+      store.put(getBytes("a" + i), getBytes("a-value" + i));
+      store.put(getBytes("b" + i), getBytes("b-value" + i));
+    }
+  }
+
+  @Test
+  public void testMetaStoreConfigDifferentFromType() throws IOException {
+
+    Configuration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
+    String dbType;
+    GenericTestUtils.setLogLevel(MetadataStoreBuilder.LOG, Level.DEBUG);
+    GenericTestUtils.LogCapturer logCapturer =
+        GenericTestUtils.LogCapturer.captureLogs(MetadataStoreBuilder.LOG);
+    if(storeImpl.equals(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB)) {
+      dbType = "RocksDB";
+    } else {
+      dbType = "LevelDB";
+    }
+
+    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
+        + "-" + dbType.toLowerCase() + "-test");
+    MetadataStore dbStore = MetadataStoreBuilder.newBuilder().setConf(conf)
+        .setCreateIfMissing(true).setDbFile(dbDir).setDBType(dbType).build();
+    assertTrue(logCapturer.getOutput().contains("Using dbType " + dbType + "" +
+        " for metastore"));
+    dbStore.close();
+    dbStore.destroy();
+    FileUtils.deleteDirectory(dbDir);
+
+  }
+
+  @Test
+  public void testdbTypeNotSet() throws IOException {
+
+    Configuration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
+    GenericTestUtils.setLogLevel(MetadataStoreBuilder.LOG, Level.DEBUG);
+    GenericTestUtils.LogCapturer logCapturer =
+        GenericTestUtils.LogCapturer.captureLogs(MetadataStoreBuilder.LOG);
+
+
+    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
+        + "-" + storeImpl.toLowerCase() + "-test");
+    MetadataStore dbStore = MetadataStoreBuilder.newBuilder().setConf(conf)
+        .setCreateIfMissing(true).setDbFile(dbDir).build();
+    assertTrue(logCapturer.getOutput().contains("dbType is null, using dbType" +
+        " " + storeImpl));
+    dbStore.close();
+    dbStore.destroy();
+    FileUtils.deleteDirectory(dbDir);
+
+  }
+
+  @After
+  public void cleanup() throws IOException {
+    store.close();
+    store.destroy();
+    FileUtils.deleteDirectory(testDir);
+  }
+
+  private byte[] getBytes(String str) {
+    return str == null ? null :
+        DFSUtilClient.string2Bytes(str);
+  }
+
+  private String getString(byte[] bytes) {
+    return bytes == null ? null :
+        DFSUtilClient.bytes2String(bytes);
+  }
+
+  @Test
+  public void testGetDelete() throws IOException {
+    for (int i=0; i<10; i++) {
+      byte[] va = store.get(getBytes("a" + i));
+      Assert.assertEquals("a-value" + i, getString(va));
+
+      byte[] vb = store.get(getBytes("b" + i));
+      Assert.assertEquals("b-value" + i, getString(vb));
+    }
+
+    String keyToDel = "del-" + UUID.randomUUID().toString();
+    store.put(getBytes(keyToDel), getBytes(keyToDel));
+    Assert.assertEquals(keyToDel, getString(store.get(getBytes(keyToDel))));
+    store.delete(getBytes(keyToDel));
+    Assert.assertEquals(null, store.get(getBytes(keyToDel)));
+  }
+
+  @Test
+  public void testPeekFrom() throws IOException {
+    // Test peek from an element that has prev as well as next
+    testPeek("a3", "a2", "a4");
+
+    // Test peek from an element that only has prev
+    testPeek("b9", "b8", null);
+
+    // Test peek from an element that only has next
+    testPeek("a0", null, "a1");
+  }
+
+  private String getExpectedValue(String key) {
+    if (key == null) {
+      return null;
+    }
+    char[] arr = key.toCharArray();
+    return new StringBuffer().append(arr[0]).append("-value")
+        .append(arr[arr.length - 1]).toString();
+  }
+
+  private void testPeek(String peekKey, String prevKey, String nextKey)
+      throws IOException {
+    // Look for current
+    String k = null;
+    String v = null;
+    ImmutablePair<byte[], byte[]> current =
+        store.peekAround(0, getBytes(peekKey));
+    if (current != null) {
+      k = getString(current.getKey());
+      v = getString(current.getValue());
+    }
+    Assert.assertEquals(peekKey, k);
+    Assert.assertEquals(v, getExpectedValue(peekKey));
+
+    // Look for prev
+    k = null;
+    v = null;
+    ImmutablePair<byte[], byte[]> prev =
+        store.peekAround(-1, getBytes(peekKey));
+    if (prev != null) {
+      k = getString(prev.getKey());
+      v = getString(prev.getValue());
+    }
+    Assert.assertEquals(prevKey, k);
+    Assert.assertEquals(v, getExpectedValue(prevKey));
+
+    // Look for next
+    k = null;
+    v = null;
+    ImmutablePair<byte[], byte[]> next =
+        store.peekAround(1, getBytes(peekKey));
+    if (next != null) {
+      k = getString(next.getKey());
+      v = getString(next.getValue());
+    }
+    Assert.assertEquals(nextKey, k);
+    Assert.assertEquals(v, getExpectedValue(nextKey));
+  }
+
+  @Test
+  public void testIterateKeys() throws IOException {
+    // iterate keys from b0
+    ArrayList<String> result = Lists.newArrayList();
+    store.iterate(getBytes("b0"), (k, v) -> {
+      // b-value{i}
+      String value = getString(v);
+      char num = value.charAt(value.length() - 1);
+      // each value adds 1
+      int i = Character.getNumericValue(num) + 1;
+      value =  value.substring(0, value.length() - 1) + i;
+      result.add(value);
+      return true;
+    });
+
+    Assert.assertFalse(result.isEmpty());
+    for (int i=0; i<result.size(); i++) {
+      Assert.assertEquals("b-value" + (i+1), result.get(i));
+    }
+
+    // iterate from a non exist key
+    result.clear();
+    store.iterate(getBytes("xyz"), (k, v) -> {
+      result.add(getString(v));
+      return true;
+    });
+    Assert.assertTrue(result.isEmpty());
+
+    // iterate from the beginning
+    result.clear();
+    store.iterate(null, (k, v) -> {
+      result.add(getString(v));
+      return true;
+    });
+    Assert.assertEquals(20, result.size());
+  }
+
+  @Test
+  public void testGetRangeKVs() throws IOException {
+    List<Map.Entry<byte[], byte[]>> result = null;
+
+    // Set empty startKey will return values from beginning.
+    result = store.getRangeKVs(null, 5);
+    Assert.assertEquals(5, result.size());
+    Assert.assertEquals("a-value2", getString(result.get(2).getValue()));
+
+    // Empty list if startKey doesn't exist.
+    result = store.getRangeKVs(getBytes("a12"), 5);
+    Assert.assertEquals(0, result.size());
+
+    // Returns max available entries after a valid startKey.
+    result = store.getRangeKVs(getBytes("b0"), MAX_GETRANGE_LENGTH);
+    Assert.assertEquals(10, result.size());
+    Assert.assertEquals("b0", getString(result.get(0).getKey()));
+    Assert.assertEquals("b-value0", getString(result.get(0).getValue()));
+    result = store.getRangeKVs(getBytes("b0"), 5);
+    Assert.assertEquals(5, result.size());
+
+    // Both startKey and count are honored.
+    result = store.getRangeKVs(getBytes("a9"), 2);
+    Assert.assertEquals(2, result.size());
+    Assert.assertEquals("a9", getString(result.get(0).getKey()));
+    Assert.assertEquals("a-value9", getString(result.get(0).getValue()));
+    Assert.assertEquals("b0", getString(result.get(1).getKey()));
+    Assert.assertEquals("b-value0", getString(result.get(1).getValue()));
+
+    // Filter keys by prefix.
+    // It should returns all "b*" entries.
+    MetadataKeyFilter filter1 = new KeyPrefixFilter("b");
+    result = store.getRangeKVs(null, 100, filter1);
+    Assert.assertEquals(10, result.size());
+    Assert.assertTrue(result.stream().allMatch(entry ->
+        new String(entry.getKey()).startsWith("b")
+    ));
+    Assert.assertEquals(20, filter1.getKeysScannedNum());
+    Assert.assertEquals(10, filter1.getKeysHintedNum());
+    result = store.getRangeKVs(null, 3, filter1);
+    Assert.assertEquals(3, result.size());
+    result = store.getRangeKVs(getBytes("b3"), 1, filter1);
+    Assert.assertEquals("b-value3", getString(result.get(0).getValue()));
+
+    // Define a customized filter that filters keys by suffix.
+    // Returns all "*2" entries.
+    MetadataKeyFilter filter2 = (preKey, currentKey, nextKey)
+        -> getString(currentKey).endsWith("2");
+    result = store.getRangeKVs(null, MAX_GETRANGE_LENGTH, filter2);
+    Assert.assertEquals(2, result.size());
+    Assert.assertEquals("a2", getString(result.get(0).getKey()));
+    Assert.assertEquals("b2", getString(result.get(1).getKey()));
+    result = store.getRangeKVs(null, 1, filter2);
+    Assert.assertEquals(1, result.size());
+    Assert.assertEquals("a2", getString(result.get(0).getKey()));
+
+    // Apply multiple filters.
+    result = store.getRangeKVs(null, MAX_GETRANGE_LENGTH, filter1, filter2);
+    Assert.assertEquals(1, result.size());
+    Assert.assertEquals("b2", getString(result.get(0).getKey()));
+    Assert.assertEquals("b-value2", getString(result.get(0).getValue()));
+
+    // If filter is null, no effect.
+    result = store.getRangeKVs(null, 1, null);
+    Assert.assertEquals(1, result.size());
+    Assert.assertEquals("a0", getString(result.get(0).getKey()));
+  }
+
+  @Test
+  public void testGetSequentialRangeKVs() throws IOException {
+    MetadataKeyFilter suffixFilter = (preKey, currentKey, nextKey)
+        -> DFSUtil.bytes2String(currentKey).endsWith("2");
+    // Suppose to return a2 and b2
+    List<Map.Entry<byte[], byte[]>> result =
+        store.getRangeKVs(null, MAX_GETRANGE_LENGTH, suffixFilter);
+    Assert.assertEquals(2, result.size());
+    Assert.assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
+    Assert.assertEquals("b2", DFSUtil.bytes2String(result.get(1).getKey()));
+
+    // Suppose to return just a2, because when it iterates to a3,
+    // the filter no long matches and it should stop from there.
+    result = store.getSequentialRangeKVs(null,
+        MAX_GETRANGE_LENGTH, suffixFilter);
+    Assert.assertEquals(1, result.size());
+    Assert.assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
+  }
+
+  @Test
+  public void testGetRangeLength() throws IOException {
+    List<Map.Entry<byte[], byte[]>> result = null;
+
+    result = store.getRangeKVs(null, 0);
+    Assert.assertEquals(0, result.size());
+
+    result = store.getRangeKVs(null, 1);
+    Assert.assertEquals(1, result.size());
+
+    // Count less than zero is invalid.
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage("Invalid count given");
+    store.getRangeKVs(null, -1);
+  }
+
+  @Test
+  public void testInvalidStartKey() throws IOException {
+    // If startKey is invalid, the returned list should be empty.
+    List<Map.Entry<byte[], byte[]>> kvs =
+        store.getRangeKVs(getBytes("unknownKey"), MAX_GETRANGE_LENGTH);
+    Assert.assertEquals(kvs.size(), 0);
+  }
+
+  @Test
+  public void testDestroyDB() throws IOException {
+    // create a new DB to test db destroy
+    Configuration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
+
+    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
+        + "-" + storeImpl.toLowerCase() + "-toDestroy");
+    MetadataStore dbStore = MetadataStoreBuilder.newBuilder()
+        .setConf(conf)
+        .setCreateIfMissing(true)
+        .setDbFile(dbDir)
+        .build();
+
+    dbStore.put(getBytes("key1"), getBytes("value1"));
+    dbStore.put(getBytes("key2"), getBytes("value2"));
+
+    Assert.assertFalse(dbStore.isEmpty());
+    Assert.assertTrue(dbDir.exists());
+    Assert.assertTrue(dbDir.listFiles().length > 0);
+
+    dbStore.destroy();
+
+    Assert.assertFalse(dbDir.exists());
+  }
+
+  @Test
+  public void testBatchWrite() throws IOException {
+    Configuration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
+
+    File dbDir = GenericTestUtils.getTestDir(getClass().getSimpleName()
+        + "-" + storeImpl.toLowerCase() + "-batchWrite");
+    MetadataStore dbStore = MetadataStoreBuilder.newBuilder()
+        .setConf(conf)
+        .setCreateIfMissing(true)
+        .setDbFile(dbDir)
+        .build();
+
+    List<String> expectedResult = Lists.newArrayList();
+    for (int i = 0; i<10; i++) {
+      dbStore.put(getBytes("batch-" + i), getBytes("batch-value-" + i));
+      expectedResult.add("batch-" + i);
+    }
+
+    BatchOperation batch = new BatchOperation();
+    batch.delete(getBytes("batch-2"));
+    batch.delete(getBytes("batch-3"));
+    batch.delete(getBytes("batch-4"));
+    batch.put(getBytes("batch-new-2"), getBytes("batch-new-value-2"));
+
+    expectedResult.remove("batch-2");
+    expectedResult.remove("batch-3");
+    expectedResult.remove("batch-4");
+    expectedResult.add("batch-new-2");
+
+    dbStore.writeBatch(batch);
+
+    Iterator<String> it = expectedResult.iterator();
+    AtomicInteger count = new AtomicInteger(0);
+    dbStore.iterate(null, (key, value) -> {
+      count.incrementAndGet();
+      return it.hasNext() && it.next().equals(getString(key));
+    });
+
+    Assert.assertEquals(8, count.get());
+  }
+}

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

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


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


[35/37] hadoop git commit: Merge trunk into HDDS-48

Posted by bh...@apache.org.
Merge trunk into HDDS-48


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

Branch: refs/heads/trunk
Commit: c275a9a6a07b2bd889bdba4d05b420027f430b34
Parents: 44e19fc 83cd84b
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Mon Jul 9 12:13:03 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Mon Jul 9 12:13:03 2018 -0700

----------------------------------------------------------------------
 .gitignore                                      |    4 +
 dev-support/bin/ozone-dist-layout-stitching     |    2 +-
 ...ExcludePrivateAnnotationsStandardDoclet.java |    6 +-
 .../hadoop-common/src/main/conf/hadoop-env.sh   |    6 +-
 .../org/apache/hadoop/conf/Configuration.java   |  458 +++---
 .../java/org/apache/hadoop/fs/FileContext.java  |    9 +-
 .../org/apache/hadoop/fs/LocalDirAllocator.java |    7 +-
 .../hadoop-common/src/site/markdown/Metrics.md  |   39 +-
 .../org/apache/hadoop/fs/TestFileContext.java   |   44 +-
 .../apache/hadoop/fs/TestLocalDirAllocator.java |   59 +
 .../src/main/compose/ozone/docker-compose.yaml  |    6 +-
 .../src/main/compose/ozone/docker-config        |    2 +-
 .../src/main/compose/ozoneperf/README.md        |    4 +-
 .../main/compose/ozoneperf/docker-compose.yaml  |    6 +-
 .../src/main/compose/ozoneperf/docker-config    |    2 +-
 .../scm/client/ContainerOperationClient.java    |  117 +-
 hadoop-hdds/common/pom.xml                      |   18 +
 .../hadoop/hdds/protocol/DatanodeDetails.java   |   13 +-
 .../apache/hadoop/hdds/scm/ScmConfigKeys.java   |    6 +-
 .../hadoop/hdds/scm/client/ScmClient.java       |   43 +-
 .../container/common/helpers/ContainerInfo.java |  167 ++-
 .../common/helpers/ContainerWithPipeline.java   |  131 ++
 .../StorageContainerLocationProtocol.java       |   18 +-
 ...rLocationProtocolClientSideTranslatorPB.java |   34 +-
 .../org/apache/hadoop/ozone/OzoneConsts.java    |   22 +-
 .../apache/hadoop/ozone/audit/AuditAction.java  |   30 +
 .../hadoop/ozone/audit/AuditEventStatus.java    |   36 +
 .../apache/hadoop/ozone/audit/AuditLogger.java  |  128 ++
 .../hadoop/ozone/audit/AuditLoggerType.java     |   37 +
 .../apache/hadoop/ozone/audit/AuditMarker.java  |   38 +
 .../apache/hadoop/ozone/audit/Auditable.java    |   32 +
 .../apache/hadoop/ozone/audit/package-info.java |  123 ++
 .../org/apache/hadoop/ozone/common/Storage.java |    6 +-
 ...rLocationProtocolServerSideTranslatorPB.java |   33 +-
 .../main/proto/ScmBlockLocationProtocol.proto   |   10 +-
 .../StorageContainerLocationProtocol.proto      |   34 +-
 hadoop-hdds/common/src/main/proto/hdds.proto    |   28 +-
 .../common/src/main/resources/ozone-default.xml |  131 +-
 .../apache/hadoop/ozone/audit/DummyAction.java  |   51 +
 .../apache/hadoop/ozone/audit/DummyEntity.java  |   57 +
 .../ozone/audit/TestOzoneAuditLogger.java       |  147 ++
 .../apache/hadoop/ozone/audit/package-info.java |   23 +
 .../common/src/test/resources/log4j2.properties |   76 +
 .../apache/hadoop/hdds/scm/HddsServerUtil.java  |   11 -
 .../DeleteBlocksCommandHandler.java             |   30 +-
 .../protocol/StorageContainerNodeProtocol.java  |    4 +-
 .../src/main/resources/webapps/static/ozone.js  |    4 +-
 .../webapps/static/templates/config.html        |    4 +-
 .../hadoop/hdds/scm/block/BlockManagerImpl.java |   80 +-
 .../block/DatanodeDeletedBlockTransactions.java |   11 +-
 .../hadoop/hdds/scm/block/DeletedBlockLog.java  |    2 +-
 .../container/CloseContainerEventHandler.java   |   35 +-
 .../hdds/scm/container/ContainerMapping.java    |  128 +-
 .../scm/container/ContainerStateManager.java    |   30 +-
 .../hadoop/hdds/scm/container/Mapping.java      |   26 +-
 .../scm/container/closer/ContainerCloser.java   |   15 +-
 .../scm/container/states/ContainerStateMap.java |   13 +-
 .../hadoop/hdds/scm/events/SCMEvents.java       |   80 ++
 .../hadoop/hdds/scm/events/package-info.java    |   23 +
 .../hadoop/hdds/scm/node/CommandQueue.java      |    2 +-
 .../hadoop/hdds/scm/node/DatanodeInfo.java      |  109 ++
 .../hdds/scm/node/HeartbeatQueueItem.java       |   98 --
 .../hadoop/hdds/scm/node/NodeManager.java       |   16 +-
 .../hadoop/hdds/scm/node/NodeStateManager.java  |  575 ++++++++
 .../hadoop/hdds/scm/node/SCMNodeManager.java    |  511 +------
 .../node/states/NodeAlreadyExistsException.java |   45 +
 .../hdds/scm/node/states/NodeException.java     |   44 +
 .../scm/node/states/NodeNotFoundException.java  |   49 +
 .../hdds/scm/node/states/NodeStateMap.java      |  281 ++++
 .../hdds/scm/pipelines/PipelineManager.java     |   27 +-
 .../hdds/scm/pipelines/PipelineSelector.java    |   16 +
 .../scm/pipelines/ratis/RatisManagerImpl.java   |    1 +
 .../standalone/StandaloneManagerImpl.java       |    1 +
 .../hdds/scm/server/SCMBlockProtocolServer.java |    2 +-
 .../scm/server/SCMClientProtocolServer.java     |   74 +-
 .../server/SCMDatanodeHeartbeatDispatcher.java  |   13 +-
 .../scm/server/SCMDatanodeProtocolServer.java   |    2 +-
 .../scm/server/StorageContainerManager.java     |    7 +-
 .../hdds/scm/block/TestDeletedBlockLog.java     |   15 +-
 .../hdds/scm/container/MockNodeManager.java     |   58 +-
 .../TestCloseContainerEventHandler.java         |   54 +-
 .../scm/container/TestContainerMapping.java     |   27 +-
 .../container/closer/TestContainerCloser.java   |   18 +-
 .../hdds/scm/node/TestContainerPlacement.java   |   16 +-
 .../hadoop/hdds/scm/node/TestNodeManager.java   |  186 +--
 .../TestSCMDatanodeHeartbeatDispatcher.java     |   20 +-
 .../testutils/ReplicationNodeManagerMock.java   |   37 +-
 .../hadoop/hdds/scm/cli/OzoneBaseCLI.java       |    2 +-
 .../cli/container/CloseContainerHandler.java    |   10 +-
 .../cli/container/DeleteContainerHandler.java   |    9 +-
 .../scm/cli/container/InfoContainerHandler.java |   11 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |   19 -
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |   46 +-
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |   15 +-
 .../hdfs/client/HdfsClientConfigKeys.java       |    3 +
 .../hdfs/client/impl/BlockReaderFactory.java    |   21 +-
 .../hdfs/client/impl/BlockReaderLocal.java      |   93 +-
 .../client/impl/BlockReaderLocalLegacy.java     |   44 +-
 .../hdfs/client/impl/BlockReaderRemote.java     |   33 +-
 .../datanode/ReplicaNotFoundException.java      |    2 +-
 .../ha/ConfiguredFailoverProxyProvider.java     |    9 +-
 .../InMemoryAliasMapFailoverProxyProvider.java  |   38 +
 .../hdfs/server/federation/router/Quota.java    |   10 +-
 .../router/RouterQuotaUpdateService.java        |   43 +-
 .../federation/router/RouterRpcServer.java      |    1 -
 .../router/TestDisableRouterQuota.java          |   94 ++
 .../federation/router/TestRouterQuota.java      |  212 ++-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |    5 +-
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |   37 +-
 .../org/apache/hadoop/hdfs/NameNodeProxies.java |   15 +-
 ...yAliasMapProtocolClientSideTranslatorPB.java |   95 +-
 .../aliasmap/InMemoryAliasMapProtocol.java      |    5 +
 .../aliasmap/InMemoryLevelDBAliasMapServer.java |   19 +-
 .../impl/InMemoryLevelDBAliasMapClient.java     |   80 +-
 .../impl/TextFileRegionAliasMap.java            |    5 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |   21 +-
 .../hdfs/server/datanode/DiskBalancer.java      |   29 +-
 .../erasurecode/StripedBlockReader.java         |    2 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |    8 +
 .../hdfs/server/namenode/NamenodeFsck.java      |    1 -
 .../src/main/resources/hdfs-default.xml         |   35 +-
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |   13 +-
 .../apache/hadoop/hdfs/MiniDFSNNTopology.java   |    2 +-
 .../hdfs/client/impl/BlockReaderTestUtil.java   |    2 -
 .../hdfs/client/impl/TestBlockReaderLocal.java  |    2 -
 .../blockmanagement/TestBlockTokenWithDFS.java  |    2 -
 .../TestNameNodePrunesMissingStorages.java      |    5 +-
 .../impl/TestInMemoryLevelDBAliasMapClient.java |    7 +
 .../datanode/TestDataNodeVolumeFailure.java     |    2 -
 .../server/diskbalancer/TestDiskBalancer.java   |   80 +-
 .../shortcircuit/TestShortCircuitCache.java     |   89 ++
 .../src/test/acceptance/basic/basic.robot       |    6 +-
 .../test/acceptance/basic/docker-compose.yaml   |    8 +-
 .../src/test/acceptance/basic/docker-config     |    4 +-
 .../src/test/acceptance/basic/ozone-shell.robot |   18 +-
 .../src/test/acceptance/commonlib.robot         |    4 +-
 .../test/acceptance/ozonefs/docker-compose.yaml |    8 +-
 .../src/test/acceptance/ozonefs/docker-config   |    4 +-
 .../src/test/acceptance/ozonefs/ozonefs.robot   |    6 +-
 .../apache/hadoop/ozone/client/BucketArgs.java  |    4 +-
 .../hadoop/ozone/client/OzoneClientFactory.java |   89 +-
 .../apache/hadoop/ozone/client/OzoneKey.java    |    2 +-
 .../apache/hadoop/ozone/client/VolumeArgs.java  |    4 +-
 .../ozone/client/io/ChunkGroupInputStream.java  |   33 +-
 .../ozone/client/io/ChunkGroupOutputStream.java |   63 +-
 .../client/rest/DefaultRestServerSelector.java  |    2 +-
 .../hadoop/ozone/client/rest/RestClient.java    |   15 +-
 .../ozone/client/rest/RestServerSelector.java   |    2 +-
 .../hadoop/ozone/client/rpc/RpcClient.java      |  142 +-
 .../ozone/client/TestHddsClientUtils.java       |   24 +-
 hadoop-ozone/common/pom.xml                     |    2 +-
 hadoop-ozone/common/src/main/bin/ozone          |    9 +-
 hadoop-ozone/common/src/main/bin/start-ozone.sh |   16 +-
 hadoop-ozone/common/src/main/bin/stop-ozone.sh  |   16 +-
 .../java/org/apache/hadoop/ozone/KsmUtils.java  |   87 --
 .../java/org/apache/hadoop/ozone/OmUtils.java   |   94 ++
 .../org/apache/hadoop/ozone/audit/OMAction.java |   51 +
 .../apache/hadoop/ozone/audit/package-info.java |   22 +
 .../apache/hadoop/ozone/freon/OzoneGetConf.java |   16 +-
 .../apache/hadoop/ozone/ksm/KSMConfigKeys.java  |   81 --
 .../hadoop/ozone/ksm/helpers/KsmBucketArgs.java |  233 ---
 .../hadoop/ozone/ksm/helpers/KsmBucketInfo.java |  235 ---
 .../hadoop/ozone/ksm/helpers/KsmKeyArgs.java    |  119 --
 .../hadoop/ozone/ksm/helpers/KsmKeyInfo.java    |  277 ----
 .../ozone/ksm/helpers/KsmKeyLocationInfo.java   |  129 --
 .../ksm/helpers/KsmKeyLocationInfoGroup.java    |  118 --
 .../ozone/ksm/helpers/KsmOzoneAclMap.java       |  110 --
 .../hadoop/ozone/ksm/helpers/KsmVolumeArgs.java |  223 ---
 .../ozone/ksm/helpers/OpenKeySession.java       |   50 -
 .../hadoop/ozone/ksm/helpers/ServiceInfo.java   |  237 ---
 .../hadoop/ozone/ksm/helpers/VolumeArgs.java    |  140 --
 .../hadoop/ozone/ksm/helpers/package-info.java  |   18 -
 .../apache/hadoop/ozone/ksm/package-info.java   |   21 -
 .../ksm/protocol/KeySpaceManagerProtocol.java   |  252 ----
 .../hadoop/ozone/ksm/protocol/package-info.java |   19 -
 ...ceManagerProtocolClientSideTranslatorPB.java |  769 ----------
 .../protocolPB/KeySpaceManagerProtocolPB.java   |   34 -
 .../ozone/ksm/protocolPB/package-info.java      |   19 -
 .../apache/hadoop/ozone/om/OMConfigKeys.java    |   81 ++
 .../hadoop/ozone/om/helpers/OmBucketArgs.java   |  233 +++
 .../hadoop/ozone/om/helpers/OmBucketInfo.java   |  235 +++
 .../hadoop/ozone/om/helpers/OmKeyArgs.java      |  119 ++
 .../hadoop/ozone/om/helpers/OmKeyInfo.java      |  277 ++++
 .../ozone/om/helpers/OmKeyLocationInfo.java     |  129 ++
 .../om/helpers/OmKeyLocationInfoGroup.java      |  118 ++
 .../hadoop/ozone/om/helpers/OmOzoneAclMap.java  |  110 ++
 .../hadoop/ozone/om/helpers/OmVolumeArgs.java   |  223 +++
 .../hadoop/ozone/om/helpers/OpenKeySession.java |   50 +
 .../hadoop/ozone/om/helpers/ServiceInfo.java    |  237 +++
 .../hadoop/ozone/om/helpers/VolumeArgs.java     |  140 ++
 .../hadoop/ozone/om/helpers/package-info.java   |   18 +
 .../apache/hadoop/ozone/om/package-info.java    |   21 +
 .../ozone/om/protocol/OzoneManagerProtocol.java |  252 ++++
 .../hadoop/ozone/om/protocol/package-info.java  |   19 +
 ...neManagerProtocolClientSideTranslatorPB.java |  769 ++++++++++
 .../om/protocolPB/OzoneManagerProtocolPB.java   |   34 +
 .../ozone/om/protocolPB/package-info.java       |   19 +
 .../hadoop/ozone/protocolPB/KSMPBHelper.java    |  113 --
 .../hadoop/ozone/protocolPB/OMPBHelper.java     |  113 ++
 .../hadoop/ozone/protocolPB/OzonePBHelper.java  |   30 +
 .../main/proto/KeySpaceManagerProtocol.proto    |  474 ------
 .../src/main/proto/OzoneManagerProtocol.proto   |  480 +++++++
 hadoop-ozone/docs/content/GettingStarted.md     |   18 +-
 hadoop-ozone/docs/content/Metrics.md            |   10 +-
 hadoop-ozone/docs/content/_index.md             |   12 +-
 hadoop-ozone/docs/static/OzoneOverview.svg      |    2 +-
 .../container/TestContainerStateManager.java    |  161 ++-
 .../apache/hadoop/ozone/MiniOzoneCluster.java   |   24 +-
 .../hadoop/ozone/MiniOzoneClusterImpl.java      |   66 +-
 .../hadoop/ozone/TestContainerOperations.java   |   11 +-
 .../ozone/TestOzoneConfigurationFields.java     |    4 +-
 .../ozone/TestStorageContainerManager.java      |   28 +-
 .../TestStorageContainerManagerHelper.java      |   22 +-
 .../ozone/client/rest/TestOzoneRestClient.java  |    6 +-
 .../ozone/client/rpc/TestOzoneRpcClient.java    |   22 +-
 .../commandhandler/TestBlockDeletion.java       |  212 +++
 .../TestCloseContainerByPipeline.java           |   97 +-
 .../TestCloseContainerHandler.java              |   14 +-
 .../ozone/ksm/TestContainerReportWithKeys.java  |  143 --
 .../apache/hadoop/ozone/ksm/TestKSMMetrcis.java |  306 ----
 .../apache/hadoop/ozone/ksm/TestKSMSQLCli.java  |  284 ----
 .../hadoop/ozone/ksm/TestKeySpaceManager.java   | 1350 ------------------
 .../ksm/TestKeySpaceManagerRestInterface.java   |  135 --
 .../ozone/ksm/TestKsmBlockVersioning.java       |  253 ----
 .../ksm/TestMultipleContainerReadWrite.java     |  215 ---
 .../ozone/om/TestContainerReportWithKeys.java   |  143 ++
 .../om/TestMultipleContainerReadWrite.java      |  215 +++
 .../hadoop/ozone/om/TestOmBlockVersioning.java  |  253 ++++
 .../apache/hadoop/ozone/om/TestOmMetrics.java   |  313 ++++
 .../apache/hadoop/ozone/om/TestOmSQLCli.java    |  284 ++++
 .../hadoop/ozone/om/TestOzoneManager.java       | 1349 +++++++++++++++++
 .../ozone/om/TestOzoneManagerRestInterface.java |  135 ++
 .../hadoop/ozone/ozShell/TestOzoneShell.java    |   14 +-
 .../hadoop/ozone/scm/TestAllocateContainer.java |    6 +-
 .../hadoop/ozone/scm/TestContainerSQLCli.java   |    3 +-
 .../ozone/scm/TestContainerSmallFile.java       |   36 +-
 .../org/apache/hadoop/ozone/scm/TestSCMCli.java |  127 +-
 .../ozone/scm/TestXceiverClientManager.java     |   62 +-
 .../ozone/scm/TestXceiverClientMetrics.java     |   14 +-
 .../hadoop/ozone/scm/node/TestQueryNode.java    |   19 +-
 .../ozone/web/TestDistributedOzoneVolumes.java  |   12 +-
 .../hadoop/ozone/web/client/TestKeys.java       |   58 +-
 .../src/test/resources/webapps/ksm/.gitkeep     |   15 -
 .../resources/webapps/ozoneManager/.gitkeep     |   15 +
 .../server/datanode/ObjectStoreHandler.java     |   33 +-
 .../ozone/web/handlers/KeyProcessTemplate.java  |    4 +-
 .../web/handlers/VolumeProcessTemplate.java     |    4 +-
 .../web/storage/DistributedStorageHandler.java  |  153 +-
 .../apache/hadoop/ozone/ksm/BucketManager.java  |   79 -
 .../hadoop/ozone/ksm/BucketManagerImpl.java     |  315 ----
 .../org/apache/hadoop/ozone/ksm/KSMMXBean.java  |   31 -
 .../hadoop/ozone/ksm/KSMMetadataManager.java    |  253 ----
 .../ozone/ksm/KSMMetadataManagerImpl.java       |  526 -------
 .../org/apache/hadoop/ozone/ksm/KSMMetrics.java |  459 ------
 .../org/apache/hadoop/ozone/ksm/KSMStorage.java |   90 --
 .../hadoop/ozone/ksm/KeyDeletingService.java    |  142 --
 .../org/apache/hadoop/ozone/ksm/KeyManager.java |  175 ---
 .../apache/hadoop/ozone/ksm/KeyManagerImpl.java |  566 --------
 .../hadoop/ozone/ksm/KeySpaceManager.java       |  914 ------------
 .../ozone/ksm/KeySpaceManagerHttpServer.java    |   78 -
 .../hadoop/ozone/ksm/OpenKeyCleanupService.java |  117 --
 .../ozone/ksm/ServiceListJSONServlet.java       |  103 --
 .../apache/hadoop/ozone/ksm/VolumeManager.java  |  100 --
 .../hadoop/ozone/ksm/VolumeManagerImpl.java     |  391 -----
 .../ozone/ksm/exceptions/KSMException.java      |  118 --
 .../ozone/ksm/exceptions/package-info.java      |   19 -
 .../apache/hadoop/ozone/ksm/package-info.java   |   21 -
 .../apache/hadoop/ozone/om/BucketManager.java   |   79 +
 .../hadoop/ozone/om/BucketManagerImpl.java      |  315 ++++
 .../hadoop/ozone/om/KeyDeletingService.java     |  142 ++
 .../org/apache/hadoop/ozone/om/KeyManager.java  |  175 +++
 .../apache/hadoop/ozone/om/KeyManagerImpl.java  |  566 ++++++++
 .../org/apache/hadoop/ozone/om/OMMXBean.java    |   31 +
 .../hadoop/ozone/om/OMMetadataManager.java      |  253 ++++
 .../org/apache/hadoop/ozone/om/OMMetrics.java   |  459 ++++++
 .../org/apache/hadoop/ozone/om/OMStorage.java   |   90 ++
 .../hadoop/ozone/om/OmMetadataManagerImpl.java  |  526 +++++++
 .../hadoop/ozone/om/OpenKeyCleanupService.java  |  117 ++
 .../apache/hadoop/ozone/om/OzoneManager.java    |  911 ++++++++++++
 .../hadoop/ozone/om/OzoneManagerHttpServer.java |   78 +
 .../hadoop/ozone/om/ServiceListJSONServlet.java |  103 ++
 .../apache/hadoop/ozone/om/VolumeManager.java   |  100 ++
 .../hadoop/ozone/om/VolumeManagerImpl.java      |  390 +++++
 .../hadoop/ozone/om/exceptions/OMException.java |  118 ++
 .../ozone/om/exceptions/package-info.java       |   19 +
 .../apache/hadoop/ozone/om/package-info.java    |   21 +
 ...ceManagerProtocolServerSideTranslatorPB.java |  559 --------
 ...neManagerProtocolServerSideTranslatorPB.java |  571 ++++++++
 .../hadoop/ozone/protocolPB/package-info.java   |    2 +-
 .../src/main/webapps/ksm/index.html             |   70 -
 .../src/main/webapps/ksm/ksm-metrics.html       |   44 -
 .../ozone-manager/src/main/webapps/ksm/ksm.js   |  110 --
 .../ozone-manager/src/main/webapps/ksm/main.css |   23 -
 .../src/main/webapps/ksm/main.html              |   18 -
 .../src/main/webapps/ozoneManager/index.html    |   70 +
 .../src/main/webapps/ozoneManager/main.css      |   23 +
 .../src/main/webapps/ozoneManager/main.html     |   18 +
 .../main/webapps/ozoneManager/om-metrics.html   |   44 +
 .../main/webapps/ozoneManager/ozoneManager.js   |  110 ++
 .../hadoop/ozone/ksm/TestBucketManagerImpl.java |  395 -----
 .../hadoop/ozone/ksm/TestChunkStreams.java      |  234 ---
 .../ksm/TestKeySpaceManagerHttpServer.java      |  141 --
 .../apache/hadoop/ozone/ksm/package-info.java   |   21 -
 .../hadoop/ozone/om/TestBucketManagerImpl.java  |  394 +++++
 .../hadoop/ozone/om/TestChunkStreams.java       |  234 +++
 .../ozone/om/TestOzoneManagerHttpServer.java    |  141 ++
 .../apache/hadoop/ozone/om/package-info.java    |   21 +
 .../hadoop/fs/ozone/contract/OzoneContract.java |    4 +-
 .../genesis/BenchMarkContainerStateMap.java     |   16 +-
 .../org/apache/hadoop/ozone/scm/cli/SQLCLI.java |  111 +-
 .../hadoop/fs/s3a/s3guard/S3GuardTool.java      |   10 +
 .../s3guard/AbstractS3GuardToolTestBase.java    |   18 +
 .../namenode/ITestProvidedImplementation.java   |  373 ++++-
 .../dev-support/findbugs-exclude.xml            |   17 +-
 .../hadoop/yarn/api/records/Resource.java       |   13 +
 .../api/records/impl/LightWeightResource.java   |   23 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |    7 +
 .../impl/pb/GetApplicationsRequestPBImpl.java   |   44 +-
 .../logaggregation/AggregatedLogFormat.java     |    6 +-
 .../timeline/RollingLevelDBTimelineStore.java   |    6 +
 .../server/timeline/TimelineDataManager.java    |    7 +-
 .../timeline/webapp/TimelineWebServices.java    |    4 +
 .../webapp/TestTimelineWebServices.java         |    2 +-
 .../amrmproxy/BroadcastAMRMProxyPolicy.java     |   11 -
 .../amrmproxy/RejectAMRMProxyPolicy.java        |    4 -
 .../TestBroadcastAMRMProxyFederationPolicy.java |   11 +-
 .../yarn/server/nodemanager/NodeManager.java    |   66 +-
 .../runtime/DockerLinuxContainerRuntime.java    |    4 +-
 .../runtime/ContainerExecutionException.java    |    6 +
 .../impl/container-executor.c                   |   30 +-
 .../container-executor/impl/utils/docker-util.c |    2 +-
 .../test/test-container-executor.c              |   20 +
 .../nodemanager/TestNodeManagerResync.java      |   56 +
 .../runtime/TestDockerContainerRuntime.java     |   10 +-
 .../conf/capacity-scheduler.xml                 |   10 +
 .../scheduler/capacity/CapacityScheduler.java   |   45 +-
 .../CapacitySchedulerConfiguration.java         |   10 +
 .../scheduler/capacity/ParentQueue.java         |   36 +-
 .../allocator/AbstractContainerAllocator.java   |   13 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |    5 +
 .../scheduler/fair/ConfigurableResource.java    |   69 +-
 .../fair/FairSchedulerConfiguration.java        |  174 ++-
 .../allocation/AllocationFileQueueParser.java   |    2 +-
 .../resourcemanager/webapp/dao/AppInfo.java     |    2 +-
 .../webapp/dao/SchedulerInfo.java               |    8 +-
 .../TestWorkPreservingRMRestart.java            |    2 +
 .../fair/TestFairSchedulerConfiguration.java    |  160 ++-
 .../webapp/TestRMWebServices.java               |   31 +-
 .../webapp/TestRMWebServicesApps.java           |   14 +-
 ...estRMWebServicesAppsCustomResourceTypes.java |  242 ++++
 .../webapp/TestRMWebServicesCapacitySched.java  |   30 +-
 .../TestRMWebServicesConfigurationMutation.java |    5 +
 .../webapp/TestRMWebServicesFairScheduler.java  |   95 +-
 .../TestRMWebServicesSchedulerActivities.java   |    2 +-
 ...ustomResourceTypesConfigurationProvider.java |  138 ++
 .../FairSchedulerJsonVerifications.java         |  139 ++
 .../FairSchedulerXmlVerifications.java          |  153 ++
 ...ervicesFairSchedulerCustomResourceTypes.java |  271 ++++
 .../webapp/helper/AppInfoJsonVerifications.java |  123 ++
 .../webapp/helper/AppInfoXmlVerifications.java  |  132 ++
 .../webapp/helper/BufferedClientResponse.java   |   57 +
 .../helper/JsonCustomResourceTypeTestcase.java  |   77 +
 .../ResourceRequestsJsonVerifications.java      |  252 ++++
 .../ResourceRequestsXmlVerifications.java       |  215 +++
 .../helper/XmlCustomResourceTypeTestCase.java   |  112 ++
 .../router/clientrm/RouterClientRMService.java  |   53 +-
 .../router/rmadmin/RouterRMAdminService.java    |   51 +-
 .../server/router/webapp/RouterWebServices.java |   48 +-
 .../clientrm/TestRouterClientRMService.java     |   60 +
 .../rmadmin/TestRouterRMAdminService.java       |   60 +
 .../router/webapp/TestRouterWebServices.java    |   65 +
 .../pom.xml                                     |   10 +
 .../storage/TestTimelineReaderHBaseDown.java    |  220 +++
 .../storage/HBaseTimelineReaderImpl.java        |   93 ++
 .../reader/TimelineFromIdConverter.java         |   93 ++
 .../reader/TimelineReaderWebServices.java       |  198 ++-
 .../TestTimelineReaderWebServicesBasicAcl.java  |  154 ++
 .../src/site/markdown/FairScheduler.md          |    6 +-
 .../src/main/webapp/app/initializers/loader.js  |   10 +-
 379 files changed, 22363 insertions(+), 15606 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c275a9a6/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c275a9a6/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --cc hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index 82d67b7,4fad5d8..0db5993
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@@ -98,11 -93,8 +98,11 @@@ public final class OzoneConsts 
    public static final String BLOCK_DB = "block.db";
    public static final String OPEN_CONTAINERS_DB = "openContainers.db";
    public static final String DELETED_BLOCK_DB = "deletedBlock.db";
-   public static final String KSM_DB_NAME = "ksm.db";
+   public static final String OM_DB_NAME = "om.db";
  
 +  public static final String STORAGE_DIR_CHUNKS = "chunks";
 +  public static final String CONTAINER_FILE_CHECKSUM_EXTENSION = ".chksm";
 +
    /**
     * Supports Bucket Versioning.
     */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c275a9a6/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c275a9a6/hadoop-hdds/common/src/main/resources/ozone-default.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c275a9a6/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
----------------------------------------------------------------------
diff --cc hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
index 4fc1cd9,d215da9..c3d1596
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
@@@ -31,13 -29,11 +31,12 @@@ import org.apache.hadoop.hdds.protocol.
  import org.apache.hadoop.hdds.protocol.proto
      .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
  import org.apache.hadoop.ozone.OzoneConsts;
 -import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
  import org.apache.hadoop.ozone.container.common.helpers
      .DeletedContainerBlocksSummary;
 -import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
 -import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
 +import org.apache.hadoop.ozone.container.common.interfaces.Container;
- import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 +import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 +import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
 +import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
  import org.apache.hadoop.ozone.container.common.statemachine
      .EndpointStateMachine;
  import org.apache.hadoop.ozone.container.common.statemachine
@@@ -167,21 -145,28 +166,28 @@@ public class DeleteBlocksCommandHandle
     * Move a bunch of blocks from a container to deleting state.
     * This is a meta update, the actual deletes happen in async mode.
     *
 +   * @param containerData - KeyValueContainerData
     * @param delTX a block deletion transaction.
 -   * @param config configuration.
     * @throws IOException if I/O error occurs.
     */
 -  private void deleteContainerBlocks(DeletedBlocksTransaction delTX,
 -      Configuration config) throws IOException {
 +  private void deleteKeyValueContainerBlocks(
 +      KeyValueContainerData containerData, DeletedBlocksTransaction delTX)
 +      throws IOException {
      long containerId = delTX.getContainerID();
 -    ContainerData containerInfo = containerManager.readContainer(containerId);
      if (LOG.isDebugEnabled()) {
        LOG.debug("Processing Container : {}, DB path : {}", containerId,
 -          containerInfo.getDBPath());
 +          containerData.getMetadataPath());
      }
  
 -    if (delTX.getTxID() < containerInfo.getDeleteTransactionId()) {
++    if (delTX.getTxID() < containerData.getDeleteTransactionId()) {
+       LOG.debug(String.format("Ignoring delete blocks for containerId: %d."
+               + " Outdated delete transactionId %d < %d", containerId,
 -          delTX.getTxID(), containerInfo.getDeleteTransactionId()));
++          delTX.getTxID(), containerData.getDeleteTransactionId()));
+       return;
+     }
+ 
      int newDeletionBlocks = 0;
 -    MetadataStore containerDB = KeyUtils.getDB(containerInfo, config);
 +    MetadataStore containerDB = KeyUtils.getDB(containerData, conf);
      for (Long blk : delTX.getLocalIDList()) {
        BatchOperation batch = new BatchOperation();
        byte[] blkBytes = Longs.toByteArray(blk);
@@@ -208,13 -203,15 +224,15 @@@
          LOG.debug("Block {} not found or already under deletion in"
                  + " container {}, skip deleting it.", blk, containerId);
        }
-       containerDB.put(DFSUtil.string2Bytes(
-           OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX + containerId),
-           Longs.toByteArray(delTX.getTxID()));
      }
  
+     containerDB.put(DFSUtil.string2Bytes(
+         OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX + delTX.getContainerID()),
+         Longs.toByteArray(delTX.getTxID()));
 -    containerManager
 -        .updateDeleteTransactionId(delTX.getContainerID(), delTX.getTxID());
++    containerData
++        .updateDeleteTransactionId(delTX.getTxID());
      // update pending deletion blocks count in in-memory container status
 -    containerManager.incrPendingDeletionBlocks(newDeletionBlocks, containerId);
 +    containerData.incrPendingDeletionBlocks(newDeletionBlocks);
    }
  
    @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c275a9a6/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c275a9a6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupInputStream.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c275a9a6/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
----------------------------------------------------------------------
diff --cc hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
index ad1e706,a30c6f4..fff8611
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
@@@ -27,13 -27,11 +27,13 @@@ import org.apache.hadoop.hdds.scm.conta
  import org.apache.hadoop.hdfs.DFSUtil;
  import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
  import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 -import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 -import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
 +import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 +import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
 +import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 +import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
  import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
- import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
- import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
+ import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+ import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
  import org.apache.hadoop.ozone.web.handlers.BucketArgs;
  import org.apache.hadoop.ozone.web.handlers.KeyArgs;
  import org.apache.hadoop.ozone.web.handlers.UserArgs;
@@@ -160,14 -158,15 +160,16 @@@ public class TestStorageContainerManage
  
    private MetadataStore getContainerMetadata(Long containerID)
        throws IOException {
-     ContainerInfo container = cluster.getStorageContainerManager()
-         .getClientProtocolServer().getContainer(containerID);
-     DatanodeDetails leadDN = container.getPipeline().getLeader();
+     ContainerWithPipeline containerWithPipeline = cluster
+         .getStorageContainerManager().getClientProtocolServer()
+         .getContainerWithPipeline(containerID);
+ 
+     DatanodeDetails leadDN = containerWithPipeline.getPipeline().getLeader();
      OzoneContainer containerServer =
          getContainerServerByDatanodeUuid(leadDN.getUuidString());
 -    ContainerData containerData = containerServer.getContainerManager()
 -        .readContainer(containerID);
 +    KeyValueContainerData containerData = (KeyValueContainerData) containerServer
 +        .getContainerSet()
 +        .getContainer(containerID).getContainerData();
      return KeyUtils.getDB(containerData, conf);
    }
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c275a9a6/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
----------------------------------------------------------------------
diff --cc hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
index b832dd2,58b831b..30b18c2
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
@@@ -32,10 -32,10 +32,10 @@@ import org.apache.hadoop.ozone.client.O
  import org.apache.hadoop.ozone.client.OzoneClient;
  import org.apache.hadoop.ozone.client.OzoneClientFactory;
  import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 -import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 +import org.apache.hadoop.ozone.container.common.impl.ContainerData;
  import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
- import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
- import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
+ import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+ import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
  import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
  import org.apache.hadoop.test.GenericTestUtils;
  import org.junit.AfterClass;
@@@ -204,14 -257,8 +257,8 @@@ public class TestCloseContainerByPipeli
          if (datanode.equals(datanodeService.getDatanodeDetails())) {
            containerData =
                datanodeService.getDatanodeStateMachine().getContainer()
 -                  .getContainerManager().readContainer(containerID);
 +                  .getContainerSet().getContainer(containerID).getContainerData();
-           if (!containerData.isOpen()) {
-             // make sure the closeContainerHandler on the Datanode is invoked
-             Assert.assertTrue(
-                 datanodeService.getDatanodeStateMachine().getCommandDispatcher()
-                     .getCloseContainerHandler().getInvocationCount() > 0);
-             return true;
-           }
+           return !containerData.isOpen();
          }
      } catch (StorageContainerException e) {
        throw new AssertionError(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c275a9a6/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
----------------------------------------------------------------------
diff --cc hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
index 114bd04,58a5154..682bd63
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
@@@ -27,9 -27,9 +27,9 @@@ import org.apache.hadoop.hdds.client.Re
  import org.apache.hadoop.hdds.client.ReplicationType;
  import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
  import org.apache.hadoop.ozone.client.rest.OzoneException;
 -import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 +import org.apache.hadoop.ozone.container.common.impl.ContainerData;
- import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
- import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
+ import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+ import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
  import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
  import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
  import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c275a9a6/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestContainerReportWithKeys.java
----------------------------------------------------------------------
diff --cc hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestContainerReportWithKeys.java
index 0000000,5481506..c25b00e
mode 000000,100644..100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestContainerReportWithKeys.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestContainerReportWithKeys.java
@@@ -1,0 -1,143 +1,143 @@@
+ /**
+  * 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.om;
+ 
+ import org.apache.commons.lang3.RandomStringUtils;
+ 
+ import org.apache.hadoop.hdds.client.ReplicationFactor;
+ import org.apache.hadoop.hdds.client.ReplicationType;
+ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+ import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
+ import org.apache.hadoop.ozone.MiniOzoneCluster;
+ import org.apache.hadoop.ozone.OzoneConfigKeys;
+ import org.apache.hadoop.ozone.OzoneConsts;
+ import org.apache.hadoop.ozone.client.*;
+ import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 -import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 -import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
++import org.apache.hadoop.ozone.container.common.impl.ContainerData;
++import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+ import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+ import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+ import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+ import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+ import org.junit.AfterClass;
+ import org.junit.BeforeClass;
+ import org.junit.Rule;
+ import org.junit.Test;
+ import org.junit.rules.ExpectedException;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import java.io.IOException;
+ 
+ /**
+  * This class tests container report with DN container state info.
+  */
+ public class TestContainerReportWithKeys {
+   private static final Logger LOG = LoggerFactory.getLogger(
+       TestContainerReportWithKeys.class);
+   private static MiniOzoneCluster cluster = null;
+   private static OzoneConfiguration conf;
+   private static StorageContainerManager scm;
+ 
+   @Rule
+   public ExpectedException exception = ExpectedException.none();
+ 
+   /**
+    * Create a MiniDFSCluster for testing.
+    * <p>
+    * Ozone is made active by setting OZONE_ENABLED = true and
+    * OZONE_HANDLER_TYPE_KEY = "distributed"
+    *
+    * @throws IOException
+    */
+   @BeforeClass
+   public static void init() throws Exception {
+     conf = new OzoneConfiguration();
+     conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
+         OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
+     cluster = MiniOzoneCluster.newBuilder(conf).build();
+     cluster.waitForClusterToBeReady();
+     scm = cluster.getStorageContainerManager();
+   }
+ 
+   /**
+    * Shutdown MiniDFSCluster.
+    */
+   @AfterClass
+   public static void shutdown() {
+     if (cluster != null) {
+       cluster.shutdown();
+     }
+   }
+ 
+   @Test
+   public void testContainerReportKeyWrite() throws Exception {
+     final String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
+     final String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
+     final String keyName = "key" + RandomStringUtils.randomNumeric(5);
+     final int keySize = 100;
+ 
+     OzoneClient client = OzoneClientFactory.getClient(conf);
+     ObjectStore objectStore = client.getObjectStore();
+     objectStore.createVolume(volumeName);
+     objectStore.getVolume(volumeName).createBucket(bucketName);
+     OzoneOutputStream key =
+         objectStore.getVolume(volumeName).getBucket(bucketName)
+             .createKey(keyName, keySize, ReplicationType.STAND_ALONE,
+                 ReplicationFactor.ONE);
+     String dataString = RandomStringUtils.randomAlphabetic(keySize);
+     key.write(dataString.getBytes());
+     key.close();
+ 
+     OmKeyArgs keyArgs = new OmKeyArgs.Builder()
+         .setVolumeName(volumeName)
+         .setBucketName(bucketName)
+         .setKeyName(keyName)
+         .setType(HddsProtos.ReplicationType.STAND_ALONE)
+         .setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(keySize)
+         .build();
+ 
+ 
+     OmKeyLocationInfo keyInfo =
+         cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
+             .get(0).getBlocksLatestVersionOnly().get(0);
+ 
+     ContainerData cd = getContainerData(keyInfo.getContainerID());
+ 
 -    LOG.info("DN Container Data:  keyCount: {} used: {} ",
 -        cd.getKeyCount(), cd.getBytesUsed());
++/*    LOG.info("DN Container Data:  keyCount: {} used: {} ",
++        cd.getKeyCount(), cd.getBytesUsed());*/
+ 
+     ContainerInfo cinfo = scm.getContainerInfo(keyInfo.getContainerID());
+ 
+     LOG.info("SCM Container Info keyCount: {} usedBytes: {}",
+         cinfo.getNumberOfKeys(), cinfo.getUsedBytes());
+   }
+ 
+ 
+   private static ContainerData getContainerData(long containerID) {
+     ContainerData containerData;
+     try {
 -      ContainerManager containerManager = cluster.getHddsDatanodes().get(0)
 -          .getDatanodeStateMachine().getContainer().getContainerManager();
 -      containerData = containerManager.readContainer(containerID);
++      ContainerSet containerManager = cluster.getHddsDatanodes().get(0)
++          .getDatanodeStateMachine().getContainer().getContainerSet();
++      containerData = containerManager.getContainer(containerID).getContainerData();
+     } catch (StorageContainerException e) {
+       throw new AssertionError(e);
+     }
+     return containerData;
+   }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c275a9a6/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java
----------------------------------------------------------------------
diff --cc hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java
index 5c62803,42bb936..a2d95e8
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java
@@@ -141,9 -144,8 +144,8 @@@ public class TestContainerSmallFile 
      ContainerProtocolCalls.writeSmallFile(client, blockID,
          "data123".getBytes(), traceID);
  
- 
      thrown.expect(StorageContainerException.class);
 -    thrown.expectMessage("Unable to find the container");
 +    thrown.expectMessage("ContainerID 8888 does not exist");
  
      // Try to read a invalid key
      ContainerProtos.GetSmallFileResponseProto response =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c275a9a6/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java
----------------------------------------------------------------------
diff --cc hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java
index 12d444a,a6bb586..cc11feb
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java
@@@ -162,21 -158,22 +163,22 @@@ public class TestSCMCli 
      // 1. Test to delete a non-empty container.
      // ****************************************
      // Create an non-empty container
-     ContainerInfo container = containerOperationClient
+     ContainerWithPipeline container = containerOperationClient
          .createContainer(xceiverClientManager.getType(),
              HddsProtos.ReplicationFactor.ONE, containerOwner);
--
 -    ContainerData cdata = ContainerData
 -        .getFromProtBuf(containerOperationClient.readContainer(
 -            container.getContainerInfo().getContainerID()), conf);
 -    KeyUtils.getDB(cdata, conf)
 +    KeyValueContainerData kvData = KeyValueContainerData
 +        .getFromProtoBuf(containerOperationClient.readContainer(
-             container.getContainerID(), container.getPipeline()));
++            container.getContainerInfo().getContainerID(), container
++                .getPipeline()));
 +    KeyUtils.getDB(kvData, conf)
-         .put(Longs.toByteArray(container.getContainerID()),
+         .put(Longs.toByteArray(container.getContainerInfo().getContainerID()),
              "someKey".getBytes());
-     Assert.assertTrue(containerExist(container.getContainerID()));
 -    Assert.assertTrue(
 -        containerExist(container.getContainerInfo().getContainerID()));
++    Assert.assertTrue(containerExist(container.getContainerInfo()
++        .getContainerID()));
  
      // Gracefully delete a container should fail because it is open.
-     delCmd = new String[] {"-container", "-delete", "-c",
-         Long.toString(container.getContainerID())};
+     delCmd = new String[]{"-container", "-delete", "-c",
+         Long.toString(container.getContainerInfo().getContainerID())};
      testErr = new ByteArrayOutputStream();
      ByteArrayOutputStream out = new ByteArrayOutputStream();
      exitCode = runCommandAndGetOutput(delCmd, out, testErr);
@@@ -275,26 -267,24 +272,27 @@@
          EXECUTION_ERROR, exitCode);
  
      // Create an empty container.
-     ContainerInfo container = containerOperationClient
+     ContainerWithPipeline container = containerOperationClient
          .createContainer(xceiverClientManager.getType(),
              HddsProtos.ReplicationFactor.ONE, containerOwner);
 -    ContainerData data = ContainerData.getFromProtBuf(containerOperationClient
 -        .readContainer(container.getContainerInfo().getContainerID()), conf);
 -
 +    KeyValueContainerData data = KeyValueContainerData
 +        .getFromProtoBuf(containerOperationClient.
-             readContainer(container.getContainerID(),
++            readContainer(container.getContainerInfo().getContainerID(),
 +                container.getPipeline()));
- 
-     info = new String[] { "-container", "-info", "-c",
-         Long.toString(container.getContainerID()) };
+     info = new String[]{"-container", "-info", "-c",
+         Long.toString(container.getContainerInfo().getContainerID())};
      ByteArrayOutputStream out = new ByteArrayOutputStream();
      exitCode = runCommandAndGetOutput(info, out, null);
      assertEquals("Expected Success, did not find it.", ResultCode.SUCCESS,
-             exitCode);
+         exitCode);
  
      String openStatus = data.isOpen() ? "OPEN" : "CLOSED";
 -    String expected = String.format(formatStr, container.getContainerInfo()
 -            .getContainerID(), openStatus, data.getDBPath(),
 -        data.getContainerPath(), "", datanodeDetails.getHostName(),
 -        datanodeDetails.getHostName());
 +    String expected =
-         String.format(formatStr, container.getContainerID(), openStatus,
-         data.getDbFile().getPath(), data.getContainerPath(), "",
-         datanodeDetails.getHostName(), datanodeDetails.getHostName());
++        String.format(formatStr, container.getContainerInfo().getContainerID
++                (), openStatus, data.getDbFile().getPath(), data
++                .getContainerPath(), "", datanodeDetails.getHostName(),
++            datanodeDetails.getHostName());
++
      assertEquals(expected, out.toString());
  
      out.reset();
@@@ -303,9 -293,9 +301,10 @@@
      container = containerOperationClient
          .createContainer(xceiverClientManager.getType(),
              HddsProtos.ReplicationFactor.ONE, containerOwner);
 -    data = ContainerData
 -        .getFromProtBuf(containerOperationClient.readContainer(
 -            container.getContainerInfo().getContainerID()), conf);
 +    data = KeyValueContainerData
 +        .getFromProtoBuf(containerOperationClient.readContainer(
-             container.getContainerID(), container.getPipeline()));
++            container.getContainerInfo().getContainerID(), container
++                .getPipeline()));
      KeyUtils.getDB(data, conf)
          .put(containerID.getBytes(), "someKey".getBytes());
  
@@@ -315,25 -305,24 +314,27 @@@
      assertEquals(ResultCode.SUCCESS, exitCode);
  
      openStatus = data.isOpen() ? "OPEN" : "CLOSED";
-     expected = String.format(formatStr, container.getContainerID(), openStatus,
-         data.getDbFile().getPath(), data.getContainerPath(), "",
-         datanodeDetails.getHostName(), datanodeDetails.getHostName());
 -    expected = String.format(formatStr, container.getContainerInfo().
 -            getContainerID(), openStatus, data.getDBPath(),
 -        data.getContainerPath(), "", datanodeDetails.getHostName(),
++
++    expected = String.format(formatStr, container.getContainerInfo()
++            .getContainerID(), openStatus, data.getDbFile().getPath(), data
++            .getContainerPath(), "", datanodeDetails.getHostName(),
+         datanodeDetails.getHostName());
      assertEquals(expected, out.toString());
  
      out.reset();
  
- 
      // Close last container and test info again.
-     containerOperationClient.closeContainer(
-         container.getContainerID(), container.getPipeline());
+     containerOperationClient
+         .closeContainer(container.getContainerInfo().getContainerID());
  
-     info = new String[] { "-container", "-info", "-c",
-         Long.toString(container.getContainerID()) };
+     info = new String[]{"-container", "-info", "-c",
+         Long.toString(container.getContainerInfo().getContainerID())};
      exitCode = runCommandAndGetOutput(info, out, null);
      assertEquals(ResultCode.SUCCESS, exitCode);
 -    data = ContainerData.getFromProtBuf(containerOperationClient
 -        .readContainer(container.getContainerInfo().getContainerID()), conf);
 +    data = KeyValueContainerData
 +        .getFromProtoBuf(containerOperationClient.readContainer(
-             container.getContainerID(), container.getPipeline()));
++            container.getContainerInfo().getContainerID(), container
++                .getPipeline()));
  
      openStatus = data.isOpen() ? "OPEN" : "CLOSED";
      expected = String

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c275a9a6/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
----------------------------------------------------------------------
diff --cc hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
index 2f592c1,a95bd0e..c144db2
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
@@@ -44,18 -44,17 +44,18 @@@ import org.apache.hadoop.ozone.client.i
  import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
  import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
  import org.apache.hadoop.ozone.client.rpc.RpcClient;
 -import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 -import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
  import org.apache.hadoop.ozone.container.common.helpers.KeyData;
 +import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 +import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 +import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
  import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
- import org.apache.hadoop.ozone.ksm.KeySpaceManager;
- import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
- import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
- import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
- import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
- import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
- import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos
+ import org.apache.hadoop.ozone.om.OzoneManager;
+ import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+ import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+ import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+ import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+ import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
      .Status;
  import org.apache.hadoop.ozone.client.rest.OzoneException;
  import org.apache.hadoop.ozone.web.utils.OzoneUtils;
@@@ -663,12 -661,11 +663,12 @@@ public class TestKeys 
    }
  
    @Test
 +  @Ignore("Needs to be fixed for new SCM and Storage design")
    public void testDeleteKey() throws Exception {
-     KeySpaceManager ksm = ozoneCluster.getKeySpaceManager();
+     OzoneManager ozoneManager = ozoneCluster.getOzoneManager();
      // To avoid interference from other test cases,
      // we collect number of existing keys at the beginning
-     int numOfExistedKeys = countKsmKeys(ksm);
+     int numOfExistedKeys = countOmKeys(ozoneManager);
  
      // Keep tracking bucket keys info while creating them
      PutHelper helper = new PutHelper(client, path);
@@@ -697,20 -694,17 +697,20 @@@
  
        // Memorize chunks that has been created,
        // so we can verify actual deletions at DN side later.
-       for (KsmKeyInfo keyInfo : createdKeys) {
-         List<KsmKeyLocationInfo> locations =
+       for (OmKeyInfo keyInfo : createdKeys) {
+         List<OmKeyLocationInfo> locations =
              keyInfo.getLatestVersionLocations().getLocationList();
-         for (KsmKeyLocationInfo location : locations) {
+         for (OmKeyLocationInfo location : locations) {
 -          KeyData keyData = new KeyData(location.getBlockID());
 -          KeyData blockInfo = cm.getContainerManager()
 -              .getKeyManager().getKey(keyData);
 -          ContainerData containerData = cm.getContainerManager()
 -              .readContainer(keyData.getContainerID());
 -          File dataDir = ContainerUtils
 -              .getDataDirectory(containerData).toFile();
 +          KeyValueHandler  keyValueHandler = (KeyValueHandler) cm
 +              .getDispatcher().getHandler(ContainerProtos.ContainerType
 +                  .KeyValueContainer);
 +          KeyValueContainer container = (KeyValueContainer) cm.getContainerSet()
 +              .getContainer(location.getBlockID().getContainerID());
 +          KeyData blockInfo = keyValueHandler
 +              .getKeyManager().getKey(container, location.getBlockID());
 +          KeyValueContainerData containerData = (KeyValueContainerData) container
 +              .getContainerData();
 +          File dataDir = new File(containerData.getChunksPath());
            for (ContainerProtos.ChunkInfo chunkInfo : blockInfo.getChunks()) {
              File chunkFile = dataDir.toPath()
                  .resolve(chunkInfo.getChunkName()).toFile();


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