Repository: hadoop
Updated Branches:
  refs/heads/HDDS-48 143dd560b -> 0e437f9b1


Revert "Create Version File in Datanode. Contributed by Bharat Viswanadham."

This reverts commit f26d3466d79125123cba00ab81481655d7bfe3c1.


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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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


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

Reply via email to