http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java
deleted file mode 100644
index c8284fd..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java
+++ /dev/null
@@ -1,138 +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.audit;
-/**
- ******************************************************************************
- *                              Important
- * 1. Any changes to classes in this package can render the logging
- * framework broken.
- * 2. The logger framework has been designed keeping in mind future
- * plans to build a log parser.
- * 3. Please exercise great caution when attempting changes in this package.
- ******************************************************************************
- *
- *
- * This package lays the foundation for Audit logging in Ozone.
- * AuditLogging in Ozone has been built using log4j2 which brings in new
- * features that facilitate turning on/off selective audit events by using
- * MarkerFilter, checking for change in logging configuration periodically
- * and reloading the changes, use of disruptor framework for improved
- * Asynchronous logging.
- *
- * The log4j2 configurations can be specified in XML, YAML, JSON and
- * Properties file. For Ozone, we are using the Properties file due to sheer
- * simplicity, readability and ease of modification.
- *
- * log4j2 configuration file can be passed to startup command with option
- * -Dlog4j.configurationFile unlike -Dlog4j.configuration in log4j 1.x
- *
- ******************************************************************************
- *          Understanding the Audit Logging framework in Ozone.
- ******************************************************************************
- * **** Auditable ***
- * This is an interface to mark an entity as auditable.
- * This interface must be implemented by entities requiring audit logging.
- * For example - OMVolumeArgs, OMBucketArgs.
- * The implementing class must override toAuditMap() to return an
- * instance of Map<Key, Value> where both Key and Value are String.
- *
- * Key: must contain printable US ASCII characters
- * May not contain a space, =, ], or "
- * If the key is multi word then use camel case.
- *
- * Value: if it is a collection/array, then it must be converted to a comma
- * delimited string
- *
- * *** AuditAction ***
- * This is an interface to define the various type of actions to be audited.
- * To ensure separation of concern, for each sub-component you must create an
- * Enum to implement AuditAction.
- * Structure of Enum can be referred from the test class DummyAction.
- *
- * For starters, we expect following 3 implementations of AuditAction:
- * OMAction - to define action types for Ozone Manager
- * SCMAction - to define action types for Storage Container manager
- * DNAction - to define action types for Datanode
- *
- * *** AuditEventStatus ***
- * Enum to define Audit event status like success and failure.
- * This is used in AuditLogger.logXXX() methods.
- *
- *  * *** AuditLogger ***
- * This is where the audit logging magic unfolds.
- * The class has 2 Markers defined - READ and WRITE.
- * These markers are used to tag when logging events.
- *
- * *** AuditLoggerType ***
- * Enum to define the various AuditLoggers in Ozone
- *
- * *** AuditMarker ***
- * Enum to define various Audit Markers used in AuditLogging.
- *
- * *** AuditMessage ***
- * Entity to define an audit message to be logged
- * It will generate a message formatted as:
- * user=xxx ip=xxx op=XXXX_XXXX {key=val, key1=val1..} ret=XXXXXX
- *
- * *** Auditor ***
- * Interface to mark an actor class as Auditor
- * Must be implemented by class where we want to log audit events
- * Implementing class must override and implement methods
- * buildAuditMessageForSuccess and buildAuditMessageForFailure.
- *
- * ****************************************************************************
- *                              Usage
- * ****************************************************************************
- * Using the AuditLogger to log events:
- * 1. Get a logger by specifying the appropriate logger type
- * Example: ExtendedLogger AUDIT = new AuditLogger(AuditLoggerType.OMLogger)
- *
- * 2. Construct an instance of AuditMessage
- *
- * 3. Log Read/Write and Success/Failure event as needed.
- * Example
- * AUDIT.logWriteSuccess(buildAuditMessageForSuccess(params))
- *
- * 4. Log Level implicitly defaults to INFO for xxxxSuccess() and ERROR for
- * xxxxFailure()
- * AUDIT.logWriteSuccess(buildAuditMessageForSuccess(params))
- * AUDIT.logWriteFailure(buildAuditMessageForSuccess(params))
- *
- * See sample invocations in src/test in the following class:
- * org.apache.hadoop.ozone.audit.TestOzoneAuditLogger
- *
- * ****************************************************************************
- *                      Defining new Logger types
- * ****************************************************************************
- * New Logger type can be added with following steps:
- * 1. Update AuditLoggerType to add the new type
- * 2. Create new Enum by implementing AuditAction if needed
- * 3. Ensure the required entity implements Auditable
- *
- * ****************************************************************************
- *                      Defining new Marker types
- * ****************************************************************************
- * New Markers can be configured as follows:
- * 1. Define new markers in AuditMarker
- * 2. Get the Marker in AuditLogger for use in the log methods, example:
- * private static final Marker WRITE_MARKER = AuditMarker.WRITE.getMarker();
- * 3. Define log methods in AuditLogger to use the new Marker type
- * 4. Call these new methods from the required classes to audit with these
- * new markers
- * 5. The marker based filtering can be configured in log4j2 configurations
- * Refer log4j2.properties in src/test/resources for a sample.
- */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/BlockGroup.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/BlockGroup.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/BlockGroup.java
deleted file mode 100644
index 7a5403f..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/BlockGroup.java
+++ /dev/null
@@ -1,97 +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.common;
-
-import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
-    .KeyBlocks;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * A group of blocks relations relevant, e.g belong to a certain object key.
- */
-public final class BlockGroup {
-
-  private String groupID;
-  private List<BlockID> blockIDs;
-  private BlockGroup(String groupID, List<BlockID> blockIDs) {
-    this.groupID = groupID;
-    this.blockIDs = blockIDs;
-  }
-
-  public List<BlockID> getBlockIDList() {
-    return blockIDs;
-  }
-
-  public String getGroupID() {
-    return groupID;
-  }
-
-  public KeyBlocks getProto() {
-    KeyBlocks.Builder kbb = KeyBlocks.newBuilder();
-    for (BlockID block : blockIDs) {
-      kbb.addBlocks(block.getProtobuf());
-    }
-    return kbb.setKey(groupID).build();
-  }
-
-  /**
-   * Parses a KeyBlocks proto to a group of blocks.
-   * @param proto KeyBlocks proto.
-   * @return a group of blocks.
-   */
-  public static BlockGroup getFromProto(KeyBlocks proto) {
-    List<BlockID> blockIDs = new ArrayList<>();
-    for (HddsProtos.BlockID block : proto.getBlocksList()) {
-      blockIDs.add(new BlockID(block.getContainerID(), block.getLocalID()));
-    }
-    return BlockGroup.newBuilder().setKeyName(proto.getKey())
-        .addAllBlockIDs(blockIDs).build();
-  }
-
-  public static Builder newBuilder() {
-    return new Builder();
-  }
-
-  /**
-   * BlockGroup instance builder.
-   */
-  public static class Builder {
-
-    private String groupID;
-    private List<BlockID> blockIDs;
-
-    public Builder setKeyName(String blockGroupID) {
-      this.groupID = blockGroupID;
-      return this;
-    }
-
-    public Builder addAllBlockIDs(List<BlockID> keyBlocks) {
-      this.blockIDs = keyBlocks;
-      return this;
-    }
-
-    public BlockGroup build() {
-      return new BlockGroup(groupID, blockIDs);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/DeleteBlockGroupResult.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/DeleteBlockGroupResult.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/DeleteBlockGroupResult.java
deleted file mode 100644
index 892b695..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/DeleteBlockGroupResult.java
+++ /dev/null
@@ -1,97 +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.common;
-
-import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.hdds.scm.container.common.helpers.DeleteBlockResult;
-import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
-    .DeleteScmBlockResult;
-import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
-    .DeleteScmBlockResult.Result;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.stream.Collectors;
-
-/**
- * Result to delete a group of blocks.
- */
-public class DeleteBlockGroupResult {
-  private String objectKey;
-  private List<DeleteBlockResult> blockResultList;
-  public DeleteBlockGroupResult(String objectKey,
-      List<DeleteBlockResult> blockResultList) {
-    this.objectKey = objectKey;
-    this.blockResultList = blockResultList;
-  }
-
-  public String getObjectKey() {
-    return objectKey;
-  }
-
-  public List<DeleteBlockResult> getBlockResultList() {
-    return blockResultList;
-  }
-
-  public List<DeleteScmBlockResult> getBlockResultProtoList() {
-    List<DeleteScmBlockResult> resultProtoList =
-        new ArrayList<>(blockResultList.size());
-    for (DeleteBlockResult result : blockResultList) {
-      DeleteScmBlockResult proto = DeleteScmBlockResult.newBuilder()
-          .setBlockID(result.getBlockID().getProtobuf())
-          .setResult(result.getResult()).build();
-      resultProtoList.add(proto);
-    }
-    return resultProtoList;
-  }
-
-  public static List<DeleteBlockResult> convertBlockResultProto(
-      List<DeleteScmBlockResult> results) {
-    List<DeleteBlockResult> protoResults = new ArrayList<>(results.size());
-    for (DeleteScmBlockResult result : results) {
-      protoResults.add(new DeleteBlockResult(BlockID.getFromProtobuf(
-          result.getBlockID()), result.getResult()));
-    }
-    return protoResults;
-  }
-
-  /**
-   * Only if all blocks are successfully deleted, this group is considered
-   * to be successfully executed.
-   *
-   * @return true if all blocks are successfully deleted, false otherwise.
-   */
-  public boolean isSuccess() {
-    for (DeleteBlockResult result : blockResultList) {
-      if (result.getResult() != Result.success) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  /**
-   * @return A list of deletion failed block IDs.
-   */
-  public List<BlockID> getFailedBlocks() {
-    List<BlockID> failedBlocks = blockResultList.stream()
-        .filter(result -> result.getResult() != Result.success)
-        .map(DeleteBlockResult::getBlockID).collect(Collectors.toList());
-    return failedBlocks;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/InconsistentStorageStateException.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/InconsistentStorageStateException.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/InconsistentStorageStateException.java
deleted file mode 100644
index 518b519..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/InconsistentStorageStateException.java
+++ /dev/null
@@ -1,51 +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.common;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-import java.io.File;
-import java.io.IOException;
-
-/**
- * The exception is thrown when file system state is inconsistent
- * and is not recoverable.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class InconsistentStorageStateException extends IOException {
-  private static final long serialVersionUID = 1L;
-
-  public InconsistentStorageStateException(String descr) {
-    super(descr);
-  }
-
-  public InconsistentStorageStateException(File dir, String descr) {
-    super("Directory " + getFilePath(dir) + " is in an inconsistent state: "
-        + descr);
-  }
-
-  private static String getFilePath(File dir) {
-    try {
-      return dir.getCanonicalPath();
-    } catch (IOException e) {
-    }
-    return dir.getPath();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/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
deleted file mode 100644
index 1826a58..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
+++ /dev/null
@@ -1,249 +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.common;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
-import org.apache.hadoop.util.Time;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.DirectoryStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.Properties;
-
-/**
- * Storage information file. This Class defines the methods to check
- * the consistency of the storage dir and the version file.
- * <p>
- * Local storage information is stored in a separate file VERSION.
- * It contains type of the node,
- * the storage layout version, the SCM id, and
- * the OM/SCM state creation time.
- *
- */
-@InterfaceAudience.Private
-public abstract class Storage {
-  private static final Logger LOG = LoggerFactory.getLogger(Storage.class);
-
-  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;
-  private final File storageDir;
-
-  private StorageState state;
-  private StorageInfo storageInfo;
-
-
-  /**
-   * Determines the state of the Version file.
-   */
-  public enum StorageState {
-    NON_EXISTENT, NOT_INITIALIZED, INITIALIZED
-  }
-
-  public Storage(NodeType type, File root, String sdName)
-      throws IOException {
-    this.nodeType = type;
-    this.root = root;
-    this.storageDir = new File(root, sdName);
-    this.state = getStorageState();
-    if (state == StorageState.INITIALIZED) {
-      this.storageInfo = new StorageInfo(type, getVersionFile());
-    } else {
-      this.storageInfo = new StorageInfo(
-          nodeType, StorageInfo.newClusterID(), Time.now());
-      setNodeProperties();
-    }
-  }
-
-  /**
-   * Gets the path of the Storage dir.
-   * @return Stoarge dir path
-   */
-  public String getStorageDir() {
-    return storageDir.getAbsoluteFile().toString();
-  }
-
-  /**
-   * Gets the state of the version file.
-   * @return the state of the Version file
-   */
-  public StorageState getState() {
-    return state;
-  }
-
-  public NodeType getNodeType() {
-    return storageInfo.getNodeType();
-  }
-
-  public String getClusterID() {
-    return storageInfo.getClusterID();
-  }
-
-  public long getCreationTime() {
-    return storageInfo.getCreationTime();
-  }
-
-  public void setClusterId(String clusterId) throws IOException {
-    if (state == StorageState.INITIALIZED) {
-      throw new IOException(
-          "Storage directory " + storageDir + " already initialized.");
-    } else {
-      storageInfo.setClusterId(clusterId);
-    }
-  }
-
-  /**
-   * Retreives the storageInfo instance to read/write the common
-   * version file properties.
-   * @return the instance of the storageInfo class
-   */
-  protected StorageInfo getStorageInfo() {
-    return storageInfo;
-  }
-
-  abstract protected Properties getNodeProperties();
-
-  /**
-   * Sets the Node properties spaecific to OM/SCM.
-   */
-  private void setNodeProperties() {
-    Properties nodeProperties = getNodeProperties();
-    if (nodeProperties != null) {
-      for (String key : nodeProperties.stringPropertyNames()) {
-        storageInfo.setProperty(key, nodeProperties.getProperty(key));
-      }
-    }
-  }
-
-  /**
-   * Directory {@code current} contains latest files defining
-   * the file system meta-data.
-   *
-   * @return the directory path
-   */
-  private File getCurrentDir() {
-    return new File(storageDir, STORAGE_DIR_CURRENT);
-  }
-
-  /**
-   * File {@code VERSION} contains the following fields:
-   * <ol>
-   * <li>node type</li>
-   * <li>OM/SCM state creation time</li>
-   * <li>other fields specific for this node type</li>
-   * </ol>
-   * The version file is always written last during storage directory updates.
-   * The existence of the version file indicates that all other files have
-   * been successfully written in the storage directory, the storage is valid
-   * and does not need to be recovered.
-   *
-   * @return the version file path
-   */
-  private File getVersionFile() {
-    return new File(getCurrentDir(), STORAGE_FILE_VERSION);
-  }
-
-
-  /**
-   * Check to see if current/ directory is empty. This method is used
-   * before determining to format the directory.
-   * @throws IOException if unable to list files under the directory.
-   */
-  private void checkEmptyCurrent() throws IOException {
-    File currentDir = getCurrentDir();
-    if (!currentDir.exists()) {
-      // if current/ does not exist, it's safe to format it.
-      return;
-    }
-    try (DirectoryStream<Path> dirStream = Files
-        .newDirectoryStream(currentDir.toPath())) {
-      if (dirStream.iterator().hasNext()) {
-        throw new InconsistentStorageStateException(getCurrentDir(),
-            "Can't initialize the storage directory because the current "
-                + "it is not empty.");
-      }
-    }
-  }
-
-  /**
-   * Check consistency of the storage directory.
-   *
-   * @return state {@link StorageState} of the storage directory
-   * @throws IOException
-   */
-  private StorageState getStorageState() throws IOException {
-    assert root != null : "root is null";
-    String rootPath = root.getCanonicalPath();
-    try { // check that storage exists
-      if (!root.exists()) {
-        // storage directory does not exist
-        LOG.warn("Storage directory " + rootPath + " does not exist");
-        return StorageState.NON_EXISTENT;
-      }
-      // or is inaccessible
-      if (!root.isDirectory()) {
-        LOG.warn(rootPath + "is not a directory");
-        return StorageState.NON_EXISTENT;
-      }
-      if (!FileUtil.canWrite(root)) {
-        LOG.warn("Cannot access storage directory " + rootPath);
-        return StorageState.NON_EXISTENT;
-      }
-    } catch (SecurityException ex) {
-      LOG.warn("Cannot access storage directory " + rootPath, ex);
-      return StorageState.NON_EXISTENT;
-    }
-
-    // check whether current directory is valid
-    File versionFile = getVersionFile();
-    boolean hasCurrent = versionFile.exists();
-
-    if (hasCurrent) {
-      return StorageState.INITIALIZED;
-    } else {
-      checkEmptyCurrent();
-      return StorageState.NOT_INITIALIZED;
-    }
-  }
-
-  /**
-   * Creates the Version file if not present,
-   * otherwise returns with IOException.
-   * @throws IOException
-   */
-  public void initialize() throws IOException {
-    if (state == StorageState.INITIALIZED) {
-      throw new IOException("Storage directory already initialized.");
-    }
-    if (!getCurrentDir().mkdirs()) {
-      throw new IOException("Cannot create directory " + getCurrentDir());
-    }
-    storageInfo.writeTo(getVersionFile());
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/StorageInfo.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/StorageInfo.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/StorageInfo.java
deleted file mode 100644
index 0e98a4c..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/StorageInfo.java
+++ /dev/null
@@ -1,183 +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.common;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.util.Properties;
-import java.util.UUID;
-
-/**
- * Common class for storage information. This class defines the common
- * properties and functions to set them , write them into the version file
- * and read them from the version file.
- *
- */
-@InterfaceAudience.Private
-public class StorageInfo {
-
-  private Properties properties = new Properties();
-
-  /**
-   * Property to hold node type.
-   */
-  private static final String NODE_TYPE = "nodeType";
-  /**
-   * Property to hold ID of the cluster.
-   */
-  private static final String CLUSTER_ID = "clusterID";
-  /**
-   * Property to hold creation time of the storage.
-   */
-  private static final String CREATION_TIME = "cTime";
-
-  /**
-   * Constructs StorageInfo instance.
-   * @param type
-   *          Type of the node using the storage
-   * @param cid
-   *          Cluster ID
-   * @param cT
-   *          Cluster creation Time
-
-   * @throws IOException
-   */
-  public StorageInfo(NodeType type, String cid, long cT)
-      throws IOException {
-    Preconditions.checkNotNull(type);
-    Preconditions.checkNotNull(cid);
-    Preconditions.checkNotNull(cT);
-    properties.setProperty(NODE_TYPE, type.name());
-    properties.setProperty(CLUSTER_ID, cid);
-    properties.setProperty(CREATION_TIME, String.valueOf(cT));
-  }
-
-  public StorageInfo(NodeType type, File propertiesFile)
-      throws IOException {
-    this.properties = readFrom(propertiesFile);
-    verifyNodeType(type);
-    verifyClusterId();
-    verifyCreationTime();
-  }
-
-  public NodeType getNodeType() {
-    return NodeType.valueOf(properties.getProperty(NODE_TYPE));
-  }
-
-  public String getClusterID() {
-    return properties.getProperty(CLUSTER_ID);
-  }
-
-  public Long  getCreationTime() {
-    String creationTime = properties.getProperty(CREATION_TIME);
-    if(creationTime != null) {
-      return Long.parseLong(creationTime);
-    }
-    return null;
-  }
-
-  public String getProperty(String key) {
-    return properties.getProperty(key);
-  }
-
-  public void setProperty(String key, String value) {
-    properties.setProperty(key, value);
-  }
-
-  public void setClusterId(String clusterId) {
-    properties.setProperty(CLUSTER_ID, clusterId);
-  }
-
-  private void verifyNodeType(NodeType type)
-      throws InconsistentStorageStateException {
-    NodeType nodeType = getNodeType();
-    Preconditions.checkNotNull(nodeType);
-    if(type != nodeType) {
-      throw new InconsistentStorageStateException("Expected NodeType: " + type 
+
-          ", but found: " + nodeType);
-    }
-  }
-
-  private void verifyClusterId()
-      throws InconsistentStorageStateException {
-    String clusterId = getClusterID();
-    Preconditions.checkNotNull(clusterId);
-    if(clusterId.isEmpty()) {
-      throw new InconsistentStorageStateException("Cluster ID not found");
-    }
-  }
-
-  private void verifyCreationTime() {
-    Long creationTime = getCreationTime();
-    Preconditions.checkNotNull(creationTime);
-  }
-
-
-  public void writeTo(File to)
-      throws IOException {
-    try (RandomAccessFile file = new RandomAccessFile(to, "rws");
-         FileOutputStream out = new FileOutputStream(file.getFD())) {
-      file.seek(0);
-    /*
-     * 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.setLength(out.getChannel().position());
-    }
-  }
-
-  private Properties readFrom(File from) throws IOException {
-    try (RandomAccessFile file = new RandomAccessFile(from, "rws");
-        FileInputStream in = new FileInputStream(file.getFD())) {
-      Properties props = new Properties();
-      file.seek(0);
-      props.load(in);
-      return props;
-    }
-  }
-
-  /**
-   * Generate new clusterID.
-   *
-   * clusterID is a persistent attribute of the cluster.
-   * It is generated when the cluster is created and remains the same
-   * during the life cycle of the cluster.  When a new SCM node is initialized,
-   * if this is a new cluster, a new clusterID is generated and stored.
-   * @return new clusterID
-   */
-  public static String newClusterID() {
-    return "CID-" + UUID.randomUUID().toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/package-info.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/package-info.java
deleted file mode 100644
index 6517e58..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/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.common;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/InvalidStateTransitionException.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/InvalidStateTransitionException.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/InvalidStateTransitionException.java
deleted file mode 100644
index 9aeff24..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/InvalidStateTransitionException.java
+++ /dev/null
@@ -1,42 +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.common.statemachine;
-
-/**
- * Class wraps invalid state transition exception.
- */
-public class InvalidStateTransitionException extends Exception {
-  private Enum<?> currentState;
-  private Enum<?> event;
-
-  public InvalidStateTransitionException(Enum<?> currentState, Enum<?> event) {
-    super("Invalid event: " + event + " at " + currentState + " state.");
-    this.currentState = currentState;
-    this.event = event;
-  }
-
-  public Enum<?> getCurrentState() {
-    return currentState;
-  }
-
-  public Enum<?> getEvent() {
-    return event;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/StateMachine.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/StateMachine.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/StateMachine.java
deleted file mode 100644
index bf8cbd5..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/StateMachine.java
+++ /dev/null
@@ -1,68 +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.common.statemachine;
-
-import com.google.common.base.Supplier;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Template class that wraps simple event driven state machine.
- * @param <STATE> states allowed
- * @param <EVENT> events allowed
- */
-public class StateMachine<STATE extends Enum<?>, EVENT extends Enum<?>> {
-  private STATE initialState;
-  private Set<STATE> finalStates;
-
-  private final LoadingCache<EVENT, Map<STATE, STATE>> transitions =
-      CacheBuilder.newBuilder().build(
-          CacheLoader.from((Supplier<Map<STATE, STATE>>) () -> new HashMap()));
-
-  public StateMachine(STATE initState, Set<STATE> finalStates) {
-    this.initialState = initState;
-    this.finalStates = finalStates;
-  }
-
-  public STATE getInitialState() {
-    return initialState;
-  }
-
-  public Set<STATE> getFinalStates() {
-    return finalStates;
-  }
-
-  public STATE getNextState(STATE from, EVENT e)
-      throws InvalidStateTransitionException {
-    STATE target = transitions.getUnchecked(e).get(from);
-    if (target == null) {
-      throw new InvalidStateTransitionException(from, e);
-    }
-    return target;
-  }
-
-  public void addTransition(STATE from, STATE to, EVENT e) {
-    transitions.getUnchecked(e).put(from, to);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/package-info.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/package-info.java
deleted file mode 100644
index 045409e..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/statemachine/package-info.java
+++ /dev/null
@@ -1,21 +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.common.statemachine;
-/**
- state machine template class for ozone.
- **/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java
deleted file mode 100644
index 0c1d427..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java
+++ /dev/null
@@ -1,255 +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.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.client.BlockID;
-import com.google.common.base.Preconditions;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.ArrayList;
-
-/**
- * Helper class to convert Protobuf to Java classes.
- */
-public class BlockData {
-  private final BlockID blockID;
-  private final Map<String, String> metadata;
-
-  /**
-   * Represent a list of chunks.
-   * In order to reduce memory usage, chunkList is declared as an
-   * {@link Object}.
-   * When #elements == 0, chunkList is null.
-   * When #elements == 1, chunkList refers to the only element.
-   * When #elements > 1, chunkList refers to the list.
-   *
-   * Please note : when we are working with blocks, we don't care what they
-   * point to. So we We don't read chunkinfo nor validate them. It is
-   * responsibility of higher layer like ozone. We just read and write data
-   * from network.
-   */
-  private Object chunkList;
-
-  /**
-   * total size of the key.
-   */
-  private long size;
-
-  /**
-   * Constructs a BlockData Object.
-   *
-   * @param blockID
-   */
-  public BlockData(BlockID blockID) {
-    this.blockID = blockID;
-    this.metadata = new TreeMap<>();
-    this.size = 0;
-  }
-
-  /**
-   * Returns a blockData object from the protobuf data.
-   *
-   * @param data - Protobuf data.
-   * @return - BlockData
-   * @throws IOException
-   */
-  public static BlockData getFromProtoBuf(ContainerProtos.BlockData data) 
throws
-      IOException {
-    BlockData blockData = new BlockData(
-        BlockID.getFromProtobuf(data.getBlockID()));
-    for (int x = 0; x < data.getMetadataCount(); x++) {
-      blockData.addMetadata(data.getMetadata(x).getKey(),
-          data.getMetadata(x).getValue());
-    }
-    blockData.setChunks(data.getChunksList());
-    if (data.hasSize()) {
-      Preconditions.checkArgument(data.getSize() == blockData.getSize());
-    }
-    return blockData;
-  }
-
-  /**
-   * Returns a Protobuf message from BlockData.
-   * @return Proto Buf Message.
-   */
-  public ContainerProtos.BlockData getProtoBufMessage() {
-    ContainerProtos.BlockData.Builder builder =
-        ContainerProtos.BlockData.newBuilder();
-    builder.setBlockID(this.blockID.getDatanodeBlockIDProtobuf());
-    for (Map.Entry<String, String> entry : metadata.entrySet()) {
-      ContainerProtos.KeyValue.Builder keyValBuilder =
-          ContainerProtos.KeyValue.newBuilder();
-      builder.addMetadata(keyValBuilder.setKey(entry.getKey())
-          .setValue(entry.getValue()).build());
-    }
-    builder.addAllChunks(getChunks());
-    builder.setSize(size);
-    return builder.build();
-  }
-
-  /**
-   * Adds metadata.
-   *
-   * @param key   - Key
-   * @param value - Value
-   * @throws IOException
-   */
-  public synchronized void addMetadata(String key, String value) throws
-      IOException {
-    if (this.metadata.containsKey(key)) {
-      throw new IOException("This key already exists. Key " + key);
-    }
-    metadata.put(key, value);
-  }
-
-  public synchronized Map<String, String> getMetadata() {
-    return Collections.unmodifiableMap(this.metadata);
-  }
-
-  /**
-   * Returns value of a key.
-   */
-  public synchronized String getValue(String key) {
-    return metadata.get(key);
-  }
-
-  /**
-   * Deletes a metadata entry from the map.
-   *
-   * @param key - Key
-   */
-  public synchronized void deleteKey(String key) {
-    metadata.remove(key);
-  }
-
-  @SuppressWarnings("unchecked")
-  private List<ContainerProtos.ChunkInfo> castChunkList() {
-    return (List<ContainerProtos.ChunkInfo>)chunkList;
-  }
-
-  /**
-   * Returns chunks list.
-   *
-   * @return list of chunkinfo.
-   */
-  public List<ContainerProtos.ChunkInfo> getChunks() {
-    return chunkList == null? Collections.emptyList()
-        : chunkList instanceof ContainerProtos.ChunkInfo?
-            Collections.singletonList((ContainerProtos.ChunkInfo)chunkList)
-        : Collections.unmodifiableList(castChunkList());
-  }
-
-  /**
-   * Adds chinkInfo to the list.
-   */
-  public void addChunk(ContainerProtos.ChunkInfo chunkInfo) {
-    if (chunkList == null) {
-      chunkList = chunkInfo;
-    } else {
-      final List<ContainerProtos.ChunkInfo> list;
-      if (chunkList instanceof ContainerProtos.ChunkInfo) {
-        list = new ArrayList<>(2);
-        list.add((ContainerProtos.ChunkInfo)chunkList);
-        chunkList = list;
-      } else {
-        list = castChunkList();
-      }
-      list.add(chunkInfo);
-    }
-    size += chunkInfo.getLen();
-  }
-
-  /**
-   * removes the chunk.
-   */
-  public boolean removeChunk(ContainerProtos.ChunkInfo chunkInfo) {
-    final boolean removed;
-    if (chunkList instanceof List) {
-      final List<ContainerProtos.ChunkInfo> list = castChunkList();
-      removed = list.remove(chunkInfo);
-      if (list.size() == 1) {
-        chunkList = list.get(0);
-      }
-    } else if (chunkInfo.equals(chunkList)) {
-      chunkList = null;
-      removed = true;
-    } else {
-      removed = false;
-    }
-
-    if (removed) {
-      size -= chunkInfo.getLen();
-    }
-    return removed;
-  }
-
-  /**
-   * Returns container ID.
-   *
-   * @return long.
-   */
-  public long getContainerID() {
-    return blockID.getContainerID();
-  }
-
-  /**
-   * Returns LocalID.
-   * @return long.
-   */
-  public long getLocalID() {
-    return blockID.getLocalID();
-  }
-
-  /**
-   * Return Block ID.
-   * @return BlockID.
-   */
-  public BlockID getBlockID() {
-    return blockID;
-  }
-
-  /**
-   * Sets Chunk list.
-   *
-   * @param chunks - List of chunks.
-   */
-  public void setChunks(List<ContainerProtos.ChunkInfo> chunks) {
-    if (chunks == null) {
-      chunkList = null;
-      size = 0L;
-    } else {
-      final int n = chunks.size();
-      chunkList = n == 0? null: n == 1? chunks.get(0): chunks;
-      size = chunks.parallelStream().mapToLong(
-          ContainerProtos.ChunkInfo::getLen).sum();
-    }
-  }
-
-  /**
-   * Get the total size of chunks allocated for the key.
-   * @return total size of the key.
-   */
-  public long getSize() {
-    return size;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java
deleted file mode 100644
index 21916b5..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ChunkInfo.java
+++ /dev/null
@@ -1,184 +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.hdds.protocol.datanode.proto.ContainerProtos;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.TreeMap;
-
-/**
- * Java class that represents ChunkInfo ProtoBuf class. This helper class 
allows
- * us to convert to and from protobuf to normal java.
- */
-public class ChunkInfo {
-  private final String chunkName;
-  private final long offset;
-  private final long len;
-  private String checksum;
-  private final Map<String, String> metadata;
-
-
-  /**
-   * Constructs a ChunkInfo.
-   *
-   * @param chunkName - File Name where chunk lives.
-   * @param offset    - offset where Chunk Starts.
-   * @param len       - Length of the Chunk.
-   */
-  public ChunkInfo(String chunkName, long offset, long len) {
-    this.chunkName = chunkName;
-    this.offset = offset;
-    this.len = len;
-    this.metadata = new TreeMap<>();
-  }
-
-  /**
-   * Adds metadata.
-   *
-   * @param key   - Key Name.
-   * @param value - Value.
-   * @throws IOException
-   */
-  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);
-    }
-  }
-
-  /**
-   * Gets a Chunkinfo class from the protobuf definitions.
-   *
-   * @param info - Protobuf class
-   * @return ChunkInfo
-   * @throws IOException
-   */
-  public static ChunkInfo getFromProtoBuf(ContainerProtos.ChunkInfo info)
-      throws IOException {
-    Preconditions.checkNotNull(info);
-
-    ChunkInfo chunkInfo = new ChunkInfo(info.getChunkName(), info.getOffset(),
-        info.getLen());
-
-    for (int x = 0; x < info.getMetadataCount(); x++) {
-      chunkInfo.addMetadata(info.getMetadata(x).getKey(),
-          info.getMetadata(x).getValue());
-    }
-
-
-    if (info.hasChecksum()) {
-      chunkInfo.setChecksum(info.getChecksum());
-    }
-    return chunkInfo;
-  }
-
-  /**
-   * Returns a ProtoBuf Message from ChunkInfo.
-   *
-   * @return Protocol Buffer Message
-   */
-  public ContainerProtos.ChunkInfo getProtoBufMessage() {
-    ContainerProtos.ChunkInfo.Builder builder = ContainerProtos
-        .ChunkInfo.newBuilder();
-
-    builder.setChunkName(this.getChunkName());
-    builder.setOffset(this.getOffset());
-    builder.setLen(this.getLen());
-    if (this.getChecksum() != null && !this.getChecksum().isEmpty()) {
-      builder.setChecksum(this.getChecksum());
-    }
-
-    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());
-    }
-
-    return builder.build();
-  }
-
-  /**
-   * Returns the chunkName.
-   *
-   * @return - String
-   */
-  public String getChunkName() {
-    return chunkName;
-  }
-
-  /**
-   * Gets the start offset of the given chunk in physical file.
-   *
-   * @return - long
-   */
-  public long getOffset() {
-    return offset;
-  }
-
-  /**
-   * Returns the length of the Chunk.
-   *
-   * @return long
-   */
-  public long getLen() {
-    return len;
-  }
-
-  /**
-   * Returns the SHA256 value of this chunk.
-   *
-   * @return - Hash String
-   */
-  public String getChecksum() {
-    return checksum;
-  }
-
-  /**
-   * Sets the Hash value of this chunk.
-   *
-   * @param checksum - Hash String.
-   */
-  public void setChecksum(String checksum) {
-    this.checksum = checksum;
-  }
-
-  /**
-   * Returns Metadata associated with this Chunk.
-   *
-   * @return - Map of Key,values.
-   */
-  public Map<String, String> getMetadata() {
-    return metadata;
-  }
-
-  @Override
-  public String toString() {
-    return "ChunkInfo{" +
-        "chunkName='" + chunkName +
-        ", offset=" + offset +
-        ", len=" + len +
-        '}';
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/package-info.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/package-info.java
deleted file mode 100644
index fa5df11..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/package-info.java
+++ /dev/null
@@ -1,23 +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;
-
-/**
- * Helper classes for the container protocol communication.
- */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/Lease.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/Lease.java 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/Lease.java
deleted file mode 100644
index dfa9315..0000000
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/Lease.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
- * <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.lease;
-
-import org.apache.hadoop.util.Time;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.Callable;
-
-/**
- * This class represents the lease created on a resource. Callback can be
- * registered on the lease which will be executed in case of timeout.
- *
- * @param <T> Resource type for which the lease can be associated
- */
-public class Lease<T> {
-
-  /**
-   * The resource for which this lease is created.
-   */
-  private final T resource;
-
-  private final long creationTime;
-
-  /**
-   * Lease lifetime in milliseconds.
-   */
-  private volatile long leaseTimeout;
-
-  private boolean expired;
-
-  /**
-   * Functions to be called in case of timeout.
-   */
-  private List<Callable<Void>> callbacks;
-
-
-  /**
-   * Creates a lease on the specified resource with given timeout.
-   *
-   * @param resource
-   *        Resource for which the lease has to be created
-   * @param timeout
-   *        Lease lifetime in milliseconds
-   */
-  public Lease(T resource, long timeout) {
-    this.resource = resource;
-    this.leaseTimeout = timeout;
-    this.callbacks = Collections.synchronizedList(new ArrayList<>());
-    this.creationTime = Time.monotonicNow();
-    this.expired = false;
-  }
-
-  /**
-   * Returns true if the lease has expired, else false.
-   *
-   * @return true if expired, else false
-   */
-  public boolean hasExpired() {
-    return expired;
-  }
-
-  /**
-   * Registers a callback which will be executed in case of timeout. Callbacks
-   * are executed in a separate Thread.
-   *
-   * @param callback
-   *        The Callable which has to be executed
-   * @throws LeaseExpiredException
-   *         If the lease has already timed out
-   */
-  public void registerCallBack(Callable<Void> callback)
-      throws LeaseExpiredException {
-    if(hasExpired()) {
-      throw new LeaseExpiredException("Resource: " + resource);
-    }
-    callbacks.add(callback);
-  }
-
-  /**
-   * Returns the time elapsed since the creation of lease.
-   *
-   * @return elapsed time in milliseconds
-   * @throws LeaseExpiredException
-   *         If the lease has already timed out
-   */
-  public long getElapsedTime() throws LeaseExpiredException {
-    if(hasExpired()) {
-      throw new LeaseExpiredException("Resource: " + resource);
-    }
-    return Time.monotonicNow() - creationTime;
-  }
-
-  /**
-   * Returns the time available before timeout.
-   *
-   * @return remaining time in milliseconds
-   * @throws LeaseExpiredException
-   *         If the lease has already timed out
-   */
-  public long getRemainingTime() throws LeaseExpiredException {
-    if(hasExpired()) {
-      throw new LeaseExpiredException("Resource: " + resource);
-    }
-    return leaseTimeout - getElapsedTime();
-  }
-
-  /**
-   * Returns total lease lifetime.
-   *
-   * @return total lifetime of lease in milliseconds
-   * @throws LeaseExpiredException
-   *         If the lease has already timed out
-   */
-  public long getLeaseLifeTime() throws LeaseExpiredException {
-    if(hasExpired()) {
-      throw new LeaseExpiredException("Resource: " + resource);
-    }
-    return leaseTimeout;
-  }
-
-  /**
-   * Renews the lease timeout period.
-   *
-   * @param timeout
-   *        Time to be added to the lease in milliseconds
-   * @throws LeaseExpiredException
-   *         If the lease has already timed out
-   */
-  public void renew(long timeout) throws LeaseExpiredException {
-    if(hasExpired()) {
-      throw new LeaseExpiredException("Resource: " + resource);
-    }
-    leaseTimeout += timeout;
-  }
-
-  @Override
-  public int hashCode() {
-    return resource.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if(obj instanceof Lease) {
-      return resource.equals(((Lease) obj).resource);
-    }
-    return false;
-  }
-
-  @Override
-  public String toString() {
-    return "Lease<" + resource.toString() + ">";
-  }
-
-  /**
-   * Returns the callbacks to be executed for the lease in case of timeout.
-   *
-   * @return callbacks to be executed
-   */
-  List<Callable<Void>> getCallbacks() {
-    return callbacks;
-  }
-
-  /**
-   * Expires/Invalidates the lease.
-   */
-  void invalidate() {
-    callbacks = null;
-    expired = true;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseAlreadyExistException.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseAlreadyExistException.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseAlreadyExistException.java
deleted file mode 100644
index a39ea22..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseAlreadyExistException.java
+++ /dev/null
@@ -1,46 +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.lease;
-
-/**
- * This exception represents that there is already a lease acquired on the
- * same resource.
- */
-public class LeaseAlreadyExistException  extends LeaseException {
-
-  /**
-   * Constructs an {@code LeaseAlreadyExistException} with {@code null}
-   * as its error detail message.
-   */
-  public LeaseAlreadyExistException() {
-    super();
-  }
-
-  /**
-   * Constructs an {@code LeaseAlreadyExistException} with the specified
-   * detail message.
-   *
-   * @param message
-   *        The detail message (which is saved for later retrieval
-   *        by the {@link #getMessage()} method)
-   */
-  public LeaseAlreadyExistException(String message) {
-    super(message);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseCallbackExecutor.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseCallbackExecutor.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseCallbackExecutor.java
deleted file mode 100644
index 1b7391b..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseCallbackExecutor.java
+++ /dev/null
@@ -1,65 +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.lease;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.List;
-import java.util.concurrent.Callable;
-
-/**
- * This class is responsible for executing the callbacks of a lease in case of
- * timeout.
- */
-public class LeaseCallbackExecutor<T> implements Runnable {
-
-  private static final Logger LOG = LoggerFactory.getLogger(Lease.class);
-
-  private final T resource;
-  private final List<Callable<Void>> callbacks;
-
-  /**
-   * Constructs LeaseCallbackExecutor instance with list of callbacks.
-   *
-   * @param resource
-   *        The resource for which the callbacks are executed
-   * @param callbacks
-   *        Callbacks to be executed by this executor
-   */
-  public LeaseCallbackExecutor(T resource, List<Callable<Void>> callbacks) {
-    this.resource = resource;
-    this.callbacks = callbacks;
-  }
-
-  @Override
-  public void run() {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Executing callbacks for lease on {}", resource);
-    }
-    for(Callable<Void> callback : callbacks) {
-      try {
-        callback.call();
-      } catch (Exception e) {
-        LOG.warn("Exception while executing callback for lease on {}",
-            resource, e);
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseException.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseException.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseException.java
deleted file mode 100644
index 418f412..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseException.java
+++ /dev/null
@@ -1,45 +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.lease;
-
-/**
- * This exception represents all lease related exceptions.
- */
-public class LeaseException extends Exception {
-
-  /**
-   * Constructs an {@code LeaseException} with {@code null}
-   * as its error detail message.
-   */
-  public LeaseException() {
-    super();
-  }
-
-  /**
-   * Constructs an {@code LeaseException} with the specified
-   * detail message.
-   *
-   * @param message
-   *        The detail message (which is saved for later retrieval
-   *        by the {@link #getMessage()} method)
-   */
-  public LeaseException(String message) {
-    super(message);
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseExpiredException.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseExpiredException.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseExpiredException.java
deleted file mode 100644
index 440a023..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseExpiredException.java
+++ /dev/null
@@ -1,45 +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.lease;
-
-/**
- * This exception represents that the lease that is being accessed has expired.
- */
-public class LeaseExpiredException extends LeaseException {
-
-  /**
-   * Constructs an {@code LeaseExpiredException} with {@code null}
-   * as its error detail message.
-   */
-  public LeaseExpiredException() {
-    super();
-  }
-
-  /**
-   * Constructs an {@code LeaseExpiredException} with the specified
-   * detail message.
-   *
-   * @param message
-   *        The detail message (which is saved for later retrieval
-   *        by the {@link #getMessage()} method)
-   */
-  public LeaseExpiredException(String message) {
-    super(message);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.java
deleted file mode 100644
index 756a41a..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.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
- * <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.lease;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-/**
- * LeaseManager is someone who can provide you leases based on your
- * requirement. If you want to return the lease back before it expires,
- * you can give it back to Lease Manager. He is the one responsible for
- * the lifecycle of leases. The resource for which lease is created
- * should have proper {@code equals} method implementation, resource
- * equality is checked while the lease is created.
- *
- * @param <T> Type of leases that this lease manager can create
- */
-public class LeaseManager<T> {
-
-  private static final Logger LOG =
-      LoggerFactory.getLogger(LeaseManager.class);
-
-  private final String name;
-  private final long defaultTimeout;
-  private Map<T, Lease<T>> activeLeases;
-  private LeaseMonitor leaseMonitor;
-  private Thread leaseMonitorThread;
-  private boolean isRunning;
-
-  /**
-   * Creates an instance of lease manager.
-   *
-   * @param name
-   *        Name for the LeaseManager instance.
-   * @param defaultTimeout
-   *        Default timeout in milliseconds to be used for lease creation.
-   */
-  public LeaseManager(String name, long defaultTimeout) {
-    this.name = name;
-    this.defaultTimeout = defaultTimeout;
-  }
-
-  /**
-   * Starts the lease manager service.
-   */
-  public void start() {
-    LOG.debug("Starting {} LeaseManager service", name);
-    activeLeases = new ConcurrentHashMap<>();
-    leaseMonitor = new LeaseMonitor();
-    leaseMonitorThread = new Thread(leaseMonitor);
-    leaseMonitorThread.setName(name + "-LeaseManager#LeaseMonitor");
-    leaseMonitorThread.setDaemon(true);
-    leaseMonitorThread.setUncaughtExceptionHandler((thread, throwable) -> {
-      // Let us just restart this thread after logging an error.
-      // if this thread is not running we cannot handle Lease expiry.
-      LOG.error("LeaseMonitor thread encountered an error. Thread: {}",
-          thread.toString(), throwable);
-      leaseMonitorThread.start();
-    });
-    LOG.debug("Starting {}-LeaseManager#LeaseMonitor Thread", name);
-    leaseMonitorThread.start();
-    isRunning = true;
-  }
-
-  /**
-   * Returns a lease for the specified resource with default timeout.
-   *
-   * @param resource
-   *        Resource for which lease has to be created
-   * @throws LeaseAlreadyExistException
-   *         If there is already a lease on the resource
-   */
-  public synchronized Lease<T> acquire(T resource)
-      throws LeaseAlreadyExistException {
-    return acquire(resource, defaultTimeout);
-  }
-
-  /**
-   * Returns a lease for the specified resource with the timeout provided.
-   *
-   * @param resource
-   *        Resource for which lease has to be created
-   * @param timeout
-   *        The timeout in milliseconds which has to be set on the lease
-   * @throws LeaseAlreadyExistException
-   *         If there is already a lease on the resource
-   */
-  public synchronized Lease<T> acquire(T resource, long timeout)
-      throws LeaseAlreadyExistException {
-    checkStatus();
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Acquiring lease on {} for {} milliseconds", resource, 
timeout);
-    }
-    if(activeLeases.containsKey(resource)) {
-      throw new LeaseAlreadyExistException("Resource: " + resource);
-    }
-    Lease<T> lease = new Lease<>(resource, timeout);
-    activeLeases.put(resource, lease);
-    leaseMonitorThread.interrupt();
-    return lease;
-  }
-
-  /**
-   * Returns a lease associated with the specified resource.
-   *
-   * @param resource
-   *        Resource for which the lease has to be returned
-   * @throws LeaseNotFoundException
-   *         If there is no active lease on the resource
-   */
-  public Lease<T> get(T resource) throws LeaseNotFoundException {
-    checkStatus();
-    Lease<T> lease = activeLeases.get(resource);
-    if(lease != null) {
-      return lease;
-    }
-    throw new LeaseNotFoundException("Resource: " + resource);
-  }
-
-  /**
-   * Releases the lease associated with the specified resource.
-   *
-   * @param resource
-   *        The for which the lease has to be released
-   * @throws LeaseNotFoundException
-   *         If there is no active lease on the resource
-   */
-  public synchronized void release(T resource)
-      throws LeaseNotFoundException {
-    checkStatus();
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Releasing lease on {}", resource);
-    }
-    Lease<T> lease = activeLeases.remove(resource);
-    if(lease == null) {
-      throw new LeaseNotFoundException("Resource: " + resource);
-    }
-    lease.invalidate();
-  }
-
-  /**
-   * Shuts down the LeaseManager and releases the resources. All the active
-   * {@link Lease} will be released (callbacks on leases will not be
-   * executed).
-   */
-  public void shutdown() {
-    checkStatus();
-    LOG.debug("Shutting down LeaseManager service");
-    leaseMonitor.disable();
-    leaseMonitorThread.interrupt();
-    for(T resource : activeLeases.keySet()) {
-      try {
-        release(resource);
-      }  catch(LeaseNotFoundException ex) {
-        //Ignore the exception, someone might have released the lease
-      }
-    }
-    isRunning = false;
-  }
-
-  /**
-   * Throws {@link LeaseManagerNotRunningException} if the service is not
-   * running.
-   */
-  private void checkStatus() {
-    if(!isRunning) {
-      throw new LeaseManagerNotRunningException("LeaseManager not running.");
-    }
-  }
-
-  /**
-   * Monitors the leases and expires them based on the timeout, also
-   * responsible for executing the callbacks of expired leases.
-   */
-  private final class LeaseMonitor implements Runnable {
-
-    private boolean monitor = true;
-    private ExecutorService executorService;
-
-    private LeaseMonitor() {
-      this.monitor = true;
-      this.executorService = Executors.newCachedThreadPool();
-    }
-
-    @Override
-    public void run() {
-      while(monitor) {
-        LOG.debug("{}-LeaseMonitor: checking for lease expiry", name);
-        long sleepTime = Long.MAX_VALUE;
-
-        for (T resource : activeLeases.keySet()) {
-          try {
-            Lease<T> lease = get(resource);
-            long remainingTime = lease.getRemainingTime();
-            if (remainingTime <= 0) {
-              //Lease has timed out
-              List<Callable<Void>> leaseCallbacks = lease.getCallbacks();
-              release(resource);
-              executorService.execute(
-                  new LeaseCallbackExecutor(resource, leaseCallbacks));
-            } else {
-              sleepTime = remainingTime > sleepTime ?
-                  sleepTime : remainingTime;
-            }
-          } catch (LeaseNotFoundException | LeaseExpiredException ex) {
-            //Ignore the exception, someone might have released the lease
-          }
-        }
-
-        try {
-          if(!Thread.interrupted()) {
-            Thread.sleep(sleepTime);
-          }
-        } catch (InterruptedException ignored) {
-          // This means a new lease is added to activeLeases.
-        }
-      }
-    }
-
-    /**
-     * Disables lease monitor, next interrupt call on the thread
-     * will stop lease monitor.
-     */
-    public void disable() {
-      monitor = false;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManagerNotRunningException.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManagerNotRunningException.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManagerNotRunningException.java
deleted file mode 100644
index ced31de..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManagerNotRunningException.java
+++ /dev/null
@@ -1,45 +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.lease;
-
-/**
- * This exception represents that there LeaseManager service is not running.
- */
-public class LeaseManagerNotRunningException  extends RuntimeException {
-
-  /**
-   * Constructs an {@code LeaseManagerNotRunningException} with {@code null}
-   * as its error detail message.
-   */
-  public LeaseManagerNotRunningException() {
-    super();
-  }
-
-  /**
-   * Constructs an {@code LeaseManagerNotRunningException} with the specified
-   * detail message.
-   *
-   * @param message
-   *        The detail message (which is saved for later retrieval
-   *        by the {@link #getMessage()} method)
-   */
-  public LeaseManagerNotRunningException(String message) {
-    super(message);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseNotFoundException.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseNotFoundException.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseNotFoundException.java
deleted file mode 100644
index c292d33..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseNotFoundException.java
+++ /dev/null
@@ -1,46 +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.lease;
-
-/**
- * This exception represents that the lease that is being accessed does not
- * exist.
- */
-public class LeaseNotFoundException extends LeaseException {
-
-  /**
-   * Constructs an {@code LeaseNotFoundException} with {@code null}
-   * as its error detail message.
-   */
-  public LeaseNotFoundException() {
-    super();
-  }
-
-  /**
-   * Constructs an {@code LeaseNotFoundException} with the specified
-   * detail message.
-   *
-   * @param message
-   *        The detail message (which is saved for later retrieval
-   *        by the {@link #getMessage()} method)
-   */
-  public LeaseNotFoundException(String message) {
-    super(message);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/package-info.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/package-info.java
deleted file mode 100644
index 48ee2e1..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/package-info.java
+++ /dev/null
@@ -1,26 +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.
- */
-
-/**
- * A generic lease management API which can be used if a service
- * needs any kind of lease management.
- */
-
-package org.apache.hadoop.ozone.lease;
-/*
- This package contains lease management related classes.
- */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c392da8/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/ActiveLock.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/ActiveLock.java 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/ActiveLock.java
deleted file mode 100644
index c302084..0000000
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lock/ActiveLock.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations 
under
- * the License.
- */
-
-package org.apache.hadoop.ozone.lock;
-
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-/**
- * Lock implementation which also maintains counter.
- */
-public final class ActiveLock {
-
-  private Lock lock;
-  private AtomicInteger count;
-
-  /**
-   * Use ActiveLock#newInstance to create instance.
-   */
-  private ActiveLock() {
-    this.lock = new ReentrantLock();
-    this.count = new AtomicInteger(0);
-  }
-
-  /**
-   * Creates a new instance of ActiveLock.
-   *
-   * @return new ActiveLock
-   */
-  public static ActiveLock newInstance() {
-    return new ActiveLock();
-  }
-
-  /**
-   * Acquires the lock.
-   *
-   * <p>If the lock is not available then the current thread becomes
-   * disabled for thread scheduling purposes and lies dormant until the
-   * lock has been acquired.
-   */
-  public void lock() {
-    lock.lock();
-  }
-
-  /**
-   * Releases the lock.
-   */
-  public void unlock() {
-    lock.unlock();
-  }
-
-  /**
-   * Increment the active count of the lock.
-   */
-  void incrementActiveCount() {
-    count.incrementAndGet();
-  }
-
-  /**
-   * Decrement the active count of the lock.
-   */
-  void decrementActiveCount() {
-    count.decrementAndGet();
-  }
-
-  /**
-   * Returns the active count on the lock.
-   *
-   * @return Number of active leases on the lock.
-   */
-  int getActiveLockCount() {
-    return count.get();
-  }
-
-  /**
-   * Resets the active count on the lock.
-   */
-  void resetCounter() {
-    count.set(0);
-  }
-
-  @Override
-  public String toString() {
-    return lock.toString();
-  }
-}


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

Reply via email to