http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/client/src/main/java/org/apache/hadoop/scm/storage/ChunkOutputStream.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/client/src/main/java/org/apache/hadoop/scm/storage/ChunkOutputStream.java
 
b/hadoop-hdsl/client/src/main/java/org/apache/hadoop/scm/storage/ChunkOutputStream.java
deleted file mode 100644
index 52a981f..0000000
--- 
a/hadoop-hdsl/client/src/main/java/org/apache/hadoop/scm/storage/ChunkOutputStream.java
+++ /dev/null
@@ -1,227 +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.scm.storage;
-
-import static org.apache.hadoop.scm.storage.ContainerProtocolCalls.putKey;
-import static org.apache.hadoop.scm.storage.ContainerProtocolCalls.writeChunk;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.util.UUID;
-
-import com.google.protobuf.ByteString;
-
-import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos.ChunkInfo;
-import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos.KeyData;
-import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.KeyValue;
-import org.apache.hadoop.scm.XceiverClientManager;
-import org.apache.hadoop.scm.XceiverClientSpi;
-
-/**
- * An {@link OutputStream} used by the REST service in combination with the
- * SCMClient to write the value of a key to a sequence
- * of container chunks.  Writes are buffered locally and periodically written 
to
- * the container as a new chunk.  In order to preserve the semantics that
- * replacement of a pre-existing key is atomic, each instance of the stream has
- * an internal unique identifier.  This unique identifier and a monotonically
- * increasing chunk index form a composite key that is used as the chunk name.
- * After all data is written, a putKey call creates or updates the 
corresponding
- * container key, and this call includes the full list of chunks that make up
- * the key data.  The list of chunks is updated all at once.  Therefore, a
- * concurrent reader never can see an intermediate state in which different
- * chunks of data from different versions of the key data are interleaved.
- * This class encapsulates all state management for buffering and writing
- * through to the container.
- */
-public class ChunkOutputStream extends OutputStream {
-
-  private final String containerKey;
-  private final String key;
-  private final String traceID;
-  private final KeyData.Builder containerKeyData;
-  private XceiverClientManager xceiverClientManager;
-  private XceiverClientSpi xceiverClient;
-  private ByteBuffer buffer;
-  private final String streamId;
-  private int chunkIndex;
-  private int chunkSize;
-
-  /**
-   * Creates a new ChunkOutputStream.
-   *
-   * @param containerKey container key
-   * @param key chunk key
-   * @param xceiverClientManager client manager that controls client
-   * @param xceiverClient client to perform container calls
-   * @param traceID container protocol call args
-   * @param chunkSize chunk size
-   */
-  public ChunkOutputStream(String containerKey, String key,
-      XceiverClientManager xceiverClientManager, XceiverClientSpi 
xceiverClient,
-      String traceID, int chunkSize) {
-    this.containerKey = containerKey;
-    this.key = key;
-    this.traceID = traceID;
-    this.chunkSize = chunkSize;
-    KeyValue keyValue = KeyValue.newBuilder()
-        .setKey("TYPE").setValue("KEY").build();
-    this.containerKeyData = KeyData.newBuilder()
-        .setContainerName(xceiverClient.getPipeline().getContainerName())
-        .setName(containerKey)
-        .addMetadata(keyValue);
-    this.xceiverClientManager = xceiverClientManager;
-    this.xceiverClient = xceiverClient;
-    this.buffer = ByteBuffer.allocate(chunkSize);
-    this.streamId = UUID.randomUUID().toString();
-    this.chunkIndex = 0;
-  }
-
-  @Override
-  public synchronized void write(int b) throws IOException {
-    checkOpen();
-    int rollbackPosition = buffer.position();
-    int rollbackLimit = buffer.limit();
-    buffer.put((byte)b);
-    if (buffer.position() == chunkSize) {
-      flushBufferToChunk(rollbackPosition, rollbackLimit);
-    }
-  }
-
-  @Override
-  public void write(byte[] b, int off, int len) throws IOException {
-    if (b == null) {
-      throw new NullPointerException();
-    }
-    if ((off < 0) || (off > b.length) || (len < 0) ||
-        ((off + len) > b.length) || ((off + len) < 0)) {
-      throw new IndexOutOfBoundsException();
-    }
-    if (len == 0) {
-      return;
-    }
-    checkOpen();
-    while (len > 0) {
-      int writeLen = Math.min(chunkSize - buffer.position(), len);
-      int rollbackPosition = buffer.position();
-      int rollbackLimit = buffer.limit();
-      buffer.put(b, off, writeLen);
-      if (buffer.position() == chunkSize) {
-        flushBufferToChunk(rollbackPosition, rollbackLimit);
-      }
-      off += writeLen;
-      len -= writeLen;
-    }
-  }
-
-  @Override
-  public synchronized void flush() throws IOException {
-    checkOpen();
-    if (buffer.position() > 0) {
-      int rollbackPosition = buffer.position();
-      int rollbackLimit = buffer.limit();
-      flushBufferToChunk(rollbackPosition, rollbackLimit);
-    }
-  }
-
-  @Override
-  public synchronized void close() throws IOException {
-    if (xceiverClientManager != null && xceiverClient != null &&
-        buffer != null) {
-      try {
-        if (buffer.position() > 0) {
-          writeChunkToContainer();
-        }
-        putKey(xceiverClient, containerKeyData.build(), traceID);
-      } catch (IOException e) {
-        throw new IOException(
-            "Unexpected Storage Container Exception: " + e.toString(), e);
-      } finally {
-        xceiverClientManager.releaseClient(xceiverClient);
-        xceiverClientManager = null;
-        xceiverClient = null;
-        buffer = null;
-      }
-    }
-
-  }
-
-  /**
-   * Checks if the stream is open.  If not, throws an exception.
-   *
-   * @throws IOException if stream is closed
-   */
-  private synchronized void checkOpen() throws IOException {
-    if (xceiverClient == null) {
-      throw new IOException("ChunkOutputStream has been closed.");
-    }
-  }
-
-  /**
-   * Attempts to flush buffered writes by writing a new chunk to the container.
-   * If successful, then clears the buffer to prepare to receive writes for a
-   * new chunk.
-   *
-   * @param rollbackPosition position to restore in buffer if write fails
-   * @param rollbackLimit limit to restore in buffer if write fails
-   * @throws IOException if there is an I/O error while performing the call
-   */
-  private synchronized void flushBufferToChunk(int rollbackPosition,
-      int rollbackLimit) throws IOException {
-    boolean success = false;
-    try {
-      writeChunkToContainer();
-      success = true;
-    } finally {
-      if (success) {
-        buffer.clear();
-      } else {
-        buffer.position(rollbackPosition);
-        buffer.limit(rollbackLimit);
-      }
-    }
-  }
-
-  /**
-   * Writes buffered data as a new chunk to the container and saves chunk
-   * information to be used later in putKey call.
-   *
-   * @throws IOException if there is an I/O error while performing the call
-   */
-  private synchronized void writeChunkToContainer() throws IOException {
-    buffer.flip();
-    ByteString data = ByteString.copyFrom(buffer);
-    ChunkInfo chunk = ChunkInfo
-        .newBuilder()
-        .setChunkName(
-            DigestUtils.md5Hex(key) + "_stream_"
-                + streamId + "_chunk_" + ++chunkIndex)
-        .setOffset(0)
-        .setLen(data.size())
-        .build();
-    try {
-      writeChunk(xceiverClient, chunk, key, data, traceID);
-    } catch (IOException e) {
-      throw new IOException(
-          "Unexpected Storage Container Exception: " + e.toString(), e);
-    }
-    containerKeyData.addChunks(chunk);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/client/src/main/java/org/apache/hadoop/scm/storage/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/client/src/main/java/org/apache/hadoop/scm/storage/package-info.java
 
b/hadoop-hdsl/client/src/main/java/org/apache/hadoop/scm/storage/package-info.java
deleted file mode 100644
index b367461..0000000
--- 
a/hadoop-hdsl/client/src/main/java/org/apache/hadoop/scm/storage/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.scm.storage;
-
-/**
- * Low level IO streams to upload/download chunks from container service.
- */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/common/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdsl/common/dev-support/findbugsExcludeFile.xml 
b/hadoop-hdsl/common/dev-support/findbugsExcludeFile.xml
deleted file mode 100644
index d93c4a1..0000000
--- a/hadoop-hdsl/common/dev-support/findbugsExcludeFile.xml
+++ /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.
--->
-<FindBugsFilter>
-  <Match>
-    <Package name="org.apache.hadoop.hdsl.protocol.proto"/>
-  </Match>
-</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdsl/common/pom.xml b/hadoop-hdsl/common/pom.xml
deleted file mode 100644
index c2323af..0000000
--- a/hadoop-hdsl/common/pom.xml
+++ /dev/null
@@ -1,129 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0";
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
-http://maven.apache.org/xsd/maven-4.0.0.xsd";>
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.hadoop</groupId>
-    <artifactId>hadoop-hdsl</artifactId>
-    <version>3.2.0-SNAPSHOT</version>
-  </parent>
-  <artifactId>hadoop-hdsl-common</artifactId>
-  <version>3.2.0-SNAPSHOT</version>
-  <description>Apache Hadoop HDSL Common utilities</description>
-  <name>Apache Hadoop HDSL Common</name>
-  <packaging>jar</packaging>
-
-  <properties>
-    <hadoop.component>hdsl</hadoop.component>
-    <is.hadoop.component>true</is.hadoop.component>
-  </properties>
-
-  <dependencies>
-
-    <dependency>
-      <groupId>org.fusesource.leveldbjni</groupId>
-      <artifactId>leveldbjni-all</artifactId>
-    </dependency>
-
-    <dependency>
-      <artifactId>ratis-server</artifactId>
-      <groupId>org.apache.ratis</groupId>
-      <exclusions>
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>slf4j-log4j12</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>io.dropwizard.metrics</groupId>
-          <artifactId>metrics-core</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <artifactId>ratis-netty</artifactId>
-      <groupId>org.apache.ratis</groupId>
-    </dependency>
-    <dependency>
-      <artifactId>ratis-grpc</artifactId>
-      <groupId>org.apache.ratis</groupId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.rocksdb</groupId>
-      <artifactId>rocksdbjni</artifactId>
-      <version>5.8.0</version>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-
-  </dependencies>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.hadoop</groupId>
-        <artifactId>hadoop-maven-plugins</artifactId>
-        <executions>
-          <execution>
-            <id>compile-protoc</id>
-            <goals>
-              <goal>protoc</goal>
-            </goals>
-            <configuration>
-              <protocVersion>${protobuf.version}</protocVersion>
-              <protocCommand>${protoc.path}</protocCommand>
-              <imports>
-                <param>
-                  
${basedir}/../../hadoop-common-project/hadoop-common/src/main/proto
-                </param>
-                <param>
-                  
${basedir}/../../hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/
-                </param>
-                <param>
-                  
${basedir}/../../hadoop-hdfs-project/hadoop-hdfs/src/main/proto/
-                </param>
-                <param>${basedir}/src/main/proto</param>
-              </imports>
-              <source>
-                <directory>${basedir}/src/main/proto</directory>
-                <includes>
-                  <include>StorageContainerLocationProtocol.proto</include>
-                  <include>DatanodeContainerProtocol.proto</include>
-                  <include>hdsl.proto</include>
-                  <include>ScmBlockLocationProtocol.proto</include>
-                </includes>
-              </source>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>findbugs-maven-plugin</artifactId>
-        <configuration>
-          
<excludeFilterFile>${basedir}/dev-support/findbugsExcludeFile.xml</excludeFilterFile>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/HdslUtils.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/HdslUtils.java 
b/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/HdslUtils.java
deleted file mode 100644
index 6446618..0000000
--- a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/HdslUtils.java
+++ /dev/null
@@ -1,273 +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.hdsl;
-
-import java.net.InetSocketAddress;
-
-import java.nio.file.Paths;
-import java.util.Collection;
-import java.util.HashSet;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.scm.ScmConfigKeys;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Strings;
-import com.google.common.net.HostAndPort;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED_DEFAULT;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * HDSL specific stateless utility functions.
- */
-public class HdslUtils {
-
-
-  private static final Logger LOG = LoggerFactory.getLogger(HdslUtils.class);
-
-  /**
-   * The service ID of the solitary Ozone SCM service.
-   */
-  public static final String OZONE_SCM_SERVICE_ID = "OzoneScmService";
-  public static final String OZONE_SCM_SERVICE_INSTANCE_ID =
-      "OzoneScmServiceInstance";
-
-  private static final int NO_PORT = -1;
-
-  private HdslUtils() {
-  }
-
-  /**
-   * Retrieve the socket address that should be used by clients to connect
-   * to the SCM.
-   *
-   * @param conf
-   * @return Target InetSocketAddress for the SCM client endpoint.
-   */
-  public static InetSocketAddress getScmAddressForClients(Configuration conf) {
-    final Optional<String> host = getHostNameFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
-
-    if (!host.isPresent()) {
-      throw new IllegalArgumentException(
-          ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY + " must be defined. See"
-              + " https://wiki.apache.org/hadoop/Ozone#Configuration for "
-              + "details"
-              + " on configuring Ozone.");
-    }
-
-    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
-
-    return NetUtils.createSocketAddr(host.get() + ":" + port
-        .or(ScmConfigKeys.OZONE_SCM_CLIENT_PORT_DEFAULT));
-  }
-
-  /**
-   * Retrieve the socket address that should be used by clients to connect
-   * to the SCM for block service. If
-   * {@link ScmConfigKeys#OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY} is not defined
-   * then {@link ScmConfigKeys#OZONE_SCM_CLIENT_ADDRESS_KEY} is used.
-   *
-   * @param conf
-   * @return Target InetSocketAddress for the SCM block client endpoint.
-   * @throws IllegalArgumentException if configuration is not defined.
-   */
-  public static InetSocketAddress getScmAddressForBlockClients(
-      Configuration conf) {
-    Optional<String> host = getHostNameFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY);
-
-    if (!host.isPresent()) {
-      host = getHostNameFromConfigKeys(conf,
-          ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY);
-      if (!host.isPresent()) {
-        throw new IllegalArgumentException(
-            ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY
-                + " must be defined. See"
-                + " https://wiki.apache.org/hadoop/Ozone#Configuration";
-                + " for details on configuring Ozone.");
-      }
-    }
-
-    final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
-        ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY);
-
-    return NetUtils.createSocketAddr(host.get() + ":" + port
-        .or(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_PORT_DEFAULT));
-  }
-
-  /**
-   * Retrieve the hostname, trying the supplied config keys in order.
-   * Each config value may be absent, or if present in the format
-   * host:port (the :port part is optional).
-   *
-   * @param conf  - Conf
-   * @param keys a list of configuration key names.
-   *
-   * @return first hostname component found from the given keys, or absent.
-   * @throws IllegalArgumentException if any values are not in the 'host'
-   *             or host:port format.
-   */
-  public static Optional<String> getHostNameFromConfigKeys(Configuration conf,
-      String... keys) {
-    for (final String key : keys) {
-      final String value = conf.getTrimmed(key);
-      final Optional<String> hostName = getHostName(value);
-      if (hostName.isPresent()) {
-        return hostName;
-      }
-    }
-    return Optional.absent();
-  }
-
-  /**
-   * Gets the hostname or Indicates that it is absent.
-   * @param value host or host:port
-   * @return hostname
-   */
-  public static Optional<String> getHostName(String value) {
-    if ((value == null) || value.isEmpty()) {
-      return Optional.absent();
-    }
-    return Optional.of(HostAndPort.fromString(value).getHostText());
-  }
-
-  /**
-   * Gets the port if there is one, throws otherwise.
-   * @param value  String in host:port format.
-   * @return Port
-   */
-  public static Optional<Integer> getHostPort(String value) {
-    if ((value == null) || value.isEmpty()) {
-      return Optional.absent();
-    }
-    int port = HostAndPort.fromString(value).getPortOrDefault(NO_PORT);
-    if (port == NO_PORT) {
-      return Optional.absent();
-    } else {
-      return Optional.of(port);
-    }
-  }
-
-  /**
-   * Retrieve the port number, trying the supplied config keys in order.
-   * Each config value may be absent, or if present in the format
-   * host:port (the :port part is optional).
-   *
-   * @param conf Conf
-   * @param keys a list of configuration key names.
-   *
-   * @return first port number component found from the given keys, or absent.
-   * @throws IllegalArgumentException if any values are not in the 'host'
-   *             or host:port format.
-   */
-  public static Optional<Integer> getPortNumberFromConfigKeys(
-      Configuration conf, String... keys) {
-    for (final String key : keys) {
-      final String value = conf.getTrimmed(key);
-      final Optional<Integer> hostPort = getHostPort(value);
-      if (hostPort.isPresent()) {
-        return hostPort;
-      }
-    }
-    return Optional.absent();
-  }
-
-  /**
-   * Retrieve the socket addresses of all storage container managers.
-   *
-   * @param conf
-   * @return A collection of SCM addresses
-   * @throws IllegalArgumentException If the configuration is invalid
-   */
-  public static Collection<InetSocketAddress> getSCMAddresses(
-      Configuration conf) throws IllegalArgumentException {
-    Collection<InetSocketAddress> addresses =
-        new HashSet<InetSocketAddress>();
-    Collection<String> names =
-        conf.getTrimmedStringCollection(ScmConfigKeys.OZONE_SCM_NAMES);
-    if (names == null || names.isEmpty()) {
-      throw new IllegalArgumentException(ScmConfigKeys.OZONE_SCM_NAMES
-          + " need to be a set of valid DNS names or IP addresses."
-          + " Null or empty address list found.");
-    }
-
-    final com.google.common.base.Optional<Integer>
-        defaultPort =  com.google.common.base.Optional.of(ScmConfigKeys
-        .OZONE_SCM_DEFAULT_PORT);
-    for (String address : names) {
-      com.google.common.base.Optional<String> hostname =
-          getHostName(address);
-      if (!hostname.isPresent()) {
-        throw new IllegalArgumentException("Invalid hostname for SCM: "
-            + hostname);
-      }
-      com.google.common.base.Optional<Integer> port =
-          getHostPort(address);
-      InetSocketAddress addr = NetUtils.createSocketAddr(hostname.get(),
-          port.or(defaultPort.get()));
-      addresses.add(addr);
-    }
-    return addresses;
-  }
-
-  public static boolean isHdslEnabled(Configuration conf) {
-    String securityEnabled =
-        conf.get(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
-            "simple");
-    boolean securityAuthorizationEnabled = conf.getBoolean(
-        CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, false);
-
-    if (securityEnabled.equals("kerberos") || securityAuthorizationEnabled) {
-      LOG.error("Ozone is not supported in a security enabled cluster. ");
-      return false;
-    } else {
-      return conf.getBoolean(OZONE_ENABLED, OZONE_ENABLED_DEFAULT);
-    }
-  }
-
-
-  /**
-   * Get the path for datanode id file.
-   *
-   * @param conf - Configuration
-   * @return the path of datanode id as string
-   */
-  public static String getDatanodeIdFilePath(Configuration conf) {
-    String dataNodeIDPath = conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID);
-    if (dataNodeIDPath == null) {
-      String metaPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
-      if (Strings.isNullOrEmpty(metaPath)) {
-        // this means meta data is not found, in theory should not happen at
-        // this point because should've failed earlier.
-        throw new IllegalArgumentException("Unable to locate meta data" +
-            "directory when getting datanode id path");
-      }
-      dataNodeIDPath = Paths.get(metaPath,
-          ScmConfigKeys.OZONE_SCM_DATANODE_ID_PATH_DEFAULT).toString();
-    }
-    return dataNodeIDPath;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/conf/OzoneConfiguration.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/conf/OzoneConfiguration.java
 
b/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/conf/OzoneConfiguration.java
deleted file mode 100644
index a185156..0000000
--- 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/conf/OzoneConfiguration.java
+++ /dev/null
@@ -1,161 +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.hdsl.conf;
-
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.List;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlElement;
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Configuration for ozone.
- */
-@InterfaceAudience.Private
-public class OzoneConfiguration extends Configuration {
-  static {
-    activate();
-  }
-
-  public OzoneConfiguration() {
-    OzoneConfiguration.activate();
-  }
-
-  public OzoneConfiguration(Configuration conf) {
-    super(conf);
-  }
-
-  public List<Property> readPropertyFromXml(URL url) throws JAXBException {
-    JAXBContext context = JAXBContext.newInstance(XMLConfiguration.class);
-    Unmarshaller um = context.createUnmarshaller();
-
-    XMLConfiguration config = (XMLConfiguration) um.unmarshal(url);
-    return config.getProperties();
-  }
-
-  /**
-   * Class to marshall/un-marshall configuration from xml files.
-   */
-  @XmlAccessorType(XmlAccessType.FIELD)
-  @XmlRootElement(name = "configuration")
-  public static class XMLConfiguration {
-
-    @XmlElement(name = "property", type = Property.class)
-    private List<Property> properties = new ArrayList<>();
-
-    public XMLConfiguration() {
-    }
-
-    public XMLConfiguration(List<Property> properties) {
-      this.properties = properties;
-    }
-
-    public List<Property> getProperties() {
-      return properties;
-    }
-
-    public void setProperties(List<Property> properties) {
-      this.properties = properties;
-    }
-  }
-
-  /**
-   * Class to marshall/un-marshall configuration properties from xml files.
-   */
-  @XmlAccessorType(XmlAccessType.FIELD)
-  @XmlRootElement(name = "property")
-  public static class Property implements Comparable<Property> {
-
-    private String name;
-    private String value;
-    private String tag;
-    private String description;
-
-    public String getName() {
-      return name;
-    }
-
-    public void setName(String name) {
-      this.name = name;
-    }
-
-    public String getValue() {
-      return value;
-    }
-
-    public void setValue(String value) {
-      this.value = value;
-    }
-
-    public String getTag() {
-      return tag;
-    }
-
-    public void setTag(String tag) {
-      this.tag = tag;
-    }
-
-    public String getDescription() {
-      return description;
-    }
-
-    public void setDescription(String description) {
-      this.description = description;
-    }
-
-    @Override
-    public int compareTo(Property o) {
-      if (this == o) {
-        return 0;
-      }
-      return this.getName().compareTo(o.getName());
-    }
-
-    @Override
-    public String toString() {
-      return this.getName() + " " + this.getValue() + this.getTag();
-    }
-
-    @Override
-    public int hashCode(){
-      return this.getName().hashCode();
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      return (obj instanceof Property) && (((Property) obj).getName())
-          .equals(this.getName());
-    }
-  }
-
-  public static void activate(){
-    // adds the default resources
-    Configuration.addDefaultResource("hdfs-default.xml");
-    Configuration.addDefaultResource("hdfs-site.xml");
-    Configuration.addDefaultResource("ozone-default.xml");
-    Configuration.addDefaultResource("ozone-site.xml");
-  }
-}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/package-info.java 
b/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/package-info.java
deleted file mode 100644
index d24d0fb..0000000
--- a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/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.hdsl;
-
-/**
- * Generic HDSL specific configurator and helper classes.
- */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/protocol/DatanodeDetails.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/protocol/DatanodeDetails.java
 
b/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/protocol/DatanodeDetails.java
deleted file mode 100644
index 7049c30..0000000
--- 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/protocol/DatanodeDetails.java
+++ /dev/null
@@ -1,422 +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.hdsl.protocol;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
-
-import java.util.UUID;
-
-/**
- * DatanodeDetails class contains details about DataNode like:
- * - UUID of the DataNode.
- * - IP and Hostname details.
- * - Port details to which the DataNode will be listening.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public final class DatanodeDetails implements Comparable<DatanodeDetails> {
-
-  /**
-   * DataNode's unique identifier in the cluster.
-   */
-  private final UUID uuid;
-
-  private String ipAddress;
-  private String hostName;
-  private Integer infoPort;
-  private Integer infoSecurePort;
-  private Integer containerPort;
-  private Integer ratisPort;
-  private Integer ozoneRestPort;
-
-
-  /**
-   * Constructs DatanodeDetails instance. DatanodeDetails.Builder is used
-   * for instantiating DatanodeDetails.
-   * @param uuid DataNode's UUID
-   * @param ipAddress IP Address of this DataNode
-   * @param hostName DataNode's hostname
-   * @param infoPort HTTP Port
-   * @param infoSecurePort HTTPS Port
-   * @param containerPort Container Port
-   * @param ratisPort Ratis Port
-   * @param ozoneRestPort Rest Port
-   */
-  private DatanodeDetails(
-      String uuid, String ipAddress, String hostName, Integer infoPort,
-      Integer infoSecurePort, Integer containerPort, Integer ratisPort,
-      Integer ozoneRestPort) {
-    this.uuid = UUID.fromString(uuid);
-    this.ipAddress = ipAddress;
-    this.hostName = hostName;
-    this.infoPort = infoPort;
-    this.infoSecurePort = infoSecurePort;
-    this.containerPort = containerPort;
-    this.ratisPort = ratisPort;
-    this.ozoneRestPort = ozoneRestPort;
-  }
-
-  /**
-   * Returns the DataNode UUID.
-   *
-   * @return UUID of DataNode
-   */
-  public UUID getUuid() {
-    return uuid;
-  }
-
-  /**
-   * Returns the string representation of DataNode UUID.
-   *
-   * @return UUID of DataNode
-   */
-  public String getUuidString() {
-    return uuid.toString();
-  }
-
-  /**
-   * Sets the IP address of Datanode.
-   *
-   * @param ip IP Address
-   */
-  public void setIpAddress(String ip) {
-    this.ipAddress = ip;
-  }
-
-  /**
-   * Returns IP address of DataNode.
-   *
-   * @return IP address
-   */
-  public String getIpAddress() {
-    return ipAddress;
-  }
-
-  /**
-   * Sets the Datanode hostname.
-   *
-   * @param host hostname
-   */
-  public void setHostName(String host) {
-    this.hostName = host;
-  }
-
-  /**
-   * Returns Hostname of DataNode.
-   *
-   * @return Hostname
-   */
-  public String getHostName() {
-    return hostName;
-  }
-
-  /**
-   * Sets the InfoPort.
-   * @param port InfoPort
-   */
-  public void setInfoPort(int port) {
-    infoPort = port;
-  }
-
-  /**
-   * Returns DataNodes Info Port.
-   *
-   * @return InfoPort
-   */
-  public int getInfoPort() {
-    return infoPort;
-  }
-
-  /**
-   * Sets the InfoSecurePort.
-   *
-   * @param port InfoSecurePort
-   */
-  public void setInfoSecurePort(int port) {
-    infoSecurePort = port;
-  }
-
-  /**
-   * Returns DataNodes Secure Info Port.
-   *
-   * @return InfoSecurePort
-   */
-  public int getInfoSecurePort() {
-    return infoSecurePort;
-  }
-
-  /**
-   * Sets the Container Port.
-   * @param port ContainerPort
-   */
-  public void setContainerPort(int port) {
-    containerPort = port;
-  }
-
-  /**
-   * Returns standalone container Port.
-   *
-   * @return Container Port
-   */
-  public int getContainerPort() {
-    return containerPort;
-  }
-
-  /**
-   * Sets Ratis Port.
-   * @param port RatisPort
-   */
-  public void setRatisPort(int port) {
-    ratisPort = port;
-  }
-
-
-  /**
-   * Returns Ratis Port.
-   * @return Ratis Port
-   */
-  public int getRatisPort() {
-    return ratisPort;
-  }
-
-
-  /**
-   * Sets OzoneRestPort.
-   * @param port OzoneRestPort
-   */
-  public void setOzoneRestPort(int port) {
-    ozoneRestPort = port;
-  }
-
-  /**
-   * Returns Ozone Rest Port.
-   * @return OzoneRestPort
-   */
-  public int getOzoneRestPort() {
-    return ozoneRestPort;
-  }
-
-  /**
-   * Returns a DatanodeDetails from the protocol buffers.
-   *
-   * @param datanodeDetailsProto - protoBuf Message
-   * @return DatanodeDetails
-   */
-  public static DatanodeDetails getFromProtoBuf(
-      HdslProtos.DatanodeDetailsProto datanodeDetailsProto) {
-    DatanodeDetails.Builder builder = newBuilder();
-    builder.setUuid(datanodeDetailsProto.getUuid());
-    if (datanodeDetailsProto.hasIpAddress()) {
-      builder.setIpAddress(datanodeDetailsProto.getIpAddress());
-    }
-    if (datanodeDetailsProto.hasHostName()) {
-      builder.setHostName(datanodeDetailsProto.getHostName());
-    }
-    if (datanodeDetailsProto.hasInfoPort()) {
-        builder.setInfoPort(datanodeDetailsProto.getInfoPort());
-    }
-    if (datanodeDetailsProto.hasInfoSecurePort()) {
-        builder.setInfoSecurePort(datanodeDetailsProto.getInfoSecurePort());
-    }
-    if (datanodeDetailsProto.hasContainerPort()) {
-      builder.setContainerPort(datanodeDetailsProto.getContainerPort());
-    }
-    if (datanodeDetailsProto.hasRatisPort()) {
-      builder.setRatisPort(datanodeDetailsProto.getRatisPort());
-    }
-    if (datanodeDetailsProto.hasOzoneRestPort()) {
-      builder.setOzoneRestPort(datanodeDetailsProto.getOzoneRestPort());
-    }
-    return builder.build();
-  }
-
-  /**
-   * Returns a DatanodeDetails protobuf message from a datanode ID.
-   * @return Hdsl.DatanodeDetailsProto
-   */
-  public HdslProtos.DatanodeDetailsProto getProtoBufMessage() {
-    HdslProtos.DatanodeDetailsProto.Builder builder =
-        HdslProtos.DatanodeDetailsProto.newBuilder()
-            .setUuid(getUuidString());
-    if (ipAddress != null) {
-      builder.setIpAddress(ipAddress);
-    }
-    if (hostName != null) {
-      builder.setHostName(hostName);
-    }
-    if (infoPort != null) {
-      builder.setInfoPort(infoPort);
-    }
-    if (infoSecurePort != null) {
-      builder.setInfoSecurePort(infoSecurePort);
-    }
-    if (containerPort != null) {
-      builder.setContainerPort(containerPort);
-    }
-    if (ratisPort != null) {
-      builder.setRatisPort(ratisPort);
-    }
-    if (ozoneRestPort != null) {
-      builder.setOzoneRestPort(ozoneRestPort);
-    }
-    return builder.build();
-  }
-
-  @Override
-  public String toString() {
-    return uuid.toString() + "{" +
-        "ip: " +
-        ipAddress +
-        ", host: " +
-        hostName +
-        "}";
-  }
-
-  @Override
-  public int compareTo(DatanodeDetails that) {
-    return this.getUuid().compareTo(that.getUuid());
-  }
-
-  /**
-   * Returns DatanodeDetails.Builder instance.
-   *
-   * @return DatanodeDetails.Builder
-   */
-  public static Builder newBuilder() {
-    return new Builder();
-  }
-
-  /**
-   * Builder class for building DatanodeDetails.
-   */
-  public static class Builder {
-    private String id;
-    private String ipAddress;
-    private String hostName;
-    private Integer infoPort;
-    private Integer infoSecurePort;
-    private Integer containerPort;
-    private Integer ratisPort;
-    private Integer ozoneRestPort;
-
-    /**
-     * Sets the DatanodeUuid.
-     *
-     * @param uuid DatanodeUuid
-     * @return DatanodeDetails.Builder
-     */
-    public Builder setUuid(String uuid) {
-      this.id = uuid;
-      return this;
-    }
-
-    /**
-     * Sets the IP address of DataNode.
-     *
-     * @param ip address
-     * @return DatanodeDetails.Builder
-     */
-    public Builder setIpAddress(String ip) {
-      this.ipAddress = ip;
-      return this;
-    }
-
-    /**
-     * Sets the hostname of DataNode.
-     *
-     * @param host hostname
-     * @return DatanodeDetails.Builder
-     */
-    public Builder setHostName(String host) {
-      this.hostName = host;
-      return this;
-    }
-
-    /**
-     * Sets the InfoPort.
-     *
-     * @param port InfoPort
-     * @return DatanodeDetails.Builder
-     */
-    public Builder setInfoPort(Integer port) {
-      this.infoPort = port;
-      return this;
-    }
-
-    /**
-     * Sets the Secure Info Port.
-     *
-     * @param port InfoSecurePort
-     * @return DatanodeDetails.Builder
-     */
-    public Builder setInfoSecurePort(Integer port) {
-      this.infoSecurePort = port;
-      return this;
-    }
-
-    /**
-     * Sets the ContainerPort.
-     *
-     * @param port ContainerPort
-     * @return DatanodeDetails.Builder
-     */
-    public Builder setContainerPort(Integer port) {
-      this.containerPort = port;
-      return this;
-    }
-
-    /**
-     * Sets the RatisPort.
-     *
-     * @param port RatisPort
-     * @return DatanodeDetails.Builder
-     */
-    public Builder setRatisPort(Integer port) {
-      this.ratisPort = port;
-      return this;
-    }
-
-    /**
-     * Sets the OzoneRestPort.
-     *
-     * @param port OzoneRestPort
-     * @return DatanodeDetails.Builder
-     */
-    public Builder setOzoneRestPort(Integer port) {
-      this.ozoneRestPort = port;
-      return this;
-    }
-
-    /**
-     * Builds and returns DatanodeDetails instance.
-     *
-     * @return DatanodeDetails
-     */
-    public DatanodeDetails build() {
-      Preconditions.checkNotNull(id);
-      return new DatanodeDetails(id, ipAddress, hostName,
-          infoPort, infoSecurePort, containerPort, ratisPort, ozoneRestPort);
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/protocol/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/protocol/package-info.java
 
b/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/protocol/package-info.java
deleted file mode 100644
index 7fd2543..0000000
--- 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/hdsl/protocol/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * This package contains HDSL protocol related classes.
- */
-package org.apache.hadoop.hdsl.protocol;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java 
b/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java
deleted file mode 100644
index ff0ac4e..0000000
--- a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/OzoneAcl.java
+++ /dev/null
@@ -1,231 +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;
-
-import java.util.Objects;
-
-/**
- * OzoneACL classes define bucket ACLs used in OZONE.
- *
- * ACLs in Ozone follow this pattern.
- * • user:name:rw
- * • group:name:rw
- * • world::rw
- */
-public class OzoneAcl {
-  private OzoneACLType type;
-  private String name;
-  private OzoneACLRights rights;
-
-  /**
-   * Constructor for OzoneAcl.
-   */
-  public OzoneAcl() {
-  }
-
-  /**
-   * Constructor for OzoneAcl.
-   *
-   * @param type - Type
-   * @param name - Name of user
-   * @param rights - Rights
-   */
-  public OzoneAcl(OzoneACLType type, String name, OzoneACLRights rights) {
-    this.name = name;
-    this.rights = rights;
-    this.type = type;
-    if (type == OzoneACLType.WORLD && name.length() != 0) {
-      throw new IllegalArgumentException("Unexpected name part in world type");
-    }
-    if (((type == OzoneACLType.USER) || (type == OzoneACLType.GROUP))
-        && (name.length() == 0)) {
-      throw new IllegalArgumentException("User or group name is required");
-    }
-  }
-
-  /**
-   * Parses an ACL string and returns the ACL object.
-   *
-   * @param acl - Acl String , Ex. user:anu:rw
-   *
-   * @return - Ozone ACLs
-   */
-  public static OzoneAcl parseAcl(String acl) throws IllegalArgumentException {
-    if ((acl == null) || acl.isEmpty()) {
-      throw new IllegalArgumentException("ACLs cannot be null or empty");
-    }
-    String[] parts = acl.trim().split(":");
-    if (parts.length < 3) {
-      throw new IllegalArgumentException("ACLs are not in expected format");
-    }
-
-    OzoneACLType aclType = OzoneACLType.valueOf(parts[0].toUpperCase());
-    OzoneACLRights rights = OzoneACLRights.getACLRight(parts[2].toLowerCase());
-
-    // TODO : Support sanitation of these user names by calling into
-    // userAuth Interface.
-    return new OzoneAcl(aclType, parts[1], rights);
-  }
-
-  @Override
-  public String toString() {
-    return type + ":" + name + ":" + OzoneACLRights.getACLRightsString(rights);
-  }
-
-  /**
-   * Returns a hash code value for the object. This method is
-   * supported for the benefit of hash tables.
-   *
-   * @return a hash code value for this object.
-   *
-   * @see Object#equals(Object)
-   * @see System#identityHashCode
-   */
-  @Override
-  public int hashCode() {
-    return Objects.hash(this.getName(), this.getRights().toString(),
-                        this.getType().toString());
-  }
-
-  /**
-   * Returns name.
-   *
-   * @return name
-   */
-  public String getName() {
-    return name;
-  }
-
-  /**
-   * Returns Rights.
-   *
-   * @return - Rights
-   */
-  public OzoneACLRights getRights() {
-    return rights;
-  }
-
-  /**
-   * Returns Type.
-   *
-   * @return type
-   */
-  public OzoneACLType getType() {
-    return type;
-  }
-
-  /**
-   * Indicates whether some other object is "equal to" this one.
-   *
-   * @param obj the reference object with which to compare.
-   *
-   * @return {@code true} if this object is the same as the obj
-   * argument; {@code false} otherwise.
-   */
-  @Override
-  public boolean equals(Object obj) {
-    if (obj == null) {
-      return false;
-    }
-    if (getClass() != obj.getClass()) {
-      return false;
-    }
-    OzoneAcl otherAcl = (OzoneAcl) obj;
-    return otherAcl.getName().equals(this.getName()) &&
-        otherAcl.getRights() == this.getRights() &&
-        otherAcl.getType() == this.getType();
-  }
-
-  /**
-   * ACL types.
-   */
-  public enum OzoneACLType {
-    USER(OzoneConsts.OZONE_ACL_USER_TYPE),
-    GROUP(OzoneConsts.OZONE_ACL_GROUP_TYPE),
-    WORLD(OzoneConsts.OZONE_ACL_WORLD_TYPE);
-
-    /**
-     * String value for this Enum.
-     */
-    private final String value;
-
-    /**
-     * Init OzoneACLtypes enum.
-     *
-     * @param val String type for this enum.
-     */
-    OzoneACLType(String val) {
-      value = val;
-    }
-  }
-
-  /**
-   * ACL rights.
-   */
-  public enum OzoneACLRights {
-    READ, WRITE, READ_WRITE;
-
-    /**
-     * Returns the ACL rights based on passed in String.
-     *
-     * @param type ACL right string
-     *
-     * @return OzoneACLRights
-     */
-    public static OzoneACLRights getACLRight(String type) {
-      if (type == null || type.isEmpty()) {
-        throw new IllegalArgumentException("ACL right cannot be empty");
-      }
-
-      switch (type) {
-      case OzoneConsts.OZONE_ACL_READ:
-        return OzoneACLRights.READ;
-      case OzoneConsts.OZONE_ACL_WRITE:
-        return OzoneACLRights.WRITE;
-      case OzoneConsts.OZONE_ACL_READ_WRITE:
-      case OzoneConsts.OZONE_ACL_WRITE_READ:
-        return OzoneACLRights.READ_WRITE;
-      default:
-        throw new IllegalArgumentException("ACL right is not recognized");
-      }
-
-    }
-
-    /**
-     * Returns String representation of ACL rights.
-     * @param acl OzoneACLRights
-     * @return String representation of acl
-     */
-    public static String getACLRightsString(OzoneACLRights acl) {
-      switch(acl) {
-      case READ:
-        return OzoneConsts.OZONE_ACL_READ;
-      case WRITE:
-        return OzoneConsts.OZONE_ACL_WRITE;
-      case READ_WRITE:
-        return OzoneConsts.OZONE_ACL_READ_WRITE;
-      default:
-        throw new IllegalArgumentException("ACL right is not recognized");
-      }
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java 
b/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
deleted file mode 100644
index bad3a84..0000000
--- 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
+++ /dev/null
@@ -1,239 +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;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.ozone.client.ReplicationFactor;
-import org.apache.hadoop.ozone.client.ReplicationType;
-
-import org.apache.hadoop.scm.ScmConfigKeys;
-
-/**
- * This class contains constants for configuration keys used in Ozone.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Unstable
-public final class OzoneConfigKeys {
-  public static final String DFS_CONTAINER_IPC_PORT =
-      "dfs.container.ipc";
-  public static final int DFS_CONTAINER_IPC_PORT_DEFAULT = 9859;
-
-  /**
-   *
-   * When set to true, allocate a random free port for ozone container,
-   * so that a mini cluster is able to launch multiple containers on a node.
-   *
-   * When set to false (default), container port is fixed as specified by
-   * DFS_CONTAINER_IPC_PORT_DEFAULT.
-   */
-  public static final String DFS_CONTAINER_IPC_RANDOM_PORT =
-      "dfs.container.ipc.random.port";
-  public static final boolean DFS_CONTAINER_IPC_RANDOM_PORT_DEFAULT =
-      false;
-
-  /**
-   * Ratis Port where containers listen to.
-   */
-  public static final String DFS_CONTAINER_RATIS_IPC_PORT =
-      "dfs.container.ratis.ipc";
-  public static final int DFS_CONTAINER_RATIS_IPC_PORT_DEFAULT = 9858;
-
-  /**
-   * When set to true, allocate a random free port for ozone container, so that
-   * a mini cluster is able to launch multiple containers on a node.
-   */
-  public static final String DFS_CONTAINER_RATIS_IPC_RANDOM_PORT =
-      "dfs.container.ratis.ipc.random.port";
-  public static final boolean DFS_CONTAINER_RATIS_IPC_RANDOM_PORT_DEFAULT =
-      false;
-
-  public static final String OZONE_LOCALSTORAGE_ROOT =
-      "ozone.localstorage.root";
-  public static final String OZONE_LOCALSTORAGE_ROOT_DEFAULT = "/tmp/ozone";
-  public static final String OZONE_ENABLED =
-      "ozone.enabled";
-  public static final boolean OZONE_ENABLED_DEFAULT = false;
-  public static final String OZONE_HANDLER_TYPE_KEY =
-      "ozone.handler.type";
-  public static final String OZONE_HANDLER_TYPE_DEFAULT = "distributed";
-  public static final String OZONE_TRACE_ENABLED_KEY =
-      "ozone.trace.enabled";
-  public static final boolean OZONE_TRACE_ENABLED_DEFAULT = false;
-
-  public static final String OZONE_METADATA_DIRS =
-      "ozone.metadata.dirs";
-
-  public static final String OZONE_METADATA_STORE_IMPL =
-      "ozone.metastore.impl";
-  public static final String OZONE_METADATA_STORE_IMPL_LEVELDB =
-      "LevelDB";
-  public static final String OZONE_METADATA_STORE_IMPL_ROCKSDB =
-      "RocksDB";
-  public static final String OZONE_METADATA_STORE_IMPL_DEFAULT =
-      OZONE_METADATA_STORE_IMPL_ROCKSDB;
-
-  public static final String OZONE_METADATA_STORE_ROCKSDB_STATISTICS =
-      "ozone.metastore.rocksdb.statistics";
-
-  public static final String  OZONE_METADATA_STORE_ROCKSDB_STATISTICS_DEFAULT =
-      "ALL";
-  public static final String OZONE_METADATA_STORE_ROCKSDB_STATISTICS_OFF =
-      "OFF";
-
-  public static final String OZONE_CONTAINER_CACHE_SIZE =
-      "ozone.container.cache.size";
-  public static final int OZONE_CONTAINER_CACHE_DEFAULT = 1024;
-
-  public static final String OZONE_SCM_BLOCK_SIZE_IN_MB =
-      "ozone.scm.block.size.in.mb";
-  public static final long OZONE_SCM_BLOCK_SIZE_DEFAULT = 256;
-
-  /**
-   * Ozone administrator users delimited by comma.
-   * If not set, only the user who launches an ozone service will be the
-   * admin user. This property must be set if ozone services are started by
-   * different users. Otherwise the RPC layer will reject calls from
-   * other servers which are started by users not in the list.
-   * */
-  public static final String OZONE_ADMINISTRATORS =
-      "ozone.administrators";
-
-  public static final String OZONE_CLIENT_PROTOCOL =
-      "ozone.client.protocol";
-
-  // This defines the overall connection limit for the connection pool used in
-  // RestClient.
-  public static final String OZONE_REST_CLIENT_HTTP_CONNECTION_MAX =
-      "ozone.rest.client.http.connection.max";
-  public static final int OZONE_REST_CLIENT_HTTP_CONNECTION_DEFAULT = 100;
-
-  // This defines the connection limit per one HTTP route/host.
-  public static final String OZONE_REST_CLIENT_HTTP_CONNECTION_PER_ROUTE_MAX =
-      "ozone.rest.client.http.connection.per-route.max";
-
-  public static final int
-      OZONE_REST_CLIENT_HTTP_CONNECTION_PER_ROUTE_MAX_DEFAULT = 20;
-
-  public static final String OZONE_CLIENT_SOCKET_TIMEOUT =
-      "ozone.client.socket.timeout";
-  public static final int OZONE_CLIENT_SOCKET_TIMEOUT_DEFAULT = 5000;
-  public static final String OZONE_CLIENT_CONNECTION_TIMEOUT =
-      "ozone.client.connection.timeout";
-  public static final int OZONE_CLIENT_CONNECTION_TIMEOUT_DEFAULT = 5000;
-
-  public static final String OZONE_REPLICATION = "ozone.replication";
-  public static final int OZONE_REPLICATION_DEFAULT =
-      ReplicationFactor.THREE.getValue();
-
-  public static final String OZONE_REPLICATION_TYPE = "ozone.replication.type";
-  public static final String OZONE_REPLICATION_TYPE_DEFAULT =
-      ReplicationType.RATIS.toString();
-
-  /**
-   * Configuration property to configure the cache size of client list calls.
-   */
-  public static final String OZONE_CLIENT_LIST_CACHE_SIZE =
-      "ozone.client.list.cache";
-  public static final int OZONE_CLIENT_LIST_CACHE_SIZE_DEFAULT = 1000;
-
-  /**
-   * Configuration properties for Ozone Block Deleting Service.
-   */
-  public static final String OZONE_BLOCK_DELETING_SERVICE_INTERVAL =
-      "ozone.block.deleting.service.interval";
-  public static final String OZONE_BLOCK_DELETING_SERVICE_INTERVAL_DEFAULT
-      = "60s";
-
-  /**
-   * The interval of open key clean service.
-   */
-  public static final String OZONE_OPEN_KEY_CLEANUP_SERVICE_INTERVAL_SECONDS =
-      "ozone.open.key.cleanup.service.interval.seconds";
-  public static final int
-      OZONE_OPEN_KEY_CLEANUP_SERVICE_INTERVAL_SECONDS_DEFAULT
-      = 24 * 3600; // a total of 24 hour
-
-  /**
-   * An open key gets cleaned up when it is being in open state for too long.
-   */
-  public static final String OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS =
-      "ozone.open.key.expire.threshold";
-  public static final int OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS_DEFAULT =
-      24 * 3600;
-
-  public static final String OZONE_BLOCK_DELETING_SERVICE_TIMEOUT =
-      "ozone.block.deleting.service.timeout";
-  public static final String OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT
-      = "300s"; // 300s for default
-
-  public static final String OZONE_KEY_PREALLOCATION_MAXSIZE =
-      "ozone.key.preallocation.maxsize";
-  public static final long OZONE_KEY_PREALLOCATION_MAXSIZE_DEFAULT
-      = 128 * OzoneConsts.MB;
-
-  public static final String OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER =
-      "ozone.block.deleting.limit.per.task";
-  public static final int OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER_DEFAULT
-      = 1000;
-
-  public static final String OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL
-      = "ozone.block.deleting.container.limit.per.interval";
-  public static final int
-      OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL_DEFAULT = 10;
-
-  public static final String OZONE_CONTAINER_REPORT_INTERVAL =
-      "ozone.container.report.interval";
-  public static final String OZONE_CONTAINER_REPORT_INTERVAL_DEFAULT =
-      "60s";
-
-  public static final String DFS_CONTAINER_RATIS_ENABLED_KEY
-      = ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY;
-  public static final boolean DFS_CONTAINER_RATIS_ENABLED_DEFAULT
-      = ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT;
-  public static final String DFS_CONTAINER_RATIS_RPC_TYPE_KEY
-      = ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY;
-  public static final String DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT
-      = ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT;
-  public static final String DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_KEY
-      = ScmConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_KEY;
-  public static final int DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_DEFAULT
-      = ScmConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_DEFAULT;
-  public static final String DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY
-      = ScmConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_KEY;
-  public static final int DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT
-      = ScmConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_SIZE_DEFAULT;
-  public static final String DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY
-      = ScmConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY;
-  public static final int DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT
-      = ScmConfigKeys.DFS_CONTAINER_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT;
-  public static final int DFS_CONTAINER_CHUNK_MAX_SIZE
-      = ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE;
-  public static final String DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR =
-      "dfs.container.ratis.datanode.storage.dir";
-
-  public static final String OZONE_SCM_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL =
-      "ozone.web.authentication.kerberos.principal";
-
-  /**
-   * There is no need to instantiate this class.
-   */
-  private OzoneConfigKeys() {
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java 
b/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
deleted file mode 100644
index 2f9e469..0000000
--- a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ /dev/null
@@ -1,167 +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;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * Set of constants used in Ozone implementation.
- */
-@InterfaceAudience.Private
-public final class OzoneConsts {
-
-
-  public static final String STORAGE_DIR = "scm";
-  public static final String SCM_ID = "scmUuid";
-
-  public static final String OZONE_SIMPLE_ROOT_USER = "root";
-  public static final String OZONE_SIMPLE_HDFS_USER = "hdfs";
-
-  /*
-   * BucketName length is used for both buckets and volume lengths
-   */
-  public static final int OZONE_MIN_BUCKET_NAME_LENGTH = 3;
-  public static final int OZONE_MAX_BUCKET_NAME_LENGTH = 63;
-
-  public static final String OZONE_ACL_USER_TYPE = "user";
-  public static final String OZONE_ACL_GROUP_TYPE = "group";
-  public static final String OZONE_ACL_WORLD_TYPE = "world";
-
-  public static final String OZONE_ACL_READ = "r";
-  public static final String OZONE_ACL_WRITE = "w";
-  public static final String OZONE_ACL_READ_WRITE = "rw";
-  public static final String OZONE_ACL_WRITE_READ = "wr";
-
-  public static final String OZONE_DATE_FORMAT =
-      "EEE, dd MMM yyyy HH:mm:ss zzz";
-  public static final String OZONE_TIME_ZONE = "GMT";
-
-  public static final String OZONE_COMPONENT = "component";
-  public static final String OZONE_FUNCTION  = "function";
-  public static final String OZONE_RESOURCE = "resource";
-  public static final String OZONE_USER = "user";
-  public static final String OZONE_REQUEST = "request";
-
-  public static final String CONTAINER_EXTENSION = ".container";
-  public static final String CONTAINER_META = ".meta";
-
-  //  container storage is in the following format.
-  //  Data Volume basePath/containers/<containerName>/metadata and
-  //  Data Volume basePath/containers/<containerName>/data/...
-  public static final String CONTAINER_PREFIX  = "containers";
-  public static final String CONTAINER_META_PATH = "metadata";
-  public static final String CONTAINER_DATA_PATH = "data";
-  public static final String CONTAINER_TEMPORARY_CHUNK_PREFIX = "tmp";
-  public static final String CONTAINER_CHUNK_NAME_DELIMITER = ".";
-  public static final String CONTAINER_ROOT_PREFIX = "repository";
-
-  public static final String FILE_HASH = "SHA-256";
-  public final static String CHUNK_OVERWRITE = "OverWriteRequested";
-
-  public static final int CHUNK_SIZE = 1 * 1024 * 1024; // 1 MB
-  public static final long KB = 1024L;
-  public static final long MB = KB * 1024L;
-  public static final long GB = MB * 1024L;
-  public static final long TB = GB * 1024L;
-
-  /**
-   * level DB names used by SCM and data nodes.
-   */
-  public static final String CONTAINER_DB_SUFFIX = "container.db";
-  public static final String SCM_CONTAINER_DB = "scm-" + CONTAINER_DB_SUFFIX;
-  public static final String DN_CONTAINER_DB = "-dn-"+ CONTAINER_DB_SUFFIX;
-  public static final String BLOCK_DB = "block.db";
-  public static final String NODEPOOL_DB = "nodepool.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";
-
-  /**
-   * Supports Bucket Versioning.
-   */
-  public enum Versioning {NOT_DEFINED, ENABLED, DISABLED}
-
-  /**
-   * Ozone handler types.
-   */
-  public static final String OZONE_HANDLER_DISTRIBUTED = "distributed";
-  public static final String OZONE_HANDLER_LOCAL = "local";
-
-  public static final String DELETING_KEY_PREFIX = "#deleting#";
-  public static final String OPEN_KEY_PREFIX = "#open#";
-  public static final String OPEN_KEY_ID_DELIMINATOR = "#";
-
-  /**
-   * KSM LevelDB prefixes.
-   *
-   * KSM DB stores metadata as KV pairs with certain prefixes,
-   * prefix is used to improve the performance to get related
-   * metadata.
-   *
-   * KSM DB Schema:
-   *  ----------------------------------------------------------
-   *  |  KEY                                     |     VALUE   |
-   *  ----------------------------------------------------------
-   *  | $userName                                |  VolumeList |
-   *  ----------------------------------------------------------
-   *  | /#volumeName                             |  VolumeInfo |
-   *  ----------------------------------------------------------
-   *  | /#volumeName/#bucketName                 |  BucketInfo |
-   *  ----------------------------------------------------------
-   *  | /volumeName/bucketName/keyName           |  KeyInfo    |
-   *  ----------------------------------------------------------
-   *  | #deleting#/volumeName/bucketName/keyName |  KeyInfo    |
-   *  ----------------------------------------------------------
-   */
-  public static final String KSM_VOLUME_PREFIX = "/#";
-  public static final String KSM_BUCKET_PREFIX = "/#";
-  public static final String KSM_KEY_PREFIX = "/";
-  public static final String KSM_USER_PREFIX = "$";
-
-  /**
-   * Max KSM Quota size of 1024 PB.
-   */
-  public static final long MAX_QUOTA_IN_BYTES = 1024L * 1024 * TB;
-
-  /**
-   * Max number of keys returned per list buckets operation.
-   */
-  public static final int MAX_LISTBUCKETS_SIZE  = 1024;
-
-  /**
-   * Max number of keys returned per list keys operation.
-   */
-  public static final int MAX_LISTKEYS_SIZE  = 1024;
-
-  /**
-   * Max number of volumes returned per list volumes operation.
-   */
-  public static final int MAX_LISTVOLUMES_SIZE = 1024;
-
-  public static final int INVALID_PORT = -1;
-
-
-  // The ServiceListJSONServlet context attribute where KeySpaceManager
-  // instance gets stored.
-  public static final String KSM_CONTEXT_ATTRIBUTE = "ozone.ksm";
-
-  private OzoneConsts() {
-    // Never Constructed
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/OzoneQuota.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/OzoneQuota.java
 
b/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/OzoneQuota.java
deleted file mode 100644
index 032dd60..0000000
--- 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/OzoneQuota.java
+++ /dev/null
@@ -1,203 +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;
-
-import org.apache.hadoop.ozone.OzoneConsts;
-
-
-/**
- * represents an OzoneQuota Object that can be applied to
- * a storage volume.
- */
-public class OzoneQuota {
-
-  public static final String OZONE_QUOTA_BYTES = "BYTES";
-  public static final String OZONE_QUOTA_MB = "MB";
-  public static final String OZONE_QUOTA_GB = "GB";
-  public static final String OZONE_QUOTA_TB = "TB";
-
-  private Units unit;
-  private long size;
-
-  /** Quota Units.*/
-  public enum Units {UNDEFINED, BYTES, KB, MB, GB, TB}
-
-  /**
-   * Returns size.
-   *
-   * @return long
-   */
-  public long getSize() {
-    return size;
-  }
-
-  /**
-   * Returns Units.
-   *
-   * @return Unit in MB, GB or TB
-   */
-  public Units getUnit() {
-    return unit;
-  }
-
-  /**
-   * Constructs a default Quota object.
-   */
-  public OzoneQuota() {
-    this.size = 0;
-    this.unit = Units.UNDEFINED;
-  }
-
-  /**
-   * Constructor for Ozone Quota.
-   *
-   * @param size Long Size
-   * @param unit MB, GB  or TB
-   */
-  public OzoneQuota(long size, Units unit) {
-    this.size = size;
-    this.unit = unit;
-  }
-
-  /**
-   * Formats a quota as a string.
-   *
-   * @param quota the quota to format
-   * @return string representation of quota
-   */
-  public static String formatQuota(OzoneQuota quota) {
-    return String.valueOf(quota.size) + quota.unit;
-  }
-
-  /**
-   * Parses a user provided string and returns the
-   * Quota Object.
-   *
-   * @param quotaString Quota String
-   *
-   * @return OzoneQuota object
-   *
-   * @throws IllegalArgumentException
-   */
-  public static OzoneQuota parseQuota(String quotaString)
-      throws IllegalArgumentException {
-
-    if ((quotaString == null) || (quotaString.isEmpty())) {
-      throw new IllegalArgumentException(
-          "Quota string cannot be null or empty.");
-    }
-
-    String uppercase = quotaString.toUpperCase().replaceAll("\\s+", "");
-    String size = "";
-    int nSize;
-    Units currUnit = Units.MB;
-    Boolean found = false;
-    if (uppercase.endsWith(OZONE_QUOTA_MB)) {
-      size = uppercase
-          .substring(0, uppercase.length() - OZONE_QUOTA_MB.length());
-      currUnit = Units.MB;
-      found = true;
-    }
-
-    if (uppercase.endsWith(OZONE_QUOTA_GB)) {
-      size = uppercase
-          .substring(0, uppercase.length() - OZONE_QUOTA_GB.length());
-      currUnit = Units.GB;
-      found = true;
-    }
-
-    if (uppercase.endsWith(OZONE_QUOTA_TB)) {
-      size = uppercase
-          .substring(0, uppercase.length() - OZONE_QUOTA_TB.length());
-      currUnit = Units.TB;
-      found = true;
-    }
-
-    if (uppercase.endsWith(OZONE_QUOTA_BYTES)) {
-      size = uppercase
-          .substring(0, uppercase.length() - OZONE_QUOTA_BYTES.length());
-      currUnit = Units.BYTES;
-      found = true;
-    }
-
-    if (!found) {
-      throw new IllegalArgumentException(
-          "Quota unit not recognized. Supported values are BYTES, MB, GB and " 
+
-              "TB.");
-    }
-
-    nSize = Integer.parseInt(size);
-    if (nSize < 0) {
-      throw new IllegalArgumentException("Quota cannot be negative.");
-    }
-
-    return new OzoneQuota(nSize, currUnit);
-  }
-
-
-  /**
-   * Returns size in Bytes or -1 if there is no Quota.
-   */
-  public long sizeInBytes() {
-    switch (this.unit) {
-    case BYTES:
-      return this.getSize();
-    case MB:
-      return this.getSize() * OzoneConsts.MB;
-    case GB:
-      return this.getSize() * OzoneConsts.GB;
-    case TB:
-      return this.getSize() * OzoneConsts.TB;
-    case UNDEFINED:
-    default:
-      return -1;
-    }
-  }
-
-  /**
-   * Returns OzoneQuota corresponding to size in bytes.
-   *
-   * @param sizeInBytes size in bytes to be converted
-   *
-   * @return OzoneQuota object
-   */
-  public static OzoneQuota getOzoneQuota(long sizeInBytes) {
-    long size;
-    Units unit;
-    if (sizeInBytes % OzoneConsts.TB == 0) {
-      size = sizeInBytes / OzoneConsts.TB;
-      unit = Units.TB;
-    } else if (sizeInBytes % OzoneConsts.GB == 0) {
-      size = sizeInBytes / OzoneConsts.GB;
-      unit = Units.GB;
-    } else if (sizeInBytes % OzoneConsts.MB == 0) {
-      size = sizeInBytes / OzoneConsts.MB;
-      unit = Units.MB;
-    } else {
-      size = sizeInBytes;
-      unit = Units.BYTES;
-    }
-    return new OzoneQuota((int)size, unit);
-  }
-
-  @Override
-  public String toString() {
-    return size + " " + unit;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/ReplicationFactor.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/ReplicationFactor.java
 
b/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/ReplicationFactor.java
deleted file mode 100644
index 971cfec..0000000
--- 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/ReplicationFactor.java
+++ /dev/null
@@ -1,63 +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;
-
-/**
- * The replication factor to be used while writing key into ozone.
- */
-public enum ReplicationFactor {
-  ONE(1),
-  THREE(3);
-
-  /**
-   * Integer representation of replication.
-   */
-  private int value;
-
-  /**
-   * Initializes ReplicationFactor with value.
-   * @param value replication value
-   */
-  ReplicationFactor(int value) {
-    this.value = value;
-  }
-
-  /**
-   * Returns enum value corresponding to the int value.
-   * @param value replication value
-   * @return ReplicationFactor
-   */
-  public static ReplicationFactor valueOf(int value) {
-    if(value == 1) {
-      return ONE;
-    }
-    if (value == 3) {
-      return THREE;
-    }
-    throw new IllegalArgumentException("Unsupported value: " + value);
-  }
-
-  /**
-   * Returns integer representation of ReplicationFactor.
-   * @return replication value
-   */
-  public int getValue() {
-    return value;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/ReplicationType.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/ReplicationType.java
 
b/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/ReplicationType.java
deleted file mode 100644
index 537c336..0000000
--- 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/ReplicationType.java
+++ /dev/null
@@ -1,28 +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;
-
-/**
- * The replication type to be used while writing key into ozone.
- */
-public enum ReplicationType {
-    RATIS,
-    STAND_ALONE,
-    CHAINED
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/package-info.java
 
b/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/package-info.java
deleted file mode 100644
index 7492838..0000000
--- 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/client/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.client;
-
-/**
- * Base property types for HDSL containers and replications.
- */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/common/BlockGroup.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/common/BlockGroup.java
 
b/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/common/BlockGroup.java
deleted file mode 100644
index 9fcc613..0000000
--- 
a/hadoop-hdsl/common/src/main/java/org/apache/hadoop/ozone/common/BlockGroup.java
+++ /dev/null
@@ -1,87 +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.hdsl.protocol.proto
-    .ScmBlockLocationProtocolProtos.KeyBlocks;
-
-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<String> blockIDs;
-  private BlockGroup(String groupID, List<String> blockIDs) {
-    this.groupID = groupID;
-    this.blockIDs = blockIDs;
-  }
-
-  public List<String> getBlockIDList() {
-    return blockIDs;
-  }
-
-  public String getGroupID() {
-    return groupID;
-  }
-
-  public KeyBlocks getProto() {
-    return KeyBlocks.newBuilder().setKey(groupID)
-        .addAllBlocks(blockIDs).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) {
-    return BlockGroup.newBuilder().setKeyName(proto.getKey())
-        .addAllBlockIDs(proto.getBlocksList()).build();
-  }
-
-  public static Builder newBuilder() {
-    return new Builder();
-  }
-
-  /**
-   * BlockGroup instance builder.
-   */
-  public static class Builder {
-
-    private String groupID;
-    private List<String> blockIDs;
-
-    public Builder setKeyName(String blockGroupID) {
-      this.groupID = blockGroupID;
-      return this;
-    }
-
-    public Builder addAllBlockIDs(List<String> keyBlocks) {
-      this.blockIDs = keyBlocks;
-      return this;
-    }
-
-    public BlockGroup build() {
-      return new BlockGroup(groupID, blockIDs);
-    }
-  }
-
-}


---------------------------------------------------------------------
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