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 2021/04/08 03:56:14 UTC

[ozone] 08/32: HDDS-4514. AllocateBlock : lookup and update open file table for the given path (#1679)

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 c10abe67cb09e92879c5ece0727d8a875d1f0db9
Author: Rakesh Radhakrishnan <ra...@apache.org>
AuthorDate: Tue Dec 22 09:07:25 2020 +0530

    HDDS-4514. AllocateBlock : lookup and update open file table for the given path (#1679)
---
 .../hadoop/fs/ozone/TestOzoneFileSystem.java       |  74 +++++++
 .../hadoop/fs/ozone/TestOzoneFileSystemV1.java     |  25 ---
 .../om/ratis/utils/OzoneManagerRatisUtils.java     |   4 +
 .../om/request/file/OMFileCreateRequestV1.java     |   3 +
 .../ozone/om/request/file/OMFileRequest.java       |  52 +++++
 .../om/request/key/OMAllocateBlockRequestV1.java   | 227 +++++++++++++++++++++
 .../ozone/om/request/key/OMKeyCommitRequestV1.java |  76 +------
 .../om/response/key/OMAllocateBlockResponse.java   |  12 ++
 ...esponse.java => OMAllocateBlockResponseV1.java} |  49 ++---
 .../om/request/file/TestOMFileCreateRequest.java   |  22 --
 .../om/request/file/TestOMFileCreateRequestV1.java |   5 +
 .../om/request/key/TestOMAllocateBlockRequest.java |  44 ++--
 .../request/key/TestOMAllocateBlockRequestV1.java  | 119 +++++++++++
 .../om/request/key/TestOMKeyCommitRequestV1.java   |   5 +
 .../om/request/key/TestOMKeyDeleteRequestV1.java   |  14 ++
 .../ozone/om/request/key/TestOMKeyRequest.java     |  25 +++
 .../response/file/TestOMFileCreateResponseV1.java  |   5 +
 .../response/key/TestOMAllocateBlockResponse.java  |  37 ++--
 ...eV1.java => TestOMAllocateBlockResponseV1.java} |  76 +++----
 .../om/response/key/TestOMKeyCommitResponseV1.java |   5 +
 .../om/response/key/TestOMKeyDeleteResponseV1.java |  16 ++
 21 files changed, 667 insertions(+), 228 deletions(-)

diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java
index 855484c..c830e07 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
 import org.apache.hadoop.fs.Trash;
 import org.apache.hadoop.fs.TrashPolicy;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -583,6 +584,7 @@ public class TestOzoneFileSystem {
   @Test
   public void testListStatusOnLargeDirectory() throws Exception {
     Path root = new Path("/");
+    deleteRootDir(); // cleanup
     Set<String> paths = new TreeSet<>();
     int numDirs = LISTING_PAGE_SIZE + LISTING_PAGE_SIZE / 2;
     for(int i = 0; i < numDirs; i++) {
@@ -592,6 +594,21 @@ public class TestOzoneFileSystem {
     }
 
     FileStatus[] fileStatuses = o3fs.listStatus(root);
+    // Added logs for debugging failures, to check any sub-path mismatches.
+    Set<String> actualPaths = new TreeSet<>();
+    ArrayList<String> actualPathList = new ArrayList<>();
+    if (rootItemCount != fileStatuses.length) {
+      for (int i = 0; i < fileStatuses.length; i++) {
+        actualPaths.add(fileStatuses[i].getPath().getName());
+        actualPathList.add(fileStatuses[i].getPath().getName());
+      }
+      if (rootItemCount != actualPathList.size()) {
+        actualPaths.removeAll(paths);
+        actualPathList.removeAll(paths);
+        LOG.info("actualPaths: {}", actualPaths);
+        LOG.info("actualPathList: {}", actualPathList);
+      }
+    }
     assertEquals(
         "Total directories listed do not match the existing directories",
         numDirs, fileStatuses.length);
@@ -602,6 +619,31 @@ public class TestOzoneFileSystem {
   }
 
   /**
+   * Cleanup files and directories.
+   *
+   * @throws IOException DB failure
+   */
+  protected void deleteRootDir() throws IOException {
+    Path root = new Path("/");
+    FileStatus[] fileStatuses = fs.listStatus(root);
+
+    rootItemCount = 0; // reset to zero
+
+    if (fileStatuses == null) {
+      return;
+    }
+
+    for (FileStatus fStatus : fileStatuses) {
+      fs.delete(fStatus.getPath(), true);
+    }
+
+    fileStatuses = fs.listStatus(root);
+    if (fileStatuses != null) {
+      Assert.assertEquals("Delete root failed!", 0, fileStatuses.length);
+    }
+  }
+
+  /**
    * Tests listStatus on a path with subdirs.
    */
   @Test
@@ -662,6 +704,38 @@ public class TestOzoneFileSystem {
   }
 
   @Test
+  public void testAllocateMoreThanOneBlock() throws IOException {
+    Path file = new Path("/file");
+    String str = "TestOzoneFileSystemV1.testSeekOnFileLength";
+    byte[] strBytes = str.getBytes();
+    long numBlockAllocationsOrg =
+            cluster.getOzoneManager().getMetrics().getNumBlockAllocates();
+
+    try (FSDataOutputStream out1 = fs.create(file, FsPermission.getDefault(),
+            true, 8, (short) 3, 1, null)) {
+      for (int i = 0; i < 100000; i++) {
+        out1.write(strBytes);
+      }
+    }
+
+    try (FSDataInputStream stream = fs.open(file)) {
+      FileStatus fileStatus = fs.getFileStatus(file);
+      long blkSize = fileStatus.getBlockSize();
+      long fileLength = fileStatus.getLen();
+      Assert.assertTrue("Block allocation should happen",
+              fileLength > blkSize);
+
+      long newNumBlockAllocations =
+              cluster.getOzoneManager().getMetrics().getNumBlockAllocates();
+
+      Assert.assertTrue("Block allocation should happen",
+              (newNumBlockAllocations > numBlockAllocationsOrg));
+
+      stream.seek(fileLength);
+      assertEquals(-1, stream.read());
+    }
+  }
+
   public void testDeleteRoot() throws IOException {
     Path dir = new Path("/dir");
     fs.mkdirs(dir);
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemV1.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemV1.java
index 212080b..2938714 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemV1.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemV1.java
@@ -407,31 +407,6 @@ public class TestOzoneFileSystemV1 extends TestOzoneFileSystem {
     }
   }
 
-  /**
-   * Cleanup files and directories.
-   *
-   * @throws IOException DB failure
-   */
-  protected void deleteRootDir() throws IOException {
-    Path root = new Path("/");
-    FileStatus[] fileStatuses = fs.listStatus(root);
-
-    rootItemCount = 0; // reset to zero
-
-    if (fileStatuses == null) {
-      return;
-    }
-
-    for (FileStatus fStatus : fileStatuses) {
-      fs.delete(fStatus.getPath(), true);
-    }
-
-    fileStatuses = fs.listStatus(root);
-    if (fileStatuses != null) {
-      Assert.assertEquals("Delete root failed!", 0, fileStatuses.length);
-    }
-  }
-
   @Override
   @Test
   @Ignore("TODO:HDDS-2939")
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 97868d8..4702181 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
@@ -37,6 +37,7 @@ import org.apache.hadoop.ozone.om.request.file.OMFileCreateRequest;
 import org.apache.hadoop.ozone.om.request.file.OMFileCreateRequestV1;
 import org.apache.hadoop.ozone.om.request.key.OMKeysDeleteRequest;
 import org.apache.hadoop.ozone.om.request.key.OMAllocateBlockRequest;
+import org.apache.hadoop.ozone.om.request.key.OMAllocateBlockRequestV1;
 import org.apache.hadoop.ozone.om.request.key.OMKeyCommitRequest;
 import org.apache.hadoop.ozone.om.request.key.OMKeyCommitRequestV1;
 import org.apache.hadoop.ozone.om.request.key.OMKeyCreateRequest;
@@ -134,6 +135,9 @@ public final class OzoneManagerRatisUtils {
     case SetBucketProperty:
       return new OMBucketSetPropertyRequest(omRequest);
     case AllocateBlock:
+      if (omLayoutVersionV1) {
+        return new OMAllocateBlockRequestV1(omRequest);
+      }
       return new OMAllocateBlockRequest(omRequest);
     case CreateKey:
       return new OMKeyCreateRequest(omRequest);
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequestV1.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequestV1.java
index 606e15b..e38908a 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequestV1.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequestV1.java
@@ -139,6 +139,9 @@ public class OMFileCreateRequestV1 extends OMFileCreateRequest {
                 pathInfoV1.getLeafNodeName());
         dbFileInfo = OMFileRequest.getOmKeyInfoFromFileTable(false,
                 omMetadataManager, dbFileKey, keyName);
+        if (dbFileInfo != null) {
+          ozoneManager.getKeyManager().refresh(dbFileInfo);
+        }
       }
 
       // check if the file or directory already existed in OM
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 fc9bab0..aadc126 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
@@ -52,6 +52,8 @@ import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nonnull;
 
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_A_FILE;
 
 /**
  * Base class for file requests.
@@ -851,4 +853,54 @@ public final class OMFileRequest {
   public static boolean isImmediateChild(long parentId, long ancestorId) {
     return parentId == ancestorId;
   }
+
+  /**
+   * Get parent id for the user given path.
+   *
+   * @param bucketId       bucket id
+   * @param pathComponents fie path elements
+   * @param keyName        user given key name
+   * @param omMetadataManager   om metadata manager
+   * @return lastKnownParentID
+   * @throws IOException DB failure or parent not exists in DirectoryTable
+   */
+  public static long getParentID(long bucketId, Iterator<Path> pathComponents,
+      String keyName, OMMetadataManager omMetadataManager) throws IOException {
+
+    long lastKnownParentId = bucketId;
+
+    // If no sub-dirs then bucketID is the root/parent.
+    if(!pathComponents.hasNext()){
+      return bucketId;
+    }
+
+    OmDirectoryInfo omDirectoryInfo;
+    while (pathComponents.hasNext()) {
+      String nodeName = pathComponents.next().toString();
+      boolean reachedLastPathComponent = !pathComponents.hasNext();
+      String dbNodeName =
+              omMetadataManager.getOzonePathKey(lastKnownParentId, nodeName);
+
+      omDirectoryInfo = omMetadataManager.
+              getDirectoryTable().get(dbNodeName);
+      if (omDirectoryInfo != null) {
+        if (reachedLastPathComponent) {
+          throw new OMException("Can not create file: " + keyName +
+                  " as there is already directory in the given path",
+                  NOT_A_FILE);
+        }
+        lastKnownParentId = omDirectoryInfo.getObjectID();
+      } else {
+        // One of the sub-dir doesn't exists in DB. Immediate parent should
+        // exists for committing the key, otherwise will fail the operation.
+        if (!reachedLastPathComponent) {
+          throw new OMException("Failed to find parent directory of "
+                  + keyName + " in DirectoryTable", KEY_NOT_FOUND);
+        }
+        break;
+      }
+    }
+
+    return lastKnownParentId;
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
new file mode 100644
index 0000000..a6a2558
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
@@ -0,0 +1,227 @@
+/**
+ * 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.key;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.OzoneConsts;
+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.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.AllocateBlockRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.AllocateBlockResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.jetbrains.annotations.NotNull;
+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.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * Handles allocate block request layout version V1.
+ */
+public class OMAllocateBlockRequestV1 extends OMAllocateBlockRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMAllocateBlockRequestV1.class);
+
+  public OMAllocateBlockRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+
+    AllocateBlockRequest allocateBlockRequest =
+            getOmRequest().getAllocateBlockRequest();
+
+    KeyArgs keyArgs =
+            allocateBlockRequest.getKeyArgs();
+
+    OzoneManagerProtocolProtos.KeyLocation blockLocation =
+            allocateBlockRequest.getKeyLocation();
+    Preconditions.checkNotNull(blockLocation);
+
+    String volumeName = keyArgs.getVolumeName();
+    String bucketName = keyArgs.getBucketName();
+    String keyName = keyArgs.getKeyName();
+    long clientID = allocateBlockRequest.getClientID();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumBlockAllocateCalls();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    Map<String, String> auditMap = buildKeyArgsAuditMap(keyArgs);
+    auditMap.put(OzoneConsts.CLIENT_ID, String.valueOf(clientID));
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    String openKeyName = null;
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+            getOmRequest());
+    OMClientResponse omClientResponse = null;
+
+    OmKeyInfo openKeyInfo = null;
+    IOException exception = null;
+    OmBucketInfo omBucketInfo = null;
+    boolean acquiredLock = false;
+
+    try {
+      keyArgs = resolveBucketLink(ozoneManager, keyArgs, auditMap);
+      volumeName = keyArgs.getVolumeName();
+      bucketName = keyArgs.getBucketName();
+
+      // check Acl
+      checkKeyAclsInOpenKeyTable(ozoneManager, volumeName, bucketName, keyName,
+          IAccessAuthorizer.ACLType.WRITE, allocateBlockRequest.getClientID());
+
+      validateBucketAndVolume(omMetadataManager, volumeName,
+          bucketName);
+
+      // Here we don't acquire bucket/volume lock because for a single client
+      // allocateBlock is called in serial fashion. With this approach, it
+      // won't make 'fail-fast' during race condition case on delete/rename op,
+      // assuming that later it will fail at the key commit operation.
+      openKeyName = getOpenKeyName(volumeName, bucketName, keyName, clientID,
+              ozoneManager);
+      openKeyInfo = getOpenKeyInfo(omMetadataManager, openKeyName, keyName);
+      if (openKeyInfo == null) {
+        throw new OMException("Open Key not found " + openKeyName,
+                KEY_NOT_FOUND);
+      }
+
+      List<OmKeyLocationInfo> newLocationList = Collections.singletonList(
+              OmKeyLocationInfo.getFromProtobuf(blockLocation));
+
+      acquiredLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
+              volumeName, bucketName);
+      omBucketInfo = getBucketInfo(omMetadataManager, volumeName, bucketName);
+      // check bucket and volume quota
+      long preAllocatedSpace = newLocationList.size()
+              * ozoneManager.getScmBlockSize()
+              * openKeyInfo.getFactor().getNumber();
+      checkBucketQuotaInBytes(omBucketInfo, preAllocatedSpace);
+      // Append new block
+      openKeyInfo.appendNewBlocks(newLocationList, false);
+
+      // Set modification time.
+      openKeyInfo.setModificationTime(keyArgs.getModificationTime());
+
+      // Set the UpdateID to current transactionLogIndex
+      openKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
+
+      // Add to cache.
+      addOpenTableCacheEntry(trxnLogIndex, omMetadataManager, openKeyName,
+              openKeyInfo);
+      omBucketInfo.incrUsedBytes(preAllocatedSpace);
+
+      omResponse.setAllocateBlockResponse(AllocateBlockResponse.newBuilder()
+              .setKeyLocation(blockLocation).build());
+      omClientResponse = getOmClientResponse(clientID, omResponse,
+              openKeyInfo, omBucketInfo.copyObject());
+      LOG.debug("Allocated block for Volume:{}, Bucket:{}, OpenKey:{}",
+              volumeName, bucketName, openKeyName);
+    } catch (IOException ex) {
+      omMetrics.incNumBlockAllocateCallFails();
+      exception = ex;
+      omClientResponse = new OMAllocateBlockResponse(createErrorOMResponse(
+              omResponse, exception));
+      LOG.error("Allocate Block failed. Volume:{}, Bucket:{}, OpenKey:{}. " +
+              "Exception:{}", volumeName, bucketName, openKeyName, exception);
+    } finally {
+      addResponseToDoubleBuffer(trxnLogIndex, omClientResponse,
+              omDoubleBufferHelper);
+      if (acquiredLock) {
+        omMetadataManager.getLock().releaseWriteLock(BUCKET_LOCK, volumeName,
+                bucketName);
+      }
+    }
+
+    auditLog(auditLogger, buildAuditMessage(OMAction.ALLOCATE_BLOCK, auditMap,
+            exception, getOmRequest().getUserInfo()));
+
+    return omClientResponse;
+  }
+
+  private OmKeyInfo getOpenKeyInfo(OMMetadataManager omMetadataManager,
+      String openKeyName, String keyName) throws IOException {
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    return OMFileRequest.getOmKeyInfoFromFileTable(true,
+            omMetadataManager, openKeyName, fileName);
+  }
+
+  private String getOpenKeyName(String volumeName, String bucketName,
+      String keyName, long clientID, OzoneManager ozoneManager)
+          throws IOException {
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketId = omBucketInfo.getObjectID();
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    Iterator<Path> pathComponents = Paths.get(keyName).iterator();
+    long parentID = OMFileRequest.getParentID(bucketId, pathComponents,
+            keyName, omMetadataManager);
+    return omMetadataManager.getOpenFileName(parentID, fileName,
+            clientID);
+  }
+
+  private void addOpenTableCacheEntry(long trxnLogIndex,
+      OMMetadataManager omMetadataManager, String openKeyName,
+      OmKeyInfo openKeyInfo) {
+    String fileName = openKeyInfo.getFileName();
+    OMFileRequest.addOpenFileTableCacheEntry(omMetadataManager, openKeyName,
+            openKeyInfo, fileName, trxnLogIndex);
+  }
+
+  @NotNull
+  private OMClientResponse getOmClientResponse(long clientID,
+      OMResponse.Builder omResponse, OmKeyInfo openKeyInfo,
+      OmBucketInfo omBucketInfo) {
+    return new OMAllocateBlockResponseV1(omResponse.build(), openKeyInfo,
+            clientID, omBucketInfo);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestV1.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestV1.java
index 8c47f7e..7d99119 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestV1.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequestV1.java
@@ -25,7 +25,6 @@ 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.helpers.OmBucketInfo;
-import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
@@ -53,7 +52,6 @@ import java.util.List;
 import java.util.Map;
 
 import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
-import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_A_FILE;
 import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
 
 /**
@@ -130,8 +128,8 @@ public class OMKeyCommitRequestV1 extends OMKeyCommitRequest {
       String fileName = OzoneFSUtils.getFileName(keyName);
       omBucketInfo = omMetadataManager.getBucketTable().get(bucketKey);
       long bucketId = omBucketInfo.getObjectID();
-      long parentID = getParentID(bucketId, pathComponents, keyName,
-              omMetadataManager, ozoneManager);
+      long parentID = OMFileRequest.getParentID(bucketId, pathComponents,
+              keyName, omMetadataManager);
       String dbFileKey = omMetadataManager.getOzonePathKey(parentID, fileName);
       dbOpenFileKey = omMetadataManager.getOpenFileName(parentID, fileName,
               commitKeyRequest.getClientID());
@@ -197,74 +195,4 @@ public class OMKeyCommitRequestV1 extends OMKeyCommitRequest {
 
     return omClientResponse;
   }
-
-
-  /**
-   * Check for directory exists with same name, if it exists throw error.
-   *
-   * @param keyName                  key name
-   * @param ozoneManager             Ozone Manager
-   * @param reachedLastPathComponent true if the path component is a fileName
-   * @throws IOException if directory exists with same name
-   */
-  private void checkDirectoryAlreadyExists(String keyName,
-                                           OzoneManager ozoneManager,
-                                           boolean reachedLastPathComponent)
-          throws IOException {
-    // Reached last component, which would be a file. Returns its parentID.
-    if (reachedLastPathComponent && ozoneManager.getEnableFileSystemPaths()) {
-      throw new OMException("Can not create file: " + keyName +
-              " as there is already directory in the given path", NOT_A_FILE);
-    }
-  }
-
-  /**
-   * Get parent id for the user given path.
-   *
-   * @param bucketId          bucket id
-   * @param pathComponents    fie path elements
-   * @param keyName           user given key name
-   * @param omMetadataManager metadata manager
-   * @return lastKnownParentID
-   * @throws IOException DB failure or parent not exists in DirectoryTable
-   */
-  private long getParentID(long bucketId, Iterator<Path> pathComponents,
-                           String keyName, OMMetadataManager omMetadataManager,
-                           OzoneManager ozoneManager)
-          throws IOException {
-
-    long lastKnownParentId = bucketId;
-
-    // If no sub-dirs then bucketID is the root/parent.
-    if(!pathComponents.hasNext()){
-      return bucketId;
-    }
-
-    OmDirectoryInfo omDirectoryInfo;
-    while (pathComponents.hasNext()) {
-      String nodeName = pathComponents.next().toString();
-      boolean reachedLastPathComponent = !pathComponents.hasNext();
-      String dbNodeName =
-              omMetadataManager.getOzonePathKey(lastKnownParentId, nodeName);
-
-      omDirectoryInfo = omMetadataManager.
-              getDirectoryTable().get(dbNodeName);
-      if (omDirectoryInfo != null) {
-        checkDirectoryAlreadyExists(keyName, ozoneManager,
-                reachedLastPathComponent);
-        lastKnownParentId = omDirectoryInfo.getObjectID();
-      } else {
-        // One of the sub-dir doesn't exists in DB. Immediate parent should
-        // exists for committing the key, otherwise will fail the operation.
-        if (!reachedLastPathComponent) {
-          throw new OMException("Failed to commit key, as parent directory of "
-                  + keyName + " entry is not found in DirectoryTable",
-                  KEY_NOT_FOUND);
-        }
-        break;
-      }
-    }
-
-    return lastKnownParentId;
-  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMAllocateBlockResponse.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMAllocateBlockResponse.java
index 4b20853..c97d702 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMAllocateBlockResponse.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMAllocateBlockResponse.java
@@ -74,4 +74,16 @@ public class OMAllocateBlockResponse extends OMClientResponse {
         omMetadataManager.getBucketKey(omKeyInfo.getVolumeName(),
             omKeyInfo.getBucketName()), omBucketInfo);
   }
+
+  protected OmKeyInfo getOmKeyInfo() {
+    return omKeyInfo;
+  }
+
+  protected OmBucketInfo getOmBucketInfo() {
+    return omBucketInfo;
+  }
+
+  protected long getClientID() {
+    return clientID;
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMAllocateBlockResponse.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMAllocateBlockResponseV1.java
similarity index 59%
copy from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMAllocateBlockResponse.java
copy to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMAllocateBlockResponseV1.java
index 4b20853..ef8b639 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMAllocateBlockResponse.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMAllocateBlockResponseV1.java
@@ -18,60 +18,43 @@
 
 package org.apache.hadoop.ozone.om.response.key;
 
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
 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.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
 
-import java.io.IOException;
 import javax.annotation.Nonnull;
+import java.io.IOException;
 
-import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_KEY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_FILE_TABLE;
 
 /**
- * Response for AllocateBlock request.
+ * Response for AllocateBlock request layout version V1.
  */
-@CleanupTableInfo(cleanupTables = {OPEN_KEY_TABLE})
-public class OMAllocateBlockResponse extends OMClientResponse {
+@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE})
+public class OMAllocateBlockResponseV1 extends OMAllocateBlockResponse {
 
-  private OmKeyInfo omKeyInfo;
-  private long clientID;
-  private OmBucketInfo omBucketInfo;
-
-  public OMAllocateBlockResponse(@Nonnull OMResponse omResponse,
+  public OMAllocateBlockResponseV1(@Nonnull OMResponse omResponse,
       @Nonnull OmKeyInfo omKeyInfo, long clientID,
       @Nonnull OmBucketInfo omBucketInfo) {
-    super(omResponse);
-    this.omKeyInfo = omKeyInfo;
-    this.clientID = clientID;
-    this.omBucketInfo = omBucketInfo;
-  }
-
-  /**
-   * For when the request is not successful.
-   * For a successful request, the other constructor should be used.
-   */
-  public OMAllocateBlockResponse(@Nonnull OMResponse omResponse) {
-    super(omResponse);
-    checkStatusNotOK();
+    super(omResponse, omKeyInfo, clientID, omBucketInfo);
   }
 
   @Override
   public void addToDBBatch(OMMetadataManager omMetadataManager,
       BatchOperation batchOperation) throws IOException {
 
-    String openKey = omMetadataManager.getOpenKey(omKeyInfo.getVolumeName(),
-        omKeyInfo.getBucketName(), omKeyInfo.getKeyName(), clientID);
-    omMetadataManager.getOpenKeyTable().putWithBatch(batchOperation, openKey,
-        omKeyInfo);
+    OMFileRequest.addToOpenFileTable(omMetadataManager, batchOperation,
+            getOmKeyInfo(), getClientID());
 
     // update bucket usedBytes.
     omMetadataManager.getBucketTable().putWithBatch(batchOperation,
-        omMetadataManager.getBucketKey(omKeyInfo.getVolumeName(),
-            omKeyInfo.getBucketName()), omBucketInfo);
+            omMetadataManager.getBucketKey(getOmKeyInfo().getVolumeName(),
+                    getOmKeyInfo().getBucketName()), getOmBucketInfo());
   }
 }
+
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMFileCreateRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMFileCreateRequest.java
index a963f88..0a76589 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMFileCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMFileCreateRequest.java
@@ -398,28 +398,6 @@ public class TestOMFileCreateRequest extends TestOMKeyRequest {
   }
 
   /**
-   * Verify path in open key table. Also, it returns OMKeyInfo for the given
-   * key path.
-   *
-   * @param key      key name
-   * @param id       client id
-   * @param doAssert if true then do assertion, otherwise it just skip.
-   * @return om key info for the given key path.
-   * @throws Exception DB failure
-   */
-  protected OmKeyInfo verifyPathInOpenKeyTable(String key, long id,
-                                               boolean doAssert)
-          throws Exception {
-    String openKey = omMetadataManager.getOpenKey(volumeName, bucketName,
-            key, id);
-    OmKeyInfo omKeyInfo = omMetadataManager.getOpenKeyTable().get(openKey);
-    if (doAssert) {
-      Assert.assertNotNull("Failed to find key in OpenKeyTable", omKeyInfo);
-    }
-    return omKeyInfo;
-  }
-
-  /**
    * Gets OMFileCreateRequest reference.
    *
    * @param omRequest om request
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMFileCreateRequestV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMFileCreateRequestV1.java
index 7ded386..046ac90 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMFileCreateRequestV1.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMFileCreateRequestV1.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
 import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
 import org.apache.hadoop.util.StringUtils;
@@ -183,6 +184,10 @@ public class TestOMFileCreateRequestV1 extends TestOMFileCreateRequest {
   protected OzoneConfiguration getOzoneConfiguration() {
     OzoneConfiguration config = super.getOzoneConfiguration();
     config.set(OMConfigKeys.OZONE_OM_LAYOUT_VERSION, "V1");
+    // omLayoutVersionV1 flag will be set while invoking OzoneManager#start()
+    // and its not invoked in this test. Hence it is explicitly setting
+    // this configuration to populate prefix tables.
+    OzoneManagerRatisUtils.setOmLayoutVersionV1(true);
     return config;
   }
 
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMAllocateBlockRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMAllocateBlockRequest.java
index 4b3d38f..9d26d0d 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMAllocateBlockRequest.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMAllocateBlockRequest.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.UUID;
 
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.jetbrains.annotations.NotNull;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -57,21 +58,19 @@ public class TestOMAllocateBlockRequest extends TestOMKeyRequest {
     TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
         omMetadataManager);
 
-    TestOMRequestUtils.addKeyToTable(true, volumeName, bucketName, keyName,
-        clientID, replicationType, replicationFactor, omMetadataManager);
+    addKeyToOpenKeyTable(volumeName, bucketName);
 
     OMRequest modifiedOmRequest =
         doPreExecute(createAllocateBlockRequest());
 
     OMAllocateBlockRequest omAllocateBlockRequest =
-        new OMAllocateBlockRequest(modifiedOmRequest);
+            getOmAllocateBlockRequest(modifiedOmRequest);
 
     // Check before calling validateAndUpdateCache. As adding DB entry has
     // not added any blocks, so size should be zero.
 
-    OmKeyInfo omKeyInfo =
-        omMetadataManager.getOpenKeyTable().get(omMetadataManager.getOpenKey(
-            volumeName, bucketName, keyName, clientID));
+    OmKeyInfo omKeyInfo = verifyPathInOpenKeyTable(keyName, clientID,
+            true);
 
     List<OmKeyLocationInfo> omKeyLocationInfo =
         omKeyInfo.getLatestVersionLocations().getLocationList();
@@ -87,10 +86,8 @@ public class TestOMAllocateBlockRequest extends TestOMKeyRequest {
 
     // Check open table whether new block is added or not.
 
-    omKeyInfo =
-        omMetadataManager.getOpenKeyTable().get(omMetadataManager.getOpenKey(
-            volumeName, bucketName, keyName, clientID));
-
+    omKeyInfo = verifyPathInOpenKeyTable(keyName, clientID,
+            true);
 
     // Check modification time
     Assert.assertEquals(modifiedOmRequest.getAllocateBlockRequest()
@@ -119,6 +116,12 @@ public class TestOMAllocateBlockRequest extends TestOMKeyRequest {
 
   }
 
+  @NotNull
+  protected OMAllocateBlockRequest getOmAllocateBlockRequest(
+          OMRequest modifiedOmRequest) {
+    return new OMAllocateBlockRequest(modifiedOmRequest);
+  }
+
   @Test
   public void testValidateAndUpdateCacheWithVolumeNotFound() throws Exception {
 
@@ -126,7 +129,7 @@ public class TestOMAllocateBlockRequest extends TestOMKeyRequest {
         doPreExecute(createAllocateBlockRequest());
 
     OMAllocateBlockRequest omAllocateBlockRequest =
-        new OMAllocateBlockRequest(modifiedOmRequest);
+            getOmAllocateBlockRequest(modifiedOmRequest);
 
 
     OMClientResponse omAllocateBlockResponse =
@@ -145,7 +148,7 @@ public class TestOMAllocateBlockRequest extends TestOMKeyRequest {
         doPreExecute(createAllocateBlockRequest());
 
     OMAllocateBlockRequest omAllocateBlockRequest =
-        new OMAllocateBlockRequest(modifiedOmRequest);
+            getOmAllocateBlockRequest(modifiedOmRequest);
 
 
     // Added only volume to DB.
@@ -168,7 +171,7 @@ public class TestOMAllocateBlockRequest extends TestOMKeyRequest {
         doPreExecute(createAllocateBlockRequest());
 
     OMAllocateBlockRequest omAllocateBlockRequest =
-        new OMAllocateBlockRequest(modifiedOmRequest);
+            getOmAllocateBlockRequest(modifiedOmRequest);
 
     // Add volume, bucket entries to DB.
     TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
@@ -190,10 +193,11 @@ public class TestOMAllocateBlockRequest extends TestOMKeyRequest {
    * @return OMRequest - modified request returned from preExecute.
    * @throws Exception
    */
-  private OMRequest doPreExecute(OMRequest originalOMRequest) throws Exception {
+  protected OMRequest doPreExecute(OMRequest originalOMRequest)
+      throws Exception {
 
     OMAllocateBlockRequest omAllocateBlockRequest =
-        new OMAllocateBlockRequest(originalOMRequest);
+            getOmAllocateBlockRequest(originalOMRequest);
 
     OMRequest modifiedOmRequest =
         omAllocateBlockRequest.preExecute(ozoneManager);
@@ -228,7 +232,7 @@ public class TestOMAllocateBlockRequest extends TestOMKeyRequest {
   }
 
 
-  private OMRequest createAllocateBlockRequest() {
+  protected OMRequest createAllocateBlockRequest() {
 
     KeyArgs keyArgs = KeyArgs.newBuilder()
         .setVolumeName(volumeName).setBucketName(bucketName)
@@ -246,4 +250,12 @@ public class TestOMAllocateBlockRequest extends TestOMKeyRequest {
         .setAllocateBlockRequest(allocateBlockRequest).build();
 
   }
+
+  protected String addKeyToOpenKeyTable(String volumeName, String bucketName)
+          throws Exception {
+    TestOMRequestUtils.addKeyToTable(true, volumeName, bucketName,
+            keyName, clientID, replicationType, replicationFactor,
+            omMetadataManager);
+    return "";
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMAllocateBlockRequestV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMAllocateBlockRequestV1.java
new file mode 100644
index 0000000..4e74979
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMAllocateBlockRequestV1.java
@@ -0,0 +1,119 @@
+/**
+ * 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.key;
+
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
+import org.jetbrains.annotations.NotNull;
+import org.junit.Assert;
+
+/**
+ * Tests OMAllocateBlockRequest class layout version V1.
+ */
+public class TestOMAllocateBlockRequestV1 extends TestOMAllocateBlockRequest {
+
+  @NotNull
+  @Override
+  protected OzoneConfiguration getOzoneConfiguration() {
+    OzoneConfiguration config = super.getOzoneConfiguration();
+    config.set(OMConfigKeys.OZONE_OM_LAYOUT_VERSION, "V1");
+    // omLayoutVersionV1 flag will be set while invoking OzoneManager#start()
+    // and its not invoked in this test. Hence it is explicitly setting
+    // this configuration to populate prefix tables.
+    OzoneManagerRatisUtils.setOmLayoutVersionV1(true);
+    return config;
+  }
+
+  protected String addKeyToOpenKeyTable(String volumeName, String bucketName)
+          throws Exception {
+    // need to initialize parentID
+    String parentDir = keyName;
+    String fileName = "file1";
+    keyName = parentDir + OzoneConsts.OM_KEY_PREFIX + fileName;
+
+    // add parentDir to dirTable
+    long parentID = TestOMRequestUtils.addParentsToDirTable(volumeName,
+            bucketName, parentDir, omMetadataManager);
+    long txnId = 50;
+    long objectId = parentID + 1;
+
+    OmKeyInfo omKeyInfoV1 =
+            TestOMRequestUtils.createOmKeyInfo(volumeName, bucketName, keyName,
+                    HddsProtos.ReplicationType.RATIS,
+                    HddsProtos.ReplicationFactor.ONE, objectId, parentID, txnId,
+                    Time.now());
+
+    // add key to openFileTable
+    TestOMRequestUtils.addFileToKeyTable(true, false,
+            fileName, omKeyInfoV1, clientID, txnLogId, omMetadataManager);
+
+    return omMetadataManager.getOzonePathKey(parentID, fileName);
+  }
+
+  @NotNull
+  protected OMAllocateBlockRequest getOmAllocateBlockRequest(
+          OzoneManagerProtocolProtos.OMRequest modifiedOmRequest) {
+    return new OMAllocateBlockRequestV1(modifiedOmRequest);
+  }
+
+  @Override
+  protected OmKeyInfo verifyPathInOpenKeyTable(String key, long id,
+      boolean doAssert) throws Exception {
+    long bucketId = TestOMRequestUtils.getBucketId(volumeName, bucketName,
+            omMetadataManager);
+    String[] pathComponents = StringUtils.split(key, '/');
+    long parentId = bucketId;
+    for (int indx = 0; indx < pathComponents.length; indx++) {
+      String pathElement = pathComponents[indx];
+      // Reached last component, which is file name
+      if (indx == pathComponents.length - 1) {
+        String dbOpenFileName = omMetadataManager.getOpenFileName(
+                parentId, pathElement, id);
+        OmKeyInfo omKeyInfo = omMetadataManager.getOpenKeyTable()
+                .get(dbOpenFileName);
+        if (doAssert) {
+          Assert.assertNotNull("Invalid key!", omKeyInfo);
+        }
+        return omKeyInfo;
+      } else {
+        // directory
+        String dbKey = omMetadataManager.getOzonePathKey(parentId,
+                pathElement);
+        OmDirectoryInfo dirInfo =
+                omMetadataManager.getDirectoryTable().get(dbKey);
+        parentId = dirInfo.getObjectID();
+      }
+    }
+    if (doAssert) {
+      Assert.fail("Invalid key!");
+    }
+    return  null;
+  }
+}
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequestV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequestV1.java
index f5168e1..ed1e2bd 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequestV1.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequestV1.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
 import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
 import org.apache.hadoop.util.Time;
@@ -87,6 +88,10 @@ public class TestOMKeyCommitRequestV1 extends TestOMKeyCommitRequest {
   protected OzoneConfiguration getOzoneConfiguration() {
     OzoneConfiguration config = super.getOzoneConfiguration();
     config.set(OMConfigKeys.OZONE_OM_LAYOUT_VERSION, "V1");
+    // omLayoutVersionV1 flag will be set while invoking OzoneManager#start()
+    // and its not invoked in this test. Hence it is explicitly setting
+    // this configuration to populate prefix tables.
+    OzoneManagerRatisUtils.setOmLayoutVersionV1(true);
     return config;
   }
 
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyDeleteRequestV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyDeleteRequestV1.java
index dbba143..7527e78 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyDeleteRequestV1.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyDeleteRequestV1.java
@@ -18,8 +18,11 @@
 
 package org.apache.hadoop.ozone.om.request.key;
 
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
 import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
 import org.apache.hadoop.util.Time;
@@ -54,4 +57,15 @@ public class TestOMKeyDeleteRequestV1 extends TestOMKeyDeleteRequest {
             fileName, omKeyInfo, -1, 50, omMetadataManager);
     return omKeyInfo.getPath();
   }
+
+  @Override
+  protected OzoneConfiguration getOzoneConfiguration() {
+    OzoneConfiguration config = super.getOzoneConfiguration();
+    config.set(OMConfigKeys.OZONE_OM_LAYOUT_VERSION, "V1");
+    // omLayoutVersionV1 flag will be set while invoking OzoneManager#start()
+    // and its not invoked in this test. Hence it is explicitly setting
+    // this configuration to populate prefix tables.
+    OzoneManagerRatisUtils.setOmLayoutVersionV1(true);
+    return config;
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java
index 4bf66bb..33f58bb 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java
@@ -27,11 +27,13 @@ import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.ozone.om.ResolvedBucket;
 import org.apache.hadoop.ozone.om.KeyManager;
 import org.apache.hadoop.ozone.om.KeyManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
 import org.apache.hadoop.ozone.om.request.OMClientRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
 import org.jetbrains.annotations.NotNull;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
@@ -179,6 +181,29 @@ public class TestOMKeyRequest {
     return new OzoneConfiguration();
   }
 
+
+  /**
+   * Verify path in open key table. Also, it returns OMKeyInfo for the given
+   * key path.
+   *
+   * @param key      key name
+   * @param id       client id
+   * @param doAssert if true then do assertion, otherwise it just skip.
+   * @return om key info for the given key path.
+   * @throws Exception DB failure
+   */
+  protected OmKeyInfo verifyPathInOpenKeyTable(String key, long id,
+                                               boolean doAssert)
+          throws Exception {
+    String openKey = omMetadataManager.getOpenKey(volumeName, bucketName,
+            key, id);
+    OmKeyInfo omKeyInfo = omMetadataManager.getOpenKeyTable().get(openKey);
+    if (doAssert) {
+      Assert.assertNotNull("Failed to find key in OpenKeyTable", omKeyInfo);
+    }
+    return omKeyInfo;
+  }
+
   @After
   public void stop() {
     omMetrics.unRegister();
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMFileCreateResponseV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMFileCreateResponseV1.java
index 19a1bb9..bc4345e 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMFileCreateResponseV1.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMFileCreateResponseV1.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
 import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
 import org.apache.hadoop.ozone.om.response.key.OMKeyCreateResponse;
 import org.apache.hadoop.ozone.om.response.key.TestOMKeyCreateResponse;
@@ -68,6 +69,10 @@ public class TestOMFileCreateResponseV1 extends TestOMKeyCreateResponse {
   protected OzoneConfiguration getOzoneConfiguration() {
     OzoneConfiguration config = super.getOzoneConfiguration();
     config.set(OMConfigKeys.OZONE_OM_LAYOUT_VERSION, "V1");
+    // omLayoutVersionV1 flag will be set while invoking OzoneManager#start()
+    // and its not invoked in this test. Hence it is explicitly setting
+    // this configuration to populate prefix tables.
+    OzoneManagerRatisUtils.setOmLayoutVersionV1(true);
     return config;
   }
 }
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMAllocateBlockResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMAllocateBlockResponse.java
index 602ec99..33c16ae 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMAllocateBlockResponse.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMAllocateBlockResponse.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.om.response.key;
 
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.util.Time;
+import org.jetbrains.annotations.NotNull;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -37,8 +38,7 @@ public class TestOMAllocateBlockResponse extends TestOMKeyResponse {
   @Test
   public void testAddToDBBatch() throws Exception {
 
-    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
-        bucketName, keyName, replicationType, replicationFactor);
+    OmKeyInfo omKeyInfo = createOmKeyInfo();
     OmBucketInfo omBucketInfo = OmBucketInfo.newBuilder()
         .setVolumeName(volumeName).setBucketName(bucketName)
         .setCreationTime(Time.now()).build();
@@ -50,11 +50,9 @@ public class TestOMAllocateBlockResponse extends TestOMKeyResponse {
         .setCmdType(OzoneManagerProtocolProtos.Type.AllocateBlock)
         .build();
     OMAllocateBlockResponse omAllocateBlockResponse =
-        new OMAllocateBlockResponse(omResponse, omKeyInfo, clientID,
-            omBucketInfo);
+            getOmAllocateBlockResponse(omKeyInfo, omBucketInfo, omResponse);
 
-    String openKey = omMetadataManager.getOpenKey(volumeName, bucketName,
-        keyName, clientID);
+    String openKey = getOpenKey();
 
     // Not adding key entry before to test whether commit is successful or not.
     Assert.assertFalse(omMetadataManager.getOpenKeyTable().isExist(openKey));
@@ -68,8 +66,7 @@ public class TestOMAllocateBlockResponse extends TestOMKeyResponse {
 
   @Test
   public void testAddToDBBatchWithErrorResponse() throws Exception {
-    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
-        bucketName, keyName, replicationType, replicationFactor);
+    OmKeyInfo omKeyInfo = createOmKeyInfo();
     OmBucketInfo omBucketInfo = OmBucketInfo.newBuilder()
         .setVolumeName(volumeName).setBucketName(bucketName)
         .setCreationTime(Time.now()).build();
@@ -81,12 +78,10 @@ public class TestOMAllocateBlockResponse extends TestOMKeyResponse {
         .setCmdType(OzoneManagerProtocolProtos.Type.AllocateBlock)
         .build();
     OMAllocateBlockResponse omAllocateBlockResponse =
-        new OMAllocateBlockResponse(omResponse, omKeyInfo, clientID,
-            omBucketInfo);
+            getOmAllocateBlockResponse(omKeyInfo, omBucketInfo, omResponse);
 
     // Before calling addToDBBatch
-    String openKey = omMetadataManager.getOpenKey(volumeName, bucketName,
-        keyName, clientID);
+    String openKey = getOpenKey();
     Assert.assertFalse(omMetadataManager.getOpenKeyTable().isExist(openKey));
 
     omAllocateBlockResponse.checkAndUpdateDB(omMetadataManager, batchOperation);
@@ -98,4 +93,22 @@ public class TestOMAllocateBlockResponse extends TestOMKeyResponse {
     Assert.assertFalse(omMetadataManager.getOpenKeyTable().isExist(openKey));
 
   }
+
+  protected OmKeyInfo createOmKeyInfo() throws Exception {
+    return TestOMRequestUtils.createOmKeyInfo(volumeName,
+            bucketName, keyName, replicationType, replicationFactor);
+  }
+
+  protected String getOpenKey() throws Exception {
+    return omMetadataManager.getOpenKey(volumeName, bucketName,
+            keyName, clientID);
+  }
+
+  @NotNull
+  protected OMAllocateBlockResponse getOmAllocateBlockResponse(
+          OmKeyInfo omKeyInfo, OmBucketInfo omBucketInfo,
+          OMResponse omResponse) {
+    return new OMAllocateBlockResponse(omResponse, omKeyInfo, clientID,
+            omBucketInfo);
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponseV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMAllocateBlockResponseV1.java
similarity index 53%
copy from hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponseV1.java
copy to hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMAllocateBlockResponseV1.java
index 369faa9..e105a37 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponseV1.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMAllocateBlockResponseV1.java
@@ -20,75 +20,54 @@ package org.apache.hadoop.ozone.om.response.key;
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
-import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
 import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
 import org.apache.hadoop.util.Time;
 import org.jetbrains.annotations.NotNull;
-import org.junit.Assert;
 
 /**
- * Tests OMKeyCommitResponse layout version V1.
+ * Tests OMAllocateBlockResponse layout version V1.
  */
-public class TestOMKeyCommitResponseV1 extends TestOMKeyCommitResponse {
+public class TestOMAllocateBlockResponseV1
+        extends TestOMAllocateBlockResponse {
 
-  @NotNull
-  protected OMKeyCommitResponse getOmKeyCommitResponse(
-          OmVolumeArgs omVolumeArgs, OmKeyInfo omKeyInfo,
-          OzoneManagerProtocolProtos.OMResponse omResponse, String openKey,
-          String ozoneKey) {
-    Assert.assertNotNull(omBucketInfo);
-    return new OMKeyCommitResponseV1(
-            omResponse, omKeyInfo, ozoneKey, openKey, omVolumeArgs,
-            omBucketInfo);
-  }
+  // logical ID, which really doesn't exist in dirTable
+  private long parentID = 10;
+  private String fileName = "file1";
 
-  @NotNull
-  @Override
-  protected OmKeyInfo getOmKeyInfo() {
-    Assert.assertNotNull(omBucketInfo);
-    return TestOMRequestUtils.createOmKeyInfo(volumeName,
-            omBucketInfo.getBucketName(), keyName, replicationType,
-            replicationFactor,
-            omBucketInfo.getObjectID() + 1,
-            omBucketInfo.getObjectID(), 100, Time.now());
-  }
+  protected OmKeyInfo createOmKeyInfo() throws Exception {
+    // need to initialize parentID
+    String parentDir = keyName;
+    keyName = parentDir + OzoneConsts.OM_KEY_PREFIX + fileName;
 
-  @NotNull
-  @Override
-  protected void addKeyToOpenKeyTable() throws Exception {
-    Assert.assertNotNull(omBucketInfo);
-    long parentID = omBucketInfo.getObjectID();
-    long objectId = parentID + 10;
+    long txnId = 50;
+    long objectId = parentID + 1;
 
     OmKeyInfo omKeyInfoV1 =
             TestOMRequestUtils.createOmKeyInfo(volumeName, bucketName, keyName,
                     HddsProtos.ReplicationType.RATIS,
-                    HddsProtos.ReplicationFactor.ONE, objectId, parentID, 100,
+                    HddsProtos.ReplicationFactor.ONE, objectId, parentID, txnId,
                     Time.now());
-
-    String fileName = OzoneFSUtils.getFileName(keyName);
-    TestOMRequestUtils.addFileToKeyTable(true, false,
-            fileName, omKeyInfoV1, clientID, txnLogId, omMetadataManager);
+    return omKeyInfoV1;
   }
 
-  @NotNull
-  @Override
-  protected String getOpenKeyName() {
-    Assert.assertNotNull(omBucketInfo);
+  protected String getOpenKey() throws Exception {
     return omMetadataManager.getOpenFileName(
-            omBucketInfo.getObjectID(), keyName, clientID);
+            parentID, fileName, clientID);
   }
 
   @NotNull
-  @Override
-  protected String getOzoneKey() {
-    Assert.assertNotNull(omBucketInfo);
-    return omMetadataManager.getOzonePathKey(omBucketInfo.getObjectID(),
-            keyName);
+  protected OMAllocateBlockResponse getOmAllocateBlockResponse(
+          OmKeyInfo omKeyInfo, OmVolumeArgs omVolumeArgs,
+          OmBucketInfo omBucketInfo, OMResponse omResponse) {
+    return new OMAllocateBlockResponseV1(omResponse, omKeyInfo, clientID,
+            omBucketInfo);
   }
 
   @NotNull
@@ -96,6 +75,11 @@ public class TestOMKeyCommitResponseV1 extends TestOMKeyCommitResponse {
   protected OzoneConfiguration getOzoneConfiguration() {
     OzoneConfiguration config = super.getOzoneConfiguration();
     config.set(OMConfigKeys.OZONE_OM_LAYOUT_VERSION, "V1");
+    // omLayoutVersionV1 flag will be set while invoking OzoneManager#start()
+    // and its not invoked in this test. Hence it is explicitly setting
+    // this configuration to populate prefix tables.
+    OzoneManagerRatisUtils.setOmLayoutVersionV1(true);
     return config;
   }
+
 }
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponseV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponseV1.java
index 369faa9..1e59ce8 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponseV1.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponseV1.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
 import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.util.Time;
@@ -96,6 +97,10 @@ public class TestOMKeyCommitResponseV1 extends TestOMKeyCommitResponse {
   protected OzoneConfiguration getOzoneConfiguration() {
     OzoneConfiguration config = super.getOzoneConfiguration();
     config.set(OMConfigKeys.OZONE_OM_LAYOUT_VERSION, "V1");
+    // omLayoutVersionV1 flag will be set while invoking OzoneManager#start()
+    // and its not invoked in this test. Hence it is explicitly setting
+    // this configuration to populate prefix tables.
+    OzoneManagerRatisUtils.setOmLayoutVersionV1(true);
     return config;
   }
 }
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyDeleteResponseV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyDeleteResponseV1.java
index 3cfec38..d35c79e 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyDeleteResponseV1.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyDeleteResponseV1.java
@@ -18,11 +18,15 @@
 
 package org.apache.hadoop.ozone.om.response.key;
 
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
 import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.util.Time;
+import org.jetbrains.annotations.NotNull;
 import org.junit.Assert;
 
 /**
@@ -67,4 +71,16 @@ public class TestOMKeyDeleteResponseV1 extends TestOMKeyDeleteResponse {
             getOmBucketInfo().getObjectID() + 1,
             getOmBucketInfo().getObjectID(), 100, Time.now());
   }
+
+  @NotNull
+  @Override
+  protected OzoneConfiguration getOzoneConfiguration() {
+    OzoneConfiguration config = super.getOzoneConfiguration();
+    config.set(OMConfigKeys.OZONE_OM_LAYOUT_VERSION, "V1");
+    // omLayoutVersionV1 flag will be set while invoking OzoneManager#start()
+    // and its not invoked in this test. Hence it is explicitly setting
+    // this configuration to populate prefix tables.
+    OzoneManagerRatisUtils.setOmLayoutVersionV1(true);
+    return config;
+  }
 }

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