Create Version File in Datanode. Contributed by Bharat Viswanadham.

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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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


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

Reply via email to