http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
 
b/hadoop-hdsl/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
deleted file mode 100644
index 187ecda..0000000
--- 
a/hadoop-hdsl/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
+++ /dev/null
@@ -1,351 +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.
- */
-
-/**
- * These .proto interfaces are private and unstable.
- * Please see http://wiki.apache.org/hadoop/Compatibility
- * for what changes are allowed for a *unstable* .proto interface.
- */
-
-option java_package = "org.apache.hadoop.hdsl.protocol.proto";
-
-option java_outer_classname = "StorageContainerDatanodeProtocolProtos";
-
-option java_generic_services = true;
-
-option java_generate_equals_and_hash = true;
-
-package hadoop.hdsl;
-
-import "hdsl.proto";
-
-
-/**
-* This message is send by data node to indicate that it is alive or it is
-* registering with the node manager.
-*/
-message SCMHeartbeatRequestProto {
-  required DatanodeDetailsProto datanodeDetails = 1;
-  optional SCMNodeReport nodeReport = 2;
-  optional ReportState containerReportState = 3;
-}
-
-enum DatanodeContainerState {
-  closed = 0;
-  open = 1;
-}
-
-/**
-NodeState contains messages from datanode to SCM saying that it has
-some information that SCM might be interested in.*/
-message ReportState {
-  enum states {
-    noContainerReports = 0;
-    completeContinerReport = 1;
-    deltaContainerReport = 2;
-  }
-  required states state = 1;
-  required int64 count = 2 [default = 0];
-}
-
-
-/**
-This message is used to persist the information about a container in the
-SCM database, This information allows SCM to startup faster and avoid having
-all container info in memory all the time.
-  */
-message ContainerPersistanceProto {
-  required DatanodeContainerState state = 1;
-  required hadoop.hdsl.Pipeline pipeline = 2;
-  required ContainerInfo info = 3;
-}
-
-/**
-This message is used to do a quick look up of which containers are effected
-if a node goes down
-*/
-message NodeContianerMapping {
-  repeated string contianerName = 1;
-}
-
-/**
-A container report contains the following information.
-*/
-message ContainerInfo {
-  required string containerName = 1;
-  optional string finalhash = 2;
-  optional int64 size = 3;
-  optional int64 used = 4;
-  optional int64 keyCount = 5;
-  // TODO: move the io count to separate message
-  optional int64 readCount = 6;
-  optional int64 writeCount = 7;
-  optional int64 readBytes = 8;
-  optional int64 writeBytes = 9;
-  required int64 containerID = 10;
-  optional hadoop.hdsl.LifeCycleState state = 11;
-}
-
-// The deleted blocks which are stored in deletedBlock.db of scm.
-message DeletedBlocksTransaction {
-  required int64 txID = 1;
-  required string containerName = 2;
-  repeated string blockID = 3;
-  // the retry time of sending deleting command to datanode.
-  required int32 count = 4;
-}
-
-/**
-A set of container reports, max count is generally set to
-8192 since that keeps the size of the reports under 1 MB.
-*/
-message ContainerReportsRequestProto {
-  enum reportType {
-    fullReport = 0;
-    deltaReport = 1;
-  }
-  required DatanodeDetailsProto datanodeDetails = 1;
-  repeated ContainerInfo reports = 2;
-  required reportType type = 3;
-}
-
-message ContainerReportsResponseProto {
-}
-
-/**
-* This message is send along with the heart beat to report datanode
-* storage utilization by SCM.
-*/
-message SCMNodeReport {
-  repeated SCMStorageReport storageReport = 1;
-}
-
-message SCMStorageReport {
-  required string storageUuid = 1;
-  optional uint64 capacity = 2 [default = 0];
-  optional uint64 scmUsed = 3 [default = 0];
-  optional uint64 remaining = 4 [default = 0];
-  //optional hadoop.hdfs.StorageTypeProto storageType = 5 [default = DISK];
-}
-
-message SCMRegisterRequestProto {
-  required DatanodeDetailsProto datanodeDetails = 1;
-  optional SCMNodeAddressList addressList = 2;
-}
-
-/**
- * Request for version info of the software stack on the server.
- */
-message SCMVersionRequestProto {
-
-}
-
-/**
-* Generic response that is send to a version request. This allows keys to be
-* added on the fly and protocol to remain stable.
-*/
-message SCMVersionResponseProto {
-  required uint32 softwareVersion = 1;
-  repeated hadoop.hdsl.KeyValue keys = 2;
-}
-
-message SCMNodeAddressList {
-  repeated string addressList = 1;
-}
-
-/**
- * Datanode ID returned by the SCM. This is similar to name node
- * registeration of a datanode.
- */
-message SCMRegisteredCmdResponseProto {
-  enum ErrorCode {
-    success = 1;
-    errorNodeNotPermitted = 2;
-  }
-  required ErrorCode errorCode = 2;
-  optional string datanodeUUID = 3;
-  optional string clusterID = 4;
-  optional SCMNodeAddressList addressList = 5;
-}
-
-/**
- * SCM informs a datanode to register itself again.
- * With recieving this command, datanode will transit to REGISTER state.
- */
-message SCMReregisterCmdResponseProto {}
-
-/**
-This command tells the data node to send in the container report when possible
-*/
-message SendContainerReportProto {
-}
-
-/**
-This command asks the datanode to close a specific container.
-*/
-message SCMCloseContainerCmdResponseProto {
-  required string containerName = 1;
-}
-
-/**
-Type of commands supported by SCM to datanode protocol.
-*/
-enum SCMCmdType {
-  versionCommand = 2;
-  registeredCommand = 3;
-  sendContainerReport = 4;
-  reregisterCommand = 5;
-  deleteBlocksCommand = 6;
-  closeContainerCommand = 7;
-}
-
-/*
- * These are commands returned by SCM for to the datanode to execute.
- */
-message SCMCommandResponseProto {
-  required SCMCmdType cmdType = 2; // Type of the command
-  optional SCMRegisteredCmdResponseProto registeredProto = 3;
-  optional SCMVersionResponseProto versionProto = 4;
-  optional SendContainerReportProto sendReport = 5;
-  optional SCMReregisterCmdResponseProto reregisterProto = 6;
-  optional SCMDeleteBlocksCmdResponseProto deleteBlocksProto = 7;
-  required string datanodeUUID = 8;
-  optional SCMCloseContainerCmdResponseProto closeContainerProto = 9;
-}
-
-
-/*
- * A group of commands for the datanode to execute
- */
-message SCMHeartbeatResponseProto {
-  repeated SCMCommandResponseProto commands = 1;
-}
-
-// HB response from SCM, contains a list of block deletion transactions.
-message SCMDeleteBlocksCmdResponseProto {
-  repeated DeletedBlocksTransaction deletedBlocksTransactions = 1;
-}
-
-// SendACK response returned by datanode to SCM, currently empty.
-message ContainerBlocksDeletionACKResponseProto {
-}
-
-// ACK message datanode sent to SCM, contains the result of
-// block deletion transactions.
-message ContainerBlocksDeletionACKProto {
-  message DeleteBlockTransactionResult {
-    required int64 txID = 1;
-    required bool success = 2;
-  }
-  repeated DeleteBlockTransactionResult results = 1;
-}
-
-/**
- * Protocol used from a datanode to StorageContainerManager.
- *
- * Please see the request and response messages for details of the RPC calls.
- *
- * Here is a simple state diagram that shows how a datanode would boot up and
- * communicate with SCM.
- *
- *           -----------------------
- *          |         Start         |
- *           ---------- ------------
- *                     |
- *                     |
- *                     |
- *                     |
- *                     |
- *                     |
- *                     |
- *           ----------v-------------
- *          |   Searching for  SCM    ------------
- *           ---------- -------------             |
- *                     |                          |
- *                     |                          |
- *                     |                ----------v-------------
- *                     |               | Register if needed     |
- *                     |                ----------- ------------
- *                     |                           |
- *                     v                           |
- *            ----------- ----------------         |
- *  ---------   Heartbeat state           <--------
- * |          --------^-------------------
- * |                  |
- * |                  |
- * |                  |
- * |                  |
- * |                  |
- * |                  |
- * |                  |
- *  ------------------
- *
- *
- *
- * Here is how this protocol is used by the datanode. When a datanode boots up
- * it moves into a stated called SEARCHING_SCM. In this state datanode is
- * trying to establish communication with the SCM. The address of the SCMs are
- * retrieved from the configuration information.
- *
- * In the SEARCHING_SCM state, only rpc call made by datanode is a getVersion
- * call to SCM. Once any of the SCMs reply, datanode checks if it has a local
- * persisted datanode ID. If it has this means that this datanode is already
- * registered with some SCM. If this file is not found, datanode assumes that
- * it needs to do a registration.
- *
- * If registration is need datanode moves into REGISTER state. It will
- * send a register call with DatanodeDetailsProto data structure and presist
- * that info.
- *
- * The response to the command contains clusterID. This information is
- * also persisted by the datanode and moves into heartbeat state.
- *
- * Once in the heartbeat state, datanode sends heartbeats and container reports
- * to SCM and process commands issued by SCM until it is shutdown.
- *
- */
-service StorageContainerDatanodeProtocolService {
-
-  /**
-  * Gets the version information from the SCM.
-  */
-  rpc getVersion (SCMVersionRequestProto) returns (SCMVersionResponseProto);
-
-  /**
-  * Registers a data node with SCM.
-  */
-  rpc register (SCMRegisterRequestProto) returns 
(SCMRegisteredCmdResponseProto);
-
-  /**
-   * Send heartbeat from datanode to SCM. HB's under SCM looks more
-   * like life line protocol than HB's under HDFS. In other words, it is
-   * extremely light weight and contains no data payload.
-   */
-  rpc sendHeartbeat (SCMHeartbeatRequestProto) returns 
(SCMHeartbeatResponseProto);
-
-  /**
-    send container reports sends the container report to SCM. This will
-    return a null command as response.
-  */
-  rpc sendContainerReport(ContainerReportsRequestProto) returns 
(ContainerReportsResponseProto);
-
-  /**
-   * Sends the block deletion ACK to SCM.
-   */
-  rpc sendContainerBlocksDeletionACK (ContainerBlocksDeletionACKProto) returns 
(ContainerBlocksDeletionACKResponseProto);
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/container-service/src/main/resources/META-INF/services/com.sun.jersey.spi.container.ContainerProvider
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/container-service/src/main/resources/META-INF/services/com.sun.jersey.spi.container.ContainerProvider
 
b/hadoop-hdsl/container-service/src/main/resources/META-INF/services/com.sun.jersey.spi.container.ContainerProvider
deleted file mode 100644
index 2e103fe..0000000
--- 
a/hadoop-hdsl/container-service/src/main/resources/META-INF/services/com.sun.jersey.spi.container.ContainerProvider
+++ /dev/null
@@ -1,16 +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.
-
-org.apache.hadoop.ozone.web.netty.ObjectStoreJerseyContainerProvider

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ContainerTestUtils.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ContainerTestUtils.java
 
b/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ContainerTestUtils.java
deleted file mode 100644
index 4501940..0000000
--- 
a/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ContainerTestUtils.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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations 
under
- * the License.
- */
-
-package org.apache.hadoop.ozone.container.common;
-
-import java.net.InetSocketAddress;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.retry.RetryPolicies;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.ozone.container.common.statemachine
-    .EndpointStateMachine;
-import org.apache.hadoop.ozone.protocolPB
-    .StorageContainerDatanodeProtocolClientSideTranslatorPB;
-import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolPB;
-import org.apache.hadoop.security.UserGroupInformation;
-
-/**
- * Helper utility to test containers.
- */
-public class ContainerTestUtils {
-
-  private ContainerTestUtils() {
-  }
-
-  /**
-   * Creates an Endpoint class for testing purpose.
-   *
-   * @param conf - Conf
-   * @param address - InetAddres
-   * @param rpcTimeout - rpcTimeOut
-   * @return EndPoint
-   * @throws Exception
-   */
-  public static EndpointStateMachine createEndpoint(Configuration conf,
-      InetSocketAddress address, int rpcTimeout) throws Exception {
-    RPC.setProtocolEngine(conf, StorageContainerDatanodeProtocolPB.class,
-        ProtobufRpcEngine.class);
-    long version =
-        RPC.getProtocolVersion(StorageContainerDatanodeProtocolPB.class);
-
-    StorageContainerDatanodeProtocolPB rpcProxy = RPC.getProtocolProxy(
-        StorageContainerDatanodeProtocolPB.class, version,
-        address, UserGroupInformation.getCurrentUser(), conf,
-        NetUtils.getDefaultSocketFactory(conf), rpcTimeout,
-        RetryPolicies.TRY_ONCE_THEN_FAIL).getProxy();
-
-    StorageContainerDatanodeProtocolClientSideTranslatorPB rpcClient =
-        new StorageContainerDatanodeProtocolClientSideTranslatorPB(rpcProxy);
-    return new EndpointStateMachine(address, rpcClient, conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java
 
b/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java
deleted file mode 100644
index 32c28aa..0000000
--- 
a/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations 
under
- * the License.
- */
-package org.apache.hadoop.ozone.container.common;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
-import org.apache.hadoop.hdsl.protocol.proto
-    .StorageContainerDatanodeProtocolProtos
-    .StorageContainerDatanodeProtocolService;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol;
-import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolPB;
-import org.apache.hadoop.ozone.protocolPB
-    .StorageContainerDatanodeProtocolServerSideTranslatorPB;
-
-import com.google.protobuf.BlockingService;
-
-/**
- * Test Endpoint class.
- */
-public final class SCMTestUtils {
-  /**
-   * Never constructed.
-   */
-  private SCMTestUtils() {
-  }
-
-  /**
-   * Starts an RPC server, if configured.
-   *
-   * @param conf configuration
-   * @param addr configured address of RPC server
-   * @param protocol RPC protocol provided by RPC server
-   * @param instance RPC protocol implementation instance
-   * @param handlerCount RPC server handler count
-   * @return RPC server
-   * @throws IOException if there is an I/O error while creating RPC server
-   */
-  private static RPC.Server startRpcServer(Configuration conf,
-      InetSocketAddress addr, Class<?>
-      protocol, BlockingService instance, int handlerCount)
-      throws IOException {
-    RPC.Server rpcServer = new RPC.Builder(conf)
-        .setProtocol(protocol)
-        .setInstance(instance)
-        .setBindAddress(addr.getHostString())
-        .setPort(addr.getPort())
-        .setNumHandlers(handlerCount)
-        .setVerbose(false)
-        .setSecretManager(null)
-        .build();
-
-    DFSUtil.addPBProtocol(conf, protocol, instance, rpcServer);
-    return rpcServer;
-  }
-
-
-  /**
-   * Start Datanode RPC server.
-   */
-  public static RPC.Server startScmRpcServer(Configuration configuration,
-      StorageContainerDatanodeProtocol server,
-      InetSocketAddress rpcServerAddresss, int handlerCount) throws
-      IOException {
-    RPC.setProtocolEngine(configuration,
-        StorageContainerDatanodeProtocolPB.class,
-        ProtobufRpcEngine.class);
-
-    BlockingService scmDatanodeService =
-        StorageContainerDatanodeProtocolService.
-            newReflectiveBlockingService(
-                new StorageContainerDatanodeProtocolServerSideTranslatorPB(
-                    server));
-
-    RPC.Server scmServer = startRpcServer(configuration, rpcServerAddresss,
-        StorageContainerDatanodeProtocolPB.class, scmDatanodeService,
-        handlerCount);
-
-    scmServer.start();
-    return scmServer;
-  }
-
-  public static InetSocketAddress getReuseableAddress() throws IOException {
-    try (ServerSocket socket = new ServerSocket(0)) {
-      socket.setReuseAddress(true);
-      int port = socket.getLocalPort();
-      String addr = InetAddress.getLoopbackAddress().getHostAddress();
-      return new InetSocketAddress(addr, port);
-    }
-  }
-
-  public static Configuration getConf() {
-    return new Configuration();
-  }
-
-  public static OzoneConfiguration getOzoneConf() {
-    return new OzoneConfiguration();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
 
b/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
deleted file mode 100644
index 4349b1a..0000000
--- 
a/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
+++ /dev/null
@@ -1,265 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations 
under
- * the License.
- */
-package org.apache.hadoop.ozone.container.common;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
-import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.DatanodeDetailsProto;
-import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol;
-import org.apache.hadoop.ozone.protocol.VersionResponse;
-import 
org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos;
-import 
org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerInfo;
-import 
org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
-import 
org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandResponseProto;
-import 
org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.ReportState;
-import 
org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMNodeReport;
-import 
org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
-import 
org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKResponseProto;
-import org.apache.hadoop.ozone.scm.VersionInfo;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * SCM RPC mock class.
- */
-public class ScmTestMock implements StorageContainerDatanodeProtocol {
-  private int rpcResponseDelay;
-  private AtomicInteger heartbeatCount = new AtomicInteger(0);
-  private AtomicInteger rpcCount = new AtomicInteger(0);
-  private ReportState reportState;
-  private AtomicInteger containerReportsCount = new AtomicInteger(0);
-
-  // Map of datanode to containers
-  private Map<DatanodeDetails, Map<String, ContainerInfo>> nodeContainers =
-      new HashMap();
-  /**
-   * Returns the number of heartbeats made to this class.
-   *
-   * @return int
-   */
-  public int getHeartbeatCount() {
-    return heartbeatCount.get();
-  }
-
-  /**
-   * Returns the number of RPC calls made to this mock class instance.
-   *
-   * @return - Number of RPC calls serviced by this class.
-   */
-  public int getRpcCount() {
-    return rpcCount.get();
-  }
-
-  /**
-   * Gets the RPC response delay.
-   *
-   * @return delay in milliseconds.
-   */
-  public int getRpcResponseDelay() {
-    return rpcResponseDelay;
-  }
-
-  /**
-   * Sets the RPC response delay.
-   *
-   * @param rpcResponseDelay - delay in milliseconds.
-   */
-  public void setRpcResponseDelay(int rpcResponseDelay) {
-    this.rpcResponseDelay = rpcResponseDelay;
-  }
-
-  /**
-   * Returns the number of container reports server has seen.
-   * @return int
-   */
-  public int getContainerReportsCount() {
-    return containerReportsCount.get();
-  }
-
-  /**
-   * Returns the number of containers that have been reported so far.
-   * @return - count of reported containers.
-   */
-  public long getContainerCount() {
-    return nodeContainers.values().parallelStream().mapToLong((containerMap)->{
-      return containerMap.size();
-    }).sum();
-  }
-
-  /**
-   * Get the number keys reported from container reports.
-   * @return - number of keys reported.
-   */
-  public long getKeyCount() {
-    return nodeContainers.values().parallelStream().mapToLong((containerMap)->{
-      return containerMap.values().parallelStream().mapToLong((container) -> {
-        return container.getKeyCount();
-      }).sum();
-    }).sum();
-  }
-
-  /**
-   * Get the number of bytes used from container reports.
-   * @return - number of bytes used.
-   */
-  public long getBytesUsed() {
-    return nodeContainers.values().parallelStream().mapToLong((containerMap)->{
-      return containerMap.values().parallelStream().mapToLong((container) -> {
-        return container.getUsed();
-      }).sum();
-    }).sum();
-  }
-
-  /**
-   * Returns SCM version.
-   *
-   * @return Version info.
-   */
-  @Override
-  public StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto
-      getVersion(StorageContainerDatanodeProtocolProtos
-      .SCMVersionRequestProto unused) throws IOException {
-    rpcCount.incrementAndGet();
-    sleepIfNeeded();
-    VersionInfo versionInfo = VersionInfo.getLatestVersion();
-    return VersionResponse.newBuilder()
-        .setVersion(versionInfo.getVersion())
-        .addValue(VersionInfo.DESCRIPTION_KEY, versionInfo.getDescription())
-        .build().getProtobufMessage();
-  }
-
-  private void sleepIfNeeded() {
-    if (getRpcResponseDelay() > 0) {
-      try {
-        Thread.sleep(getRpcResponseDelay());
-      } catch (InterruptedException ex) {
-        // Just ignore this exception.
-      }
-    }
-  }
-
-  /**
-   * Used by data node to send a Heartbeat.
-   *
-   * @param datanodeDetailsProto - DatanodeDetailsProto.
-   * @param nodeReport - node report.
-   * @return - SCMHeartbeatResponseProto
-   * @throws IOException
-   */
-  @Override
-  public StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto
-      sendHeartbeat(DatanodeDetailsProto datanodeDetailsProto,
-                    SCMNodeReport nodeReport, ReportState scmReportState)
-      throws IOException {
-    rpcCount.incrementAndGet();
-    heartbeatCount.incrementAndGet();
-    this.reportState = scmReportState;
-    sleepIfNeeded();
-    List<SCMCommandResponseProto>
-        cmdResponses = new LinkedList<>();
-    return SCMHeartbeatResponseProto.newBuilder().addAllCommands(cmdResponses)
-        .build();
-  }
-
-  /**
-   * Register Datanode.
-   *
-   * @param datanodeDetailsProto DatanodDetailsProto.
-   * @param scmAddresses - List of SCMs this datanode is configured to
-   * communicate.
-   * @return SCM Command.
-   */
-  @Override
-  public StorageContainerDatanodeProtocolProtos
-      .SCMRegisteredCmdResponseProto register(
-          DatanodeDetailsProto datanodeDetailsProto, String[] scmAddresses)
-      throws IOException {
-    rpcCount.incrementAndGet();
-    sleepIfNeeded();
-    return StorageContainerDatanodeProtocolProtos
-        .SCMRegisteredCmdResponseProto
-        .newBuilder().setClusterID(UUID.randomUUID().toString())
-        .setDatanodeUUID(datanodeDetailsProto.getUuid()).setErrorCode(
-            StorageContainerDatanodeProtocolProtos
-                .SCMRegisteredCmdResponseProto.ErrorCode.success).build();
-  }
-
-  /**
-   * Send a container report.
-   *
-   * @param reports -- Container report
-   * @return HeartbeatResponse.nullcommand.
-   * @throws IOException
-   */
-  @Override
-  public StorageContainerDatanodeProtocolProtos.ContainerReportsResponseProto
-      sendContainerReport(StorageContainerDatanodeProtocolProtos
-      .ContainerReportsRequestProto reports) throws IOException {
-    Preconditions.checkNotNull(reports);
-    containerReportsCount.incrementAndGet();
-
-    DatanodeDetails datanode = DatanodeDetails.getFromProtoBuf(
-        reports.getDatanodeDetails());
-    if (reports.getReportsCount() > 0) {
-      Map containers = nodeContainers.get(datanode);
-      if (containers == null) {
-        containers = new LinkedHashMap();
-        nodeContainers.put(datanode, containers);
-      }
-
-      for (StorageContainerDatanodeProtocolProtos.ContainerInfo report:
-          reports.getReportsList()) {
-        containers.put(report.getContainerName(), report);
-      }
-    }
-
-    return StorageContainerDatanodeProtocolProtos
-        .ContainerReportsResponseProto.newBuilder().build();
-  }
-
-  @Override
-  public ContainerBlocksDeletionACKResponseProto 
sendContainerBlocksDeletionACK(
-      ContainerBlocksDeletionACKProto request) throws IOException {
-    return ContainerBlocksDeletionACKResponseProto
-        .newBuilder().getDefaultInstanceForType();
-  }
-
-  public ReportState getReportState() {
-    return this.reportState;
-  }
-
-  /**
-   * Reset the mock Scm for test to get a fresh start without rebuild MockScm.
-   */
-  public void reset() {
-    heartbeatCount.set(0);
-    rpcCount.set(0);
-    reportState = ReportState.newBuilder()
-        .setState(ReportState.states.noContainerReports)
-        .setCount(0).build();
-    containerReportsCount.set(0);
-    nodeContainers.clear();
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
 
b/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
deleted file mode 100644
index f05ba49..0000000
--- 
a/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
+++ /dev/null
@@ -1,375 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations 
under
- * the License.
- */
-package org.apache.hadoop.ozone.container.common;
-
-import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
-import org.apache.hadoop.scm.ScmConfigKeys;
-import 
org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
-import 
org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine;
-import 
org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
-import org.apache.hadoop.ozone.container.common.states.DatanodeState;
-import 
org.apache.hadoop.ozone.container.common.states.datanode.InitDatanodeState;
-import 
org.apache.hadoop.ozone.container.common.states.datanode.RunningDatanodeState;
-import 
org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.util.concurrent.HadoopExecutors;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.nio.file.Paths;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
-import static org.apache.hadoop.scm.ScmConfigKeys
-    .OZONE_SCM_HEARTBEAT_RPC_TIMEOUT;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Tests the datanode state machine class and its states.
- */
-public class TestDatanodeStateMachine {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(TestDatanodeStateMachine.class);
-  private final int scmServerCount = 3;
-  private List<String> serverAddresses;
-  private List<RPC.Server> scmServers;
-  private List<ScmTestMock> mockServers;
-  private ExecutorService executorService;
-  private Configuration conf;
-  private File testRoot;
-
-  @Before
-  public void setUp() throws Exception {
-    conf = SCMTestUtils.getConf();
-    conf.setTimeDuration(OZONE_SCM_HEARTBEAT_RPC_TIMEOUT, 500,
-        TimeUnit.MILLISECONDS);
-    conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, true);
-    serverAddresses = new LinkedList<>();
-    scmServers = new LinkedList<>();
-    mockServers = new LinkedList<>();
-    for (int x = 0; x < scmServerCount; x++) {
-      int port = SCMTestUtils.getReuseableAddress().getPort();
-      String address = "127.0.0.1";
-      serverAddresses.add(address + ":" + port);
-      ScmTestMock mock = new ScmTestMock();
-
-      scmServers.add(SCMTestUtils.startScmRpcServer(conf, mock,
-          new InetSocketAddress(address, port), 10));
-      mockServers.add(mock);
-    }
-
-    conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES,
-        serverAddresses.toArray(new String[0]));
-
-    String path = GenericTestUtils
-        .getTempPath(TestDatanodeStateMachine.class.getSimpleName());
-    testRoot = new File(path);
-    if (!testRoot.mkdirs()) {
-      LOG.info("Required directories {} already exist.", testRoot);
-    }
-
-    File dataDir = new File(testRoot, "data");
-    conf.set(DFS_DATANODE_DATA_DIR_KEY, dataDir.getAbsolutePath());
-    if (!dataDir.mkdirs()) {
-      LOG.info("Data dir create failed.");
-    }
-    conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS,
-        new File(testRoot, "scm").getAbsolutePath());
-    path = Paths.get(path.toString(),
-        TestDatanodeStateMachine.class.getSimpleName() + ".id").toString();
-    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ID, path);
-    executorService = HadoopExecutors.newCachedThreadPool(
-        new ThreadFactoryBuilder().setDaemon(true)
-            .setNameFormat("Test Data Node State Machine Thread - 
%d").build());
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    try {
-      if (executorService != null) {
-        executorService.shutdown();
-        try {
-          if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) {
-            executorService.shutdownNow();
-          }
-
-          if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) {
-            LOG.error("Unable to shutdown properly.");
-          }
-        } catch (InterruptedException e) {
-          LOG.error("Error attempting to shutdown.", e);
-          executorService.shutdownNow();
-        }
-      }
-      for (RPC.Server s : scmServers) {
-        s.stop();
-      }
-    } catch (Exception e) {
-      //ignore all execption from the shutdown
-    } finally {
-      testRoot.delete();
-    }
-  }
-
-  /**
-   * Assert that starting statemachine executes the Init State.
-   *
-   * @throws InterruptedException
-   */
-  @Test
-  public void testStartStopDatanodeStateMachine() throws IOException,
-      InterruptedException, TimeoutException {
-    try (DatanodeStateMachine stateMachine =
-        new DatanodeStateMachine(getNewDatanodeDetails(), conf)) {
-      stateMachine.startDaemon();
-      SCMConnectionManager connectionManager =
-          stateMachine.getConnectionManager();
-      GenericTestUtils.waitFor(() -> connectionManager.getValues().size() == 3,
-          1000, 30000);
-
-      stateMachine.stopDaemon();
-      assertTrue(stateMachine.isDaemonStopped());
-    }
-  }
-
-  /**
-   * This test explores the state machine by invoking each call in sequence 
just
-   * like as if the state machine would call it. Because this is a test we are
-   * able to verify each of the assumptions.
-   * <p>
-   * Here is what happens at High level.
-   * <p>
-   * 1. We start the datanodeStateMachine in the INIT State.
-   * <p>
-   * 2. We invoke the INIT state task.
-   * <p>
-   * 3. That creates a set of RPC endpoints that are ready to connect to SCMs.
-   * <p>
-   * 4. We assert that we have moved to the running state for the
-   * DatanodeStateMachine.
-   * <p>
-   * 5. We get the task for the Running State - Executing that running state,
-   * makes the first network call in of the state machine. The Endpoint is in
-   * the GETVERSION State and we invoke the task.
-   * <p>
-   * 6. We assert that this call was a success by checking that each of the
-   * endponts now have version response that it got from the SCM server that it
-   * was talking to and also each of the mock server serviced one RPC call.
-   * <p>
-   * 7. Since the Register is done now, next calls to get task will return
-   * HeartbeatTask, which sends heartbeats to SCM. We assert that we get right
-   * task from sub-system below.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testDatanodeStateContext() throws IOException,
-      InterruptedException, ExecutionException, TimeoutException {
-    // There is no mini cluster started in this test,
-    // create a ID file so that state machine could load a fake datanode ID.
-    File idPath = new File(
-        conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID));
-    idPath.delete();
-    DatanodeDetails datanodeDetails = getNewDatanodeDetails();
-    datanodeDetails.setContainerPort(
-        OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
-    ContainerUtils.writeDatanodeDetailsTo(datanodeDetails, idPath);
-
-    try (DatanodeStateMachine stateMachine =
-             new DatanodeStateMachine(datanodeDetails, conf)) {
-      DatanodeStateMachine.DatanodeStates currentState =
-          stateMachine.getContext().getState();
-      Assert.assertEquals(DatanodeStateMachine.DatanodeStates.INIT,
-          currentState);
-
-      DatanodeState<DatanodeStateMachine.DatanodeStates> task =
-          stateMachine.getContext().getTask();
-      Assert.assertEquals(InitDatanodeState.class, task.getClass());
-
-      task.execute(executorService);
-      DatanodeStateMachine.DatanodeStates newState =
-          task.await(2, TimeUnit.SECONDS);
-
-      for (EndpointStateMachine endpoint :
-          stateMachine.getConnectionManager().getValues()) {
-        // We assert that each of the is in State GETVERSION.
-        Assert.assertEquals(EndpointStateMachine.EndPointStates.GETVERSION,
-            endpoint.getState());
-      }
-
-      // The Datanode has moved into Running State, since endpoints are 
created.
-      // We move to running state when we are ready to issue RPC calls to SCMs.
-      Assert.assertEquals(DatanodeStateMachine.DatanodeStates.RUNNING,
-          newState);
-
-      // If we had called context.execute instead of calling into each state
-      // this would have happened automatically.
-      stateMachine.getContext().setState(newState);
-      task = stateMachine.getContext().getTask();
-      Assert.assertEquals(RunningDatanodeState.class, task.getClass());
-
-      // This execute will invoke getVersion calls against all SCM endpoints
-      // that we know of.
-
-      task.execute(executorService);
-      newState = task.await(10, TimeUnit.SECONDS);
-      // If we are in running state, we should be in running.
-      Assert.assertEquals(DatanodeStateMachine.DatanodeStates.RUNNING,
-          newState);
-
-      for (EndpointStateMachine endpoint :
-          stateMachine.getConnectionManager().getValues()) {
-
-        // Since the earlier task.execute called into GetVersion, the
-        // endPointState Machine should move to REGISTER state.
-        Assert.assertEquals(EndpointStateMachine.EndPointStates.REGISTER,
-            endpoint.getState());
-
-        // We assert that each of the end points have gotten a version from the
-        // SCM Server.
-        Assert.assertNotNull(endpoint.getVersion());
-      }
-
-      // We can also assert that all mock servers have received only one RPC
-      // call at this point of time.
-      for (ScmTestMock mock : mockServers) {
-        Assert.assertEquals(1, mock.getRpcCount());
-      }
-
-      // This task is the Running task, but running task executes tasks based
-      // on the state of Endpoints, hence this next call will be a Register at
-      // the endpoint RPC level.
-      task = stateMachine.getContext().getTask();
-      task.execute(executorService);
-      newState = task.await(2, TimeUnit.SECONDS);
-
-      // If we are in running state, we should be in running.
-      Assert.assertEquals(DatanodeStateMachine.DatanodeStates.RUNNING,
-          newState);
-
-      for (ScmTestMock mock : mockServers) {
-        Assert.assertEquals(2, mock.getRpcCount());
-      }
-
-      // This task is the Running task, but running task executes tasks based
-      // on the state of Endpoints, hence this next call will be a
-      // HeartbeatTask at the endpoint RPC level.
-      task = stateMachine.getContext().getTask();
-      task.execute(executorService);
-      newState = task.await(2, TimeUnit.SECONDS);
-
-      // If we are in running state, we should be in running.
-      Assert.assertEquals(DatanodeStateMachine.DatanodeStates.RUNNING,
-          newState);
-
-
-      for (ScmTestMock mock : mockServers) {
-        Assert.assertEquals(1, mock.getHeartbeatCount());
-        // Assert that heartbeat did indeed carry that State that we said
-        // have in the datanode.
-        Assert.assertEquals(mock.getReportState().getState().getNumber(),
-            StorageContainerDatanodeProtocolProtos.ReportState.states
-                .noContainerReports.getNumber());
-      }
-    }
-  }
-
-  /**
-   * Test state transition with a list of invalid scm configurations,
-   * and verify the state transits to SHUTDOWN each time.
-   */
-  @Test
-  public void testDatanodeStateMachineWithInvalidConfiguration()
-      throws Exception {
-    LinkedList<Map.Entry<String, String>> confList =
-        new LinkedList<Map.Entry<String, String>>();
-    confList.add(Maps.immutableEntry(ScmConfigKeys.OZONE_SCM_NAMES, ""));
-
-    // Invalid ozone.scm.names
-    /** Empty **/
-    confList.add(Maps.immutableEntry(
-        ScmConfigKeys.OZONE_SCM_NAMES, ""));
-    /** Invalid schema **/
-    confList.add(Maps.immutableEntry(
-        ScmConfigKeys.OZONE_SCM_NAMES, "x..y"));
-    /** Invalid port **/
-    confList.add(Maps.immutableEntry(
-        ScmConfigKeys.OZONE_SCM_NAMES, "scm:xyz"));
-    /** Port out of range **/
-    confList.add(Maps.immutableEntry(
-        ScmConfigKeys.OZONE_SCM_NAMES, "scm:123456"));
-    // Invalid ozone.scm.datanode.id
-    /** Empty **/
-    confList.add(Maps.immutableEntry(
-        ScmConfigKeys.OZONE_SCM_DATANODE_ID, ""));
-
-    confList.forEach((entry) -> {
-      Configuration perTestConf = new Configuration(conf);
-      perTestConf.setStrings(entry.getKey(), entry.getValue());
-      LOG.info("Test with {} = {}", entry.getKey(), entry.getValue());
-      try (DatanodeStateMachine stateMachine = new DatanodeStateMachine(
-          getNewDatanodeDetails(), perTestConf)) {
-        DatanodeStateMachine.DatanodeStates currentState =
-            stateMachine.getContext().getState();
-        Assert.assertEquals(DatanodeStateMachine.DatanodeStates.INIT,
-            currentState);
-        DatanodeState<DatanodeStateMachine.DatanodeStates> task =
-            stateMachine.getContext().getTask();
-        task.execute(executorService);
-        DatanodeStateMachine.DatanodeStates newState =
-            task.await(2, TimeUnit.SECONDS);
-        Assert.assertEquals(DatanodeStateMachine.DatanodeStates.SHUTDOWN,
-            newState);
-      } catch (Exception e) {
-        Assert.fail("Unexpected exception found");
-      }
-    });
-  }
-
-  private DatanodeDetails getNewDatanodeDetails() {
-    return DatanodeDetails.newBuilder()
-        .setUuid(UUID.randomUUID().toString())
-        .setHostName("localhost")
-        .setIpAddress("127.0.0.1")
-        .setInfoPort(0)
-        .setInfoSecurePort(0)
-        .setContainerPort(0)
-        .setRatisPort(0)
-        .setOzoneRestPort(0)
-        .build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
 
b/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
deleted file mode 100644
index 61bd65f..0000000
--- 
a/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations 
under
- * the License.
- */
-package org.apache.hadoop.ozone.container.testutils;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
-import 
org.apache.hadoop.ozone.container.common.statemachine.background.BlockDeletingService;
-
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * A test class implementation for {@link BlockDeletingService}.
- */
-public class BlockDeletingServiceTestImpl
-    extends BlockDeletingService {
-
-  // the service timeout
-  private static final int SERVICE_TIMEOUT_IN_MILLISECONDS = 0;
-
-  // tests only
-  private CountDownLatch latch;
-  private Thread testingThread;
-  private AtomicInteger numOfProcessed = new AtomicInteger(0);
-
-  public BlockDeletingServiceTestImpl(ContainerManager containerManager,
-      int serviceInterval, Configuration conf) {
-    super(containerManager, serviceInterval,
-        SERVICE_TIMEOUT_IN_MILLISECONDS, conf);
-  }
-
-  @VisibleForTesting
-  public void runDeletingTasks() {
-    if (latch.getCount() > 0) {
-      this.latch.countDown();
-    } else {
-      throw new IllegalStateException("Count already reaches zero");
-    }
-  }
-
-  @VisibleForTesting
-  public boolean isStarted() {
-    return latch != null && testingThread.isAlive();
-  }
-
-  public int getTimesOfProcessed() {
-    return numOfProcessed.get();
-  }
-
-  // Override the implementation to start a single on-call control thread.
-  @Override public void start() {
-    PeriodicalTask svc = new PeriodicalTask();
-    // In test mode, relies on a latch countdown to runDeletingTasks tasks.
-    Runnable r = () -> {
-      while (true) {
-        latch = new CountDownLatch(1);
-        try {
-          latch.await();
-        } catch (InterruptedException e) {
-          break;
-        }
-        Future<?> future = this.getExecutorService().submit(svc);
-        try {
-          // for tests, we only wait for 3s for completion
-          future.get(3, TimeUnit.SECONDS);
-          numOfProcessed.incrementAndGet();
-        } catch (Exception e) {
-          return;
-        }
-      }
-    };
-
-    testingThread = new ThreadFactoryBuilder()
-        .setDaemon(true)
-        .build()
-        .newThread(r);
-    testingThread.start();
-  }
-
-  @Override
-  public void shutdown() {
-    testingThread.interrupt();
-    super.shutdown();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/package-info.java
 
b/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/package-info.java
deleted file mode 100644
index 4e8a90b..0000000
--- 
a/hadoop-hdsl/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/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.ozone.container.testutils;
-// Helper classes for ozone and container tests.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/framework/README.md
----------------------------------------------------------------------
diff --git a/hadoop-hdsl/framework/README.md b/hadoop-hdsl/framework/README.md
deleted file mode 100644
index 1f4d217..0000000
--- a/hadoop-hdsl/framework/README.md
+++ /dev/null
@@ -1,24 +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.
--->
-
-# Server framework for HDSL/Ozone
-
-This project contains generic utilities and resources for all the HDSL/Ozone 
-server-side components.
-
-The project is shared between the server/service projects but not with the 
-client packages.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/framework/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdsl/framework/pom.xml b/hadoop-hdsl/framework/pom.xml
deleted file mode 100644
index 44ef7fe..0000000
--- a/hadoop-hdsl/framework/pom.xml
+++ /dev/null
@@ -1,91 +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-server-framework</artifactId>
-  <version>3.2.0-SNAPSHOT</version>
-  <description>Apache Hadoop HDSL Common utilities for server side
-    components
-  </description>
-  <name>Apache Hadoop HDSL Server Common</name>
-  <packaging>jar</packaging>
-
-  <properties>
-    <hadoop.component>hdsl</hadoop.component>
-    <is.hadoop.component>true</is.hadoop.component>
-  </properties>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdsl-common</artifactId>
-      <scope>provided</scope>
-    </dependency>
-  </dependencies>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.rat</groupId>
-        <artifactId>apache-rat-plugin</artifactId>
-        <configuration>
-          <excludes>
-            <exclude>.gitattributes</exclude>
-            <exclude>.idea/**</exclude>
-            <exclude>src/main/conf/*</exclude>
-            <exclude>src/main/webapps/static/angular-1.6.4.min.js</exclude>
-            
<exclude>src/main/webapps/static/angular-nvd3-1.0.9.min.js</exclude>
-            
<exclude>src/main/webapps/static/angular-route-1.6.4.min.js</exclude>
-            <exclude>src/main/webapps/static/d3-3.5.17.min.js</exclude>
-            <exclude>src/main/webapps/static/nvd3-1.8.5.min.css</exclude>
-            <exclude>src/main/webapps/static/nvd3-1.8.5.min.css.map</exclude>
-            <exclude>src/main/webapps/static/nvd3-1.8.5.min.js</exclude>
-            <exclude>src/main/webapps/static/nvd3-1.8.5.min.js.map</exclude>
-          </excludes>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-antrun-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>copy web resources</id>
-            <phase>compile</phase>
-            <goals>
-              <goal>run</goal>
-            </goals>
-            <configuration>
-              <target>
-                <copy 
toDir="${project.build.directory}/classes/webapps/static">
-                  <fileset
-                          
dir="${basedir}/../../hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static">
-                  </fileset>
-                </copy>
-              </target>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/BaseHttpServer.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/BaseHttpServer.java
 
b/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/BaseHttpServer.java
deleted file mode 100644
index f9c3991..0000000
--- 
a/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/BaseHttpServer.java
+++ /dev/null
@@ -1,218 +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.server;
-
-import com.google.common.base.Optional;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.http.HttpConfig;
-import org.apache.hadoop.http.HttpServer2;
-import org.apache.hadoop.net.NetUtils;
-import org.eclipse.jetty.webapp.WebAppContext;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.servlet.http.HttpServlet;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import static org.apache.hadoop.hdsl.HdslUtils.getHostNameFromConfigKeys;
-import static org.apache.hadoop.hdsl.HdslUtils.getPortNumberFromConfigKeys;
-
-/**
- * Base class for HTTP server of the Ozone related components.
- */
-public abstract class BaseHttpServer {
-
-  private static final Logger LOG =
-      LoggerFactory.getLogger(BaseHttpServer.class);
-
-  private HttpServer2 httpServer;
-  private final Configuration conf;
-
-  private InetSocketAddress httpAddress;
-  private InetSocketAddress httpsAddress;
-
-  private HttpConfig.Policy policy;
-
-  private String name;
-
-  public BaseHttpServer(Configuration conf, String name) throws IOException {
-    this.name = name;
-    this.conf = conf;
-    if (isEnabled()) {
-      policy = DFSUtil.getHttpPolicy(conf);
-      if (policy.isHttpEnabled()) {
-        this.httpAddress = getHttpBindAddress();
-      }
-      if (policy.isHttpsEnabled()) {
-        this.httpsAddress = getHttpsBindAddress();
-      }
-      HttpServer2.Builder builder = null;
-      builder = DFSUtil.httpServerTemplateForNNAndJN(conf, this.httpAddress,
-          this.httpsAddress, name, getSpnegoPrincipal(), getKeytabFile());
-
-      final boolean xFrameEnabled = conf.getBoolean(
-          DFSConfigKeys.DFS_XFRAME_OPTION_ENABLED,
-          DFSConfigKeys.DFS_XFRAME_OPTION_ENABLED_DEFAULT);
-
-      final String xFrameOptionValue = conf.getTrimmed(
-          DFSConfigKeys.DFS_XFRAME_OPTION_VALUE,
-          DFSConfigKeys.DFS_XFRAME_OPTION_VALUE_DEFAULT);
-
-      
builder.configureXFrame(xFrameEnabled).setXFrameOption(xFrameOptionValue);
-
-      httpServer = builder.build();
-
-    }
-
-  }
-
-  /**
-   * Add a servlet to BaseHttpServer.
-   * @param servletName The name of the servlet
-   * @param pathSpec The path spec for the servlet
-   * @param clazz The servlet class
-   */
-  protected void addServlet(String servletName, String pathSpec,
-                            Class<? extends HttpServlet> clazz) {
-    httpServer.addServlet(servletName, pathSpec, clazz);
-  }
-
-  /**
-   * Returns the WebAppContext associated with this HttpServer.
-   * @return WebAppContext
-   */
-  protected WebAppContext getWebAppContext() {
-    return httpServer.getWebAppContext();
-  }
-
-  protected InetSocketAddress getBindAddress(String bindHostKey,
-      String addressKey, String bindHostDefault, int bindPortdefault) {
-    final Optional<String> bindHost =
-        getHostNameFromConfigKeys(conf, bindHostKey);
-
-    final Optional<Integer> addressPort =
-        getPortNumberFromConfigKeys(conf, addressKey);
-
-    final Optional<String> addresHost =
-        getHostNameFromConfigKeys(conf, addressKey);
-
-    String hostName = bindHost.or(addresHost).or(bindHostDefault);
-
-    return NetUtils.createSocketAddr(
-        hostName + ":" + addressPort.or(bindPortdefault));
-  }
-
-  /**
-   * Retrieve the socket address that should be used by clients to connect
-   * to the  HTTPS web interface.
-   *
-   * @return Target InetSocketAddress for the Ozone HTTPS endpoint.
-   */
-  public InetSocketAddress getHttpsBindAddress() {
-    return getBindAddress(getHttpsBindHostKey(), getHttpsAddressKey(),
-        getBindHostDefault(), getHttpsBindPortDefault());
-  }
-
-  /**
-   * Retrieve the socket address that should be used by clients to connect
-   * to the  HTTP web interface.
-   * <p>
-   * * @return Target InetSocketAddress for the Ozone HTTP endpoint.
-   */
-  public InetSocketAddress getHttpBindAddress() {
-    return getBindAddress(getHttpBindHostKey(), getHttpAddressKey(),
-        getBindHostDefault(), getHttpBindPortDefault());
-
-  }
-
-  public void start() throws IOException {
-    if (httpServer != null && isEnabled()) {
-      httpServer.start();
-      updateConnectorAddress();
-    }
-
-  }
-
-  private boolean isEnabled() {
-    return conf.getBoolean(getEnabledKey(), true);
-  }
-
-  public void stop() throws Exception {
-    if (httpServer != null) {
-      httpServer.stop();
-    }
-  }
-
-  /**
-   * Update the configured listen address based on the real port
-   * <p>
-   * (eg. replace :0 with real port)
-   */
-  public void updateConnectorAddress() {
-    int connIdx = 0;
-    if (policy.isHttpEnabled()) {
-      httpAddress = httpServer.getConnectorAddress(connIdx++);
-      String realAddress = NetUtils.getHostPortString(httpAddress);
-      conf.set(getHttpAddressKey(), realAddress);
-      LOG.info(
-          String.format("HTTP server of %s is listening at http://%s";,
-              name.toUpperCase(), realAddress));
-    }
-
-    if (policy.isHttpsEnabled()) {
-      httpsAddress = httpServer.getConnectorAddress(connIdx);
-      String realAddress = NetUtils.getHostPortString(httpsAddress);
-      conf.set(getHttpsAddressKey(), realAddress);
-      LOG.info(
-          String.format("HTTP server of %s is listening at https://%s";,
-              name.toUpperCase(), realAddress));
-    }
-  }
-
-  public InetSocketAddress getHttpAddress() {
-    return httpAddress;
-  }
-
-  public InetSocketAddress getHttpsAddress() {
-    return httpsAddress;
-  }
-
-  protected abstract String getHttpAddressKey();
-
-  protected abstract String getHttpsAddressKey();
-
-  protected abstract String getHttpBindHostKey();
-
-  protected abstract String getHttpsBindHostKey();
-
-  protected abstract String getBindHostDefault();
-
-  protected abstract int getHttpBindPortDefault();
-
-  protected abstract int getHttpsBindPortDefault();
-
-  protected abstract String getKeytabFile();
-
-  protected abstract String getSpnegoPrincipal();
-
-  protected abstract String getEnabledKey();
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/ServerUtils.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/ServerUtils.java
 
b/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/ServerUtils.java
deleted file mode 100644
index f315ecb..0000000
--- 
a/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/ServerUtils.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <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.server;
-
-import java.io.File;
-import java.net.InetSocketAddress;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-
-import com.google.common.base.Preconditions;
-import org.apache.http.client.methods.HttpRequestBase;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Generic utilities for all HDSL/Ozone servers.
- */
-public class ServerUtils {
-
-  private static final Logger LOG = LoggerFactory.getLogger(
-      ServerUtils.class);
-
-  private ServerUtils() {
-  }
-
-  /**
-   * Checks that a given value is with a range.
-   *
-   * For example, sanitizeUserArgs(17, 3, 5, 10)
-   * ensures that 17 is greater/equal than 3 * 5 and less/equal to 3 * 10.
-   *
-   * @param valueTocheck  - value to check
-   * @param baseValue     - the base value that is being used.
-   * @param minFactor     - range min - a 2 here makes us ensure that value
-   *                        valueTocheck is at least twice the baseValue.
-   * @param maxFactor     - range max
-   * @return long
-   */
-  public static long sanitizeUserArgs(long valueTocheck, long baseValue,
-      long minFactor, long maxFactor)
-      throws IllegalArgumentException {
-    if ((valueTocheck >= (baseValue * minFactor)) &&
-        (valueTocheck <= (baseValue * maxFactor))) {
-      return valueTocheck;
-    }
-    String errMsg = String.format("%d is not within min = %d or max = " +
-        "%d", valueTocheck, baseValue * minFactor, baseValue * maxFactor);
-    throw new IllegalArgumentException(errMsg);
-  }
-
-
-  /**
-   * After starting an RPC server, updates configuration with the actual
-   * listening address of that server. The listening address may be different
-   * from the configured address if, for example, the configured address uses
-   * port 0 to request use of an ephemeral port.
-   *
-   * @param conf configuration to update
-   * @param rpcAddressKey configuration key for RPC server address
-   * @param addr configured address
-   * @param rpcServer started RPC server.
-   */
-  public static InetSocketAddress updateRPCListenAddress(
-      OzoneConfiguration conf, String rpcAddressKey,
-      InetSocketAddress addr, RPC.Server rpcServer) {
-    return updateListenAddress(conf, rpcAddressKey, addr,
-        rpcServer.getListenerAddress());
-  }
-
-
-  /**
-   * After starting an server, updates configuration with the actual
-   * listening address of that server. The listening address may be different
-   * from the configured address if, for example, the configured address uses
-   * port 0 to request use of an ephemeral port.
-   *
-   * @param conf       configuration to update
-   * @param addressKey configuration key for RPC server address
-   * @param addr       configured address
-   * @param listenAddr the real listening address.
-   */
-  public static InetSocketAddress updateListenAddress(OzoneConfiguration conf,
-      String addressKey, InetSocketAddress addr, InetSocketAddress listenAddr) 
{
-    InetSocketAddress updatedAddr = new InetSocketAddress(addr.getHostString(),
-        listenAddr.getPort());
-    conf.set(addressKey,
-        addr.getHostString() + ":" + listenAddr.getPort());
-    return updatedAddr;
-  }
-
-
-  /**
-   * Releases a http connection if the request is not null.
-   * @param request
-   */
-  public static void releaseConnection(HttpRequestBase request) {
-    if (request != null) {
-      request.releaseConnection();
-    }
-  }
-
-
-  /**
-   * Checks and creates Ozone Metadir Path if it does not exist.
-   *
-   * @param conf - Configuration
-   *
-   * @return File MetaDir
-   */
-  public static File getOzoneMetaDirPath(Configuration conf) {
-    String metaDirPath = conf.getTrimmed(OzoneConfigKeys
-        .OZONE_METADATA_DIRS);
-    Preconditions.checkNotNull(metaDirPath);
-    File dirPath = new File(metaDirPath);
-    if (!dirPath.exists() && !dirPath.mkdirs()) {
-      throw new IllegalArgumentException("Unable to create paths. Path: " +
-          dirPath);
-    }
-    return dirPath;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/ServiceRuntimeInfo.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/ServiceRuntimeInfo.java
 
b/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/ServiceRuntimeInfo.java
deleted file mode 100644
index f9c57ea..0000000
--- 
a/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/ServiceRuntimeInfo.java
+++ /dev/null
@@ -1,64 +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.server;
-
-/**
- * Common runtime information for any service components.
- *
- * Note: it's intentional to not use MXBean or MBean as a suffix  of the name.
- *
- * Most of the services extends the ServiceRuntimeInfoImpl class and also
- * implements a specific MXBean interface which extends this interface.
- *
- * This inheritance from multiple path could confuse the jmx system and
- * some jmx properties could be disappeared.
- *
- * The solution is to always extend this interface and use the jmx naming
- * convention in the new interface..
- */
-public interface ServiceRuntimeInfo {
-
-  /**
-   * Gets the version of Hadoop.
-   *
-   * @return the version
-   */
-  String getVersion();
-
-  /**
-   * Get the version of software running on the Namenode.
-   *
-   * @return a string representing the version
-   */
-  String getSoftwareVersion();
-
-  /**
-   * Get the compilation information which contains date, user and branch.
-   *
-   * @return the compilation information, as a JSON string.
-   */
-  String getCompileInfo();
-
-  /**
-   * Gets the NN start time in milliseconds.
-   *
-   * @return the NN start time in msec
-   */
-  long getStartedTimeInMillis();
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/ServiceRuntimeInfoImpl.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/ServiceRuntimeInfoImpl.java
 
b/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/ServiceRuntimeInfoImpl.java
deleted file mode 100644
index 92f00b1..0000000
--- 
a/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/ServiceRuntimeInfoImpl.java
+++ /dev/null
@@ -1,55 +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.server;
-
-import org.apache.hadoop.util.VersionInfo;
-
-/**
- * Helper base class to report the standard version and runtime information.
- *
- */
-public class ServiceRuntimeInfoImpl implements ServiceRuntimeInfo {
-
-  private long startedTimeInMillis;
-
-  @Override
-  public String getVersion() {
-    return VersionInfo.getVersion() + ", r" + VersionInfo.getRevision();
-  }
-
-  @Override
-  public String getSoftwareVersion() {
-    return VersionInfo.getVersion();
-  }
-
-  @Override
-  public String getCompileInfo() {
-    return VersionInfo.getDate() + " by " + VersionInfo.getUser() + " from "
-        + VersionInfo.getBranch();
-  }
-
-  @Override
-  public long getStartedTimeInMillis() {
-    return startedTimeInMillis;
-  }
-
-  public void setStartTime() {
-    startedTimeInMillis = System.currentTimeMillis();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/package-info.java
 
b/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/package-info.java
deleted file mode 100644
index b38215c..0000000
--- 
a/hadoop-hdsl/framework/src/main/java/org/apache/hadoop/hdsl/server/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.server;
-
-/**
- * Common server side utilities for all the hdsl/ozone server components.
- */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/651a05a1/hadoop-hdsl/framework/src/main/resources/webapps/datanode/dn.js
----------------------------------------------------------------------
diff --git a/hadoop-hdsl/framework/src/main/resources/webapps/datanode/dn.js 
b/hadoop-hdsl/framework/src/main/resources/webapps/datanode/dn.js
deleted file mode 100644
index 3b67167..0000000
--- a/hadoop-hdsl/framework/src/main/resources/webapps/datanode/dn.js
+++ /dev/null
@@ -1,92 +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.
- */
-(function () {
-  "use strict";
-
-  var data = {ozone: {enabled: false}};
-
-  dust.loadSource(dust.compile($('#tmpl-dn').html(), 'dn'));
-
-  function loadDatanodeInfo() {
-    $.get('/jmx?qry=Hadoop:service=DataNode,name=DataNodeInfo', function(resp) 
{
-      data.dn = workaround(resp.beans[0]);
-      data.dn.HostName = resp.beans[0]['DatanodeHostname'];
-      render();
-    }).fail(show_err_msg);
-  }
-
-  function loadOzoneScmInfo() {
-        
$.get('/jmx?qry=Hadoop:service=OzoneDataNode,name=SCMConnectionManager', 
function (resp) {
-            if (resp.beans.length > 0) {
-                data.ozone.SCMServers = resp.beans[0].SCMServers;
-                data.ozone.enabled = true;
-                render();
-            }
-        }).fail(show_err_msg);
-  }
-
-  function loadOzoneStorageInfo() {
-        
$.get('/jmx?qry=Hadoop:service=OzoneDataNode,name=ContainerLocationManager', 
function (resp) {
-            if (resp.beans.length > 0) {
-                data.ozone.LocationReport = resp.beans[0].LocationReport;
-                data.ozone.enabled = true;
-                render();
-            }
-        }).fail(show_err_msg);
-    }
-
-  function workaround(dn) {
-    function node_map_to_array(nodes) {
-      var res = [];
-      for (var n in nodes) {
-        var p = nodes[n];
-        p.name = n;
-        res.push(p);
-      }
-      return res;
-    }
-
-    dn.VolumeInfo = node_map_to_array(JSON.parse(dn.VolumeInfo));
-    dn.BPServiceActorInfo = JSON.parse(dn.BPServiceActorInfo);
-
-    return dn;
-  }
-
-  function render() {
-    var base = dust.makeBase({
-      'helper_relative_time' : function (chunk, ctx, bodies, params) {
-        var value = dust.helpers.tap(params.value, chunk, ctx);
-        return chunk.write(moment().subtract(Number(value), 
'seconds').fromNow(true));
-      }
-    });
-    dust.render('dn', base.push(data), function(err, out) {
-      $('#tab-overview').html(out);
-      $('#tab-overview').addClass('active');
-    });
-  }
-
-  function show_err_msg() {
-    $('#alert-panel-body').html("Failed to load datanode information");
-    $('#alert-panel').show();
-  }
-
-    loadDatanodeInfo();
-    loadOzoneScmInfo();
-    loadOzoneStorageInfo();
-
-})();


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