You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by ra...@apache.org on 2020/11/20 10:15:38 UTC

[ozone] 01/04: HDDS-2949: mkdir : store directory entries in a separate table (#1404)

This is an automated email from the ASF dual-hosted git repository.

rakeshr pushed a commit to branch HDDS-2939
in repository https://gitbox.apache.org/repos/asf/ozone.git

commit fbd570763152eb55386f0115fa0a3e29322df62c
Author: Rakesh Radhakrishnan <ra...@apache.org>
AuthorDate: Fri Oct 2 00:45:19 2020 +0530

    HDDS-2949: mkdir : store directory entries in a separate table (#1404)
---
 .../common/src/main/resources/ozone-default.xml    |  10 +
 .../org/apache/hadoop/ozone/om/OMConfigKeys.java   |   7 +
 .../hadoop/ozone/om/helpers/OmDirectoryInfo.java   | 266 +++++++++
 .../hadoop/ozone/om/helpers/OzoneFSUtils.java      |  15 +
 .../apache/hadoop/fs/ozone/TestOzoneDirectory.java | 207 +++++++
 .../src/main/proto/OmClientProtocol.proto          |  11 +
 .../apache/hadoop/ozone/om/OMMetadataManager.java  |  17 +
 .../java/org/apache/hadoop/ozone/om/OMMetrics.java |   4 +
 .../hadoop/ozone/om/OmMetadataManagerImpl.java     |  24 +
 .../org/apache/hadoop/ozone/om/OzoneManager.java   |  14 +
 .../ozone/om/codec/OmDirectoryInfoCodec.java       |  60 ++
 .../om/ratis/utils/OzoneManagerRatisUtils.java     |  18 +
 .../om/request/file/OMDirectoryCreateRequest.java  |   3 +
 ...equest.java => OMDirectoryCreateRequestV1.java} | 321 +++++-----
 .../ozone/om/request/file/OMFileRequest.java       | 198 +++++++
 .../response/file/OMDirectoryCreateResponseV1.java | 103 ++++
 .../ozone/om/request/TestOMRequestUtils.java       |  37 ++
 .../file/TestOMDirectoryCreateRequestV1.java       | 649 +++++++++++++++++++++
 .../file/TestOMDirectoryCreateResponseV1.java      |  88 +++
 19 files changed, 1867 insertions(+), 185 deletions(-)

diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index d8402f7..d42869b 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -2475,4 +2475,14 @@
       filesystem semantics.
     </description>
   </property>
+
+  <property>
+    <name>ozone.om.layout.version</name>
+    <tag>OZONE, OM</tag>
+    <value>V0</value>
+    <description>Temporary workaround for OM upgrade and will be replaced once
+      upgrade HDDS-3698 story reaches consensus. Defaulting to 'V0' so that
+      existing unit test cases won't be affected. New OM version should be 'V1'
+    </description>
+  </property>
 </configuration>
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
index f16679a..db3c453 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
@@ -246,4 +246,11 @@ public final class OMConfigKeys {
       "ozone.om.enable.filesystem.paths";
   public static final boolean OZONE_OM_ENABLE_FILESYSTEM_PATHS_DEFAULT =
       false;
+
+  // TODO: Temporary workaround for OM upgrade path and will be replaced once
+  //  upgrade HDDS-3698 story reaches consensus. Defaulting to 'V0' so that
+  //  existing unit test cases won't be affected. New OM version should be 'V1'.
+  public static final String OZONE_OM_LAYOUT_VERSION =
+          "ozone.om.layout.version";
+  public static final String OZONE_OM_LAYOUT_VERSION_DEFAULT = "V0";
 }
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmDirectoryInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmDirectoryInfo.java
new file mode 100644
index 0000000..4c82047
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmDirectoryInfo.java
@@ -0,0 +1,266 @@
+/**
+ * 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.om.helpers;
+
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+import java.util.*;
+
+/**
+ * This class represents the directory information by keeping each component
+ * in the user given path and a pointer to its parent directory element in the
+ * path. Also, it stores directory node related metdata details.
+ */
+public class OmDirectoryInfo extends WithObjectID {
+  private long parentObjectID; // pointer to parent directory
+
+  private String name; // directory name
+
+  private long creationTime;
+  private long modificationTime;
+
+  private List<OzoneAcl> acls;
+
+  public OmDirectoryInfo(Builder builder) {
+    this.name = builder.name;
+    this.acls = builder.acls;
+    this.metadata = builder.metadata;
+    this.objectID = builder.objectID;
+    this.updateID = builder.updateID;
+    this.parentObjectID = builder.parentObjectID;
+    this.creationTime = builder.creationTime;
+    this.modificationTime = builder.modificationTime;
+  }
+
+  /**
+   * Returns new builder class that builds a OmPrefixInfo.
+   *
+   * @return Builder
+   */
+  public static OmDirectoryInfo.Builder newBuilder() {
+    return new OmDirectoryInfo.Builder();
+  }
+
+  /**
+   * Builder for Directory Info.
+   */
+  public static class Builder {
+    private long parentObjectID; // pointer to parent directory
+
+    private long objectID;
+    private long updateID;
+
+    private String name;
+
+    private long creationTime;
+    private long modificationTime;
+
+    private List<OzoneAcl> acls;
+    private Map<String, String> metadata;
+
+    public Builder() {
+      //Default values
+      this.acls = new LinkedList<>();
+      this.metadata = new HashMap<>();
+    }
+
+    public Builder setParentObjectID(long parentObjectId) {
+      this.parentObjectID = parentObjectId;
+      return this;
+    }
+
+    public Builder setObjectID(long objectId) {
+      this.objectID = objectId;
+      return this;
+    }
+
+    public Builder setUpdateID(long updateId) {
+      this.updateID = updateId;
+      return this;
+    }
+
+    public Builder setName(String dirName) {
+      this.name = dirName;
+      return this;
+    }
+
+    public Builder setCreationTime(long newCreationTime) {
+      this.creationTime = newCreationTime;
+      return this;
+    }
+
+    public Builder setModificationTime(long newModificationTime) {
+      this.modificationTime = newModificationTime;
+      return this;
+    }
+
+    public Builder setAcls(List<OzoneAcl> listOfAcls) {
+      if (listOfAcls != null) {
+        this.acls.addAll(listOfAcls);
+      }
+      return this;
+    }
+
+    public Builder addAcl(OzoneAcl ozoneAcl) {
+      if (ozoneAcl != null) {
+        this.acls.add(ozoneAcl);
+      }
+      return this;
+    }
+
+    public Builder addMetadata(String key, String value) {
+      metadata.put(key, value);
+      return this;
+    }
+
+    public Builder addAllMetadata(Map<String, String> additionalMetadata) {
+      if (additionalMetadata != null) {
+        metadata.putAll(additionalMetadata);
+      }
+      return this;
+    }
+
+    public OmDirectoryInfo build() {
+      return new OmDirectoryInfo(this);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return getPath() + ":" + getObjectID();
+  }
+
+  public long getParentObjectID() {
+    return parentObjectID;
+  }
+
+  public String getPath() {
+    return getParentObjectID() + OzoneConsts.OM_KEY_PREFIX + getName();
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public long getCreationTime() {
+    return creationTime;
+  }
+
+  public long getModificationTime() {
+    return modificationTime;
+  }
+
+  public List<OzoneAcl> getAcls() {
+    return acls;
+  }
+
+  /**
+   * Creates DirectoryInfo protobuf from OmDirectoryInfo.
+   */
+  public OzoneManagerProtocolProtos.DirectoryInfo getProtobuf() {
+    OzoneManagerProtocolProtos.DirectoryInfo.Builder pib =
+            OzoneManagerProtocolProtos.DirectoryInfo.newBuilder().setName(name)
+                    .setCreationTime(creationTime)
+                    .setModificationTime(modificationTime)
+                    .addAllMetadata(KeyValueUtil.toProtobuf(metadata))
+                    .setObjectID(objectID)
+                    .setUpdateID(updateID)
+                    .setParentID(parentObjectID);
+    if (acls != null) {
+      pib.addAllAcls(OzoneAclUtil.toProtobuf(acls));
+    }
+    return pib.build();
+  }
+
+  /**
+   * Parses DirectoryInfo protobuf and creates OmPrefixInfo.
+   * @param dirInfo
+   * @return instance of OmDirectoryInfo
+   */
+  public static OmDirectoryInfo getFromProtobuf(
+          OzoneManagerProtocolProtos.DirectoryInfo dirInfo) {
+    OmDirectoryInfo.Builder opib = OmDirectoryInfo.newBuilder()
+            .setName(dirInfo.getName())
+            .setCreationTime(dirInfo.getCreationTime())
+            .setModificationTime(dirInfo.getModificationTime())
+            .setAcls(OzoneAclUtil.fromProtobuf(dirInfo.getAclsList()));
+    if (dirInfo.getMetadataList() != null) {
+      opib.addAllMetadata(KeyValueUtil
+              .getFromProtobuf(dirInfo.getMetadataList()));
+    }
+    if (dirInfo.hasObjectID()) {
+      opib.setObjectID(dirInfo.getObjectID());
+    }
+    if (dirInfo.hasParentID()) {
+      opib.setParentObjectID(dirInfo.getParentID());
+    }
+    if (dirInfo.hasUpdateID()) {
+      opib.setUpdateID(dirInfo.getUpdateID());
+    }
+    return opib.build();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    OmDirectoryInfo omDirInfo = (OmDirectoryInfo) o;
+    return creationTime == omDirInfo.creationTime &&
+            modificationTime == omDirInfo.modificationTime &&
+            name.equals(omDirInfo.name) &&
+            Objects.equals(metadata, omDirInfo.metadata) &&
+            Objects.equals(acls, omDirInfo.acls) &&
+            objectID == omDirInfo.objectID &&
+            updateID == omDirInfo.updateID &&
+            parentObjectID == omDirInfo.parentObjectID;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(objectID, parentObjectID, name);
+  }
+
+  /**
+   * Return a new copy of the object.
+   */
+  public OmDirectoryInfo copyObject() {
+    OmDirectoryInfo.Builder builder = new Builder()
+            .setName(name)
+            .setCreationTime(creationTime)
+            .setModificationTime(modificationTime)
+            .setParentObjectID(parentObjectID)
+            .setObjectID(objectID)
+            .setUpdateID(updateID);
+
+    acls.forEach(acl -> builder.addAcl(new OzoneAcl(acl.getType(),
+            acl.getName(), (BitSet) acl.getAclBitSet().clone(),
+            acl.getAclScope())));
+
+    if (metadata != null) {
+      metadata.forEach((k, v) -> builder.addMetadata(k, v));
+    }
+
+    return builder.build();
+  }
+}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneFSUtils.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneFSUtils.java
index d1491ed..96df56f 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneFSUtils.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneFSUtils.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.om.helpers;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.StringUtils;
 
+import javax.annotation.Nonnull;
 import java.nio.file.Paths;
 
 import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
@@ -116,4 +117,18 @@ public final class OzoneFSUtils {
     }
     return true;
   }
+
+  /**
+   * The function returns leaf node name from the given absolute path. For
+   * example, the given key path '/a/b/c/d/e/file1' then it returns leaf node
+   * name 'file1'.
+   */
+  public static String getFileName(@Nonnull String keyName) {
+    java.nio.file.Path fileName = Paths.get(keyName).getFileName();
+    if (fileName != null) {
+      return fileName.toString();
+    }
+    // failed to converts a path key
+    return keyName;
+  }
 }
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneDirectory.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneDirectory.java
new file mode 100644
index 0000000..87e9f09
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneDirectory.java
@@ -0,0 +1,207 @@
+/*
+ * 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.fs.ozone;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.util.StringUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
+import static org.junit.Assert.fail;
+
+/**
+ * Test verifies the entries and operations in directory table.
+ */
+public class TestOzoneDirectory {
+
+  @Rule
+  public Timeout timeout = new Timeout(300000);
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(TestOzoneDirectory.class);
+
+  private MiniOzoneCluster cluster;
+  private FileSystem fs;
+  private OzoneFileSystem o3fs;
+  private String volumeName;
+  private String bucketName;
+
+  @Test(timeout = 300_000)
+  public void testMultiLevelDirs() throws Exception {
+    setupOzoneFileSystem();
+    // Op 1. create dir -> /d1/d2/d3/d4/
+    // Op 2. create dir -> /d1/d2/d3/d4/d5
+    // Op 3. create dir -> /d1/d2/d3/d4/d6
+    Path parent = new Path("/d1/d2/d3/d4/");
+    fs.mkdirs(parent);
+
+    OMMetadataManager omMgr = cluster.getOzoneManager().getMetadataManager();
+    OmBucketInfo omBucketInfo = omMgr.getBucketTable().get(
+            omMgr.getBucketKey(volumeName, bucketName));
+    Assert.assertNotNull("Failed to find bucketInfo", omBucketInfo);
+
+    ArrayList<String> dirKeys = new ArrayList<>();
+    long d1ObjectID = verifyDirKey(omBucketInfo.getObjectID(), "d1", "/d1",
+            dirKeys, omMgr);
+    long d2ObjectID = verifyDirKey(d1ObjectID, "d2", "/d1/d2", dirKeys,
+            omMgr);
+    long d3ObjectID = verifyDirKey(d2ObjectID, "d3", "/d1/d2/d3",
+            dirKeys, omMgr);
+    long d4ObjectID = verifyDirKey(d3ObjectID, "d4", "/d1/d2/d3/d4",
+            dirKeys, omMgr);
+
+    Assert.assertEquals("Wrong OM numKeys metrics",
+            4, cluster.getOzoneManager().getMetrics().getNumKeys());
+
+    // verify entries in directory table
+    TableIterator<String, ? extends
+            Table.KeyValue<String, OmDirectoryInfo>> iterator =
+            omMgr.getDirectoryTable().iterator();
+    iterator.seekToFirst();
+    int count = dirKeys.size();
+    Assert.assertEquals("Unexpected directory table entries!", 4, count);
+    while (iterator.hasNext()) {
+      count--;
+      Table.KeyValue<String, OmDirectoryInfo> value = iterator.next();
+      verifyKeyFormat(value.getKey(), dirKeys);
+    }
+    Assert.assertEquals("Unexpected directory table entries!", 0, count);
+
+    // verify entries in key table
+    TableIterator<String, ? extends
+            Table.KeyValue<String, OmKeyInfo>> keyTableItr =
+            omMgr.getKeyTable().iterator();
+    while (keyTableItr.hasNext()) {
+      fail("Shouldn't add any entries in KeyTable!");
+    }
+
+    // create sub-dirs under same parent
+    Path subDir5 = new Path("/d1/d2/d3/d4/d5");
+    fs.mkdirs(subDir5);
+    Path subDir6 = new Path("/d1/d2/d3/d4/d6");
+    fs.mkdirs(subDir6);
+    long d5ObjectID = verifyDirKey(d4ObjectID, "d5",
+            "/d1/d2/d3/d4/d5", dirKeys, omMgr);
+    long d6ObjectID = verifyDirKey(d4ObjectID, "d6",
+            "/d1/d2/d3/d4/d6", dirKeys, omMgr);
+    Assert.assertTrue("Wrong objectIds for sub-dirs[" + d5ObjectID +
+                    "/d5, " + d6ObjectID + "/d6] of same parent!",
+            d5ObjectID != d6ObjectID);
+
+    Assert.assertEquals("Wrong OM numKeys metrics",
+            6, cluster.getOzoneManager().getMetrics().getNumKeys());
+  }
+
+  /**
+   * Verify key name format and the DB key existence in the expected dirKeys
+   * list.
+   *
+   * @param key     table keyName
+   * @param dirKeys expected keyName
+   */
+  private void verifyKeyFormat(String key, ArrayList<String> dirKeys) {
+    String[] keyParts = StringUtils.split(key,
+            OzoneConsts.OM_KEY_PREFIX.charAt(0));
+    Assert.assertEquals("Invalid KeyName", 2, keyParts.length);
+    boolean removed = dirKeys.remove(key);
+    Assert.assertTrue("Key:" + key + " doesn't exists in directory table!",
+            removed);
+  }
+
+  long verifyDirKey(long parentId, String dirKey, String absolutePath,
+                    ArrayList<String> dirKeys, OMMetadataManager omMgr)
+          throws Exception {
+    String dbKey = parentId + "/" + dirKey;
+    dirKeys.add(dbKey);
+    OmDirectoryInfo dirInfo = omMgr.getDirectoryTable().get(dbKey);
+    Assert.assertNotNull("Failed to find " + absolutePath +
+            " using dbKey: " + dbKey, dirInfo);
+    Assert.assertEquals("Parent Id mismatches", parentId,
+            dirInfo.getParentObjectID());
+    Assert.assertEquals("Mismatches directory name", dirKey,
+            dirInfo.getName());
+    Assert.assertTrue("Mismatches directory creation time param",
+            dirInfo.getCreationTime() > 0);
+    Assert.assertEquals("Mismatches directory modification time param",
+            dirInfo.getCreationTime(), dirInfo.getModificationTime());
+    Assert.assertEquals("Wrong representation!",
+            dbKey + ":" + dirInfo.getObjectID(), dirInfo.toString());
+    return dirInfo.getObjectID();
+  }
+
+  private void setupOzoneFileSystem()
+          throws IOException, TimeoutException, InterruptedException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    conf.setInt(FS_TRASH_INTERVAL_KEY, 1);
+    conf.set(OMConfigKeys.OZONE_OM_LAYOUT_VERSION, "V1");
+    cluster = MiniOzoneCluster.newBuilder(conf)
+            .setNumDatanodes(3)
+            .build();
+    cluster.waitForClusterToBeReady();
+    // create a volume and a bucket to be used by OzoneFileSystem
+    OzoneBucket bucket = TestDataUtil.createVolumeAndBucket(cluster);
+    volumeName = bucket.getVolumeName();
+    bucketName = bucket.getName();
+
+    String rootPath = String.format("%s://%s.%s/",
+            OzoneConsts.OZONE_URI_SCHEME, bucket.getName(),
+            bucket.getVolumeName());
+
+    // Set the fs.defaultFS and start the filesystem
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath);
+    // Set the number of keys to be processed during batch operate.
+    conf.setInt(OZONE_FS_ITERATE_BATCH_SIZE, 5);
+    fs = FileSystem.get(conf);
+  }
+
+  @After
+  public void tearDown() {
+    IOUtils.closeQuietly(fs);
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+}
\ No newline at end of file
diff --git a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
index b347dc1..abbee63 100644
--- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
+++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
@@ -770,6 +770,17 @@ message KeyInfo {
     optional uint64 updateID = 15;
 }
 
+message DirectoryInfo {
+    required string name = 1;
+    required uint64 creationTime = 2;
+    required uint64 modificationTime = 3;
+    repeated hadoop.hdds.KeyValue metadata = 4;
+    repeated OzoneAclInfo acls = 5;
+    required uint64 objectID = 6;
+    required uint64 updateID = 7;
+    required uint64 parentID = 8;
+}
+
 message RepeatedKeyInfo {
     repeated KeyInfo keyInfo = 1;
 }
diff --git a/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
index 5e4e75b..cdb59d2 100644
--- a/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
+++ b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
@@ -24,6 +24,7 @@ import java.util.Set;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmPrefixInfo;
@@ -370,6 +371,12 @@ public interface OMMetadataManager {
       String bucketName, String prefix) throws IOException;
 
   /**
+   * Gets the DirectoryTable.
+   * @return Table.
+   */
+  Table<String, OmDirectoryInfo> getDirectoryTable();
+
+  /**
    * Return table mapped to the specified table name.
    * @param tableName
    * @return Table
@@ -387,4 +394,14 @@ public interface OMMetadataManager {
    * @return table names in OM DB.
    */
   Set<String> listTableNames();
+
+  /**
+   * Given a volume, bucket and a key, return the corresponding DB prefixKey
+   * key.
+   *
+   * @param parentObjectId - parent object Id
+   * @param pathComponentName   - path component name
+   * @return DB directory key as String.
+   */
+  String getOzonePathKey(long parentObjectId, String pathComponentName);
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java
index 3504b91..f4897d4 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java
@@ -205,6 +205,10 @@ public class OMMetrics {
     numKeys.incr();
   }
 
+  public void incNumKeys(long val) {
+    numKeys.incr(val);
+  }
+
   public void decNumKeys() {
     numKeys.incr(-1);
   }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
index 27a3662..0fd8c76 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.ozone.om.codec.OMTransactionInfoCodec;
 import org.apache.hadoop.ozone.om.codec.OmBucketInfoCodec;
+import org.apache.hadoop.ozone.om.codec.OmDirectoryInfoCodec;
 import org.apache.hadoop.ozone.om.codec.OmKeyInfoCodec;
 import org.apache.hadoop.ozone.om.codec.OmMultipartKeyInfoCodec;
 import org.apache.hadoop.ozone.om.codec.OmPrefixInfoCodec;
@@ -60,6 +61,7 @@ import org.apache.hadoop.ozone.om.codec.UserVolumeInfoCodec;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
@@ -126,6 +128,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
    * |----------------------------------------------------------------------|
    * |  multipartInfoTable| /volumeName/bucketName/keyName/uploadId ->...   |
    * |----------------------------------------------------------------------|
+   * |  directoryTable    | parentId/directoryName -> DirectoryInfo         |
    * |----------------------------------------------------------------------|
    * |  transactionInfoTable | #TRANSACTIONINFO -> OMTransactionInfo        |
    * |----------------------------------------------------------------------|
@@ -141,6 +144,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
   public static final String S3_SECRET_TABLE = "s3SecretTable";
   public static final String DELEGATION_TOKEN_TABLE = "dTokenTable";
   public static final String PREFIX_TABLE = "prefixTable";
+  public static final String DIRECTORY_TABLE = "directoryTable";
   public static final String TRANSACTION_INFO_TABLE =
       "transactionInfoTable";
 
@@ -159,6 +163,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
   private Table s3SecretTable;
   private Table dTokenTable;
   private Table prefixTable;
+  private Table dirTable;
   private Table transactionInfoTable;
   private boolean isRatisEnabled;
   private boolean ignorePipelineinKey;
@@ -243,6 +248,11 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
   }
 
   @Override
+  public Table<String, OmDirectoryInfo> getDirectoryTable() {
+    return dirTable;
+  }
+
+  @Override
   public Table<String, OmMultipartKeyInfo> getMultipartInfoTable() {
     return multipartInfoTable;
   }
@@ -334,6 +344,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
         .addTable(DELEGATION_TOKEN_TABLE)
         .addTable(S3_SECRET_TABLE)
         .addTable(PREFIX_TABLE)
+        .addTable(DIRECTORY_TABLE)
         .addTable(TRANSACTION_INFO_TABLE)
         .addCodec(OzoneTokenIdentifier.class, new TokenIdentifierCodec())
         .addCodec(OmKeyInfo.class, new OmKeyInfoCodec(true))
@@ -345,6 +356,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
         .addCodec(OmMultipartKeyInfo.class, new OmMultipartKeyInfoCodec())
         .addCodec(S3SecretValue.class, new S3SecretValueCodec())
         .addCodec(OmPrefixInfo.class, new OmPrefixInfoCodec())
+        .addCodec(OmDirectoryInfo.class, new OmDirectoryInfoCodec())
         .addCodec(OMTransactionInfo.class, new OMTransactionInfoCodec());
   }
 
@@ -400,6 +412,10 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
         OmPrefixInfo.class);
     checkTableStatus(prefixTable, PREFIX_TABLE);
 
+    dirTable = this.store.getTable(DIRECTORY_TABLE, String.class,
+            OmDirectoryInfo.class);
+    checkTableStatus(dirTable, DIRECTORY_TABLE);
+
     transactionInfoTable = this.store.getTable(TRANSACTION_INFO_TABLE,
         String.class, OMTransactionInfo.class);
     checkTableStatus(transactionInfoTable, TRANSACTION_INFO_TABLE);
@@ -1152,4 +1168,12 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
     return tableMap.keySet();
   }
 
+  @Override
+  public String getOzonePathKey(long parentObjectId, String pathComponentName) {
+    StringBuilder builder = new StringBuilder();
+    builder.append(parentObjectId);
+    builder.append(OM_KEY_PREFIX).append(pathComponentName);
+    return builder.toString();
+  }
+
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index 2d2c9f4..94893dc 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -222,6 +222,8 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HANDLER_COUNT_DEF
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HANDLER_COUNT_KEY;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_KERBEROS_KEYTAB_FILE_KEY;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_LAYOUT_VERSION;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_LAYOUT_VERSION_DEFAULT;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_METRICS_SAVE_INTERVAL;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_METRICS_SAVE_INTERVAL_DEFAULT;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_USER_MAX_VOLUME;
@@ -1149,6 +1151,10 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
       omRatisServer.start();
     }
 
+    // TODO: Temporary workaround for OM upgrade path and will be replaced once
+    //  upgrade HDDS-3698 story reaches consensus.
+    getOMLayoutVersion();
+
     metadataManager.start(configuration);
     startSecretManagerIfNecessary();
 
@@ -3685,6 +3691,14 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
         OZONE_OM_ENABLE_FILESYSTEM_PATHS_DEFAULT);
   }
 
+  private void getOMLayoutVersion() {
+    String version = configuration.getTrimmed(OZONE_OM_LAYOUT_VERSION,
+            OZONE_OM_LAYOUT_VERSION_DEFAULT);
+    boolean omLayoutVersionV1 =
+            StringUtils.equalsIgnoreCase(version, "V1");
+    OzoneManagerRatisUtils.setOmLayoutVersionV1(omLayoutVersionV1);
+  }
+
   /**
    * Create volume which is required for S3Gateway operations.
    * @throws IOException
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/codec/OmDirectoryInfoCodec.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/codec/OmDirectoryInfoCodec.java
new file mode 100644
index 0000000..ba592a9
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/codec/OmDirectoryInfoCodec.java
@@ -0,0 +1,60 @@
+/**
+ * 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.om.codec;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hdds.utils.db.Codec;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DirectoryInfo;
+
+import java.io.IOException;
+
+/**
+ * Codec to encode OmDirectoryInfo as byte array.
+ */
+public class OmDirectoryInfoCodec implements Codec<OmDirectoryInfo> {
+
+  @Override
+  public byte[] toPersistedFormat(OmDirectoryInfo object) throws IOException {
+    Preconditions
+            .checkNotNull(object, "Null object can't be converted " +
+                    "to byte array.");
+    return object.getProtobuf().toByteArray();
+  }
+
+  @Override
+  public OmDirectoryInfo fromPersistedFormat(byte[] rawData)
+          throws IOException {
+    Preconditions
+            .checkNotNull(rawData,
+                    "Null byte array can't converted to real object.");
+    try {
+      return OmDirectoryInfo.getFromProtobuf(DirectoryInfo.parseFrom(rawData));
+    } catch (InvalidProtocolBufferException e) {
+      throw new IllegalArgumentException(
+              "Can't encode the the raw data from the byte array", e);
+    }
+  }
+
+  @Override
+  public OmDirectoryInfo copyObject(OmDirectoryInfo object) {
+    return object.copyObject();
+  }
+}
+
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java
index 681c0da..1ea225b 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.ozone.om.request.bucket.acl.OMBucketAddAclRequest;
 import org.apache.hadoop.ozone.om.request.bucket.acl.OMBucketRemoveAclRequest;
 import org.apache.hadoop.ozone.om.request.bucket.acl.OMBucketSetAclRequest;
 import org.apache.hadoop.ozone.om.request.file.OMDirectoryCreateRequest;
+import org.apache.hadoop.ozone.om.request.file.OMDirectoryCreateRequestV1;
 import org.apache.hadoop.ozone.om.request.file.OMFileCreateRequest;
 import org.apache.hadoop.ozone.om.request.key.OMKeysDeleteRequest;
 import org.apache.hadoop.ozone.om.request.key.OMAllocateBlockRequest;
@@ -83,8 +84,22 @@ import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.TRANSACTION_INFO_
  */
 public final class OzoneManagerRatisUtils {
 
+  // TODO: Temporary workaround for OM upgrade path and will be replaced once
+  //  upgrade HDDS-3698 story reaches consensus.
+  private static boolean omLayoutVersionV1 = true;
+
   private OzoneManagerRatisUtils() {
   }
+
+  /**
+   * Sets layout version.
+   *
+   * @param layoutVersionV1 om layout version
+   */
+  public static void setOmLayoutVersionV1(boolean layoutVersionV1) {
+    OzoneManagerRatisUtils.omLayoutVersionV1 = layoutVersionV1;
+  }
+
   /**
    * Create OMClientRequest which encapsulates the OMRequest.
    * @param omRequest
@@ -133,6 +148,9 @@ public final class OzoneManagerRatisUtils {
     case RenameKeys:
       return new OMKeysRenameRequest(omRequest);
     case CreateDirectory:
+      if (omLayoutVersionV1) {
+        return new OMDirectoryCreateRequestV1(omRequest);
+      }
       return new OMDirectoryCreateRequest(omRequest);
     case CreateFile:
       return new OMFileCreateRequest(omRequest);
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java
index 4595236..a4330f4 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java
@@ -358,4 +358,7 @@ public class OMDirectoryCreateRequest extends OMKeyRequest {
         .setUpdateID(objectId);
   }
 
+  static long getMaxNumOfRecursiveDirs() {
+    return MAX_NUM_OF_RECURSIVE_DIRS;
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequestV1.java
similarity index 56%
copy from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java
copy to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequestV1.java
index 4595236..8b0727a 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequestV1.java
@@ -18,109 +18,64 @@
 
 package org.apache.hadoop.ozone.om.request.file;
 
-import java.io.IOException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
 import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
 import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
-import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
-import org.apache.hadoop.ozone.om.helpers.OzoneAclUtil;
-import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
-import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
-import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
-import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
-import org.apache.hadoop.ozone.security.acl.OzoneObj;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.ozone.audit.AuditLogger;
 import org.apache.hadoop.ozone.audit.OMAction;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.OMMetrics;
 import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
-import org.apache.hadoop.ozone.om.request.key.OMKeyRequest;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneAclUtil;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
-import org.apache.hadoop.ozone.om.response.file.OMDirectoryCreateResponse;
+import org.apache.hadoop.ozone.om.response.file.OMDirectoryCreateResponseV1;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
-    .CreateDirectoryRequest;
+        .CreateDirectoryRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
-    .CreateDirectoryResponse;
+        .CreateDirectoryResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
-    .KeyArgs;
+        .KeyArgs;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
-    .OMRequest;
+        .OMRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
-    .OMResponse;
+        .OMResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
-    .Status;
-import org.apache.hadoop.util.Time;
-import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
-import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+        .Status;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
 
 import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_ALREADY_EXISTS;
 import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_KEY_NAME;
 import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
-import static org.apache.hadoop.ozone.om.request.file.OMFileRequest.OMDirectoryResult.DIRECTORY_EXISTS_IN_GIVENPATH;
-import static org.apache.hadoop.ozone.om.request.file.OMFileRequest.OMDirectoryResult.FILE_EXISTS_IN_GIVENPATH;
-import static org.apache.hadoop.ozone.om.request.file.OMFileRequest.OMDirectoryResult.NONE;
-import static org.apache.hadoop.ozone.om.request.file.OMFileRequest.OMDirectoryResult.FILE_EXISTS;
+import static org.apache.hadoop.ozone.om.request.file.OMFileRequest.OMDirectoryResult.*;
+
 /**
- * Handle create directory request.
+ * Handle create directory request. It will add path components to the directory
+ * table and maintains file system semantics.
  */
-public class OMDirectoryCreateRequest extends OMKeyRequest {
+public class OMDirectoryCreateRequestV1 extends OMDirectoryCreateRequest {
 
   private static final Logger LOG =
-      LoggerFactory.getLogger(OMDirectoryCreateRequest.class);
-
-  // The maximum number of directories which can be created through a single
-  // transaction (recursive directory creations) is 2^8 - 1 as only 8
-  // bits are set aside for this in ObjectID.
-  private static final long MAX_NUM_OF_RECURSIVE_DIRS = 255;
-
-  /**
-   * Stores the result of request execution in
-   * OMClientRequest#validateAndUpdateCache.
-   */
-  public enum Result {
-    SUCCESS, // The request was executed successfully
-
-    DIRECTORY_ALREADY_EXISTS, // Directory key already exists in DB
+      LoggerFactory.getLogger(OMDirectoryCreateRequestV1.class);
 
-    FAILURE // The request failed and exception was thrown
-  }
-
-  public OMDirectoryCreateRequest(OMRequest omRequest) {
+  public OMDirectoryCreateRequestV1(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
-  public OMRequest preExecute(OzoneManager ozoneManager) {
-    CreateDirectoryRequest createDirectoryRequest =
-        getOmRequest().getCreateDirectoryRequest();
-    Preconditions.checkNotNull(createDirectoryRequest);
-
-    KeyArgs.Builder newKeyArgs = createDirectoryRequest.getKeyArgs()
-        .toBuilder().setModificationTime(Time.now());
-
-    CreateDirectoryRequest.Builder newCreateDirectoryRequest =
-        createDirectoryRequest.toBuilder().setKeyArgs(newKeyArgs);
-
-    return getOmRequest().toBuilder().setCreateDirectoryRequest(
-        newCreateDirectoryRequest).setUserInfo(getUserInfo()).build();
-
-  }
-
-  @Override
   public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
 
@@ -131,6 +86,7 @@ public class OMDirectoryCreateRequest extends OMKeyRequest {
     String volumeName = keyArgs.getVolumeName();
     String bucketName = keyArgs.getBucketName();
     String keyName = keyArgs.getKeyName();
+    int numKeysCreated = 0;
 
     OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
         getOmRequest());
@@ -147,7 +103,7 @@ public class OMDirectoryCreateRequest extends OMKeyRequest {
     IOException exception = null;
     OMClientResponse omClientResponse = null;
     Result result = Result.FAILURE;
-    List<OmKeyInfo> missingParentInfos;
+    List<OmDirectoryInfo> missingParentInfos;
 
     try {
       keyArgs = resolveBucketLink(ozoneManager, keyArgs, auditMap);
@@ -174,47 +130,50 @@ public class OMDirectoryCreateRequest extends OMKeyRequest {
 
       // Need to check if any files exist in the given path, if they exist we
       // cannot create a directory with the given key.
-      OMFileRequest.OMPathInfo omPathInfo =
-          OMFileRequest.verifyFilesInPath(omMetadataManager, volumeName,
+      // Verify the path against directory table
+      OMFileRequest.OMPathInfoV1 omPathInfo =
+          OMFileRequest.verifyDirectoryKeysInPath(omMetadataManager, volumeName,
               bucketName, keyName, keyPath);
       OMFileRequest.OMDirectoryResult omDirectoryResult =
           omPathInfo.getDirectoryResult();
 
-      OmKeyInfo dirKeyInfo = null;
       if (omDirectoryResult == FILE_EXISTS ||
           omDirectoryResult == FILE_EXISTS_IN_GIVENPATH) {
-        throw new OMException("Unable to create directory: " +keyName
-            + " in volume/bucket: " + volumeName + "/" + bucketName,
+        throw new OMException("Unable to create directory: " + keyName
+            + " in volume/bucket: " + volumeName + "/" + bucketName + " as " +
+                "file:" + omPathInfo.getFileExistsInPath() + " already exists",
             FILE_ALREADY_EXISTS);
       } else if (omDirectoryResult == DIRECTORY_EXISTS_IN_GIVENPATH ||
           omDirectoryResult == NONE) {
-        List<String> missingParents = omPathInfo.getMissingParents();
-        long baseObjId = ozoneManager.getObjectIdFromTxId(trxnLogIndex);
-        List<OzoneAcl> inheritAcls = omPathInfo.getAcls();
-
-        dirKeyInfo = createDirectoryKeyInfoWithACL(keyName,
-            keyArgs, baseObjId,
-            OzoneAclUtil.fromProtobuf(keyArgs.getAclsList()), trxnLogIndex);
-
-        missingParentInfos = getAllParentInfo(ozoneManager, keyArgs,
-            missingParents, inheritAcls, trxnLogIndex);
-
-        OMFileRequest.addKeyTableCacheEntries(omMetadataManager, volumeName,
-            bucketName, Optional.of(dirKeyInfo),
-            Optional.of(missingParentInfos), trxnLogIndex);
-        result = Result.SUCCESS;
-        omClientResponse = new OMDirectoryCreateResponse(omResponse.build(),
-            dirKeyInfo, missingParentInfos, result);
+
+        // prepare all missing parents
+        missingParentInfos = OMDirectoryCreateRequestV1.getAllParentDirInfo(
+                ozoneManager, keyArgs, omPathInfo, trxnLogIndex);
+        // prepare leafNode dir
+        OmDirectoryInfo dirInfo = createDirectoryInfoWithACL(
+                omPathInfo.getLeafNodeName(),
+                keyArgs, omPathInfo.getLeafNodeObjectId(),
+                omPathInfo.getLastKnownParentId(), trxnLogIndex,
+                OzoneAclUtil.fromProtobuf(keyArgs.getAclsList()));
+        OMFileRequest.addDirectoryTableCacheEntries(omMetadataManager,
+                Optional.of(dirInfo), Optional.of(missingParentInfos),
+                trxnLogIndex);
+
+        // total number of keys created.
+        numKeysCreated = missingParentInfos.size() + 1;
+
+        result = OMDirectoryCreateRequest.Result.SUCCESS;
+        omClientResponse = new OMDirectoryCreateResponseV1(omResponse.build(),
+                dirInfo, missingParentInfos, result);
       } else {
-        // omDirectoryResult == DIRECTORY_EXITS
         result = Result.DIRECTORY_ALREADY_EXISTS;
         omResponse.setStatus(Status.DIRECTORY_ALREADY_EXISTS);
-        omClientResponse = new OMDirectoryCreateResponse(omResponse.build(),
+        omClientResponse = new OMDirectoryCreateResponseV1(omResponse.build(),
             result);
       }
     } catch (IOException ex) {
       exception = ex;
-      omClientResponse = new OMDirectoryCreateResponse(
+      omClientResponse = new OMDirectoryCreateResponseV1(
           createErrorOMResponse(omResponse, exception), result);
     } finally {
       addResponseToDoubleBuffer(trxnLogIndex, omClientResponse,
@@ -228,134 +187,126 @@ public class OMDirectoryCreateRequest extends OMKeyRequest {
     auditLog(auditLogger, buildAuditMessage(OMAction.CREATE_DIRECTORY,
         auditMap, exception, userInfo));
 
-    logResult(createDirectoryRequest, keyArgs, omMetrics, result, trxnLogIndex,
-        exception);
+    logResult(createDirectoryRequest, keyArgs, omMetrics, numKeysCreated,
+            result, exception);
 
     return omClientResponse;
   }
 
+  private void logResult(CreateDirectoryRequest createDirectoryRequest,
+                         KeyArgs keyArgs, OMMetrics omMetrics, int numKeys,
+                         Result result,
+                         IOException exception) {
+
+    String volumeName = keyArgs.getVolumeName();
+    String bucketName = keyArgs.getBucketName();
+    String keyName = keyArgs.getKeyName();
+
+    switch (result) {
+    case SUCCESS:
+      omMetrics.incNumKeys(numKeys);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Directory created. Volume:{}, Bucket:{}, Key:{}",
+            volumeName, bucketName, keyName);
+      }
+      break;
+    case DIRECTORY_ALREADY_EXISTS:
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Directory already exists. Volume:{}, Bucket:{}, Key{}",
+            volumeName, bucketName, keyName, exception);
+      }
+      break;
+    case FAILURE:
+      omMetrics.incNumCreateDirectoryFails();
+      LOG.error("Directory creation failed. Volume:{}, Bucket:{}, Key{}. " +
+          "Exception:{}", volumeName, bucketName, keyName, exception);
+      break;
+    default:
+      LOG.error("Unrecognized Result for OMDirectoryCreateRequest: {}",
+          createDirectoryRequest);
+    }
+  }
+
   /**
-   * Construct OmKeyInfo for every parent directory in missing list.
+   * Construct OmDirectoryInfo for every parent directory in missing list.
    * @param ozoneManager
    * @param keyArgs
-   * @param missingParents list of parent directories to be created
-   * @param inheritAcls ACLs to be assigned to each new parent dir
+   * @param pathInfo list of parent directories to be created and its ACLs
    * @param trxnLogIndex
    * @return
    * @throws IOException
    */
-  public static List<OmKeyInfo> getAllParentInfo(OzoneManager ozoneManager,
-      KeyArgs keyArgs, List<String> missingParents, List<OzoneAcl> inheritAcls,
-      long trxnLogIndex) throws IOException {
+  public static List<OmDirectoryInfo> getAllParentDirInfo(
+          OzoneManager ozoneManager, KeyArgs keyArgs,
+          OMFileRequest.OMPathInfoV1 pathInfo, long trxnLogIndex)
+          throws IOException {
     OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
-    List<OmKeyInfo> missingParentInfos = new ArrayList<>();
+    List<OmDirectoryInfo> missingParentInfos = new ArrayList<>();
 
     // The base id is left shifted by 8 bits for creating space to
     // create (2^8 - 1) object ids in every request.
     // maxObjId represents the largest object id allocation possible inside
     // the transaction.
     long baseObjId = ozoneManager.getObjectIdFromTxId(trxnLogIndex);
-    long maxObjId = baseObjId + MAX_NUM_OF_RECURSIVE_DIRS;
-    long objectCount = 1; // baseObjID is used by the leaf directory
+    long maxObjId = baseObjId + getMaxNumOfRecursiveDirs();
+    long objectCount = 1;
 
     String volumeName = keyArgs.getVolumeName();
     String bucketName = keyArgs.getBucketName();
     String keyName = keyArgs.getKeyName();
 
+    long lastKnownParentId = pathInfo.getLastKnownParentId();
+    List<String> missingParents = pathInfo.getMissingParents();
+    List<OzoneAcl> inheritAcls = pathInfo.getAcls();
     for (String missingKey : missingParents) {
       long nextObjId = baseObjId + objectCount;
       if (nextObjId > maxObjId) {
         throw new OMException("Too many directories in path. Exceeds limit of "
-            + MAX_NUM_OF_RECURSIVE_DIRS + ". Unable to create directory: "
+            + getMaxNumOfRecursiveDirs() + ". Unable to create directory: "
             + keyName + " in volume/bucket: " + volumeName + "/" + bucketName,
             INVALID_KEY_NAME);
       }
 
-      LOG.debug("missing parent {} getting added to KeyTable", missingKey);
-      // what about keyArgs for parent directories? TODO
-      OmKeyInfo parentKeyInfo = createDirectoryKeyInfoWithACL(
-          missingKey, keyArgs, nextObjId, inheritAcls, trxnLogIndex);
+      LOG.debug("missing parent {} getting added to DirectoryTable",
+              missingKey);
+      OmDirectoryInfo dirInfo = createDirectoryInfoWithACL(missingKey,
+              keyArgs, nextObjId, lastKnownParentId, trxnLogIndex, inheritAcls);
       objectCount++;
 
-      missingParentInfos.add(parentKeyInfo);
-      omMetadataManager.getKeyTable().addCacheEntry(
-          new CacheKey<>(omMetadataManager.getOzoneKey(volumeName,
-              bucketName, parentKeyInfo.getKeyName())),
-          new CacheValue<>(Optional.of(parentKeyInfo),
-              trxnLogIndex));
-    }
-
-    return missingParentInfos;
-  }
+      missingParentInfos.add(dirInfo);
 
-  private void logResult(CreateDirectoryRequest createDirectoryRequest,
-      KeyArgs keyArgs, OMMetrics omMetrics, Result result, long trxnLogIndex,
-      IOException exception) {
-
-    String volumeName = keyArgs.getVolumeName();
-    String bucketName = keyArgs.getBucketName();
-    String keyName = keyArgs.getKeyName();
-
-    switch (result) {
-    case SUCCESS:
-      omMetrics.incNumKeys();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Directory created. Volume:{}, Bucket:{}, Key:{}",
-            volumeName, bucketName, keyName);
-      }
-      break;
-    case DIRECTORY_ALREADY_EXISTS:
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Directory already exists. Volume:{}, Bucket:{}, Key{}",
-            volumeName, bucketName, keyName, exception);
-      }
-      break;
-    case FAILURE:
-      omMetrics.incNumCreateDirectoryFails();
-      LOG.error("Directory creation failed. Volume:{}, Bucket:{}, Key{}. " +
-          "Exception:{}", volumeName, bucketName, keyName, exception);
-      break;
-    default:
-      LOG.error("Unrecognized Result for OMDirectoryCreateRequest: {}",
-          createDirectoryRequest);
+      // updating id for the next sub-dir
+      lastKnownParentId = nextObjId;
     }
+    pathInfo.setLastKnownParentId(lastKnownParentId);
+    pathInfo.setLeafNodeObjectId(baseObjId + objectCount);
+    return missingParentInfos;
   }
 
   /**
-   * fill in a KeyInfo for a new directory entry in OM database.
+   * Fill in a DirectoryInfo for a new directory entry in OM database.
    * without initializing ACLs from the KeyArgs - used for intermediate
    * directories which get created internally/recursively during file
    * and directory create.
-   * @param keyName
+   * @param dirName
    * @param keyArgs
    * @param objectId
-   * @param transactionIndex
-   * @return the OmKeyInfo structure
+   * @param parentObjectId
+   * @param inheritAcls
+   * @return the OmDirectoryInfo structure
    */
-  public static OmKeyInfo createDirectoryKeyInfoWithACL(
-      String keyName, KeyArgs keyArgs, long objectId,
-      List<OzoneAcl> inheritAcls, long transactionIndex) {
-    return dirKeyInfoBuilderNoACL(keyName, keyArgs, objectId)
-        .setAcls(inheritAcls).setUpdateID(transactionIndex).build();
+  public static OmDirectoryInfo createDirectoryInfoWithACL(
+          String dirName, KeyArgs keyArgs, long objectId,
+          long parentObjectId, long transactionIndex,
+          List<OzoneAcl> inheritAcls) {
+
+    return OmDirectoryInfo.newBuilder()
+            .setName(dirName)
+            .setCreationTime(keyArgs.getModificationTime())
+            .setModificationTime(keyArgs.getModificationTime())
+            .setObjectID(objectId)
+            .setUpdateID(transactionIndex)
+            .setParentObjectID(parentObjectId)
+            .setAcls(inheritAcls).build();
   }
-
-  private static OmKeyInfo.Builder dirKeyInfoBuilderNoACL(String keyName,
-      KeyArgs keyArgs, long objectId) {
-    String dirName = OzoneFSUtils.addTrailingSlashIfNeeded(keyName);
-
-    return new OmKeyInfo.Builder()
-        .setVolumeName(keyArgs.getVolumeName())
-        .setBucketName(keyArgs.getBucketName())
-        .setKeyName(dirName)
-        .setOmKeyLocationInfos(Collections.singletonList(
-            new OmKeyLocationInfoGroup(0, new ArrayList<>())))
-        .setCreationTime(keyArgs.getModificationTime())
-        .setModificationTime(keyArgs.getModificationTime())
-        .setDataSize(0)
-        .setReplicationType(HddsProtos.ReplicationType.RATIS)
-        .setReplicationFactor(HddsProtos.ReplicationFactor.ONE)
-        .setObjectID(objectId)
-        .setUpdateID(objectId);
-  }
-
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java
index f020f12..d5543ba 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java
@@ -21,14 +21,19 @@ package org.apache.hadoop.ozone.om.request.file;
 import java.io.IOException;
 import java.nio.file.Path;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 
 import com.google.common.base.Optional;
 import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
 import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
 import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -127,6 +132,169 @@ public final class OMFileRequest {
   }
 
   /**
+   * Verify any dir/key exist in the given path in the specified
+   * volume/bucket by iterating through directory table.
+   *
+   * @param omMetadataManager OM Metadata manager
+   * @param volumeName        volume name
+   * @param bucketName        bucket name
+   * @param keyName           key name
+   * @param keyPath           path
+   * @return OMPathInfoV1 path info object
+   * @throws IOException on DB failure
+   */
+  public static OMPathInfoV1 verifyDirectoryKeysInPath(
+          @Nonnull OMMetadataManager omMetadataManager,
+          @Nonnull String volumeName,
+          @Nonnull String bucketName, @Nonnull String keyName,
+          @Nonnull Path keyPath) throws IOException {
+
+    String leafNodeName = OzoneFSUtils.getFileName(keyName);
+    List<String> missing = new ArrayList<>();
+
+    // Found no files/ directories in the given path.
+    OMDirectoryResult result = OMDirectoryResult.NONE;
+
+    Iterator<Path> elements = keyPath.iterator();
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    // by default, inherit bucket ACLs
+    List<OzoneAcl> inheritAcls = omBucketInfo.getAcls();
+
+    long lastKnownParentId = omBucketInfo.getObjectID();
+    String dbDirName = ""; // absolute path for trace logs
+    // for better logging
+    StringBuilder fullKeyPath = new StringBuilder(bucketKey);
+    while (elements.hasNext()) {
+      String fileName = elements.next().toString();
+      fullKeyPath.append(OzoneConsts.OM_KEY_PREFIX);
+      fullKeyPath.append(fileName);
+      if (missing.size() > 0) {
+        // Add all the sub-dirs to the missing list except the leaf element.
+        // For example, /vol1/buck1/a/b/c/d/e/f/file1.txt.
+        // Assume /vol1/buck1/a/b/c exists, then add d, e, f into missing list.
+        if(elements.hasNext()){
+          // skips leaf node.
+          missing.add(fileName);
+        }
+        continue;
+      }
+
+      // For example, /vol1/buck1/a/b/c/d/e/f/file1.txt
+      // 1. Do lookup on directoryTable. If not exists goto next step.
+      // 2. Do look on keyTable. If not exists goto next step.
+      // 3. Add 'sub-dir' to missing parents list
+      String dbNodeName = omMetadataManager.getOzonePathKey(
+              lastKnownParentId, fileName);
+      OmDirectoryInfo omDirInfo = omMetadataManager.getDirectoryTable().
+              get(dbNodeName);
+      if (omDirInfo != null) {
+        dbDirName += omDirInfo.getName() + OzoneConsts.OZONE_URI_DELIMITER;
+        if (elements.hasNext()) {
+          result = OMDirectoryResult.DIRECTORY_EXISTS_IN_GIVENPATH;
+          lastKnownParentId = omDirInfo.getObjectID();
+          inheritAcls = omDirInfo.getAcls();
+          continue;
+        } else {
+          // Checked all the sub-dirs till the leaf node.
+          // Found a directory in the given path.
+          result = OMDirectoryResult.DIRECTORY_EXISTS;
+        }
+      } else {
+        // Get parentID from the lastKnownParent. For any files, directly under
+        // the bucket, the parent is the bucketID. Say, "/vol1/buck1/file1"
+        // TODO: Need to add UT for this case along with OMFileCreateRequest.
+        if (omMetadataManager.getKeyTable().isExist(dbNodeName)) {
+          if (elements.hasNext()) {
+            // Found a file in the given key name.
+            result = OMDirectoryResult.FILE_EXISTS_IN_GIVENPATH;
+          } else {
+            // Checked all the sub-dirs till the leaf file.
+            // Found a file with the given key name.
+            result = OMDirectoryResult.FILE_EXISTS;
+          }
+          break; // Skip directory traversal as it hits key.
+        }
+
+        // Add to missing list, there is no such file/directory with given name.
+        if (elements.hasNext()) {
+          missing.add(fileName);
+        }
+      }
+    }
+
+    LOG.trace("verifyFiles/Directories in Path : " + "/" + volumeName
+            + "/" + bucketName + "/" + keyName + ":" + result);
+
+    if (result == OMDirectoryResult.FILE_EXISTS_IN_GIVENPATH || result ==
+            OMDirectoryResult.FILE_EXISTS) {
+      return new OMPathInfoV1(leafNodeName, lastKnownParentId, missing,
+              result, inheritAcls, fullKeyPath.toString());
+    }
+
+    String dbDirKeyName = omMetadataManager.getOzoneDirKey(volumeName,
+            bucketName, dbDirName);
+    LOG.trace("Acls inherited from parent " + dbDirKeyName + " are : "
+            + inheritAcls);
+
+    return new OMPathInfoV1(leafNodeName, lastKnownParentId, missing,
+            result, inheritAcls);
+  }
+
+  /**
+   * Class to return the results from verifyDirectoryKeysInPath.
+   * Includes the list of missing intermediate directories and
+   * the directory search result code.
+   */
+  public static class OMPathInfoV1 extends OMPathInfo{
+    private String leafNodeName;
+    private long lastKnownParentId;
+    private long leafNodeObjectId;
+    private String fileExistsInPath;
+
+    public OMPathInfoV1(String leafNodeName, long lastKnownParentId,
+                        List missingParents, OMDirectoryResult result,
+                        List<OzoneAcl> aclList, String fileExistsInPath) {
+      super(missingParents, result, aclList);
+      this.leafNodeName = leafNodeName;
+      this.lastKnownParentId = lastKnownParentId;
+      this.fileExistsInPath = fileExistsInPath;
+    }
+
+    public OMPathInfoV1(String leafNodeName, long lastKnownParentId,
+                        List missingParents, OMDirectoryResult result,
+                        List<OzoneAcl> aclList) {
+      this(leafNodeName, lastKnownParentId, missingParents, result, aclList,
+              "");
+    }
+
+    public String getLeafNodeName() {
+      return leafNodeName;
+    }
+
+    public long getLeafNodeObjectId() {
+      return leafNodeObjectId;
+    }
+
+    public void setLeafNodeObjectId(long leafNodeObjectId) {
+      this.leafNodeObjectId = leafNodeObjectId;
+    }
+
+    public void setLastKnownParentId(long lastKnownParentId) {
+      this.lastKnownParentId = lastKnownParentId;
+    }
+
+    public long getLastKnownParentId() {
+      return lastKnownParentId;
+    }
+
+    public String getFileExistsInPath() {
+      return fileExistsInPath;
+    }
+  }
+
+  /**
    * Class to return the results from verifyFilesInPath.
    * Includes the list of missing intermediate directories and
    * the directory search result code.
@@ -224,4 +392,34 @@ public final class OMFileRequest {
           new CacheValue<>(keyInfo, index));
     }
   }
+
+  /**
+   * Adding directory info to the Table cache.
+   *
+   * @param omMetadataManager  OM Metdata Manager
+   * @param dirInfo            directory info
+   * @param missingParentInfos list of the parents to be added to DB
+   * @param trxnLogIndex       transaction log index
+   */
+  public static void addDirectoryTableCacheEntries(
+          OMMetadataManager omMetadataManager,
+          Optional<OmDirectoryInfo> dirInfo,
+          Optional<List<OmDirectoryInfo>> missingParentInfos,
+          long trxnLogIndex) {
+    for (OmDirectoryInfo subDirInfo : missingParentInfos.get()) {
+      omMetadataManager.getDirectoryTable().addCacheEntry(
+              new CacheKey<>(omMetadataManager.getOzonePathKey(
+                      subDirInfo.getParentObjectID(), subDirInfo.getName())),
+              new CacheValue<>(Optional.of(subDirInfo), trxnLogIndex));
+    }
+
+    if (dirInfo.isPresent()) {
+      omMetadataManager.getDirectoryTable().addCacheEntry(
+              new CacheKey<>(omMetadataManager.getOzonePathKey(
+                      dirInfo.get().getParentObjectID(),
+                      dirInfo.get().getName())),
+              new CacheValue<>(dirInfo, trxnLogIndex));
+    }
+  }
+
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/file/OMDirectoryCreateResponseV1.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/file/OMDirectoryCreateResponseV1.java
new file mode 100644
index 0000000..4e93fa7
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/file/OMDirectoryCreateResponseV1.java
@@ -0,0 +1,103 @@
+/**
+ * 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.om.response.file;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.request.file.OMDirectoryCreateRequest.Result;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE;
+
+/**
+ * Response for create directory request.
+ */
+@CleanupTableInfo(cleanupTables = {DIRECTORY_TABLE})
+public class OMDirectoryCreateResponseV1 extends OMClientResponse {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMDirectoryCreateResponseV1.class);
+
+  private OmDirectoryInfo dirInfo;
+  private List<OmDirectoryInfo> parentDirInfos;
+  private Result result;
+
+  public OMDirectoryCreateResponseV1(@Nonnull OMResponse omResponse,
+                                     @Nonnull OmDirectoryInfo dirInfo,
+                                     @Nonnull List<OmDirectoryInfo> pDirInfos,
+                                     @Nonnull Result result) {
+    super(omResponse);
+    this.dirInfo = dirInfo;
+    this.parentDirInfos = pDirInfos;
+    this.result = result;
+  }
+
+  /**
+   * For when the request is not successful or the directory already exists.
+   */
+  public OMDirectoryCreateResponseV1(@Nonnull OMResponse omResponse,
+                                     @Nonnull Result result) {
+    super(omResponse);
+    this.result = result;
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+                              BatchOperation batchOperation)
+          throws IOException {
+    addToDirectoryTable(omMetadataManager, batchOperation);
+  }
+
+  private void addToDirectoryTable(OMMetadataManager omMetadataManager,
+                                BatchOperation batchOperation)
+          throws IOException {
+    if (dirInfo != null) {
+      if (parentDirInfos != null) {
+        for (OmDirectoryInfo parentDirInfo : parentDirInfos) {
+          String parentKey = omMetadataManager
+                  .getOzonePathKey(parentDirInfo.getParentObjectID(),
+                          parentDirInfo.getName());
+          LOG.debug("putWithBatch parent : dir {} info : {}", parentKey,
+                  parentDirInfo);
+          omMetadataManager.getDirectoryTable()
+                  .putWithBatch(batchOperation, parentKey, parentDirInfo);
+        }
+      }
+
+      String dirKey = omMetadataManager.getOzonePathKey(
+              dirInfo.getParentObjectID(), dirInfo.getName());
+      omMetadataManager.getDirectoryTable().putWithBatch(batchOperation, dirKey,
+              dirInfo);
+    } else {
+      // When directory already exists, we don't add it to cache. And it is
+      // not an error, in this case dirKeyInfo will be null.
+      LOG.debug("Response Status is OK, dirKeyInfo is null in " +
+              "OMDirectoryCreateResponseV1");
+    }
+  }
+}
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMRequestUtils.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMRequestUtils.java
index ce1b2b6..b32a116 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMRequestUtils.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMRequestUtils.java
@@ -38,6 +38,8 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
 import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
@@ -231,6 +233,25 @@ public final class TestOMRequestUtils {
   }
 
   /**
+   * Add dir key entry to DirectoryTable.
+   *
+   * @throws Exception
+   */
+  public static void addDirKeyToDirTable(boolean addToCache,
+                                         OmDirectoryInfo omDirInfo,
+                                         long trxnLogIndex,
+                                         OMMetadataManager omMetadataManager)
+          throws Exception {
+    String ozoneKey = omDirInfo.getPath();
+    if (addToCache) {
+      omMetadataManager.getDirectoryTable().addCacheEntry(
+              new CacheKey<>(ozoneKey),
+              new CacheValue<>(Optional.of(omDirInfo), trxnLogIndex));
+    }
+    omMetadataManager.getDirectoryTable().put(ozoneKey, omDirInfo);
+  }
+
+  /**
    * Create OmKeyInfo.
    */
   public static OmKeyInfo createOmKeyInfo(String volumeName, String bucketName,
@@ -241,6 +262,22 @@ public final class TestOMRequestUtils {
   }
 
   /**
+   * Create OmDirectoryInfo.
+   */
+  public static OmDirectoryInfo createOmDirectoryInfo(String keyName,
+                                                      long objectID,
+                                                      long parentObjID) {
+    return new OmDirectoryInfo.Builder()
+            .setName(keyName)
+            .setCreationTime(Time.now())
+            .setModificationTime(Time.now())
+            .setObjectID(objectID)
+            .setParentObjectID(parentObjID)
+            .setUpdateID(objectID)
+            .build();
+  }
+
+  /**
    * Create OmKeyInfo.
    */
   public static OmKeyInfo createOmKeyInfo(String volumeName, String bucketName,
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMDirectoryCreateRequestV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMDirectoryCreateRequestV1.java
new file mode 100644
index 0000000..77cf74b
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMDirectoryCreateRequestV1.java
@@ -0,0 +1,649 @@
+/**
+ * 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.om.request.file;
+
+import com.google.common.base.Optional;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.AuditMessage;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.ResolvedBucket;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateDirectoryRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.jetbrains.annotations.NotNull;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status.VOLUME_NOT_FOUND;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test OM directory create request V1 layout version.
+ */
+public class TestOMDirectoryCreateRequestV1 {
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  private OzoneManager ozoneManager;
+  private OMMetrics omMetrics;
+  private OMMetadataManager omMetadataManager;
+  private AuditLogger auditLogger;
+  // Just setting ozoneManagerDoubleBuffer which does nothing.
+  private OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper =
+          ((response, transactionIndex) -> {
+            return null;
+          });
+
+  @Before
+  public void setup() throws Exception {
+    ozoneManager = Mockito.mock(OzoneManager.class);
+    omMetrics = OMMetrics.create();
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
+            folder.newFolder().getAbsolutePath());
+    omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration);
+    when(ozoneManager.getMetrics()).thenReturn(omMetrics);
+    when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager);
+    auditLogger = Mockito.mock(AuditLogger.class);
+    when(ozoneManager.getAuditLogger()).thenReturn(auditLogger);
+    Mockito.doNothing().when(auditLogger).logWrite(any(AuditMessage.class));
+    when(ozoneManager.resolveBucketLink(any(KeyArgs.class),
+            any(OMClientRequest.class)))
+            .thenReturn(new ResolvedBucket(Pair.of("", ""), Pair.of("", "")));
+  }
+
+  @After
+  public void stop() {
+    omMetrics.unRegister();
+    Mockito.framework().clearInlineMocks();
+  }
+
+  @Test
+  public void testPreExecute() throws Exception {
+    String volumeName = "vol1";
+    String bucketName = "bucket1";
+    String keyName = "a/b/c";
+
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+            omMetadataManager);
+
+    OMRequest omRequest = createDirectoryRequest(volumeName, bucketName,
+            keyName);
+    OMDirectoryCreateRequestV1 omDirectoryCreateRequestV1 =
+            new OMDirectoryCreateRequestV1(omRequest);
+
+    OMRequest modifiedOmRequest =
+            omDirectoryCreateRequestV1.preExecute(ozoneManager);
+
+    // As in preExecute, we modify original request.
+    Assert.assertNotEquals(omRequest, modifiedOmRequest);
+  }
+
+  @Test
+  public void testValidateAndUpdateCache() throws Exception {
+    String volumeName = "vol1";
+    String bucketName = "bucket1";
+    List<String> dirs = new ArrayList<String>();
+    String keyName = createDirKey(dirs, 3);
+
+    // Add volume and bucket entries to DB.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+            omMetadataManager);
+
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketID = omBucketInfo.getObjectID();
+
+    OMRequest omRequest = createDirectoryRequest(volumeName, bucketName,
+            keyName);
+    OMDirectoryCreateRequestV1 omDirCreateRequestV1 =
+            new OMDirectoryCreateRequestV1(omRequest);
+
+    OMRequest modifiedOmRequest = omDirCreateRequestV1.preExecute(ozoneManager);
+
+    omDirCreateRequestV1 = new OMDirectoryCreateRequestV1(modifiedOmRequest);
+
+    OMClientResponse omClientResponse =
+            omDirCreateRequestV1.validateAndUpdateCache(ozoneManager, 100L,
+                    ozoneManagerDoubleBufferHelper);
+
+    Assert.assertTrue(omClientResponse.getOMResponse().getStatus()
+            == OzoneManagerProtocolProtos.Status.OK);
+    verifyDirectoriesInDB(dirs, bucketID);
+  }
+
+  @Test
+  public void testValidateAndUpdateCacheWithVolumeNotFound() throws Exception {
+    String volumeName = "vol1";
+    String bucketName = "bucket1";
+    List<String> dirs = new ArrayList<String>();
+    String keyName = createDirKey(dirs, 3);
+
+    OMRequest omRequest = createDirectoryRequest(volumeName, bucketName,
+            keyName);
+    OMDirectoryCreateRequestV1 omDirCreateRequestV1 =
+            new OMDirectoryCreateRequestV1(omRequest);
+
+    OMRequest modifiedOmRequest = omDirCreateRequestV1.preExecute(ozoneManager);
+
+    omDirCreateRequestV1 = new OMDirectoryCreateRequestV1(modifiedOmRequest);
+
+    OMClientResponse omClientResponse =
+            omDirCreateRequestV1.validateAndUpdateCache(ozoneManager, 100L,
+                    ozoneManagerDoubleBufferHelper);
+
+    Assert.assertEquals(VOLUME_NOT_FOUND,
+            omClientResponse.getOMResponse().getStatus());
+
+    // Key should not exist in DB
+    Assert.assertTrue("Unexpected directory entries!",
+            omMetadataManager.getDirectoryTable().isEmpty());
+
+  }
+
+  @Test
+  public void testValidateAndUpdateCacheWithBucketNotFound() throws Exception {
+    String volumeName = "vol1";
+    String bucketName = "bucket1";
+    List<String> dirs = new ArrayList<String>();
+    String keyName = createDirKey(dirs, 3);
+
+    OMRequest omRequest = createDirectoryRequest(volumeName, bucketName,
+            keyName);
+    OMDirectoryCreateRequestV1 omDirCreateRequestV1 =
+            new OMDirectoryCreateRequestV1(omRequest);
+
+    OMRequest modifiedOmRequest = omDirCreateRequestV1.preExecute(ozoneManager);
+
+    omDirCreateRequestV1 = new OMDirectoryCreateRequestV1(modifiedOmRequest);
+    TestOMRequestUtils.addVolumeToDB(volumeName, omMetadataManager);
+
+    OMClientResponse omClientResponse =
+            omDirCreateRequestV1.validateAndUpdateCache(ozoneManager, 100L,
+                    ozoneManagerDoubleBufferHelper);
+
+    Assert.assertTrue(omClientResponse.getOMResponse().getStatus()
+            == OzoneManagerProtocolProtos.Status.BUCKET_NOT_FOUND);
+
+    // Key should not exist in DB
+    Assert.assertTrue("Unexpected directory entries!",
+            omMetadataManager.getDirectoryTable().isEmpty());
+  }
+
+  @Test
+  public void testValidateAndUpdateCacheWithSubDirectoryInPath()
+          throws Exception {
+    String volumeName = "vol1";
+    String bucketName = "bucket1";
+    List<String> dirs = new ArrayList<String>();
+    String keyName = createDirKey(dirs, 3);
+
+    // Add volume and bucket entries to DB.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+            omMetadataManager);
+
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketID = omBucketInfo.getObjectID();
+    int objID = 100;
+
+    //1. Create root
+    OmDirectoryInfo omDirInfo =
+            TestOMRequestUtils.createOmDirectoryInfo(dirs.get(0), objID++,
+                    bucketID);
+    TestOMRequestUtils.addDirKeyToDirTable(true, omDirInfo, 5000,
+            omMetadataManager);
+    //2. Create sub-directory under root
+    omDirInfo = TestOMRequestUtils.createOmDirectoryInfo(dirs.get(1), objID++,
+            omDirInfo.getObjectID());
+    TestOMRequestUtils.addDirKeyToDirTable(true, omDirInfo, 5000,
+            omMetadataManager);
+
+    OMRequest omRequest = createDirectoryRequest(volumeName, bucketName,
+            keyName);
+    OMDirectoryCreateRequestV1 omDirCreateRequestV1 =
+            new OMDirectoryCreateRequestV1(omRequest);
+
+    OMRequest modifiedOmRequest = omDirCreateRequestV1.preExecute(ozoneManager);
+
+    omDirCreateRequestV1 = new OMDirectoryCreateRequestV1(modifiedOmRequest);
+
+    OMClientResponse omClientResponse =
+            omDirCreateRequestV1.validateAndUpdateCache(ozoneManager, 100L,
+                    ozoneManagerDoubleBufferHelper);
+
+    Assert.assertTrue(omClientResponse.getOMResponse().getStatus()
+            == OzoneManagerProtocolProtos.Status.OK);
+
+    // Key should exist in DB and cache.
+    verifyDirectoriesInDB(dirs, bucketID);
+  }
+
+  @Test
+  public void testValidateAndUpdateCacheWithDirectoryAlreadyExists()
+          throws Exception {
+    String volumeName = "vol1";
+    String bucketName = "bucket1";
+    List<String> dirs = new ArrayList<String>();
+    String keyName = createDirKey(dirs, 3);
+
+    // Add volume and bucket entries to DB.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+            omMetadataManager);
+
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketID = omBucketInfo.getObjectID();
+
+    // bucketID is the parent
+    long parentID = bucketID;
+
+    // add all the directories into DirectoryTable
+    for (int indx = 0; indx < dirs.size(); indx++) {
+      long objID = 100 + indx;
+      long txnID = 5000 + indx;
+      // for index=0, parentID is bucketID
+      OmDirectoryInfo omDirInfo = TestOMRequestUtils.createOmDirectoryInfo(
+              dirs.get(indx), objID, parentID);
+      TestOMRequestUtils.addDirKeyToDirTable(false, omDirInfo,
+              txnID, omMetadataManager);
+
+      parentID = omDirInfo.getObjectID();
+    }
+
+    OMRequest omRequest = createDirectoryRequest(volumeName, bucketName,
+            keyName);
+    OMDirectoryCreateRequestV1 omDirCreateRequestV1 =
+            new OMDirectoryCreateRequestV1(omRequest);
+
+    OMRequest modifiedOmRequest = omDirCreateRequestV1.preExecute(ozoneManager);
+
+    omDirCreateRequestV1 = new OMDirectoryCreateRequestV1(modifiedOmRequest);
+
+    OMClientResponse omClientResponse =
+            omDirCreateRequestV1.validateAndUpdateCache(ozoneManager, 100L,
+                    ozoneManagerDoubleBufferHelper);
+
+    Assert.assertTrue(omClientResponse.getOMResponse().getStatus()
+            == OzoneManagerProtocolProtos.Status.DIRECTORY_ALREADY_EXISTS);
+
+    Assert.assertEquals("Wrong OM numKeys metrics",
+            0, ozoneManager.getMetrics().getNumKeys());
+
+    // Key should exist in DB and doesn't added to cache.
+    verifyDirectoriesInDB(dirs, bucketID);
+    verifyDirectoriesNotInCache(dirs, bucketID);
+  }
+
+  /**
+   * Case: File exists with the same name as the requested directory.
+   * Say, requested to createDir '/a/b/c' and there is a file exists with
+   * same name.
+   */
+  @Test
+  public void testValidateAndUpdateCacheWithFilesInPath() throws Exception {
+    String volumeName = "vol1";
+    String bucketName = "bucket1";
+    List<String> dirs = new ArrayList<String>();
+    String keyName = createDirKey(dirs, 3);
+
+    // Add volume and bucket entries to DB.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+            omMetadataManager);
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long parentID = omBucketInfo.getObjectID();
+
+    // add all the parent directories into DirectoryTable. This won't create
+    // the leaf node and this will be used in CreateDirectoryReq.
+    for (int indx = 0; indx < dirs.size() - 1; indx++) {
+      long objID = 100 + indx;
+      long txnID = 5000 + indx;
+      // for index=0, parentID is bucketID
+      OmDirectoryInfo omDirInfo = TestOMRequestUtils.createOmDirectoryInfo(
+              dirs.get(indx), objID, parentID);
+      TestOMRequestUtils.addDirKeyToDirTable(false, omDirInfo,
+              txnID, omMetadataManager);
+
+      parentID = omDirInfo.getObjectID();
+    }
+
+    long objID = parentID + 100;
+    long txnID = 50000;
+
+    // Add a file into the FileTable, this is to simulate "file exists" check.
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
+            bucketName, keyName, HddsProtos.ReplicationType.RATIS,
+            HddsProtos.ReplicationFactor.THREE, objID++);
+    String ozoneFileName = parentID + "/" + dirs.get(dirs.size() - 1);
+    omMetadataManager.getKeyTable().addCacheEntry(new CacheKey<>(ozoneFileName),
+            new CacheValue<>(Optional.of(omKeyInfo), ++txnID));
+    omMetadataManager.getKeyTable().put(ozoneFileName, omKeyInfo);
+
+    OMRequest omRequest = createDirectoryRequest(volumeName, bucketName,
+            keyName);
+    OMDirectoryCreateRequestV1 omDirCreateRequestV1 =
+            new OMDirectoryCreateRequestV1(omRequest);
+
+    OMRequest modifiedOmRequest =
+            omDirCreateRequestV1.preExecute(ozoneManager);
+
+    omDirCreateRequestV1 = new OMDirectoryCreateRequestV1(modifiedOmRequest);
+
+    OMClientResponse omClientResponse =
+            omDirCreateRequestV1.validateAndUpdateCache(ozoneManager, 100L,
+                    ozoneManagerDoubleBufferHelper);
+
+    Assert.assertTrue(omClientResponse.getOMResponse().getStatus()
+            == OzoneManagerProtocolProtos.Status.FILE_ALREADY_EXISTS);
+
+    Assert.assertEquals("Wrong OM numKeys metrics",
+            0, ozoneManager.getMetrics().getNumKeys());
+
+    // Key should not exist in DB
+    Assert.assertNotNull(omMetadataManager.getKeyTable().get(ozoneFileName));
+    // Key should not exist in DB
+    Assert.assertEquals("Wrong directories count!", 3,
+            omMetadataManager.getDirectoryTable().getEstimatedKeyCount());
+  }
+
+
+  /**
+   * Case: File exists in the given path.
+   * Say, requested to createDir '/a/b/c/d' and there is a file '/a/b' exists
+   * in the given path.
+   */
+  @Test
+  public void testValidateAndUpdateCacheWithFileExistsInGivenPath()
+          throws Exception {
+    String volumeName = "vol1";
+    String bucketName = "bucket1";
+    List<String> dirs = new ArrayList<String>();
+    String keyName = createDirKey(dirs, 3);
+
+    // Add volume and bucket entries to DB.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+            omMetadataManager);
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long parentID = omBucketInfo.getObjectID();
+
+    long objID = parentID + 100;
+    long txnID = 5000;
+
+    // for index=0, parentID is bucketID
+    OmDirectoryInfo omDirInfo = TestOMRequestUtils.createOmDirectoryInfo(
+            dirs.get(0), objID++, parentID);
+    TestOMRequestUtils.addDirKeyToDirTable(true, omDirInfo,
+            txnID, omMetadataManager);
+    parentID = omDirInfo.getObjectID();
+
+    // Add a key in second level.
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
+            bucketName, keyName, HddsProtos.ReplicationType.RATIS,
+            HddsProtos.ReplicationFactor.THREE, objID++);
+    String ozoneKey = parentID + "/" + dirs.get(1);
+    omMetadataManager.getKeyTable().addCacheEntry(new CacheKey<>(ozoneKey),
+            new CacheValue<>(Optional.of(omKeyInfo), ++txnID));
+    omMetadataManager.getKeyTable().put(ozoneKey, omKeyInfo);
+
+    OMRequest omRequest = createDirectoryRequest(volumeName, bucketName,
+            keyName);
+    OMDirectoryCreateRequestV1 omDirCreateRequestV1 =
+            new OMDirectoryCreateRequestV1(omRequest);
+
+    OMRequest modifiedOmRequest =
+            omDirCreateRequestV1.preExecute(ozoneManager);
+
+    omDirCreateRequestV1 = new OMDirectoryCreateRequestV1(modifiedOmRequest);
+
+    OMClientResponse omClientResponse =
+            omDirCreateRequestV1.validateAndUpdateCache(ozoneManager, 100L,
+                    ozoneManagerDoubleBufferHelper);
+
+    Assert.assertTrue("Invalid response code:" +
+                    omClientResponse.getOMResponse().getStatus(),
+            omClientResponse.getOMResponse().getStatus()
+                    == OzoneManagerProtocolProtos.Status.FILE_ALREADY_EXISTS);
+
+    Assert.assertEquals("Wrong OM numKeys metrics",
+            0, ozoneManager.getMetrics().getNumKeys());
+
+    // Key should not exist in DB
+    Assert.assertTrue(omMetadataManager.getKeyTable().get(ozoneKey) != null);
+    // Key should not exist in DB
+    Assert.assertEquals("Wrong directories count!",
+            1, omMetadataManager.getDirectoryTable().getEstimatedKeyCount());
+  }
+
+  @Test
+  public void testCreateDirectoryUptoLimitOfMaxLevels255() throws Exception {
+    String volumeName = "vol1";
+    String bucketName = "bucket1";
+    List<String> dirs = new ArrayList<String>();
+    String keyName = createDirKey(dirs, 255);
+
+    // Add volume and bucket entries to DB.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+            omMetadataManager);
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketID = omBucketInfo.getObjectID();
+
+    OMRequest omRequest = createDirectoryRequest(volumeName, bucketName,
+            OzoneFSUtils.addTrailingSlashIfNeeded(keyName));
+    OMDirectoryCreateRequestV1 omDirCreateRequestV1 =
+            new OMDirectoryCreateRequestV1(omRequest);
+
+    OMRequest modifiedOmRequest = omDirCreateRequestV1.preExecute(ozoneManager);
+
+    omDirCreateRequestV1 = new OMDirectoryCreateRequestV1(modifiedOmRequest);
+
+    Assert.assertEquals(0L, omMetrics.getNumKeys());
+    OMClientResponse omClientResponse =
+            omDirCreateRequestV1.validateAndUpdateCache(ozoneManager, 100L,
+                    ozoneManagerDoubleBufferHelper);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.OK,
+            omClientResponse.getOMResponse().getStatus());
+
+    verifyDirectoriesInDB(dirs, bucketID);
+
+    Assert.assertEquals(dirs.size(), omMetrics.getNumKeys());
+  }
+
+  @Test
+  public void testCreateDirectoryExceedLimitOfMaxLevels255() throws Exception {
+    String volumeName = "vol1";
+    String bucketName = "bucket1";
+    List<String> dirs = new ArrayList<String>();
+    String keyName = createDirKey(dirs, 256);
+
+    // Add volume and bucket entries to DB.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+            omMetadataManager);
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketID = omBucketInfo.getObjectID();
+
+    OMRequest omRequest = createDirectoryRequest(volumeName, bucketName,
+            OzoneFSUtils.addTrailingSlashIfNeeded(keyName));
+    OMDirectoryCreateRequestV1 omDirCreateRequestV1 =
+            new OMDirectoryCreateRequestV1(omRequest);
+
+    OMRequest modifiedOmRequest = omDirCreateRequestV1.preExecute(ozoneManager);
+
+    omDirCreateRequestV1 = new OMDirectoryCreateRequestV1(modifiedOmRequest);
+
+    Assert.assertEquals(0L, omMetrics.getNumKeys());
+    OMClientResponse omClientResponse =
+            omDirCreateRequestV1.validateAndUpdateCache(ozoneManager,
+                    100L, ozoneManagerDoubleBufferHelper);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.INVALID_KEY_NAME,
+            omClientResponse.getOMResponse().getStatus());
+
+    Assert.assertEquals("Unexpected directories!", 0,
+            omMetadataManager.getDirectoryTable().getEstimatedKeyCount());
+
+    Assert.assertEquals(0, omMetrics.getNumKeys());
+  }
+
+  @Test
+  public void testCreateDirectoryOMMetric() throws Exception {
+    String volumeName = "vol1";
+    String bucketName = "bucket1";
+    List<String> dirs = new ArrayList<String>();
+    String keyName = createDirKey(dirs, 3);
+
+    // Add volume and bucket entries to DB.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+            omMetadataManager);
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketID = omBucketInfo.getObjectID();
+
+    OMRequest omRequest = createDirectoryRequest(volumeName, bucketName,
+            OzoneFSUtils.addTrailingSlashIfNeeded(keyName));
+    OMDirectoryCreateRequestV1 omDirCreateRequestV1 =
+            new OMDirectoryCreateRequestV1(omRequest);
+
+    OMRequest modifiedOmRequest = omDirCreateRequestV1.preExecute(ozoneManager);
+
+    omDirCreateRequestV1 = new OMDirectoryCreateRequestV1(modifiedOmRequest);
+
+    Assert.assertEquals(0L, omMetrics.getNumKeys());
+    OMClientResponse omClientResponse =
+            omDirCreateRequestV1.validateAndUpdateCache(ozoneManager, 100L,
+                    ozoneManagerDoubleBufferHelper);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.OK,
+            omClientResponse.getOMResponse().getStatus());
+
+    verifyDirectoriesInDB(dirs, bucketID);
+
+    Assert.assertEquals(dirs.size(), omMetrics.getNumKeys());
+  }
+
+
+  @NotNull
+  private String createDirKey(List<String> dirs, int depth) {
+    String keyName = RandomStringUtils.randomAlphabetic(5);
+    dirs.add(keyName);
+    for (int i = 0; i < depth; i++) {
+      String dirName = RandomStringUtils.randomAlphabetic(5);
+      dirs.add(dirName);
+      keyName += "/" + dirName;
+    }
+    return keyName;
+  }
+
+  private void verifyDirectoriesInDB(List<String> dirs, long bucketID)
+          throws IOException {
+    // bucketID is the parent
+    long parentID = bucketID;
+    for (int indx = 0; indx < dirs.size(); indx++) {
+      String dirName = dirs.get(indx);
+      String dbKey = "";
+      // for index=0, parentID is bucketID
+      dbKey = omMetadataManager.getOzonePathKey(parentID, dirName);
+      OmDirectoryInfo omDirInfo =
+              omMetadataManager.getDirectoryTable().get(dbKey);
+      Assert.assertNotNull("Invalid directory!", omDirInfo);
+      Assert.assertEquals("Invalid directory!", dirName, omDirInfo.getName());
+      Assert.assertEquals("Invalid dir path!",
+              parentID + "/" + dirName, omDirInfo.getPath());
+      parentID = omDirInfo.getObjectID();
+    }
+  }
+
+  private void verifyDirectoriesNotInCache(List<String> dirs, long bucketID)
+          throws IOException {
+    // bucketID is the parent
+    long parentID = bucketID;
+    for (int indx = 0; indx < dirs.size(); indx++) {
+      String dirName = dirs.get(indx);
+      String dbKey = "";
+      // for index=0, parentID is bucketID
+      dbKey = omMetadataManager.getOzonePathKey(parentID, dirName);
+      CacheValue<OmDirectoryInfo> omDirInfoCacheValue =
+              omMetadataManager.getDirectoryTable()
+                      .getCacheValue(new CacheKey<>(dbKey));
+      Assert.assertNull("Unexpected directory!", omDirInfoCacheValue);
+    }
+  }
+
+  /**
+   * Create OMRequest which encapsulates CreateDirectory request.
+   *
+   * @param volumeName
+   * @param bucketName
+   * @param keyName
+   * @return OMRequest
+   */
+  private OMRequest createDirectoryRequest(String volumeName, String bucketName,
+                                           String keyName) {
+    return OMRequest.newBuilder().setCreateDirectoryRequest(
+            CreateDirectoryRequest.newBuilder().setKeyArgs(
+                    KeyArgs.newBuilder().setVolumeName(volumeName)
+                            .setBucketName(bucketName).setKeyName(keyName)))
+            .setCmdType(OzoneManagerProtocolProtos.Type.CreateDirectory)
+            .setClientId(UUID.randomUUID().toString()).build();
+  }
+
+}
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponseV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponseV1.java
new file mode 100644
index 0000000..0a1114a
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponseV1.java
@@ -0,0 +1,88 @@
+/**
+ * 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.om.response.file;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.apache.hadoop.ozone.om.request.file.OMDirectoryCreateRequestV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.util.ArrayList;
+import java.util.UUID;
+
+/**
+ * Tests OMDirectoryCreateResponseV1 new layout version.
+ */
+public class TestOMDirectoryCreateResponseV1 {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  private OMMetadataManager omMetadataManager;
+  private BatchOperation batchOperation;
+
+  @Before
+  public void setup() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
+        folder.newFolder().getAbsolutePath());
+    omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration);
+    batchOperation = omMetadataManager.getStore().initBatchOperation();
+  }
+
+  @Test
+  public void testAddToDBBatch() throws Exception {
+
+    String volumeName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+
+    long parentID = 100;
+    OmDirectoryInfo omDirInfo =
+            TestOMRequestUtils.createOmDirectoryInfo(keyName, 500, parentID);
+
+    OMResponse omResponse = OMResponse.newBuilder().setCreateDirectoryResponse(
+        OzoneManagerProtocolProtos.CreateDirectoryResponse.getDefaultInstance())
+            .setStatus(OzoneManagerProtocolProtos.Status.OK)
+            .setCmdType(OzoneManagerProtocolProtos.Type.CreateDirectory)
+            .build();
+
+    OMDirectoryCreateResponseV1 omDirectoryCreateResponseV1 =
+        new OMDirectoryCreateResponseV1(omResponse, omDirInfo,
+            new ArrayList<>(), OMDirectoryCreateRequestV1.Result.SUCCESS);
+
+    omDirectoryCreateResponseV1.addToDBBatch(omMetadataManager, batchOperation);
+
+    // Do manual commit and see whether addToBatch is successful or not.
+    omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+    Assert.assertNotNull(omMetadataManager.getDirectoryTable().get(
+            omMetadataManager.getOzonePathKey(parentID, keyName)));
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@ozone.apache.org
For additional commands, e-mail: commits-help@ozone.apache.org