Просмотр исходного кода

HDDS-1638. Implement Key Write Requests to use Cache and DoubleBuffer. (#956)

Bharat Viswanadham 5 лет назад
Родитель
Сommit
4848280437
52 измененных файлов с 4279 добавлено и 466 удалено
  1. 8 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/Table.java
  2. 5 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TypedTable.java
  3. 0 1
      hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/DummyAction.java
  4. 0 1
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
  5. 13 4
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
  6. 25 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java
  7. 11 2
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
  8. 7 0
      hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
  9. 1 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneHAClusterImpl.java
  10. 2 2
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHA.java
  11. 6 6
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
  12. 52 9
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
  13. 77 3
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
  14. 3 3
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ScmClient.java
  15. 7 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerDoubleBuffer.java
  16. 7 1
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java
  17. 0 114
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java
  18. 15 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java
  19. 1 1
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/OMClientRequest.java
  20. 34 3
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/RequestAuditor.java
  21. 248 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java
  22. 192 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java
  23. 374 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java
  24. 163 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyDeleteRequest.java
  25. 202 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequest.java
  26. 205 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java
  27. 23 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/package-info.java
  28. 59 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMAllocateBlockResponse.java
  29. 66 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponse.java
  30. 61 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCreateResponse.java
  31. 78 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyDeleteResponse.java
  32. 67 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyRenameResponse.java
  33. 23 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/package-info.java
  34. 5 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerHARequestHandlerImpl.java
  35. 36 23
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java
  36. 0 292
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerStateMachine.java
  37. 69 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMRequestUtils.java
  38. 1 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/bucket/TestOMBucketCreateRequest.java
  39. 241 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMAllocateBlockRequest.java
  40. 300 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequest.java
  41. 325 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequest.java
  42. 166 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyDeleteRequest.java
  43. 225 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRenameRequest.java
  44. 151 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java
  45. 23 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/package-info.java
  46. 93 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMAllocateBlockResponse.java
  47. 108 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponse.java
  48. 92 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCreateResponse.java
  49. 163 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyDeleteResponse.java
  50. 148 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyRenameResponse.java
  51. 75 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyResponse.java
  52. 23 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/package-info.java

+ 8 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/Table.java

@@ -123,6 +123,14 @@ public interface Table<KEY, VALUE> extends AutoCloseable {
     throw new NotImplementedException("addCacheEntry is not implemented");
   }
 
+  /**
+   * Get the cache value from table cache.
+   * @param cacheKey
+   */
+  default CacheValue<VALUE> getCacheValue(CacheKey<KEY> cacheKey) {
+    throw new NotImplementedException("getCacheValue is not implemented");
+  }
+
   /**
    * Removes all the entries from the table cache which are having epoch value
    * less

+ 5 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TypedTable.java

@@ -158,6 +158,11 @@ public class TypedTable<KEY, VALUE> implements Table<KEY, VALUE> {
     cache.put(cacheKey, cacheValue);
   }
 
+  @Override
+  public CacheValue<VALUE> getCacheValue(CacheKey<KEY> cacheKey) {
+    return cache.get(cacheKey);
+  }
+
   public Iterator<Map.Entry<CacheKey<KEY>, CacheValue<VALUE>>> cacheIterator() {
     return cache.iterator();
   }

+ 0 - 1
hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/DummyAction.java

@@ -24,7 +24,6 @@ public enum DummyAction implements AuditAction {
 
   CREATE_VOLUME,
   CREATE_BUCKET,
-  CREATE_KEY,
   READ_VOLUME,
   READ_BUCKET,
   READ_KEY,

+ 0 - 1
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java

@@ -30,7 +30,6 @@ public enum OMAction implements AuditAction {
   COMMIT_KEY,
   CREATE_VOLUME,
   CREATE_BUCKET,
-  CREATE_KEY,
   DELETE_VOLUME,
   DELETE_BUCKET,
   DELETE_KEY,

+ 13 - 4
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java

@@ -165,17 +165,21 @@ public final class OmKeyInfo extends WithMetadata {
    * part of the latest version, not a new version.
    *
    * @param newLocationList the list of new blocks to be added.
+   * @param updateTime if true, will update modification time.
    * @throws IOException
    */
   public synchronized void appendNewBlocks(
-      List<OmKeyLocationInfo> newLocationList) throws IOException {
+      List<OmKeyLocationInfo> newLocationList, boolean updateTime)
+      throws IOException {
     if (keyLocationVersions.size() == 0) {
       throw new IOException("Appending new block, but no version exist");
     }
     OmKeyLocationInfoGroup currentLatestVersion =
         keyLocationVersions.get(keyLocationVersions.size() - 1);
     currentLatestVersion.appendNewBlocks(newLocationList);
-    setModificationTime(Time.now());
+    if (updateTime) {
+      setModificationTime(Time.now());
+    }
   }
 
   /**
@@ -183,10 +187,12 @@ public final class OmKeyInfo extends WithMetadata {
    * version to the all version list.
    *
    * @param newLocationList the list of new blocks to be added.
+   * @param updateTime - if true, updates modification time.
    * @throws IOException
    */
   public synchronized long addNewVersion(
-      List<OmKeyLocationInfo> newLocationList) throws IOException {
+      List<OmKeyLocationInfo> newLocationList, boolean updateTime)
+      throws IOException {
     long latestVersionNum;
     if (keyLocationVersions.size() == 0) {
       // no version exist, these blocks are the very first version.
@@ -202,7 +208,10 @@ public final class OmKeyInfo extends WithMetadata {
       keyLocationVersions.add(newVersion);
       latestVersionNum = newVersion.getVersion();
     }
-    setModificationTime(Time.now());
+
+    if (updateTime) {
+      setModificationTime(Time.now());
+    }
     return latestVersionNum;
   }
 

+ 25 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java

@@ -23,6 +23,8 @@ import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocation;
 import org.apache.hadoop.security.token.Token;
 
+import java.util.Objects;
+
 /**
  * One key can be too huge to fit in one container. In which case it gets split
  * into a number of subkeys. This class represents one such subkey instance.
@@ -202,4 +204,27 @@ public final class OmKeyLocationInfo {
         ", pipeline=" + pipeline +
         ", createVersion=" + createVersion + '}';
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    OmKeyLocationInfo that = (OmKeyLocationInfo) o;
+    return length == that.length &&
+        offset == that.offset &&
+        createVersion == that.createVersion &&
+        Objects.equals(blockID, that.blockID) &&
+        Objects.equals(token, that.token) &&
+        Objects.equals(pipeline, that.pipeline);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(blockID, length, offset, token, createVersion,
+        pipeline);
+  }
 }

+ 11 - 2
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java

@@ -726,11 +726,20 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
   public OmKeyLocationInfo allocateBlock(OmKeyArgs args, long clientId,
       ExcludeList excludeList) throws IOException {
     AllocateBlockRequest.Builder req = AllocateBlockRequest.newBuilder();
-    KeyArgs keyArgs = KeyArgs.newBuilder()
+    KeyArgs.Builder keyArgs = KeyArgs.newBuilder()
         .setVolumeName(args.getVolumeName())
         .setBucketName(args.getBucketName())
         .setKeyName(args.getKeyName())
-        .setDataSize(args.getDataSize()).build();
+        .setDataSize(args.getDataSize());
+
+    if (args.getFactor() != null) {
+      keyArgs.setFactor(args.getFactor());
+    }
+
+    if (args.getType() != null) {
+      keyArgs.setType(args.getType());
+    }
+
     req.setKeyArgs(keyArgs);
     req.setClientID(clientId);
     req.setExcludeList(excludeList.getProtoBuf());

+ 7 - 0
hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto

@@ -617,6 +617,10 @@ message KeyArgs {
     optional uint32 multipartNumber = 10;
     repeated hadoop.hdds.KeyValue metadata = 11;
     repeated OzoneAclInfo acls = 12;
+    // This will be set when the request is received in pre-Execute. This
+    // value is used in setting creation/modification time depending on the
+    // request type.
+    optional uint64 modificationTime = 13;
 }
 
 message KeyLocation {
@@ -712,6 +716,9 @@ message ListStatusResponse {
 
 message CreateKeyRequest {
     required KeyArgs keyArgs = 1;
+    // Set in OM HA during preExecute step. This way all OM's use same ID in
+    // OM HA.
+    optional uint64 clientID = 2;
 }
 
 message CreateKeyResponse {

+ 1 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneHAClusterImpl.java

@@ -103,7 +103,7 @@ public final class MiniOzoneHAClusterImpl extends MiniOzoneClusterImpl {
   public void stop() {
     for (OzoneManager ozoneManager : ozoneManagers) {
       if (ozoneManager != null) {
-        LOG.info("Stopping the OzoneManager " + ozoneManager.getOMNodId());
+        LOG.info("Stopping the OzoneManager " + ozoneManager.getOMNodeId());
         ozoneManager.stop();
         ozoneManager.join();
       }

+ 2 - 2
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHA.java

@@ -479,7 +479,7 @@ public class TestOzoneManagerHA {
         }
       }
       Assert.assertTrue("There is no OM Client Proxy corresponding to OM " +
-              "node" + cluster.getOzoneManager(i).getOMNodId(),
+              "node" + cluster.getOzoneManager(i).getOMNodeId(),
           omClientProxyExists);
     }
   }
@@ -604,7 +604,7 @@ public class TestOzoneManagerHA {
 
       // Failover to the OM node that the objectStore points to
       omFailoverProxyProvider.performFailoverIfRequired(
-          ozoneManager.getOMNodId());
+          ozoneManager.getOMNodeId());
 
       // A read request should result in the proxyProvider failing over to
       // leader node.

+ 6 - 6
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java

@@ -282,7 +282,7 @@ public class KeyManagerImpl implements KeyManager {
 
     OmKeyLocationInfo omKeyLocationInfo =
         OmKeyLocationInfo.getFromProtobuf(keyLocation);
-    keyInfo.appendNewBlocks(Collections.singletonList(omKeyLocationInfo));
+    keyInfo.appendNewBlocks(Collections.singletonList(omKeyLocationInfo), true);
     keyInfo.updateModifcationTime();
     metadataManager.getOpenKeyTable().put(openKey, keyInfo);
     return omKeyLocationInfo;
@@ -317,7 +317,7 @@ public class KeyManagerImpl implements KeyManager {
     // If om is not managing via ratis, write to db, otherwise write to DB
     // will happen via ratis apply transaction.
     if (!isRatisEnabled) {
-      keyInfo.appendNewBlocks(locationInfos);
+      keyInfo.appendNewBlocks(locationInfos, true);
       keyInfo.updateModifcationTime();
       metadataManager.getOpenKeyTable().put(openKey, keyInfo);
     }
@@ -486,7 +486,7 @@ public class KeyManagerImpl implements KeyManager {
     if (size > 0) {
       List<OmKeyLocationInfo> locationInfos =
           allocateBlock(keyInfo, new ExcludeList(), size);
-      keyInfo.appendNewBlocks(locationInfos);
+      keyInfo.appendNewBlocks(locationInfos, true);
     }
 
     // When OM is not managed via ratis we should write in to Om db in
@@ -509,7 +509,7 @@ public class KeyManagerImpl implements KeyManager {
       // the key already exist, the new blocks will be added as new version
       // when locations.size = 0, the new version will have identical blocks
       // as its previous version
-      keyInfo.addNewVersion(locations);
+      keyInfo.addNewVersion(locations, true);
       keyInfo.setDataSize(size + keyInfo.getDataSize());
     }
     return keyInfo;
@@ -632,8 +632,8 @@ public class KeyManagerImpl implements KeyManager {
       validateBucket(volumeName, bucketName);
       OmKeyInfo keyInfo = metadataManager.getOpenKeyTable().get(openKey);
       if (keyInfo == null) {
-        throw new OMException("Commit a key without corresponding entry " +
-            objectKey, KEY_NOT_FOUND);
+        throw new OMException("Failed to commit key, as " + openKey + "entry " +
+            "is not found in the openKey table", KEY_NOT_FOUND);
       }
       keyInfo.setDataSize(args.getDataSize());
       keyInfo.setModificationTime(Time.now());

+ 52 - 9
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java

@@ -473,12 +473,26 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
     try (TableIterator<String, ? extends KeyValue<String, OmBucketInfo>>
         bucketIter = bucketTable.iterator()) {
       KeyValue<String, OmBucketInfo> kv = bucketIter.seek(volumePrefix);
-      // During iteration from DB, check in mean time if this bucket is not
-      // marked for delete.
-      if (kv != null && kv.getKey().startsWith(volumePrefix) &&
-          bucketTable.get(kv.getKey()) != null) {
-        return false; // we found at least one bucket with this volume prefix.
+
+      if (kv != null) {
+        // Check the entry in db is not marked for delete. This can happen
+        // while entry is marked for delete, but it is not flushed to DB.
+        CacheValue<OmBucketInfo> cacheValue =
+            bucketTable.getCacheValue(new CacheKey(kv.getKey()));
+        if (cacheValue != null) {
+          if (kv.getKey().startsWith(volumePrefix)
+              && cacheValue.getCacheValue() != null) {
+            return false; // we found at least one bucket with this volume
+            // prefix.
+          }
+        } else {
+          if (kv.getKey().startsWith(volumePrefix)) {
+            return false; // we found at least one bucket with this volume
+            // prefix.
+          }
+        }
       }
+
     }
     return true;
   }
@@ -495,14 +509,43 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
   public boolean isBucketEmpty(String volume, String bucket)
       throws IOException {
     String keyPrefix = getBucketKey(volume, bucket);
-    //TODO: When Key ops are converted in to HA model, use cache also to
-    // determine bucket is empty or not.
+
+    // First check in key table cache.
+    Iterator<Map.Entry<CacheKey<String>, CacheValue<OmKeyInfo>>> iterator =
+        ((TypedTable< String, OmKeyInfo>) keyTable).cacheIterator();
+    while (iterator.hasNext()) {
+      Map.Entry< CacheKey<String>, CacheValue<OmKeyInfo>> entry =
+          iterator.next();
+      String key = entry.getKey().getCacheKey();
+      OmKeyInfo omKeyInfo = entry.getValue().getCacheValue();
+      // Making sure that entry is not for delete key request.
+      if (key.startsWith(keyPrefix) && omKeyInfo != null) {
+        return false;
+      }
+    }
     try (TableIterator<String, ? extends KeyValue<String, OmKeyInfo>> keyIter =
         keyTable.iterator()) {
       KeyValue<String, OmKeyInfo> kv = keyIter.seek(keyPrefix);
-      if (kv != null && kv.getKey().startsWith(keyPrefix)) {
-        return false; // we found at least one key with this vol/bucket prefix.
+
+      if (kv != null) {
+        // Check the entry in db is not marked for delete. This can happen
+        // while entry is marked for delete, but it is not flushed to DB.
+        CacheValue<OmKeyInfo> cacheValue =
+            keyTable.getCacheValue(new CacheKey(kv.getKey()));
+        if (cacheValue != null) {
+          if (kv.getKey().startsWith(keyPrefix)
+              && cacheValue.getCacheValue() != null) {
+            return false; // we found at least one key with this vol/bucket
+            // prefix.
+          }
+        } else {
+          if (kv.getKey().startsWith(keyPrefix)) {
+            return false; // we found at least one key with this vol/bucket
+            // prefix.
+          }
+        }
       }
+
     }
     return true;
   }

+ 77 - 3
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java

@@ -33,6 +33,7 @@ import java.util.Collection;
 import java.util.Objects;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -101,6 +102,7 @@ import org.apache.hadoop.ozone.audit.AuditLoggerType;
 import org.apache.hadoop.ozone.audit.AuditMessage;
 import org.apache.hadoop.ozone.audit.Auditor;
 import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.common.Storage.StorageState;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
@@ -170,6 +172,8 @@ import java.util.Timer;
 import java.util.TimerTask;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_BLOCK_TOKEN_ENABLED;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_BLOCK_TOKEN_ENABLED_DEFAULT;
 import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForBlockClients;
 import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
 import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForSecurityProtocol;
@@ -177,9 +181,15 @@ import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY
 import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
 import static org.apache.hadoop.hdds.server.ServerUtils.updateRPCListenAddress;
 import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumCountWithFixedSleep;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_AUTHORIZER_CLASS;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_BLOCKS_MAX;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_BLOCKS_MAX_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
 import static org.apache.hadoop.ozone.OzoneConsts.OM_METRICS_FILE;
 import static org.apache.hadoop.ozone.OzoneConsts.OM_METRICS_TEMP_FILE;
 import static org.apache.hadoop.ozone.OzoneConsts.OM_RATIS_SNAPSHOT_INDEX;
@@ -275,6 +285,13 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   private long maxUserVolumeCount;
 
 
+  private final ScmClient scmClient;
+  private final long scmBlockSize;
+  private final int preallocateBlocksMax;
+  private final boolean grpcBlockTokenEnabled;
+  private final boolean useRatisForReplication;
+
+
   private OzoneManager(OzoneConfiguration conf) throws IOException,
       AuthenticationException {
     super(OzoneVersionInfo.OZONE_VERSION_INFO);
@@ -385,8 +402,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     omRpcServer = getRpcServer(conf);
     omRpcAddress = updateRPCListenAddress(configuration,
         OZONE_OM_ADDRESS_KEY, omNodeRpcAddr, omRpcServer);
-    keyManager = new KeyManagerImpl(
-        new ScmClient(scmBlockClient, scmContainerClient), metadataManager,
+    this.scmClient = new ScmClient(scmBlockClient, scmContainerClient);
+    keyManager = new KeyManagerImpl(scmClient, metadataManager,
         configuration, omStorage.getOmId(), blockTokenMgr, getKmsProvider());
 
     prefixManager = new PrefixManagerImpl(metadataManager);
@@ -404,6 +421,63 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
       accessAuthorizer = null;
     }
     omMetaDir = OmUtils.getOmDbDir(configuration);
+
+    this.scmBlockSize = (long) conf
+        .getStorageSize(OZONE_SCM_BLOCK_SIZE, OZONE_SCM_BLOCK_SIZE_DEFAULT,
+            StorageUnit.BYTES);
+    this.preallocateBlocksMax = conf.getInt(
+        OZONE_KEY_PREALLOCATION_BLOCKS_MAX,
+        OZONE_KEY_PREALLOCATION_BLOCKS_MAX_DEFAULT);
+    this.grpcBlockTokenEnabled = conf.getBoolean(
+        HDDS_BLOCK_TOKEN_ENABLED,
+        HDDS_BLOCK_TOKEN_ENABLED_DEFAULT);
+    this.useRatisForReplication = conf.getBoolean(
+        DFS_CONTAINER_RATIS_ENABLED_KEY, DFS_CONTAINER_RATIS_ENABLED_DEFAULT);
+  }
+
+  /**
+   * Return configuration value of
+   * {@link OzoneConfigKeys#DFS_CONTAINER_RATIS_ENABLED_KEY}.
+   */
+  public boolean shouldUseRatis() {
+    return useRatisForReplication;
+  }
+
+  /**
+   * Return scmClient.
+   */
+  public ScmClient getScmClient() {
+    return scmClient;
+  }
+
+  /**
+   * Return SecretManager for OM.
+   */
+  public OzoneBlockTokenSecretManager getBlockTokenSecretManager() {
+    return blockTokenMgr;
+  }
+
+  /**
+   * Return config value of {@link OzoneConfigKeys#OZONE_SCM_BLOCK_SIZE}.
+   */
+  public long getScmBlockSize() {
+    return scmBlockSize;
+  }
+
+  /**
+   * Return config value of
+   * {@link OzoneConfigKeys#OZONE_KEY_PREALLOCATION_BLOCKS_MAX}.
+   */
+  public int getPreallocateBlocksMax() {
+    return preallocateBlocksMax;
+  }
+
+  /**
+   * Return config value of
+   * {@link HddsConfigKeys#HDDS_BLOCK_TOKEN_ENABLED}.
+   */
+  public boolean isGrpcBlockTokenEnabled() {
+    return grpcBlockTokenEnabled;
   }
 
   /**
@@ -3057,7 +3131,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     OzoneManager.testSecureOmFlag = testSecureOmFlag;
   }
 
-  public String getOMNodId() {
+  public String getOMNodeId() {
     return omNodeDetails.getOMNodeId();
   }
 

+ 3 - 3
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ScmClient.java

@@ -25,8 +25,8 @@ import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
  */
 public class ScmClient {
 
-  private ScmBlockLocationProtocol blockClient;
-  private StorageContainerLocationProtocol containerClient;
+  private final ScmBlockLocationProtocol blockClient;
+  private final StorageContainerLocationProtocol containerClient;
 
   ScmClient(ScmBlockLocationProtocol blockClient,
             StorageContainerLocationProtocol containerClient) {
@@ -34,7 +34,7 @@ public class ScmClient {
     this.blockClient = blockClient;
   }
 
-  ScmBlockLocationProtocol getBlockClient() {
+  public ScmBlockLocationProtocol getBlockClient() {
     return this.blockClient;
   }
 

+ 7 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerDoubleBuffer.java

@@ -168,6 +168,13 @@ public class OzoneManagerDoubleBuffer {
     omMetadataManager.getBucketTable().cleanupCache(lastRatisTransactionIndex);
     omMetadataManager.getVolumeTable().cleanupCache(lastRatisTransactionIndex);
     omMetadataManager.getUserTable().cleanupCache(lastRatisTransactionIndex);
+
+    //TODO: Optimization we can do here is for key transactions we can only
+    // cleanup cache when it is key commit transaction. In this way all
+    // intermediate transactions for a key will be read from in-memory cache.
+    omMetadataManager.getOpenKeyTable().cleanupCache(lastRatisTransactionIndex);
+    omMetadataManager.getKeyTable().cleanupCache(lastRatisTransactionIndex);
+    omMetadataManager.getDeletedTable().cleanupCache(lastRatisTransactionIndex);
   }
 
   /**

+ 7 - 1
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java

@@ -569,7 +569,13 @@ public final class OzoneManagerRatisServer {
       } else if (thisNodeRole.equals(RaftPeerRole.FOLLOWER)) {
         ByteString leaderNodeId = roleInfoProto.getFollowerInfo()
             .getLeaderInfo().getId().getId();
-        RaftPeerId leaderPeerId = RaftPeerId.valueOf(leaderNodeId);
+        // There may be a chance, here we get leaderNodeId as null. For
+        // example, in 3 node OM Ratis, if 2 OM nodes are down, there will
+        // be no leader.
+        RaftPeerId leaderPeerId = null;
+        if (leaderNodeId != null && !leaderNodeId.isEmpty()) {
+          leaderPeerId = RaftPeerId.valueOf(leaderNodeId);
+        }
 
         setServerRole(thisNodeRole, leaderPeerId);
 

+ 0 - 114
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java

@@ -190,10 +190,6 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
       RaftClientRequest raftClientRequest, OMRequest omRequest) {
 
     switch (omRequest.getCmdType()) {
-    case AllocateBlock:
-      return handleAllocateBlock(raftClientRequest, omRequest);
-    case CreateKey:
-      return handleCreateKeyRequest(raftClientRequest, omRequest);
     case InitiateMultiPartUpload:
       return handleInitiateMultipartUpload(raftClientRequest, omRequest);
     default:
@@ -243,116 +239,6 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
         .build();
   }
 
-  /**
-   * Handle createKey Request, which needs a special handling. This request
-   * needs to be executed on the leader, and the response received from this
-   * request we need to create a ApplyKeyRequest and create a
-   * TransactionContext object.
-   */
-  private TransactionContext handleCreateKeyRequest(
-      RaftClientRequest raftClientRequest, OMRequest omRequest) {
-    OMResponse omResponse = handler.handle(omRequest);
-
-    // TODO: if not success should we retry depending on the error if it is
-    //  retriable?
-    if (!omResponse.getSuccess()) {
-      TransactionContext transactionContext = TransactionContext.newBuilder()
-          .setClientRequest(raftClientRequest)
-          .setStateMachine(this)
-          .setServerRole(RaftProtos.RaftPeerRole.LEADER)
-          .build();
-      transactionContext.setException(
-          constructExceptionForFailedRequest(omResponse));
-      return transactionContext;
-    }
-
-    // Get original request
-    OzoneManagerProtocolProtos.CreateKeyRequest createKeyRequest =
-        omRequest.getCreateKeyRequest();
-
-    // Create Applykey Request.
-    OzoneManagerProtocolProtos.ApplyCreateKeyRequest applyCreateKeyRequest =
-        OzoneManagerProtocolProtos.ApplyCreateKeyRequest.newBuilder()
-            .setCreateKeyRequest(createKeyRequest)
-            .setCreateKeyResponse(omResponse.getCreateKeyResponse()).build();
-
-    OMRequest.Builder newOmRequest =
-        OMRequest.newBuilder().setCmdType(
-            OzoneManagerProtocolProtos.Type.ApplyCreateKey)
-            .setApplyCreateKeyRequest(applyCreateKeyRequest)
-            .setClientId(omRequest.getClientId());
-
-    if (omRequest.hasTraceID()) {
-      newOmRequest.setTraceID(omRequest.getTraceID());
-    }
-
-    ByteString messageContent =
-        ByteString.copyFrom(newOmRequest.build().toByteArray());
-
-    return TransactionContext.newBuilder()
-        .setClientRequest(raftClientRequest)
-        .setStateMachine(this)
-        .setServerRole(RaftProtos.RaftPeerRole.LEADER)
-        .setLogData(messageContent)
-        .build();
-  }
-
-  /**
-   * Handle AllocateBlock Request, which needs a special handling. This
-   * request needs to be executed on the leader, where it connects to SCM and
-   * get Block information.
-   * @param raftClientRequest
-   * @param omRequest
-   * @return TransactionContext
-   */
-  private TransactionContext handleAllocateBlock(
-      RaftClientRequest raftClientRequest, OMRequest omRequest) {
-    OMResponse omResponse = handler.handle(omRequest);
-
-    // If request is failed, no need to proceed further.
-    // Setting the exception with omResponse message and code.
-
-    // TODO: the allocate block fails when scm is in safe mode or when scm is
-    //  down, but that error is not correctly received in OM end, once that
-    //  is fixed, we need to see how to handle this failure case or how we
-    //  need to retry or how to handle this scenario. For other errors like
-    //  KEY_NOT_FOUND, we don't need a retry/
-    if (!omResponse.getSuccess()) {
-      TransactionContext transactionContext = TransactionContext.newBuilder()
-          .setClientRequest(raftClientRequest)
-          .setStateMachine(this)
-          .setServerRole(RaftProtos.RaftPeerRole.LEADER)
-          .build();
-      transactionContext.setException(
-          constructExceptionForFailedRequest(omResponse));
-      return transactionContext;
-    }
-
-    // Get original request
-    OzoneManagerProtocolProtos.AllocateBlockRequest allocateBlockRequest =
-        omRequest.getAllocateBlockRequest();
-
-    // Create new AllocateBlockRequest with keyLocation set.
-    OzoneManagerProtocolProtos.AllocateBlockRequest newAllocateBlockRequest =
-        OzoneManagerProtocolProtos.AllocateBlockRequest.newBuilder().
-            mergeFrom(allocateBlockRequest)
-            .setKeyLocation(
-                omResponse.getAllocateBlockResponse().getKeyLocation()).build();
-
-    OMRequest newOmRequest = omRequest.toBuilder().setCmdType(
-        OzoneManagerProtocolProtos.Type.AllocateBlock)
-        .setAllocateBlockRequest(newAllocateBlockRequest).build();
-
-    ByteString messageContent = ByteString.copyFrom(newOmRequest.toByteArray());
-
-    return TransactionContext.newBuilder()
-        .setClientRequest(raftClientRequest)
-        .setStateMachine(this)
-        .setServerRole(RaftProtos.RaftPeerRole.LEADER)
-        .setLogData(messageContent)
-        .build();
-  }
-
   /**
    * Construct IOException message for failed requests in StartTransaction.
    * @param omResponse

+ 15 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java

@@ -23,6 +23,11 @@ import org.apache.hadoop.ozone.om.request.bucket.OMBucketCreateRequest;
 import org.apache.hadoop.ozone.om.request.bucket.OMBucketDeleteRequest;
 import org.apache.hadoop.ozone.om.request.bucket.OMBucketSetPropertyRequest;
 import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.key.OMAllocateBlockRequest;
+import org.apache.hadoop.ozone.om.request.key.OMKeyCommitRequest;
+import org.apache.hadoop.ozone.om.request.key.OMKeyCreateRequest;
+import org.apache.hadoop.ozone.om.request.key.OMKeyDeleteRequest;
+import org.apache.hadoop.ozone.om.request.key.OMKeyRenameRequest;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeCreateRequest;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeDeleteRequest;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeSetOwnerRequest;
@@ -75,6 +80,16 @@ public final class OzoneManagerRatisUtils {
       return new OMBucketDeleteRequest(omRequest);
     case SetBucketProperty:
       return new OMBucketSetPropertyRequest(omRequest);
+    case AllocateBlock:
+      return new OMAllocateBlockRequest(omRequest);
+    case CreateKey:
+      return new OMKeyCreateRequest(omRequest);
+    case CommitKey:
+      return new OMKeyCommitRequest(omRequest);
+    case DeleteKey:
+      return new OMKeyDeleteRequest(omRequest);
+    case RenameKey:
+      return new OMKeyRenameRequest(omRequest);
     default:
       // TODO: will update once all request types are implemented.
       return null;

+ 1 - 1
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/OMClientRequest.java

@@ -91,7 +91,7 @@ public abstract class OMClientRequest implements RequestAuditor {
   }
 
   /**
-   * Get User information from the OMRequest.
+   * Get User information which needs to be set in the OMRequest object.
    * @return User Info.
    */
   public OzoneManagerProtocolProtos.UserInfo getUserInfo() {

+ 34 - 3
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/RequestAuditor.java

@@ -18,11 +18,17 @@
 
 package org.apache.hadoop.ozone.om.request;
 
+import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.Map;
 
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.audit.AuditAction;
 import org.apache.hadoop.ozone.audit.AuditMessage;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .KeyArgs;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .UserInfo;
 
 /**
  * Interface for OM Requests to convert to audit objects.
@@ -38,8 +44,7 @@ public interface RequestAuditor {
    * @return
    */
   AuditMessage buildAuditMessage(AuditAction op,
-      Map<String, String> auditMap, Throwable throwable,
-      OzoneManagerProtocolProtos.UserInfo userInfo);
+      Map<String, String> auditMap, Throwable throwable, UserInfo userInfo);
 
   /**
    * Build auditMap with specified volume.
@@ -47,4 +52,30 @@ public interface RequestAuditor {
    * @return auditMap.
    */
   Map<String, String> buildVolumeAuditMap(String volume);
+
+  /**
+   * Build auditMap for KeyArgs.
+   * @param keyArgs
+   */
+  default Map<String, String> buildKeyArgsAuditMap(KeyArgs keyArgs) {
+
+    if (keyArgs == null) {
+      return new HashMap<>(0);
+    } else {
+      Map< String, String > auditMap = new LinkedHashMap<>();
+      auditMap.put(OzoneConsts.VOLUME, keyArgs.getVolumeName());
+      auditMap.put(OzoneConsts.BUCKET, keyArgs.getBucketName());
+      auditMap.put(OzoneConsts.KEY, keyArgs.getKeyName());
+      auditMap.put(OzoneConsts.DATA_SIZE,
+          String.valueOf(keyArgs.getDataSize()));
+      auditMap.put(OzoneConsts.REPLICATION_TYPE,
+          (keyArgs.getType() != null) ? keyArgs.getType().name() : null);
+      auditMap.put(OzoneConsts.REPLICATION_FACTOR,
+          (keyArgs.getFactor() != null) ? keyArgs.getFactor().name() : null);
+      auditMap.put(OzoneConsts.KEY_LOCATION_INFO,
+          (keyArgs.getKeyLocationsList() != null) ?
+              keyArgs.getKeyLocationsList().toString() : null);
+      return auditMap;
+    }
+  }
 }

+ 248 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java

@@ -0,0 +1,248 @@
+/**
+ * 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 java.io.IOException;
+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.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+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.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponse;
+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.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.utils.db.cache.CacheKey;
+import org.apache.hadoop.utils.db.cache.CacheValue;
+
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes
+    .KEY_NOT_FOUND;
+
+/**
+ * Handles allocate block request.
+ */
+public class OMAllocateBlockRequest extends OMClientRequest
+    implements OMKeyRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMAllocateBlockRequest.class);
+
+  public OMAllocateBlockRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+
+    AllocateBlockRequest allocateBlockRequest =
+        getOmRequest().getAllocateBlockRequest();
+
+    Preconditions.checkNotNull(allocateBlockRequest);
+
+    KeyArgs keyArgs = allocateBlockRequest.getKeyArgs();
+
+    ExcludeList excludeList = new ExcludeList();
+    if (allocateBlockRequest.hasExcludeList()) {
+      excludeList =
+          ExcludeList.getFromProtoBuf(allocateBlockRequest.getExcludeList());
+    }
+
+    // TODO: Here we are allocating block with out any check for key exist in
+    //  open table or not and also with out any authorization checks.
+    //  Assumption here is that allocateBlocks with out openKey will be less.
+    //  There is a chance some one can misuse this api to flood allocateBlock
+    //  calls. But currently allocateBlock is internally called from
+    //  BlockOutputStreamEntryPool, so we are fine for now. But if one some
+    //  one uses direct omclient we might be in trouble.
+
+
+    // To allocate atleast one block passing requested size and scmBlockSize
+    // as same value. When allocating block requested size is same as
+    // scmBlockSize.
+    List<OmKeyLocationInfo> omKeyLocationInfoList =
+        allocateBlock(ozoneManager.getScmClient(),
+            ozoneManager.getBlockTokenSecretManager(), keyArgs.getType(),
+            keyArgs.getFactor(), excludeList, ozoneManager.getScmBlockSize(),
+            ozoneManager.getScmBlockSize(),
+            ozoneManager.getPreallocateBlocksMax(),
+            ozoneManager.isGrpcBlockTokenEnabled(), ozoneManager.getOMNodeId());
+
+    // Set modification time
+    KeyArgs.Builder newKeyArgs = keyArgs.toBuilder()
+        .setModificationTime(Time.now());
+
+    AllocateBlockRequest.Builder newAllocatedBlockRequest =
+        AllocateBlockRequest.newBuilder()
+            .setClientID(allocateBlockRequest.getClientID())
+            .setKeyArgs(newKeyArgs);
+
+
+
+    if (allocateBlockRequest.hasExcludeList()) {
+      newAllocatedBlockRequest.setExcludeList(
+          allocateBlockRequest.getExcludeList());
+    }
+
+    // Add allocated block info.
+    newAllocatedBlockRequest.setKeyLocation(
+        omKeyLocationInfoList.get(0).getProtobuf());
+
+    return getOmRequest().toBuilder().setUserInfo(getUserInfo())
+        .setAllocateBlockRequest(newAllocatedBlockRequest).build();
+
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex) {
+
+    OzoneManagerProtocolProtos.AllocateBlockRequest allocateBlockRequest =
+        getOmRequest().getAllocateBlockRequest();
+
+    OzoneManagerProtocolProtos.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.incNumAddAllocateBlockCalls();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    Map<String, String> auditMap = buildKeyArgsAuditMap(keyArgs);
+    auditMap.put(OzoneConsts.CLIENT_ID, String.valueOf(clientID));
+
+    OMResponse.Builder omResponse = OMResponse.newBuilder().setCmdType(
+        OzoneManagerProtocolProtos.Type.AllocateBlock).setStatus(
+        OzoneManagerProtocolProtos.Status.OK).setSuccess(true);
+
+    try {
+      // check Acl
+      if (ozoneManager.getAclsEnabled()) {
+        checkAcls(ozoneManager, OzoneObj.ResourceType.KEY,
+            OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.WRITE,
+            volumeName, bucketName, keyName);
+      }
+    } catch (IOException ex) {
+      LOG.error("AllocateBlock failed for Key: {} in volume/bucket:{}/{}",
+          keyName, bucketName, volumeName, ex);
+      omMetrics.incNumBlockAllocateCallFails();
+      auditLog(auditLogger, buildAuditMessage(OMAction.ALLOCATE_BLOCK, auditMap,
+          ex, getOmRequest().getUserInfo()));
+      return new OMAllocateBlockResponse(null, -1L,
+          createErrorOMResponse(omResponse, ex));
+    }
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    try {
+      validateBucketAndVolume(omMetadataManager, volumeName,
+          bucketName);
+    } catch (IOException ex) {
+      LOG.error("AllocateBlock failed for Key: {} in volume/bucket:{}/{}",
+          keyName, bucketName, volumeName, ex);
+      omMetrics.incNumBlockAllocateCallFails();
+      auditLog(auditLogger, buildAuditMessage(OMAction.ALLOCATE_BLOCK, auditMap,
+          ex, getOmRequest().getUserInfo()));
+      return new OMAllocateBlockResponse(null, -1L,
+          createErrorOMResponse(omResponse, ex));
+    }
+
+    String openKey = omMetadataManager.getOpenKey(
+        volumeName, bucketName, keyName, clientID);
+
+    IOException exception = null;
+    OmKeyInfo omKeyInfo =  null;
+
+    // Here we don't acquire bucket/volume lock because for a single client
+    // allocateBlock is called in serial fashion.
+    try {
+      omKeyInfo = omMetadataManager.getOpenKeyTable().get(openKey);
+      if (omKeyInfo == null) {
+        throw new OMException("Open Key not found " + openKey, KEY_NOT_FOUND);
+      }
+
+      // Append new block
+      omKeyInfo.appendNewBlocks(Collections.singletonList(
+          OmKeyLocationInfo.getFromProtobuf(blockLocation)), false);
+
+      // Set modification time.
+      omKeyInfo.setModificationTime(keyArgs.getModificationTime());
+
+      // Add to cache.
+      omMetadataManager.getOpenKeyTable().addCacheEntry(
+          new CacheKey<>(openKey), new CacheValue<>(Optional.of(omKeyInfo),
+              transactionLogIndex));
+
+    } catch (IOException ex) {
+      exception = ex;
+    }
+
+    auditLog(auditLogger, buildAuditMessage(OMAction.ALLOCATE_BLOCK, auditMap,
+        exception, getOmRequest().getUserInfo()));
+
+
+    if (exception == null) {
+      omResponse.setAllocateBlockResponse(AllocateBlockResponse.newBuilder()
+          .setKeyLocation(blockLocation).build());
+      return new OMAllocateBlockResponse(omKeyInfo, clientID,
+          omResponse.build());
+    } else {
+      omMetrics.incNumBlockAllocateCallFails();
+      return new OMAllocateBlockResponse(null, -1L,
+          createErrorOMResponse(omResponse, exception));
+    }
+
+  }
+
+}

+ 192 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCommitRequest.java

@@ -0,0 +1,192 @@
+/**
+ * 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 java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyCommitResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyCreateResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .CommitKeyRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .CommitKeyResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .KeyArgs;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMRequest;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.utils.db.cache.CacheKey;
+import org.apache.hadoop.utils.db.cache.CacheValue;
+
+
+
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+
+/**
+ * Handles CommitKey request.
+ */
+public class OMKeyCommitRequest extends OMClientRequest
+    implements OMKeyRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMKeyCommitRequest.class);
+
+  public OMKeyCommitRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+    CommitKeyRequest commitKeyRequest = getOmRequest().getCommitKeyRequest();
+    Preconditions.checkNotNull(commitKeyRequest);
+
+    KeyArgs keyArgs = commitKeyRequest.getKeyArgs();
+
+    KeyArgs.Builder newKeyArgs =
+        keyArgs.toBuilder().setModificationTime(Time.now());
+
+    return getOmRequest().toBuilder()
+        .setCommitKeyRequest(commitKeyRequest.toBuilder()
+            .setKeyArgs(newKeyArgs)).setUserInfo(getUserInfo()).build();
+
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex) {
+
+    CommitKeyRequest commitKeyRequest = getOmRequest().getCommitKeyRequest();
+
+    KeyArgs commitKeyArgs = commitKeyRequest.getKeyArgs();
+
+    String volumeName = commitKeyArgs.getVolumeName();
+    String bucketName = commitKeyArgs.getBucketName();
+    String keyName = commitKeyArgs.getKeyName();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumKeyCommits();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    Map<String, String> auditMap = buildKeyArgsAuditMap(commitKeyArgs);
+
+    OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OzoneManagerProtocolProtos.OMResponse.newBuilder().setCmdType(
+            OzoneManagerProtocolProtos.Type.CommitKey).setStatus(
+            OzoneManagerProtocolProtos.Status.OK).setSuccess(true);
+
+    try {
+      // check Acl
+      if (ozoneManager.getAclsEnabled()) {
+        checkAcls(ozoneManager, OzoneObj.ResourceType.KEY,
+            OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.WRITE,
+            volumeName, bucketName, keyName);
+      }
+    } catch (IOException ex) {
+      LOG.error("CommitKey failed for Key: {} in volume/bucket:{}/{}",
+          keyName, bucketName, volumeName, ex);
+      omMetrics.incNumKeyCommitFails();
+      auditLog(auditLogger, buildAuditMessage(OMAction.COMMIT_KEY, auditMap,
+          ex, getOmRequest().getUserInfo()));
+      return new OMKeyCreateResponse(null, -1L,
+          createErrorOMResponse(omResponse, ex));
+    }
+
+    List<OmKeyLocationInfo> locationInfoList = commitKeyArgs
+        .getKeyLocationsList().stream().map(OmKeyLocationInfo::getFromProtobuf)
+        .collect(Collectors.toList());
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    String dbOzoneKey = omMetadataManager.getOzoneKey(volumeName, bucketName,
+        keyName);
+    String dbOpenKey = omMetadataManager.getOpenKey(volumeName, bucketName,
+        keyName, commitKeyRequest.getClientID());
+
+    omMetadataManager.getLock().acquireBucketLock(volumeName, bucketName);
+
+    IOException exception = null;
+    OmKeyInfo omKeyInfo = null;
+    try {
+      validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
+      omKeyInfo = omMetadataManager.getOpenKeyTable().get(dbOpenKey);
+      if (omKeyInfo == null) {
+        throw new OMException("Failed to commit key, as " + dbOpenKey +
+            "entry is not found in the openKey table", KEY_NOT_FOUND);
+      }
+      omKeyInfo.setDataSize(commitKeyArgs.getDataSize());
+
+      omKeyInfo.setModificationTime(commitKeyArgs.getModificationTime());
+
+      //update the block length for each block
+      omKeyInfo.updateLocationInfoList(locationInfoList);
+
+      // Add to cache of open key table and key table.
+      omMetadataManager.getOpenKeyTable().addCacheEntry(
+          new CacheKey<>(dbOpenKey),
+          new CacheValue<>(Optional.absent(), transactionLogIndex));
+
+      omMetadataManager.getKeyTable().addCacheEntry(
+          new CacheKey<>(dbOzoneKey),
+          new CacheValue<>(Optional.of(omKeyInfo), transactionLogIndex));
+
+    } catch (IOException ex) {
+      exception = ex;
+    } finally {
+      omMetadataManager.getLock().releaseBucketLock(volumeName, bucketName);
+    }
+
+    // Performing audit logging outside of the lock.
+    auditLog(auditLogger, buildAuditMessage(OMAction.COMMIT_KEY, auditMap,
+        exception, getOmRequest().getUserInfo()));
+
+    // return response after releasing lock.
+    if (exception == null) {
+      omResponse.setCommitKeyResponse(CommitKeyResponse.newBuilder().build());
+      return new OMKeyCommitResponse(omKeyInfo, commitKeyRequest.getClientID(),
+          omResponse.build());
+    } else {
+      omMetrics.incNumKeyCommitFails();
+      return new OMKeyCommitResponse(null, -1L,
+          createErrorOMResponse(omResponse, exception));
+    }
+
+  }
+}

+ 374 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java

@@ -0,0 +1,374 @@
+/**
+ * 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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+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.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyCreateResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .CreateKeyResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .CreateKeyRequest;
+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.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.utils.UniqueId;
+import org.apache.hadoop.utils.db.cache.CacheKey;
+import org.apache.hadoop.utils.db.cache.CacheValue;
+
+/**
+ * Handles CreateKey request.
+ */
+
+public class OMKeyCreateRequest extends OMClientRequest
+    implements OMKeyRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMKeyCreateRequest.class);
+
+  public OMKeyCreateRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+    CreateKeyRequest createKeyRequest = getOmRequest().getCreateKeyRequest();
+    Preconditions.checkNotNull(createKeyRequest);
+
+    KeyArgs keyArgs = createKeyRequest.getKeyArgs();
+
+    // We cannot allocate block for multipart upload part when
+    // createMultipartKey is called, as we will not know type and factor with
+    // which initiateMultipartUpload has started for this key. When
+    // allocateBlock call happen's we shall know type and factor, as we set
+    // the type and factor read from multipart table, and set the KeyInfo in
+    // validateAndUpdateCache and return to the client. TODO: See if we can fix
+    //  this. We do not call allocateBlock in openKey for multipart upload.
+
+    CreateKeyRequest.Builder newCreateKeyRequest = null;
+    KeyArgs.Builder newKeyArgs = null;
+    if (!keyArgs.getIsMultipartKey()) {
+
+      long scmBlockSize = ozoneManager.getScmBlockSize();
+
+      // NOTE size of a key is not a hard limit on anything, it is a value that
+      // client should expect, in terms of current size of key. If client sets
+      // a value, then this value is used, otherwise, we allocate a single
+      // block which is the current size, if read by the client.
+      final long requestedSize = keyArgs.getDataSize() > 0 ?
+          keyArgs.getDataSize() : scmBlockSize;
+
+      boolean useRatis = ozoneManager.shouldUseRatis();
+
+      HddsProtos.ReplicationFactor factor = keyArgs.getFactor();
+      if (factor == null) {
+        factor = useRatis ? HddsProtos.ReplicationFactor.THREE :
+            HddsProtos.ReplicationFactor.ONE;
+      }
+
+      HddsProtos.ReplicationType type = keyArgs.getType();
+      if (type == null) {
+        type = useRatis ? HddsProtos.ReplicationType.RATIS :
+            HddsProtos.ReplicationType.STAND_ALONE;
+      }
+
+      // TODO: Here we are allocating block with out any check for
+      //  bucket/key/volume or not and also with out any authorization checks.
+      //  As for a client for the first time this can be executed on any OM,
+      //  till leader is identified.
+
+      List< OmKeyLocationInfo > omKeyLocationInfoList =
+          allocateBlock(ozoneManager.getScmClient(),
+              ozoneManager.getBlockTokenSecretManager(), type, factor,
+              new ExcludeList(), requestedSize, scmBlockSize,
+              ozoneManager.getPreallocateBlocksMax(),
+              ozoneManager.isGrpcBlockTokenEnabled(),
+              ozoneManager.getOMNodeId());
+
+      newKeyArgs = keyArgs.toBuilder().setModificationTime(Time.now())
+              .setType(type).setFactor(factor)
+              .setDataSize(requestedSize);
+
+      newKeyArgs.addAllKeyLocations(omKeyLocationInfoList.stream()
+          .map(OmKeyLocationInfo::getProtobuf).collect(Collectors.toList()));
+    } else {
+      newKeyArgs = keyArgs.toBuilder().setModificationTime(Time.now());
+    }
+
+    newCreateKeyRequest =
+        createKeyRequest.toBuilder().setKeyArgs(newKeyArgs)
+            .setClientID(UniqueId.next());
+
+    return getOmRequest().toBuilder()
+        .setCreateKeyRequest(newCreateKeyRequest).setUserInfo(getUserInfo())
+        .build();
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex) {
+    CreateKeyRequest createKeyRequest = getOmRequest().getCreateKeyRequest();
+
+
+    KeyArgs keyArgs = createKeyRequest.getKeyArgs();
+
+
+    String volumeName = keyArgs.getVolumeName();
+    String bucketName = keyArgs.getBucketName();
+    String keyName = keyArgs.getKeyName();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumKeyAllocates();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    Map<String, String> auditMap = buildKeyArgsAuditMap(keyArgs);
+
+    OMResponse.Builder omResponse = OMResponse.newBuilder().setCmdType(
+            OzoneManagerProtocolProtos.Type.CreateKey).setStatus(
+            OzoneManagerProtocolProtos.Status.OK).setSuccess(true);
+
+    try {
+      // check Acl
+      if (ozoneManager.getAclsEnabled()) {
+        checkAcls(ozoneManager, OzoneObj.ResourceType.KEY,
+            OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.WRITE,
+            volumeName, bucketName, keyName);
+      }
+    } catch (IOException ex) {
+      LOG.error("Open failed for Key: {} in volume/bucket:{}/{}",
+          keyName, bucketName, volumeName, ex);
+      omMetrics.incNumKeyAllocateFails();
+      auditLog(auditLogger, buildAuditMessage(OMAction.ALLOCATE_KEY, auditMap,
+          ex, getOmRequest().getUserInfo()));
+      return new OMKeyCreateResponse(null, -1L,
+          createErrorOMResponse(omResponse, ex));
+    }
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    String dbOpenKeyName = omMetadataManager.getOpenKey(volumeName,
+        bucketName, keyName, createKeyRequest.getClientID());
+    String dbKeyName = omMetadataManager.getOzoneKey(volumeName, bucketName,
+        keyName);
+    String dbBucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+
+    OmKeyInfo omKeyInfo = null;
+    final List< OmKeyLocationInfo > locations = new ArrayList<>();
+    FileEncryptionInfo encryptionInfo = null;
+    long openVersion = 0L;
+    IOException exception = null;
+    omMetadataManager.getLock().acquireBucketLock(volumeName, bucketName);
+    try {
+      validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
+      //TODO: We can optimize this get here, if getKmsProvider is null, then
+      // bucket encryptionInfo will be not set. If this assumption holds
+      // true, we can avoid get from bucket table.
+      OmBucketInfo bucketInfo =
+          omMetadataManager.getBucketTable().get(dbBucketKey);
+      encryptionInfo = getFileEncryptionInfo(ozoneManager, bucketInfo);
+      omKeyInfo = prepareKeyInfo(omMetadataManager, keyArgs, dbKeyName,
+          keyArgs.getDataSize(), locations, encryptionInfo);
+    } catch (IOException ex) {
+      LOG.error("Key open failed for volume:{} bucket:{} key:{}",
+          volumeName, bucketName, keyName, ex);
+      exception = ex;
+    } finally {
+      omMetadataManager.getLock().releaseBucketLock(volumeName, bucketName);
+    }
+
+
+    if (exception == null) {
+      if (omKeyInfo == null) {
+        // the key does not exist, create a new object, the new blocks are the
+        // version 0
+        omKeyInfo = createKeyInfo(keyArgs, locations, keyArgs.getFactor(),
+            keyArgs.getType(), keyArgs.getDataSize(),
+            encryptionInfo);
+      }
+
+      openVersion = omKeyInfo.getLatestVersionLocations().getVersion();
+
+      try {
+        omKeyInfo.appendNewBlocks(keyArgs.getKeyLocationsList().stream()
+            .map(OmKeyLocationInfo::getFromProtobuf)
+            .collect(Collectors.toList()), false);
+
+      } catch (IOException ex) {
+        LOG.error("Open failed for Key: {} in volume/bucket:{}/{}",
+            keyName, bucketName, volumeName, ex);
+        omMetrics.incNumKeyAllocateFails();
+        auditLog(auditLogger, buildAuditMessage(OMAction.ALLOCATE_KEY, auditMap,
+            ex, getOmRequest().getUserInfo()));
+        return new OMKeyCreateResponse(null, -1L,
+            createErrorOMResponse(omResponse, ex));
+      }
+
+      // Add to cache entry can be done outside of lock for this openKey.
+      // Even if bucket gets deleted, when commitKey we shall identify if
+      // bucket gets deleted.
+      omMetadataManager.getOpenKeyTable().addCacheEntry(
+          new CacheKey<>(dbOpenKeyName),
+          new CacheValue<>(Optional.of(omKeyInfo), transactionLogIndex));
+
+      LOG.debug("Key {} allocated in volume/bucket: {}/{}", keyName, volumeName,
+          bucketName);
+
+      auditLog(auditLogger, buildAuditMessage(OMAction.ALLOCATE_KEY, auditMap,
+          exception, getOmRequest().getUserInfo()));
+
+      long clientID = createKeyRequest.getClientID();
+
+      omResponse.setCreateKeyResponse(CreateKeyResponse.newBuilder()
+          .setKeyInfo(omKeyInfo.getProtobuf())
+          .setID(clientID).setOpenVersion(openVersion)
+          .build());
+
+      return new OMKeyCreateResponse(omKeyInfo, clientID, omResponse.build());
+
+    } else {
+      auditLog(auditLogger, buildAuditMessage(OMAction.ALLOCATE_KEY, auditMap,
+          exception, getOmRequest().getUserInfo()));
+      LOG.error("Open failed for Key: {} in volume/bucket:{}/{}",
+          keyName, bucketName, volumeName, exception);
+      omMetrics.incNumKeyAllocateFails();
+      return new OMKeyCreateResponse(null, -1L,
+          createErrorOMResponse(omResponse, exception));
+    }
+  }
+
+  private OmKeyInfo prepareKeyInfo(OMMetadataManager omMetadataManager,
+      KeyArgs keyArgs, String dbKeyName, long size,
+      List<OmKeyLocationInfo> locations, FileEncryptionInfo encInfo)
+      throws IOException {
+    OmKeyInfo keyInfo = null;
+    if (keyArgs.getIsMultipartKey()) {
+      keyInfo = prepareMultipartKeyInfo(omMetadataManager, keyArgs, size,
+          locations, encInfo);
+      //TODO args.getMetadata
+    } else if (omMetadataManager.getKeyTable().isExist(dbKeyName)) {
+      // TODO: Need to be fixed, as when key already exists, we are
+      //  appending new blocks to existing key.
+      keyInfo = omMetadataManager.getKeyTable().get(dbKeyName);
+      // the key already exist, the new blocks will be added as new version
+      // when locations.size = 0, the new version will have identical blocks
+      // as its previous version
+      keyInfo.addNewVersion(locations, false);
+      keyInfo.setDataSize(size + keyInfo.getDataSize());
+      // The modification time is set in preExecute, use the same as
+      // modification time when key already exists.
+      keyInfo.setModificationTime(keyArgs.getModificationTime());
+    }
+    return keyInfo;
+  }
+
+  private OmKeyInfo prepareMultipartKeyInfo(OMMetadataManager omMetadataManager,
+      KeyArgs args, long size, List<OmKeyLocationInfo> locations,
+      FileEncryptionInfo encInfo) throws IOException {
+    HddsProtos.ReplicationFactor factor;
+    HddsProtos.ReplicationType type;
+
+    Preconditions.checkArgument(args.getMultipartNumber() > 0,
+        "PartNumber Should be greater than zero");
+    // When key is multipart upload part key, we should take replication
+    // type and replication factor from original key which has done
+    // initiate multipart upload. If we have not found any such, we throw
+    // error no such multipart upload.
+    String uploadID = args.getMultipartUploadID();
+    Preconditions.checkNotNull(uploadID);
+    String multipartKey = omMetadataManager
+        .getMultipartKey(args.getVolumeName(), args.getBucketName(),
+            args.getKeyName(), uploadID);
+    OmKeyInfo partKeyInfo = omMetadataManager.getOpenKeyTable().get(
+        multipartKey);
+    if (partKeyInfo == null) {
+      throw new OMException("No such Multipart upload is with specified " +
+          "uploadId " + uploadID,
+          OMException.ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR);
+    } else {
+      factor = partKeyInfo.getFactor();
+      type = partKeyInfo.getType();
+    }
+    // For this upload part we don't need to check in KeyTable. As this
+    // is not an actual key, it is a part of the key.
+    return createKeyInfo(args, locations, factor, type, size, encInfo);
+  }
+
+  /**
+   * Create OmKeyInfo object.
+   * @param keyArgs
+   * @param locations
+   * @param factor
+   * @param type
+   * @param size
+   * @param encInfo
+   * @return
+   */
+  private OmKeyInfo createKeyInfo(KeyArgs keyArgs,
+      List<OmKeyLocationInfo> locations, HddsProtos.ReplicationFactor factor,
+      HddsProtos.ReplicationType type, long size, FileEncryptionInfo encInfo) {
+    OmKeyInfo.Builder builder = new OmKeyInfo.Builder()
+        .setVolumeName(keyArgs.getVolumeName())
+        .setBucketName(keyArgs.getBucketName())
+        .setKeyName(keyArgs.getKeyName())
+        .setOmKeyLocationInfos(Collections.singletonList(
+            new OmKeyLocationInfoGroup(0, locations)))
+        .setCreationTime(keyArgs.getModificationTime())
+        .setModificationTime(keyArgs.getModificationTime())
+        .setDataSize(size)
+        .setReplicationType(type)
+        .setReplicationFactor(factor)
+        .setFileEncryptionInfo(encInfo);
+    if(keyArgs.getAclsList() != null) {
+      builder.setAcls(keyArgs.getAclsList());
+    }
+    return builder.build();
+  }
+}

+ 163 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyDeleteRequest.java

@@ -0,0 +1,163 @@
+/**
+ * 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 java.io.IOException;
+import java.util.Map;
+
+import com.google.common.base.Optional;
+
+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.OmKeyInfo;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyCreateResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .DeleteKeyRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .DeleteKeyResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMRequest;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.utils.db.cache.CacheKey;
+import org.apache.hadoop.utils.db.cache.CacheValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes
+    .KEY_NOT_FOUND;
+
+/**
+ * Handles DeleteKey request.
+ */
+public class OMKeyDeleteRequest extends OMClientRequest
+    implements OMKeyRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMKeyDeleteRequest.class);
+
+  public OMKeyDeleteRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex) {
+    DeleteKeyRequest deleteKeyRequest = getOmRequest().getDeleteKeyRequest();
+
+    OzoneManagerProtocolProtos.KeyArgs deleteKeyArgs =
+        deleteKeyRequest.getKeyArgs();
+
+    String volumeName = deleteKeyArgs.getVolumeName();
+    String bucketName = deleteKeyArgs.getBucketName();
+    String keyName = deleteKeyArgs.getKeyName();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumKeyDeletes();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+    OzoneManagerProtocolProtos.UserInfo userInfo = getOmRequest().getUserInfo();
+
+    Map<String, String> auditMap = buildKeyArgsAuditMap(deleteKeyArgs);
+
+    OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OzoneManagerProtocolProtos.OMResponse.newBuilder().setCmdType(
+            OzoneManagerProtocolProtos.Type.DeleteKey).setStatus(
+            OzoneManagerProtocolProtos.Status.OK).setSuccess(true);
+
+    try {
+      // check Acl
+      if (ozoneManager.getAclsEnabled()) {
+        checkAcls(ozoneManager, OzoneObj.ResourceType.KEY,
+            OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.DELETE,
+            volumeName, bucketName, keyName);
+      }
+    } catch (IOException ex) {
+      LOG.error("Delete failed for Key: {} in volume/bucket:{}/{}",
+          keyName, bucketName, volumeName, ex);
+      omMetrics.incNumKeyDeleteFails();
+      auditLog(auditLogger, buildAuditMessage(OMAction.DELETE_KEY, auditMap,
+          ex, userInfo));
+      return new OMKeyCreateResponse(null, -1L,
+          createErrorOMResponse(omResponse, ex));
+    }
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+
+    String objectKey = omMetadataManager.getOzoneKey(
+        volumeName, bucketName, keyName);
+
+    omMetadataManager.getLock().acquireBucketLock(volumeName, bucketName);
+    IOException exception = null;
+    OmKeyInfo omKeyInfo = null;
+    try {
+
+      // Not doing bucket/volume checks here. In this way we can avoid db
+      // checks for them.
+      // TODO: Once we have volume/bucket full cache, we can add
+      // them back, as these checks will be inexpensive at that time.
+      omKeyInfo = omMetadataManager.getKeyTable().get(objectKey);
+
+      if (omKeyInfo == null) {
+        throw new OMException("Key not found", KEY_NOT_FOUND);
+      }
+
+      // Update table cache.
+      omMetadataManager.getKeyTable().addCacheEntry(
+          new CacheKey<>(omMetadataManager.getOzoneKey(volumeName, bucketName,
+              keyName)),
+          new CacheValue<>(Optional.absent(), transactionLogIndex));
+
+      // No need to add cache entries to delete table. As delete table will
+      // be used by DeleteKeyService only, not used for any client response
+      // validation, so we don't need to add to cache.
+      // TODO: Revisit if we need it later.
+
+    } catch (IOException ex) {
+      exception = ex;
+    } finally {
+      omMetadataManager.getLock().releaseBucketLock(volumeName, bucketName);
+    }
+
+    // Performing audit logging outside of the lock.
+    auditLog(auditLogger, buildAuditMessage(OMAction.DELETE_KEY, auditMap,
+        exception, userInfo));
+
+    // return response.
+    if (exception == null) {
+      omMetrics.decNumKeys();
+      return new OMKeyDeleteResponse(omKeyInfo,
+          omResponse.setDeleteKeyResponse(
+              DeleteKeyResponse.newBuilder()).build());
+    } else {
+      omMetrics.incNumKeyDeleteFails();
+      return new OMKeyDeleteResponse(null,
+          createErrorOMResponse(omResponse, exception));
+    }
+
+  }
+}

+ 202 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequest.java

@@ -0,0 +1,202 @@
+/**
+ * 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.Optional;
+import com.google.common.base.Preconditions;
+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.OmKeyInfo;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+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
+    .RenameKeyRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .RenameKeyResponse;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.utils.db.Table;
+import org.apache.hadoop.utils.db.cache.CacheKey;
+import org.apache.hadoop.utils.db.cache.CacheValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+
+/**
+ * Handles rename key request.
+ */
+public class OMKeyRenameRequest extends OMClientRequest
+    implements OMKeyRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMKeyRenameRequest.class);
+
+  public OMKeyRenameRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+
+    RenameKeyRequest renameKeyRequest = getOmRequest().getRenameKeyRequest();
+    Preconditions.checkNotNull(renameKeyRequest);
+
+    // Set modification time.
+    KeyArgs.Builder newKeyArgs = renameKeyRequest.getKeyArgs().toBuilder()
+            .setModificationTime(Time.now());
+
+    return getOmRequest().toBuilder()
+        .setRenameKeyRequest(renameKeyRequest.toBuilder()
+            .setKeyArgs(newKeyArgs)).setUserInfo(getUserInfo()).build();
+
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex) {
+
+    RenameKeyRequest renameKeyRequest = getOmRequest().getRenameKeyRequest();
+
+    OzoneManagerProtocolProtos.KeyArgs renameKeyArgs =
+        renameKeyRequest.getKeyArgs();
+
+    String volumeName = renameKeyArgs.getVolumeName();
+    String bucketName = renameKeyArgs.getBucketName();
+    String fromKeyName = renameKeyArgs.getKeyName();
+    String toKeyName = renameKeyRequest.getToKeyName();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumKeyRenames();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    Map<String, String> auditMap = buildKeyArgsAuditMap(renameKeyArgs);
+
+    OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OzoneManagerProtocolProtos.OMResponse.newBuilder().setCmdType(
+            OzoneManagerProtocolProtos.Type.CommitKey).setStatus(
+            OzoneManagerProtocolProtos.Status.OK).setSuccess(true);
+
+    try {
+      if (toKeyName.length() == 0 || fromKeyName.length() == 0) {
+        throw new OMException("Key name is empty",
+            OMException.ResultCodes.INVALID_KEY_NAME);
+      }
+      // check Acl
+      if (ozoneManager.getAclsEnabled()) {
+        checkAcls(ozoneManager, OzoneObj.ResourceType.KEY,
+            OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.WRITE,
+            volumeName, bucketName, fromKeyName);
+      }
+    } catch (IOException ex) {
+      LOG.error(
+          "Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}. "
+              + "Key: {} not found.", volumeName, bucketName, fromKeyName,
+          toKeyName, fromKeyName);
+      omMetrics.incNumKeyRenameFails();
+      auditLog(auditLogger, buildAuditMessage(OMAction.RENAME_KEY, auditMap,
+          ex, getOmRequest().getUserInfo()));
+      return new OMKeyRenameResponse(null, null, null,
+          createErrorOMResponse(omResponse, ex));
+    }
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    omMetadataManager.getLock().acquireBucketLock(volumeName, bucketName);
+
+    IOException exception = null;
+    OmKeyInfo fromKeyValue = null;
+    try {
+
+      // Not doing bucket/volume checks here. In this way we can avoid db
+      // checks for them.
+      // TODO: Once we have volume/bucket full cache, we can add
+      // them back, as these checks will be inexpensive at that time.
+
+      // fromKeyName should exist
+      String fromKey = omMetadataManager.getOzoneKey(
+          volumeName, bucketName, fromKeyName);
+      fromKeyValue = omMetadataManager.getKeyTable().get(fromKey);
+      if (fromKeyValue == null) {
+        // TODO: Add support for renaming open key
+        throw new OMException("Key not found " + fromKey, KEY_NOT_FOUND);
+      }
+
+      // toKeyName should not exist
+      String toKey =
+          omMetadataManager.getOzoneKey(volumeName, bucketName, toKeyName);
+      OmKeyInfo toKeyValue = omMetadataManager.getKeyTable().get(toKey);
+      if (toKeyValue != null) {
+        throw new OMException("Key already exists " + toKeyName,
+            OMException.ResultCodes.KEY_ALREADY_EXISTS);
+      }
+
+      fromKeyValue.setKeyName(toKeyName);
+
+      //Set modification time
+      fromKeyValue.setModificationTime(renameKeyArgs.getModificationTime());
+
+      // Add to cache.
+      // fromKey should be deleted, toKey should be added with newly updated
+      // omKeyInfo.
+      Table<String, OmKeyInfo> keyTable = omMetadataManager.getKeyTable();
+
+      keyTable.addCacheEntry(new CacheKey<>(fromKey),
+          new CacheValue<>(Optional.absent(), transactionLogIndex));
+
+      keyTable.addCacheEntry(new CacheKey<>(toKey),
+          new CacheValue<>(Optional.of(fromKeyValue), transactionLogIndex));
+
+    } catch (IOException ex) {
+      exception = ex;
+    } finally {
+      omMetadataManager.getLock().releaseBucketLock(volumeName, bucketName);
+    }
+
+
+    auditLog(auditLogger, buildAuditMessage(OMAction.RENAME_KEY, auditMap,
+        exception, getOmRequest().getUserInfo()));
+
+    if (exception == null) {
+      return new OMKeyRenameResponse(fromKeyValue, toKeyName, fromKeyName,
+          omResponse.setRenameKeyResponse(
+              RenameKeyResponse.newBuilder()).build());
+    } else {
+      LOG.error(
+          "Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}. "
+              + "Key: {} not found.", volumeName, bucketName, fromKeyName,
+          toKeyName, fromKeyName);
+      return new OMKeyRenameResponse(null, null, null,
+          createErrorOMResponse(omResponse, exception));
+    }
+  }
+}

+ 205 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java

@@ -0,0 +1,205 @@
+/**
+ * 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.crypto.key.KeyProviderCryptoExtension
+    .EncryptedKeyVersion;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.ScmClient;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.BucketEncryptionKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.security.OzoneBlockTokenSecretManager;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes
+    .BUCKET_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes
+    .VOLUME_NOT_FOUND;
+import static org.apache.hadoop.util.Time.monotonicNow;
+
+/**
+ * Interface for key write requests.
+ */
+public interface OMKeyRequest {
+
+  Logger LOG = LoggerFactory.getLogger(OMKeyRequest.class);
+
+  /**
+   * This methods avoids multiple rpc calls to SCM by allocating multiple blocks
+   * in one rpc call.
+   * @throws IOException
+   */
+  @SuppressWarnings("parameternumber")
+  default List< OmKeyLocationInfo > allocateBlock(ScmClient scmClient,
+      OzoneBlockTokenSecretManager secretManager,
+      HddsProtos.ReplicationType replicationType,
+      HddsProtos.ReplicationFactor replicationFactor,
+      ExcludeList excludeList, long requestedSize, long scmBlockSize,
+      int preallocateBlocksMax, boolean grpcBlockTokenEnabled, String omID)
+      throws IOException {
+
+    int numBlocks = Math.min((int) ((requestedSize - 1) / scmBlockSize + 1),
+        preallocateBlocksMax);
+
+    List<OmKeyLocationInfo> locationInfos = new ArrayList<>(numBlocks);
+    String remoteUser = getRemoteUser().getShortUserName();
+    List<AllocatedBlock> allocatedBlocks;
+    try {
+      allocatedBlocks = scmClient.getBlockClient()
+          .allocateBlock(scmBlockSize, numBlocks, replicationType,
+              replicationFactor, omID, excludeList);
+    } catch (SCMException ex) {
+      if (ex.getResult()
+          .equals(SCMException.ResultCodes.SAFE_MODE_EXCEPTION)) {
+        throw new OMException(ex.getMessage(),
+            OMException.ResultCodes.SCM_IN_SAFE_MODE);
+      }
+      throw ex;
+    }
+    for (AllocatedBlock allocatedBlock : allocatedBlocks) {
+      OmKeyLocationInfo.Builder builder = new OmKeyLocationInfo.Builder()
+          .setBlockID(new BlockID(allocatedBlock.getBlockID()))
+          .setLength(scmBlockSize)
+          .setOffset(0)
+          .setPipeline(allocatedBlock.getPipeline());
+      if (grpcBlockTokenEnabled) {
+        builder.setToken(secretManager
+            .generateToken(remoteUser, allocatedBlock.getBlockID().toString(),
+                getAclForUser(remoteUser), scmBlockSize));
+      }
+      locationInfos.add(builder.build());
+    }
+    return locationInfos;
+  }
+
+  /* Optimize ugi lookup for RPC operations to avoid a trip through
+   * UGI.getCurrentUser which is synch'ed.
+   */
+  default UserGroupInformation getRemoteUser() throws IOException {
+    UserGroupInformation ugi = Server.getRemoteUser();
+    return (ugi != null) ? ugi : UserGroupInformation.getCurrentUser();
+  }
+
+  /**
+   * Return acl for user.
+   * @param user
+   *
+   * */
+  default EnumSet< HddsProtos.BlockTokenSecretProto.AccessModeProto>
+      getAclForUser(String user) {
+    // TODO: Return correct acl for user.
+    return EnumSet.allOf(
+        HddsProtos.BlockTokenSecretProto.AccessModeProto.class);
+  }
+
+  /**
+   * Validate bucket and volume exists or not.
+   * @param omMetadataManager
+   * @param volumeName
+   * @param bucketName
+   * @throws IOException
+   */
+  default void validateBucketAndVolume(OMMetadataManager omMetadataManager,
+      String volumeName, String bucketName)
+      throws IOException {
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    // Check if bucket exists
+    if (!omMetadataManager.getBucketTable().isExist(bucketKey)) {
+      String volumeKey = omMetadataManager.getVolumeKey(volumeName);
+      // If the volume also does not exist, we should throw volume not found
+      // exception
+      if (!omMetadataManager.getVolumeTable().isExist(volumeKey)) {
+        throw new OMException("Volume not found " + volumeName,
+            VOLUME_NOT_FOUND);
+      }
+
+      // if the volume exists but bucket does not exist, throw bucket not found
+      // exception
+      throw new OMException("Bucket not found " + bucketName, BUCKET_NOT_FOUND);
+    }
+  }
+
+  default FileEncryptionInfo getFileEncryptionInfo(
+      OzoneManager ozoneManager, OmBucketInfo bucketInfo) throws IOException {
+    FileEncryptionInfo encInfo = null;
+    BucketEncryptionKeyInfo ezInfo = bucketInfo.getEncryptionKeyInfo();
+    if (ezInfo != null) {
+      if (ozoneManager.getKmsProvider() == null) {
+        throw new OMException("Invalid KMS provider, check configuration " +
+            CommonConfigurationKeys.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+            OMException.ResultCodes.INVALID_KMS_PROVIDER);
+      }
+
+      final String ezKeyName = ezInfo.getKeyName();
+      EncryptedKeyVersion edek = generateEDEK(ozoneManager, ezKeyName);
+      encInfo = new FileEncryptionInfo(ezInfo.getSuite(), ezInfo.getVersion(),
+          edek.getEncryptedKeyVersion().getMaterial(),
+          edek.getEncryptedKeyIv(), ezKeyName,
+          edek.getEncryptionKeyVersionName());
+    }
+    return encInfo;
+  }
+
+  default EncryptedKeyVersion generateEDEK(OzoneManager ozoneManager,
+      String ezKeyName) throws IOException {
+    if (ezKeyName == null) {
+      return null;
+    }
+    long generateEDEKStartTime = monotonicNow();
+    EncryptedKeyVersion edek = SecurityUtil.doAsLoginUser(
+        new PrivilegedExceptionAction<EncryptedKeyVersion >() {
+          @Override
+          public EncryptedKeyVersion run() throws IOException {
+            try {
+              return ozoneManager.getKmsProvider()
+                  .generateEncryptedKey(ezKeyName);
+            } catch (GeneralSecurityException e) {
+              throw new IOException(e);
+            }
+          }
+        });
+    long generateEDEKTime = monotonicNow() - generateEDEKStartTime;
+    LOG.debug("generateEDEK takes {} ms", generateEDEKTime);
+    Preconditions.checkNotNull(edek);
+    return edek;
+  }
+
+}

+ 23 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/package-info.java

@@ -0,0 +1,23 @@
+/*
+ * 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 contains classes related to key requests.
+ */
+package org.apache.hadoop.ozone.om.request.key;

+ 59 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMAllocateBlockResponse.java

@@ -0,0 +1,59 @@
+/**
+ * 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.key;
+
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMResponse;
+import org.apache.hadoop.utils.db.BatchOperation;
+
+import java.io.IOException;
+
+/**
+ * Response for AllocateBlock request.
+ */
+public class OMAllocateBlockResponse extends OMClientResponse {
+
+  private final OmKeyInfo omKeyInfo;
+  private final long clientID;
+
+  public OMAllocateBlockResponse(OmKeyInfo omKeyInfo,
+      long clientID, OMResponse omResponse) {
+    super(omResponse);
+    this.omKeyInfo = omKeyInfo;
+    this.clientID = clientID;
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    // For OmResponse with failure, this should do nothing. This method is
+    // not called in failure scenario in OM code.
+    if (getOMResponse().getStatus() == OzoneManagerProtocolProtos.Status.OK) {
+      String openKey = omMetadataManager.getOpenKey(omKeyInfo.getVolumeName(),
+          omKeyInfo.getBucketName(), omKeyInfo.getKeyName(), clientID);
+      omMetadataManager.getOpenKeyTable().putWithBatch(batchOperation, openKey,
+          omKeyInfo);
+    }
+  }
+}

+ 66 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCommitResponse.java

@@ -0,0 +1,66 @@
+/**
+ * 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.key;
+
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.utils.db.BatchOperation;
+
+import java.io.IOException;
+
+/**
+ * Response for CommitKey request.
+ */
+public class OMKeyCommitResponse extends OMClientResponse {
+
+  private OmKeyInfo omKeyInfo;
+  private long openKeySessionID;
+
+  public OMKeyCommitResponse(OmKeyInfo omKeyInfo, long openKeySessionID,
+      OzoneManagerProtocolProtos.OMResponse omResponse) {
+    super(omResponse);
+    this.omKeyInfo = omKeyInfo;
+    this.openKeySessionID = openKeySessionID;
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    // For OmResponse with failure, this should do nothing. This method is
+    // not called in failure scenario in OM code.
+    if (getOMResponse().getStatus() == OzoneManagerProtocolProtos.Status.OK) {
+      String volumeName = omKeyInfo.getVolumeName();
+      String bucketName = omKeyInfo.getBucketName();
+      String keyName = omKeyInfo.getKeyName();
+      String openKey = omMetadataManager.getOpenKey(volumeName,
+          bucketName, keyName, openKeySessionID);
+      String ozoneKey = omMetadataManager.getOzoneKey(volumeName, bucketName,
+          keyName);
+
+      // Delete from open key table and add entry to key table.
+      omMetadataManager.getOpenKeyTable().deleteWithBatch(batchOperation,
+          openKey);
+      omMetadataManager.getKeyTable().putWithBatch(batchOperation, ozoneKey,
+          omKeyInfo);
+    }
+  }
+}

+ 61 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyCreateResponse.java

@@ -0,0 +1,61 @@
+/**
+ * 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.key;
+
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMResponse;
+import org.apache.hadoop.utils.db.BatchOperation;
+
+import java.io.IOException;
+
+/**
+ * Response for CreateKey request.
+ */
+public class OMKeyCreateResponse extends OMClientResponse {
+
+  private OmKeyInfo omKeyInfo;
+  private long openKeySessionID;
+
+  public OMKeyCreateResponse(OmKeyInfo omKeyInfo, long openKeySessionID,
+      OMResponse omResponse) {
+    super(omResponse);
+    this.omKeyInfo = omKeyInfo;
+    this.openKeySessionID = openKeySessionID;
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    // For OmResponse with failure, this should do nothing. This method is
+    // not called in failure scenario in OM code.
+    if (getOMResponse().getStatus() == OzoneManagerProtocolProtos.Status.OK) {
+      String openKey = omMetadataManager.getOpenKey(omKeyInfo.getVolumeName(),
+          omKeyInfo.getBucketName(), omKeyInfo.getKeyName(),
+          openKeySessionID);
+      omMetadataManager.getOpenKeyTable().putWithBatch(batchOperation,
+          openKey, omKeyInfo);
+    }
+  }
+}
+

+ 78 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyDeleteResponse.java

@@ -0,0 +1,78 @@
+/**
+ * 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.key;
+
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMResponse;
+import org.apache.hadoop.utils.db.BatchOperation;
+
+import java.io.IOException;
+
+/**
+ * Response for DeleteKey request.
+ */
+public class OMKeyDeleteResponse extends OMClientResponse {
+  private OmKeyInfo omKeyInfo;
+
+  public OMKeyDeleteResponse(OmKeyInfo omKeyInfo, OMResponse omResponse) {
+    super(omResponse);
+    this.omKeyInfo = omKeyInfo;
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    // For OmResponse with failure, this should do nothing. This method is
+    // not called in failure scenario in OM code.
+    if (getOMResponse().getStatus() == OzoneManagerProtocolProtos.Status.OK) {
+      String ozoneKey = omMetadataManager.getOzoneKey(omKeyInfo.getVolumeName(),
+          omKeyInfo.getBucketName(), omKeyInfo.getKeyName());
+      omMetadataManager.getKeyTable().deleteWithBatch(batchOperation,
+          ozoneKey);
+
+      // If Key is not empty add this to delete table.
+      if (!isKeyEmpty(omKeyInfo)) {
+        omMetadataManager.getDeletedTable().putWithBatch(batchOperation,
+            ozoneKey, omKeyInfo);
+      }
+    }
+  }
+
+  /**
+   * Check if the key is empty or not. Key will be empty if it does not have
+   * blocks.
+   * @param keyInfo
+   * @return if empty true, else false.
+   */
+  private boolean isKeyEmpty(OmKeyInfo keyInfo) {
+    for (OmKeyLocationInfoGroup keyLocationList : keyInfo
+        .getKeyLocationVersions()) {
+      if (keyLocationList.getLocationList().size() != 0) {
+        return false;
+      }
+    }
+    return true;
+  }
+}

+ 67 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyRenameResponse.java

@@ -0,0 +1,67 @@
+/**
+ * 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.key;
+
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMResponse;
+import org.apache.hadoop.utils.db.BatchOperation;
+
+import java.io.IOException;
+
+/**
+ * Response for RenameKey request.
+ */
+public class OMKeyRenameResponse extends OMClientResponse {
+
+  private final OmKeyInfo renameKeyInfo;
+  private final String toKeyName;
+  private final String fromKeyName;
+
+  public OMKeyRenameResponse(OmKeyInfo renameKeyInfo, String toKeyName,
+      String fromKeyName, OMResponse omResponse) {
+    super(omResponse);
+    this.renameKeyInfo = renameKeyInfo;
+    this.toKeyName = toKeyName;
+    this.fromKeyName = fromKeyName;
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+    // For OmResponse with failure, this should do nothing. This method is
+    // not called in failure scenario in OM code.
+    if (getOMResponse().getStatus() == OzoneManagerProtocolProtos.Status.OK) {
+
+      // If both from and toKeyName are equal do nothing
+      if (!toKeyName.equals(fromKeyName)) {
+        String volumeName = renameKeyInfo.getVolumeName();
+        String bucketName = renameKeyInfo.getBucketName();
+        omMetadataManager.getKeyTable().deleteWithBatch(batchOperation,
+            omMetadataManager.getOzoneKey(volumeName, bucketName, fromKeyName));
+        omMetadataManager.getKeyTable().putWithBatch(batchOperation,
+            omMetadataManager.getOzoneKey(volumeName, bucketName, toKeyName),
+            renameKeyInfo);
+      }
+    }
+  }
+}

+ 23 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/package-info.java

@@ -0,0 +1,23 @@
+/*
+ * 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 contains classes related to key responses.
+ */
+package org.apache.hadoop.ozone.om.response.key;

+ 5 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerHARequestHandlerImpl.java

@@ -103,6 +103,11 @@ public class OzoneManagerHARequestHandlerImpl
     case CreateBucket:
     case DeleteBucket:
     case SetBucketProperty:
+    case AllocateBlock:
+    case CreateKey:
+    case CommitKey:
+    case DeleteKey:
+    case RenameKey:
       //TODO: We don't need to pass transactionID, this will be removed when
       // complete write requests is changed to new model. And also we can
       // return OMClientResponse, then adding to doubleBuffer can be taken

+ 36 - 23
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java

@@ -86,17 +86,26 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements
           return submitReadRequestToOM(request);
         } else {
           // PreExecute if needed.
-          try {
-            OMClientRequest omClientRequest =
-                OzoneManagerRatisUtils.createClientRequest(request);
-            if (omClientRequest != null) {
-              request = omClientRequest.preExecute(ozoneManager);
+          if (omRatisServer.isLeader()) {
+            try {
+              OMClientRequest omClientRequest =
+                  OzoneManagerRatisUtils.createClientRequest(request);
+              if (omClientRequest != null) {
+                request = omClientRequest.preExecute(ozoneManager);
+              }
+            } catch (IOException ex) {
+              // As some of the preExecute returns error. So handle here.
+              return createErrorResponse(request, ex);
             }
-          } catch (IOException ex) {
-            // As some of the preExecute returns error. So handle here.
-            return createErrorResponse(request, ex);
+            return submitRequestToRatis(request);
+          } else {
+            // throw not leader exception. This is being done, so to avoid
+            // unnecessary execution of preExecute on follower OM's. This
+            // will be helpful in the case like where we we reduce the
+            // chance of allocate blocks on follower OM's. Right now our
+            // leader status is updated every 1 second.
+            throw createNotLeaderException();
           }
-          return submitRequestToRatis(request);
         }
       } else {
         return submitRequestDirectlyToOM(request);
@@ -153,24 +162,28 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements
     if (omRatisServer.isLeader()) {
       return handler.handle(request);
     } else {
-      RaftPeerId raftPeerId = omRatisServer.getRaftPeerId();
-      Optional<RaftPeerId> leaderRaftPeerId = omRatisServer
-          .getCachedLeaderPeerId();
+      throw createNotLeaderException();
+    }
+  }
 
-      NotLeaderException notLeaderException;
-      if (leaderRaftPeerId.isPresent()) {
-        notLeaderException = new NotLeaderException(raftPeerId.toString());
-      } else {
-        notLeaderException = new NotLeaderException(
-            raftPeerId.toString(), leaderRaftPeerId.toString());
-      }
+  private ServiceException createNotLeaderException() {
+    RaftPeerId raftPeerId = omRatisServer.getRaftPeerId();
+    Optional<RaftPeerId> leaderRaftPeerId = omRatisServer
+        .getCachedLeaderPeerId();
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(notLeaderException.getMessage());
-      }
+    NotLeaderException notLeaderException;
+    if (leaderRaftPeerId.isPresent()) {
+      notLeaderException = new NotLeaderException(raftPeerId.toString());
+    } else {
+      notLeaderException = new NotLeaderException(
+          raftPeerId.toString(), leaderRaftPeerId.toString());
+    }
 
-      throw new ServiceException(notLeaderException);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(notLeaderException.getMessage());
     }
+
+    return new ServiceException(notLeaderException);
   }
 
   /**

+ 0 - 292
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerStateMachine.java

@@ -1,292 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.ozone.om.ratis;
-
-import org.apache.commons.lang3.RandomUtils;
-import org.apache.hadoop.hdds.HddsConfigKeys;
-import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
-import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.om.OMConfigKeys;
-import org.apache.hadoop.ozone.om.OMMetadataManager;
-import org.apache.hadoop.ozone.om.OMNodeDetails;
-import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
-import org.apache.hadoop.ozone.om.OzoneManager;
-import org.apache.hadoop.ozone.om.exceptions.OMException;
-import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
-import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
-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
-    .KeyArgs;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
-    .OMRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
-    .OMResponse;
-import org.apache.hadoop.ozone.protocolPB.OzoneManagerHARequestHandler;
-import org.apache.hadoop.ozone.protocolPB.OzoneManagerHARequestHandlerImpl;
-import org.apache.ratis.proto.RaftProtos;
-import org.apache.ratis.protocol.ClientId;
-import org.apache.ratis.protocol.Message;
-import org.apache.ratis.protocol.RaftClientRequest;
-import org.apache.ratis.protocol.RaftGroupId;
-import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.statemachine.TransactionContext;
-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.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.UUID;
-
-import static org.mockito.Mockito.when;
-
-/**
- * This class tests OzoneManagerStateMachine.
- */
-public class TestOzoneManagerStateMachine {
-
-  private OzoneConfiguration conf;
-  private OzoneManagerRatisServer omRatisServer;
-  private String omID;
-  private OzoneManagerHARequestHandler requestHandler;
-  private RaftGroupId raftGroupId;
-  private OzoneManagerStateMachine ozoneManagerStateMachine;
-  private OMMetadataManager omMetadataManager;
-  private OzoneManager ozoneManager;
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-
-  @Before
-  public void setup() throws Exception {
-    conf = new OzoneConfiguration();
-    omID = UUID.randomUUID().toString();
-    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS,
-        temporaryFolder.newFolder().toString());
-    int ratisPort = conf.getInt(
-        OMConfigKeys.OZONE_OM_RATIS_PORT_KEY,
-        OMConfigKeys.OZONE_OM_RATIS_PORT_DEFAULT);
-    InetSocketAddress rpcAddress = new InetSocketAddress(
-        InetAddress.getLocalHost(), 0);
-    OMNodeDetails omNodeDetails = new OMNodeDetails.Builder()
-        .setRpcAddress(rpcAddress)
-        .setRatisPort(ratisPort)
-        .setOMNodeId(omID)
-        .setOMServiceId(OzoneConsts.OM_SERVICE_ID_DEFAULT)
-        .build();
-    ozoneManager = Mockito.mock(OzoneManager.class);
-    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
-    ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
-        temporaryFolder.newFolder().getAbsolutePath());
-    omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration);
-    when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager);
-    // Starts a single node Ratis server
-    omRatisServer = OzoneManagerRatisServer.newOMRatisServer(conf, ozoneManager,
-        omNodeDetails, Collections.emptyList());
-
-
-    ozoneManagerStateMachine =
-        new OzoneManagerStateMachine(omRatisServer);
-
-    requestHandler = Mockito.mock(OzoneManagerHARequestHandlerImpl.class);
-    raftGroupId = omRatisServer.getRaftGroup().getGroupId();
-
-    ozoneManagerStateMachine.setHandler(requestHandler);
-    ozoneManagerStateMachine.setRaftGroupId(raftGroupId);
-
-  }
-
-  @Test
-  public void testAllocateBlockRequestWithSuccess() throws Exception {
-
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    String keyName = UUID.randomUUID().toString();
-    long allocateBlockClientId = RandomUtils.nextLong();
-    String clientId = UUID.randomUUID().toString();
-
-
-    OMRequest omRequest = createOmRequestForAllocateBlock(volumeName,
-        bucketName, keyName, allocateBlockClientId, clientId);
-
-    OzoneManagerProtocolProtos.OMResponse omResponse =
-        createOmResponseForAllocateBlock(true);
-
-    when(requestHandler.handle(omRequest)).thenReturn(omResponse);
-
-
-    RaftClientRequest raftClientRequest =
-        new RaftClientRequest(ClientId.randomId(),
-            RaftPeerId.valueOf("random"), raftGroupId, 1,
-            Message.valueOf(
-                OMRatisHelper.convertRequestToByteString(omRequest)),
-            RaftClientRequest.Type.valueOf(
-                RaftProtos.WriteRequestTypeProto.getDefaultInstance()), null);
-
-    TransactionContext transactionContext =
-        ozoneManagerStateMachine.startTransaction(raftClientRequest);
-
-
-    OMRequest newOmRequest = OMRatisHelper.convertByteStringToOMRequest(
-        transactionContext.getStateMachineLogEntry().getLogData());
-
-    Assert.assertTrue(newOmRequest.hasAllocateBlockRequest());
-    checkModifiedOmRequest(omRequest, newOmRequest);
-
-    // Check this keyLocation, and the keyLocation is same as from OmResponse.
-    Assert.assertTrue(newOmRequest.getAllocateBlockRequest().hasKeyLocation());
-
-    Assert.assertEquals(omResponse.getAllocateBlockResponse().getKeyLocation(),
-        newOmRequest.getAllocateBlockRequest().getKeyLocation());
-
-  }
-
-
-  private OMRequest createOmRequestForAllocateBlock(String volumeName,
-      String bucketName, String keyName, long allocateClientId,
-      String clientId) {
-    //Create AllocateBlockRequest
-    AllocateBlockRequest.Builder req = AllocateBlockRequest.newBuilder();
-    KeyArgs keyArgs = KeyArgs.newBuilder()
-        .setVolumeName(volumeName)
-        .setBucketName(bucketName)
-        .setKeyName(keyName)
-        .setDataSize(100).build();
-    req.setKeyArgs(keyArgs);
-    req.setClientID(allocateClientId);
-    req.setExcludeList(HddsProtos.ExcludeListProto.getDefaultInstance());
-    return OMRequest.newBuilder()
-        .setCmdType(OzoneManagerProtocolProtos.Type.AllocateBlock)
-        .setAllocateBlockRequest(req)
-        .setClientId(clientId)
-        .build();
-  }
-
-  private OMResponse createOmResponseForAllocateBlock(boolean status) {
-    OmKeyLocationInfo newLocation = new OmKeyLocationInfo.Builder().setBlockID(
-        new BlockID(RandomUtils.nextLong(),
-            RandomUtils.nextLong()))
-        .setLength(RandomUtils.nextLong())
-        .setOffset(0).setPipeline(
-            Pipeline.newBuilder().setId(PipelineID.randomId())
-                .setType(HddsProtos.ReplicationType.RATIS)
-                .setFactor(HddsProtos.ReplicationFactor.THREE)
-                .setState(Pipeline.PipelineState.OPEN)
-                .setNodes(new ArrayList<>()).build()).build();
-
-    OzoneManagerProtocolProtos.AllocateBlockResponse.Builder resp =
-        OzoneManagerProtocolProtos.AllocateBlockResponse.newBuilder();
-    resp.setKeyLocation(newLocation.getProtobuf());
-
-
-    if (status) {
-      return OzoneManagerProtocolProtos.OMResponse.newBuilder().setSuccess(true)
-          .setAllocateBlockResponse(resp)
-          .setCmdType(OzoneManagerProtocolProtos.Type.AllocateBlock)
-          .setStatus(OzoneManagerProtocolProtos.Status.OK)
-          .setSuccess(status).build();
-    } else {
-      return OzoneManagerProtocolProtos.OMResponse.newBuilder().setSuccess(true)
-          .setAllocateBlockResponse(resp)
-          .setCmdType(OzoneManagerProtocolProtos.Type.AllocateBlock)
-          .setStatus(OzoneManagerProtocolProtos.Status.SCM_IN_SAFE_MODE)
-          .setMessage("Scm in Safe mode")
-          .setSuccess(status).build();
-    }
-
-  }
-  @Test
-  public void testAllocateBlockWithFailure() throws Exception{
-    String volumeName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
-    String keyName = UUID.randomUUID().toString();
-    long allocateBlockClientId = RandomUtils.nextLong();
-    String clientId = UUID.randomUUID().toString();
-
-
-    OMRequest omRequest = createOmRequestForAllocateBlock(volumeName,
-        bucketName, keyName, allocateBlockClientId, clientId);
-
-    OzoneManagerProtocolProtos.OMResponse omResponse =
-        createOmResponseForAllocateBlock(false);
-
-    when(requestHandler.handle(omRequest)).thenReturn(omResponse);
-
-
-    RaftClientRequest raftClientRequest =
-        new RaftClientRequest(ClientId.randomId(),
-            RaftPeerId.valueOf("random"), raftGroupId, 1,
-            Message.valueOf(
-                OMRatisHelper.convertRequestToByteString(omRequest)),
-            RaftClientRequest.Type.valueOf(
-                RaftProtos.WriteRequestTypeProto.getDefaultInstance()), null);
-
-    TransactionContext transactionContext =
-        ozoneManagerStateMachine.startTransaction(raftClientRequest);
-
-
-    OMRequest newOmRequest = OMRatisHelper.convertByteStringToOMRequest(
-        transactionContext.getStateMachineLogEntry().getLogData());
-
-    Assert.assertTrue(newOmRequest.hasAllocateBlockRequest());
-    checkModifiedOmRequest(omRequest, newOmRequest);
-
-    // As the request failed, check for keyLocation and  the transaction
-    // context error message
-    Assert.assertFalse(newOmRequest.getAllocateBlockRequest().hasKeyLocation());
-    Assert.assertEquals("Scm in Safe mode " + OMException.STATUS_CODE
-            + OMException.ResultCodes.SCM_IN_SAFE_MODE,
-        transactionContext.getException().getMessage());
-    Assert.assertTrue(transactionContext.getException() instanceof IOException);
-
-  }
-
-  private void checkModifiedOmRequest(OMRequest omRequest,
-      OMRequest newOmRequest) {
-    Assert.assertTrue(newOmRequest.getAllocateBlockRequest()
-        .getKeyArgs().getBucketName().equals(
-            omRequest.getAllocateBlockRequest().getKeyArgs().getBucketName()));
-    Assert.assertTrue(omRequest.getAllocateBlockRequest()
-        .getKeyArgs().getVolumeName().equals(
-            newOmRequest.getAllocateBlockRequest().getKeyArgs()
-                .getVolumeName()));
-    Assert.assertTrue(omRequest.getAllocateBlockRequest()
-        .getKeyArgs().getKeyName().equals(
-            newOmRequest.getAllocateBlockRequest().getKeyArgs().getKeyName()));
-    Assert.assertEquals(omRequest.getAllocateBlockRequest()
-            .getKeyArgs().getDataSize(),
-        newOmRequest.getAllocateBlockRequest().getKeyArgs().getDataSize());
-    Assert.assertEquals(omRequest.getAllocateBlockRequest()
-            .getClientID(),
-        newOmRequest.getAllocateBlockRequest().getClientID());
-    Assert.assertEquals(omRequest.getClientId(), newOmRequest.getClientId());
-  }
-}

+ 69 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMRequestUtils.java

@@ -21,12 +21,15 @@ package org.apache.hadoop.ozone.om.request;
 
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
 
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 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.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
@@ -64,6 +67,72 @@ public final class TestOMRequestUtils {
         omMetadataManager.getBucketKey(volumeName, bucketName), omBucketInfo);
   }
 
+  /**
+   * Add key entry to KeyTable. if openKeyTable flag is true, add's entries
+   * to openKeyTable, else add's it to keyTable.
+   * @param openKeyTable
+   * @param volumeName
+   * @param bucketName
+   * @param keyName
+   * @param clientID
+   * @param replicationType
+   * @param replicationFactor
+   * @param omMetadataManager
+   * @throws Exception
+   */
+  @SuppressWarnings("parameterNumber")
+  public static void addKeyToTable(boolean openKeyTable, String volumeName,
+      String bucketName,
+      String keyName, long clientID,
+      HddsProtos.ReplicationType replicationType,
+      HddsProtos.ReplicationFactor replicationFactor,
+      OMMetadataManager omMetadataManager) throws Exception {
+
+
+    OmKeyInfo.Builder builder = new OmKeyInfo.Builder()
+        .setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setKeyName(keyName)
+        .setOmKeyLocationInfos(Collections.singletonList(
+            new OmKeyLocationInfoGroup(0, new ArrayList<>())))
+        .setCreationTime(Time.now())
+        .setModificationTime(Time.now())
+        .setDataSize(1000L)
+        .setReplicationType(replicationType)
+        .setReplicationFactor(replicationFactor);
+
+    if (openKeyTable) {
+      omMetadataManager.getOpenKeyTable().put(
+          omMetadataManager.getOpenKey(volumeName, bucketName, keyName,
+              clientID), builder.build());
+    } else {
+      omMetadataManager.getKeyTable().put(omMetadataManager.getOzoneKey(
+          volumeName, bucketName, keyName), builder.build());
+    }
+
+  }
+
+  /**
+   * Create OmKeyInfo.
+   */
+
+  public static OmKeyInfo createOmKeyInfo(String volumeName, String bucketName,
+      String keyName, HddsProtos.ReplicationType replicationType,
+      HddsProtos.ReplicationFactor replicationFactor) {
+    return new OmKeyInfo.Builder()
+        .setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setKeyName(keyName)
+        .setOmKeyLocationInfos(Collections.singletonList(
+            new OmKeyLocationInfoGroup(0, new ArrayList<>())))
+        .setCreationTime(Time.now())
+        .setModificationTime(Time.now())
+        .setDataSize(1000L)
+        .setReplicationType(replicationType)
+        .setReplicationFactor(replicationFactor).build();
+  }
+
+
   /**
    * Add volume creation entry to OM DB.
    * @param volumeName

+ 1 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/bucket/TestOMBucketCreateRequest.java

@@ -69,6 +69,7 @@ public class TestOMBucketCreateRequest {
 
   @Before
   public void setup() throws Exception {
+
     ozoneManager = Mockito.mock(OzoneManager.class);
     omMetrics = OMMetrics.create();
     OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();

+ 241 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMAllocateBlockRequest.java

@@ -0,0 +1,241 @@
+/**
+ * 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 java.util.List;
+import java.util.UUID;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+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
+    .AllocateBlockRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .KeyArgs;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMRequest;
+
+/**
+ * Tests OMAllocateBlockRequest class.
+ */
+public class TestOMAllocateBlockRequest extends TestOMKeyRequest {
+
+  @Test
+  public void testPreExecute() throws Exception {
+
+    doPreExecute(createAllocateBlockRequest());
+
+  }
+
+  @Test
+  public void testValidateAndUpdateCache() throws Exception {
+
+    OMRequest modifiedOmRequest =
+        doPreExecute(createAllocateBlockRequest());
+
+    OMAllocateBlockRequest omAllocateBlockRequest =
+        new OMAllocateBlockRequest(modifiedOmRequest);
+
+
+    // Add volume, bucket, key entries to DB.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+        omMetadataManager);
+
+    TestOMRequestUtils.addKeyToTable(true, volumeName, bucketName, keyName,
+        clientID, replicationType, replicationFactor, omMetadataManager);
+
+    // 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));
+
+    List<OmKeyLocationInfo> omKeyLocationInfo =
+        omKeyInfo.getLatestVersionLocations().getLocationList();
+
+    Assert.assertTrue(omKeyLocationInfo.size() == 0);
+
+    OMClientResponse omAllocateBlockResponse =
+        omAllocateBlockRequest.validateAndUpdateCache(ozoneManager, 100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.OK,
+        omAllocateBlockResponse.getOMResponse().getStatus());
+
+    // Check open table whether new block is added or not.
+
+    omKeyInfo =
+        omMetadataManager.getOpenKeyTable().get(omMetadataManager.getOpenKey(
+            volumeName, bucketName, keyName, clientID));
+
+
+    // Check modification time
+    Assert.assertEquals(modifiedOmRequest.getAllocateBlockRequest()
+        .getKeyArgs().getModificationTime(), omKeyInfo.getModificationTime());
+    Assert.assertNotEquals(omKeyInfo.getCreationTime(),
+        omKeyInfo.getModificationTime());
+
+    // Check data of the block
+    OzoneManagerProtocolProtos.KeyLocation keyLocation =
+        modifiedOmRequest.getAllocateBlockRequest().getKeyLocation();
+
+    omKeyLocationInfo =
+        omKeyInfo.getLatestVersionLocations().getLocationList();
+
+    Assert.assertTrue(omKeyLocationInfo.size() == 1);
+
+    Assert.assertEquals(keyLocation.getBlockID().getContainerBlockID()
+        .getContainerID(), omKeyLocationInfo.get(0).getContainerID());
+
+    Assert.assertEquals(keyLocation.getBlockID().getContainerBlockID()
+            .getLocalID(), omKeyLocationInfo.get(0).getLocalID());
+
+  }
+
+  @Test
+  public void testValidateAndUpdateCacheWithVolumeNotFound() throws Exception {
+
+    OMRequest modifiedOmRequest =
+        doPreExecute(createAllocateBlockRequest());
+
+    OMAllocateBlockRequest omAllocateBlockRequest =
+        new OMAllocateBlockRequest(modifiedOmRequest);
+
+
+    OMClientResponse omAllocateBlockResponse =
+        omAllocateBlockRequest.validateAndUpdateCache(ozoneManager, 100L);
+
+    Assert.assertTrue(omAllocateBlockResponse.getOMResponse().getStatus()
+        == OzoneManagerProtocolProtos.Status.VOLUME_NOT_FOUND);
+
+  }
+
+  @Test
+  public void testValidateAndUpdateCacheWithBucketNotFound() throws Exception {
+
+    OMRequest modifiedOmRequest =
+        doPreExecute(createAllocateBlockRequest());
+
+    OMAllocateBlockRequest omAllocateBlockRequest =
+        new OMAllocateBlockRequest(modifiedOmRequest);
+
+
+    // Added only volume to DB.
+    TestOMRequestUtils.addVolumeToDB(volumeName, "ozone", omMetadataManager);
+
+    OMClientResponse omAllocateBlockResponse =
+        omAllocateBlockRequest.validateAndUpdateCache(ozoneManager, 100L);
+
+    Assert.assertTrue(omAllocateBlockResponse.getOMResponse().getStatus()
+        == OzoneManagerProtocolProtos.Status.BUCKET_NOT_FOUND);
+
+  }
+
+  @Test
+  public void testValidateAndUpdateCacheWithKeyNotFound() throws Exception {
+
+    OMRequest modifiedOmRequest =
+        doPreExecute(createAllocateBlockRequest());
+
+    OMAllocateBlockRequest omAllocateBlockRequest =
+        new OMAllocateBlockRequest(modifiedOmRequest);
+
+    // Add volume, bucket entries to DB.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+        omMetadataManager);
+
+
+    OMClientResponse omAllocateBlockResponse =
+        omAllocateBlockRequest.validateAndUpdateCache(ozoneManager, 100L);
+
+    Assert.assertTrue(omAllocateBlockResponse.getOMResponse().getStatus()
+        == OzoneManagerProtocolProtos.Status.KEY_NOT_FOUND);
+
+  }
+
+  /**
+   * This method calls preExecute and verify the modified request.
+   * @param originalOMRequest
+   * @return OMRequest - modified request returned from preExecute.
+   * @throws Exception
+   */
+  private OMRequest doPreExecute(OMRequest originalOMRequest) throws Exception {
+
+    OMAllocateBlockRequest omAllocateBlockRequest =
+        new OMAllocateBlockRequest(originalOMRequest);
+
+    OMRequest modifiedOmRequest =
+        omAllocateBlockRequest.preExecute(ozoneManager);
+
+
+    Assert.assertEquals(originalOMRequest.getCmdType(),
+        modifiedOmRequest.getCmdType());
+    Assert.assertEquals(originalOMRequest.getClientId(),
+        modifiedOmRequest.getClientId());
+
+    Assert.assertTrue(modifiedOmRequest.hasAllocateBlockRequest());
+    AllocateBlockRequest allocateBlockRequest =
+        modifiedOmRequest.getAllocateBlockRequest();
+    // Time should be set
+    Assert.assertTrue(allocateBlockRequest.getKeyArgs()
+        .getModificationTime() > 0);
+
+    // KeyLocation should be set.
+    Assert.assertTrue(allocateBlockRequest.hasKeyLocation());
+    Assert.assertEquals(containerID,
+        allocateBlockRequest.getKeyLocation().getBlockID()
+            .getContainerBlockID().getContainerID());
+    Assert.assertEquals(localID,
+        allocateBlockRequest.getKeyLocation().getBlockID()
+            .getContainerBlockID().getLocalID());
+    Assert.assertTrue(allocateBlockRequest.getKeyLocation().hasPipeline());
+
+    Assert.assertEquals(allocateBlockRequest.getClientID(),
+        allocateBlockRequest.getClientID());
+
+    return modifiedOmRequest;
+  }
+
+
+  private OMRequest createAllocateBlockRequest() {
+
+    KeyArgs keyArgs = KeyArgs.newBuilder()
+        .setVolumeName(volumeName).setBucketName(bucketName)
+        .setKeyName(keyName)
+        .setFactor(replicationFactor).setType(replicationType)
+        .build();
+
+    AllocateBlockRequest allocateBlockRequest =
+        AllocateBlockRequest.newBuilder().setClientID(clientID)
+            .setKeyArgs(keyArgs).build();
+
+    return OMRequest.newBuilder()
+        .setCmdType(OzoneManagerProtocolProtos.Type.AllocateBlock)
+        .setClientId(UUID.randomUUID().toString())
+        .setAllocateBlockRequest(allocateBlockRequest).build();
+
+  }
+}

+ 300 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCommitRequest.java

@@ -0,0 +1,300 @@
+/**
+ * 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 java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+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
+    .CommitKeyRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .KeyArgs;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .KeyLocation;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMRequest;
+
+
+
+/**
+ * Class tests OMKeyCommitRequest class.
+ */
+public class TestOMKeyCommitRequest extends TestOMKeyRequest {
+
+  @Test
+  public void testPreExecute() throws Exception {
+    doPreExecute(createCommitKeyRequest());
+  }
+
+  @Test
+  public void testValidateAndUpdateCache() throws Exception {
+
+    OMRequest modifiedOmRequest =
+        doPreExecute(createCommitKeyRequest());
+
+    OMKeyCommitRequest omKeyCommitRequest =
+        new OMKeyCommitRequest(modifiedOmRequest);
+
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+        omMetadataManager);
+
+    TestOMRequestUtils.addKeyToTable(true, volumeName, bucketName, keyName,
+        clientID, replicationType, replicationFactor, omMetadataManager);
+
+
+    String ozoneKey = omMetadataManager.getOzoneKey(volumeName, bucketName,
+        keyName);
+
+    // Key should not be there in key table, as validateAndUpdateCache is
+    // still not called.
+    OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable().get(ozoneKey);
+
+    Assert.assertNull(omKeyInfo);
+
+    OMClientResponse omClientResponse =
+        omKeyCommitRequest.validateAndUpdateCache(ozoneManager,
+        100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.OK,
+        omClientResponse.getOMResponse().getStatus());
+
+    // Entry should be deleted from openKey Table.
+    omKeyInfo = omMetadataManager.getOpenKeyTable().get(ozoneKey);
+    Assert.assertNull(omKeyInfo);
+
+    // Now entry should be created in key Table.
+    omKeyInfo = omMetadataManager.getKeyTable().get(ozoneKey);
+
+    Assert.assertNotNull(omKeyInfo);
+
+    // Check modification time
+
+    CommitKeyRequest commitKeyRequest = modifiedOmRequest.getCommitKeyRequest();
+    Assert.assertEquals(commitKeyRequest.getKeyArgs().getModificationTime(),
+        omKeyInfo.getModificationTime());
+
+    // Check block location.
+    List<OmKeyLocationInfo> locationInfoListFromCommitKeyRequest =
+        commitKeyRequest.getKeyArgs()
+        .getKeyLocationsList().stream().map(OmKeyLocationInfo::getFromProtobuf)
+        .collect(Collectors.toList());
+
+    Assert.assertEquals(locationInfoListFromCommitKeyRequest,
+        omKeyInfo.getLatestVersionLocations().getLocationList());
+
+  }
+
+
+
+  @Test
+  public void testValidateAndUpdateCacheWithVolumeNotFound() throws Exception {
+
+    OMRequest modifiedOmRequest =
+        doPreExecute(createCommitKeyRequest());
+
+    OMKeyCommitRequest omKeyCommitRequest =
+        new OMKeyCommitRequest(modifiedOmRequest);
+
+    String ozoneKey = omMetadataManager.getOzoneKey(volumeName, bucketName,
+        keyName);
+
+    // Key should not be there in key table, as validateAndUpdateCache is
+    // still not called.
+    OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable().get(ozoneKey);
+
+    Assert.assertNull(omKeyInfo);
+
+    OMClientResponse omClientResponse =
+        omKeyCommitRequest.validateAndUpdateCache(ozoneManager,
+            100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.VOLUME_NOT_FOUND,
+        omClientResponse.getOMResponse().getStatus());
+
+    omKeyInfo = omMetadataManager.getKeyTable().get(ozoneKey);
+
+    Assert.assertNull(omKeyInfo);
+  }
+
+  @Test
+  public void testValidateAndUpdateCacheWithBucketNotFound() throws Exception {
+
+    OMRequest modifiedOmRequest =
+        doPreExecute(createCommitKeyRequest());
+
+    OMKeyCommitRequest omKeyCommitRequest =
+        new OMKeyCommitRequest(modifiedOmRequest);
+
+
+    TestOMRequestUtils.addVolumeToDB(volumeName, "ozone", omMetadataManager);
+    String ozoneKey = omMetadataManager.getOzoneKey(volumeName, bucketName,
+        keyName);
+
+    // Key should not be there in key table, as validateAndUpdateCache is
+    // still not called.
+    OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable().get(ozoneKey);
+
+    Assert.assertNull(omKeyInfo);
+
+    OMClientResponse omClientResponse =
+        omKeyCommitRequest.validateAndUpdateCache(ozoneManager,
+            100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.BUCKET_NOT_FOUND,
+        omClientResponse.getOMResponse().getStatus());
+
+    omKeyInfo = omMetadataManager.getKeyTable().get(ozoneKey);
+
+    Assert.assertNull(omKeyInfo);
+  }
+
+  @Test
+  public void testValidateAndUpdateCacheWithKeyNotFound() throws Exception {
+
+    OMRequest modifiedOmRequest =
+        doPreExecute(createCommitKeyRequest());
+
+    OMKeyCommitRequest omKeyCommitRequest =
+        new OMKeyCommitRequest(modifiedOmRequest);
+
+
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+        omMetadataManager);
+
+    String ozoneKey = omMetadataManager.getOzoneKey(volumeName, bucketName,
+        keyName);
+
+    // Key should not be there in key table, as validateAndUpdateCache is
+    // still not called.
+    OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable().get(ozoneKey);
+
+    Assert.assertNull(omKeyInfo);
+
+    OMClientResponse omClientResponse =
+        omKeyCommitRequest.validateAndUpdateCache(ozoneManager,
+            100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.KEY_NOT_FOUND,
+        omClientResponse.getOMResponse().getStatus());
+
+    omKeyInfo = omMetadataManager.getKeyTable().get(ozoneKey);
+
+    Assert.assertNull(omKeyInfo);
+  }
+
+  /**
+   * This method calls preExecute and verify the modified request.
+   * @param originalOMRequest
+   * @return OMRequest - modified request returned from preExecute.
+   * @throws Exception
+   */
+  private OMRequest doPreExecute(OMRequest originalOMRequest) throws Exception {
+
+    OMKeyCommitRequest omKeyCommitRequest =
+        new OMKeyCommitRequest(originalOMRequest);
+
+    OMRequest modifiedOmRequest = omKeyCommitRequest.preExecute(ozoneManager);
+
+    Assert.assertTrue(modifiedOmRequest.hasCommitKeyRequest());
+    KeyArgs originalKeyArgs =
+        originalOMRequest.getCommitKeyRequest().getKeyArgs();
+    KeyArgs modifiedKeyArgs =
+        modifiedOmRequest.getCommitKeyRequest().getKeyArgs();
+    verifyKeyArgs(originalKeyArgs, modifiedKeyArgs);
+    return modifiedOmRequest;
+  }
+
+  /**
+   * Verify KeyArgs.
+   * @param originalKeyArgs
+   * @param modifiedKeyArgs
+   */
+  private void verifyKeyArgs(KeyArgs originalKeyArgs, KeyArgs modifiedKeyArgs) {
+
+    // Check modification time is set or not.
+    Assert.assertTrue(modifiedKeyArgs.getModificationTime() > 0);
+    Assert.assertTrue(originalKeyArgs.getModificationTime() == 0);
+
+    Assert.assertEquals(originalKeyArgs.getVolumeName(),
+        modifiedKeyArgs.getVolumeName());
+    Assert.assertEquals(originalKeyArgs.getBucketName(),
+        modifiedKeyArgs.getBucketName());
+    Assert.assertEquals(originalKeyArgs.getKeyName(),
+        modifiedKeyArgs.getKeyName());
+    Assert.assertEquals(originalKeyArgs.getDataSize(),
+        modifiedKeyArgs.getDataSize());
+    Assert.assertEquals(originalKeyArgs.getKeyLocationsList(),
+        modifiedKeyArgs.getKeyLocationsList());
+    Assert.assertEquals(originalKeyArgs.getType(),
+        modifiedKeyArgs.getType());
+    Assert.assertEquals(originalKeyArgs.getFactor(),
+        modifiedKeyArgs.getFactor());
+  }
+
+  /**
+   * Create OMRequest which encapsulates CommitKeyRequest.
+   */
+  private OMRequest createCommitKeyRequest() {
+    KeyArgs keyArgs =
+        KeyArgs.newBuilder().setDataSize(dataSize).setVolumeName(volumeName)
+            .setKeyName(keyName).setBucketName(bucketName)
+            .setType(replicationType).setFactor(replicationFactor)
+            .addAllKeyLocations(getKeyLocation()).build();
+
+    CommitKeyRequest commitKeyRequest =
+        CommitKeyRequest.newBuilder().setKeyArgs(keyArgs)
+            .setClientID(clientID).build();
+
+    return OMRequest.newBuilder()
+        .setCmdType(OzoneManagerProtocolProtos.Type.CommitKey)
+        .setCommitKeyRequest(commitKeyRequest)
+        .setClientId(UUID.randomUUID().toString()).build();
+  }
+
+  /**
+   * Create KeyLocation list.
+   */
+  private List<KeyLocation> getKeyLocation() {
+    List<KeyLocation> keyLocations = new ArrayList<>();
+
+    for (int i=0; i < 5; i++) {
+      KeyLocation keyLocation =
+          KeyLocation.newBuilder()
+              .setBlockID(HddsProtos.BlockID.newBuilder()
+                  .setContainerBlockID(HddsProtos.ContainerBlockID.newBuilder()
+                      .setContainerID(i+1000).setLocalID(i+100).build()))
+              .setOffset(0).setLength(200).build();
+      keyLocations.add(keyLocation);
+    }
+    return keyLocations;
+  }
+
+}

+ 325 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequest.java

@@ -0,0 +1,325 @@
+/**
+ * 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 java.util.List;
+import java.util.UUID;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+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
+    .CreateKeyRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .KeyArgs;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMRequest;
+
+/**
+ * Tests OMCreateKeyRequest class.
+ */
+public class TestOMKeyCreateRequest extends TestOMKeyRequest {
+
+  @Test
+  public void testPreExecuteWithNormalKey() throws Exception {
+    doPreExecute(createKeyRequest(false, 0));
+  }
+
+  @Test
+  public void testPreExecuteWithMultipartKey() throws Exception {
+    doPreExecute(createKeyRequest(true, 1));
+  }
+
+
+  @Test
+  public void testValidateAndUpdateCache() throws Exception {
+
+    OMRequest modifiedOmRequest =
+        doPreExecute(createKeyRequest(false, 0));
+
+    OMKeyCreateRequest omKeyCreateRequest =
+        new OMKeyCreateRequest(modifiedOmRequest);
+
+    // Add volume and bucket entries to DB.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+        omMetadataManager);
+
+    long id = modifiedOmRequest.getCreateKeyRequest().getClientID();
+
+    String openKey = omMetadataManager.getOpenKey(volumeName, bucketName,
+        keyName, id);
+
+    // Before calling
+    OmKeyInfo omKeyInfo = omMetadataManager.getOpenKeyTable().get(openKey);
+
+    Assert.assertNull(omKeyInfo);
+
+    OMClientResponse omKeyCreateResponse =
+        omKeyCreateRequest.validateAndUpdateCache(ozoneManager, 100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.OK,
+        omKeyCreateResponse.getOMResponse().getStatus());
+
+    // Check open table whether key is added or not.
+
+    omKeyInfo = omMetadataManager.getOpenKeyTable().get(openKey);
+
+    Assert.assertNotNull(omKeyInfo);
+
+    List<OmKeyLocationInfo> omKeyLocationInfoList =
+        omKeyInfo.getLatestVersionLocations().getLocationList();
+    Assert.assertTrue(omKeyLocationInfoList.size() == 1);
+
+    OmKeyLocationInfo omKeyLocationInfo = omKeyLocationInfoList.get(0);
+
+    // Check modification time
+    Assert.assertEquals(modifiedOmRequest.getCreateKeyRequest()
+        .getKeyArgs().getModificationTime(), omKeyInfo.getModificationTime());
+
+    Assert.assertEquals(omKeyInfo.getModificationTime(),
+        omKeyInfo.getCreationTime());
+
+
+    // Check data of the block
+    OzoneManagerProtocolProtos.KeyLocation keyLocation =
+        modifiedOmRequest.getCreateKeyRequest().getKeyArgs().getKeyLocations(0);
+
+    Assert.assertEquals(keyLocation.getBlockID().getContainerBlockID()
+        .getContainerID(), omKeyLocationInfo.getContainerID());
+    Assert.assertEquals(keyLocation.getBlockID().getContainerBlockID()
+        .getLocalID(), omKeyLocationInfo.getLocalID());
+
+  }
+
+  @Test
+  public void testValidateAndUpdateCacheWithNoSuchMultipartUploadError()
+      throws Exception {
+
+
+    int partNumber = 1;
+    OMRequest modifiedOmRequest =
+        doPreExecute(createKeyRequest(true, partNumber));
+
+    OMKeyCreateRequest omKeyCreateRequest =
+        new OMKeyCreateRequest(modifiedOmRequest);
+
+    // Add volume and bucket entries to DB.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+        omMetadataManager);
+
+    long id = modifiedOmRequest.getCreateKeyRequest().getClientID();
+
+    String openKey = omMetadataManager.getOpenKey(volumeName, bucketName,
+        keyName, id);
+
+    // Before calling
+    OmKeyInfo omKeyInfo = omMetadataManager.getOpenKeyTable().get(openKey);
+
+    Assert.assertNull(omKeyInfo);
+
+    OMClientResponse omKeyCreateResponse =
+        omKeyCreateRequest.validateAndUpdateCache(ozoneManager, 100L);
+
+    Assert.assertEquals(
+        OzoneManagerProtocolProtos.Status.NO_SUCH_MULTIPART_UPLOAD_ERROR,
+        omKeyCreateResponse.getOMResponse().getStatus());
+
+    // As we got error, no entry should be created in openKeyTable.
+
+    omKeyInfo = omMetadataManager.getOpenKeyTable().get(openKey);
+
+    Assert.assertNull(omKeyInfo);
+  }
+
+
+
+  @Test
+  public void testValidateAndUpdateCacheWithVolumeNotFound() throws Exception {
+
+    OMRequest modifiedOmRequest =
+        doPreExecute(createKeyRequest(false, 0));
+
+    OMKeyCreateRequest omKeyCreateRequest =
+        new OMKeyCreateRequest(modifiedOmRequest);
+
+
+    long id = modifiedOmRequest.getCreateKeyRequest().getClientID();
+
+    String openKey = omMetadataManager.getOpenKey(volumeName, bucketName,
+        keyName, id);
+
+
+    // Before calling
+    OmKeyInfo omKeyInfo = omMetadataManager.getOpenKeyTable().get(openKey);
+
+    Assert.assertNull(omKeyInfo);
+
+    OMClientResponse omKeyCreateResponse =
+        omKeyCreateRequest.validateAndUpdateCache(ozoneManager, 100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.VOLUME_NOT_FOUND,
+        omKeyCreateResponse.getOMResponse().getStatus());
+
+
+    // As We got an error, openKey Table should not have entry.
+    omKeyInfo = omMetadataManager.getOpenKeyTable().get(openKey);
+
+    Assert.assertNull(omKeyInfo);
+
+  }
+
+
+  @Test
+  public void testValidateAndUpdateCacheWithBucketNotFound() throws Exception {
+
+
+    OMRequest modifiedOmRequest =
+        doPreExecute(createKeyRequest(
+            false, 0));
+
+    OMKeyCreateRequest omKeyCreateRequest =
+        new OMKeyCreateRequest(modifiedOmRequest);
+
+
+    long id = modifiedOmRequest.getCreateKeyRequest().getClientID();
+
+    String openKey = omMetadataManager.getOpenKey(volumeName, bucketName,
+        keyName, id);
+
+    TestOMRequestUtils.addVolumeToDB(volumeName, "ozone", omMetadataManager);
+
+    // Before calling
+    OmKeyInfo omKeyInfo = omMetadataManager.getOpenKeyTable().get(openKey);
+
+    Assert.assertNull(omKeyInfo);
+
+    OMClientResponse omKeyCreateResponse =
+        omKeyCreateRequest.validateAndUpdateCache(ozoneManager, 100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.BUCKET_NOT_FOUND,
+        omKeyCreateResponse.getOMResponse().getStatus());
+
+
+    // As We got an error, openKey Table should not have entry.
+    omKeyInfo = omMetadataManager.getOpenKeyTable().get(openKey);
+
+    Assert.assertNull(omKeyInfo);
+
+  }
+
+
+
+  /**
+   * This method calls preExecute and verify the modified request.
+   * @param originalOMRequest
+   * @return OMRequest - modified request returned from preExecute.
+   * @throws Exception
+   */
+  private OMRequest doPreExecute(OMRequest originalOMRequest) throws Exception {
+
+    OMKeyCreateRequest omKeyCreateRequest =
+        new OMKeyCreateRequest(originalOMRequest);
+
+    OMRequest modifiedOmRequest =
+        omKeyCreateRequest.preExecute(ozoneManager);
+
+    Assert.assertEquals(originalOMRequest.getCmdType(),
+        modifiedOmRequest.getCmdType());
+    Assert.assertEquals(originalOMRequest.getClientId(),
+        modifiedOmRequest.getClientId());
+
+    Assert.assertTrue(modifiedOmRequest.hasCreateKeyRequest());
+
+    CreateKeyRequest createKeyRequest =
+        modifiedOmRequest.getCreateKeyRequest();
+
+    KeyArgs keyArgs = createKeyRequest.getKeyArgs();
+    // Time should be set
+    Assert.assertTrue(keyArgs.getModificationTime() > 0);
+
+
+    // Client ID should be set.
+    Assert.assertTrue(createKeyRequest.hasClientID());
+    Assert.assertTrue(createKeyRequest.getClientID() > 0);
+
+
+    if (!originalOMRequest.getCreateKeyRequest().getKeyArgs()
+        .getIsMultipartKey()) {
+
+      // As our data size is 100, and scmBlockSize is default to 1000, so we
+      // shall have only one block.
+      List< OzoneManagerProtocolProtos.KeyLocation> keyLocations =
+          keyArgs.getKeyLocationsList();
+      // KeyLocation should be set.
+      Assert.assertTrue(keyLocations.size() == 1);
+      Assert.assertEquals(containerID,
+          keyLocations.get(0).getBlockID().getContainerBlockID()
+              .getContainerID());
+      Assert.assertEquals(localID,
+          keyLocations.get(0).getBlockID().getContainerBlockID()
+              .getLocalID());
+      Assert.assertTrue(keyLocations.get(0).hasPipeline());
+
+      Assert.assertEquals(0, keyLocations.get(0).getOffset());
+
+      Assert.assertEquals(scmBlockSize, keyLocations.get(0).getLength());
+    } else {
+      // We don't create blocks for multipart key in createKey preExecute.
+      Assert.assertTrue(keyArgs.getKeyLocationsList().size() == 0);
+    }
+
+    return modifiedOmRequest;
+
+  }
+
+  /**
+   * Create OMRequest which encapsulates CreateKeyRequest.
+   * @param isMultipartKey
+   * @param partNumber
+   * @return OMRequest.
+   */
+
+  @SuppressWarnings("parameterNumber")
+  private OMRequest createKeyRequest(boolean isMultipartKey, int partNumber) {
+
+    KeyArgs.Builder keyArgs = KeyArgs.newBuilder()
+        .setVolumeName(volumeName).setBucketName(bucketName)
+        .setKeyName(keyName).setIsMultipartKey(isMultipartKey)
+        .setFactor(replicationFactor).setType(replicationType);
+
+    if (isMultipartKey) {
+      keyArgs.setDataSize(dataSize).setMultipartNumber(partNumber);
+    }
+
+    OzoneManagerProtocolProtos.CreateKeyRequest createKeyRequest =
+        CreateKeyRequest.newBuilder().setKeyArgs(keyArgs).build();
+
+    return OMRequest.newBuilder()
+        .setCmdType(OzoneManagerProtocolProtos.Type.CreateKey)
+        .setClientId(UUID.randomUUID().toString())
+        .setCreateKeyRequest(createKeyRequest).build();
+
+  }
+
+}

+ 166 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyDeleteRequest.java

@@ -0,0 +1,166 @@
+/**
+ * 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 java.util.UUID;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+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
+    .DeleteKeyRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .KeyArgs;
+
+/**
+ * Tests OmKeyDelete request.
+ */
+public class TestOMKeyDeleteRequest extends TestOMKeyRequest {
+
+  @Test
+  public void testPreExecute() throws Exception {
+    doPreExecute(createDeleteKeyRequest());
+  }
+
+  @Test
+  public void testValidateAndUpdateCache() throws Exception {
+    OMRequest modifiedOmRequest =
+        doPreExecute(createDeleteKeyRequest());
+
+    OMKeyDeleteRequest omKeyDeleteRequest =
+        new OMKeyDeleteRequest(modifiedOmRequest);
+
+
+    // Add volume, bucket and key entries to OM DB.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+        omMetadataManager);
+
+    TestOMRequestUtils.addKeyToTable(false, volumeName, bucketName, keyName,
+        clientID, replicationType, replicationFactor, omMetadataManager);
+
+
+    String ozoneKey = omMetadataManager.getOzoneKey(volumeName, bucketName,
+        keyName);
+
+    OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable().get(ozoneKey);
+
+    // As we added manually to key table.
+    Assert.assertNotNull(omKeyInfo);
+
+    OMClientResponse omClientResponse =
+        omKeyDeleteRequest.validateAndUpdateCache(ozoneManager,
+        100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.OK,
+        omClientResponse.getOMResponse().getStatus());
+    // Now after calling validateAndUpdateCache, it should be deleted.
+
+    omKeyInfo = omMetadataManager.getKeyTable().get(ozoneKey);
+
+    Assert.assertNull(omKeyInfo);
+
+  }
+
+
+  @Test
+  public void testValidateAndUpdateCacheWithKeyNotFound() throws Exception {
+    OMRequest modifiedOmRequest =
+        doPreExecute(createDeleteKeyRequest());
+
+    OMKeyDeleteRequest omKeyDeleteRequest =
+        new OMKeyDeleteRequest(modifiedOmRequest);
+
+    // Add only volume and bucket entry to DB.
+    // In actual implementation we don't check for bucket/volume exists
+    // during delete key.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+        omMetadataManager);
+
+
+    OMClientResponse omClientResponse =
+        omKeyDeleteRequest.validateAndUpdateCache(ozoneManager,
+            100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.KEY_NOT_FOUND,
+        omClientResponse.getOMResponse().getStatus());
+  }
+
+  @Test
+  public void testValidateAndUpdateCacheWithOutVolumeAndBucket()
+      throws Exception {
+    OMRequest modifiedOmRequest =
+        doPreExecute(createDeleteKeyRequest());
+
+    OMKeyDeleteRequest omKeyDeleteRequest =
+        new OMKeyDeleteRequest(modifiedOmRequest);
+
+    // In actual implementation we don't check for bucket/volume exists
+    // during delete key. So it should still return error KEY_NOT_FOUND
+
+    OMClientResponse omClientResponse =
+        omKeyDeleteRequest.validateAndUpdateCache(ozoneManager,
+            100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.KEY_NOT_FOUND,
+        omClientResponse.getOMResponse().getStatus());
+  }
+
+
+  /**
+   * This method calls preExecute and verify the modified request.
+   * @param originalOmRequest
+   * @return OMRequest - modified request returned from preExecute.
+   * @throws Exception
+   */
+  private OMRequest doPreExecute(OMRequest originalOmRequest) throws Exception {
+
+    OMKeyDeleteRequest omKeyDeleteRequest =
+        new OMKeyDeleteRequest(originalOmRequest);
+
+    OMRequest modifiedOmRequest = omKeyDeleteRequest.preExecute(ozoneManager);
+
+    // Will not be equal, as UserInfo will be set.
+    Assert.assertNotEquals(originalOmRequest, modifiedOmRequest);
+
+    return modifiedOmRequest;
+  }
+
+  /**
+   * Create OMRequest which encapsulates DeleteKeyRequest.
+   * @return OMRequest
+   */
+  private OMRequest createDeleteKeyRequest() {
+    KeyArgs keyArgs = KeyArgs.newBuilder().setBucketName(bucketName)
+        .setVolumeName(volumeName).setKeyName(keyName).build();
+
+    DeleteKeyRequest deleteKeyRequest =
+        DeleteKeyRequest.newBuilder().setKeyArgs(keyArgs).build();
+
+    return OMRequest.newBuilder().setDeleteKeyRequest(deleteKeyRequest)
+        .setCmdType(OzoneManagerProtocolProtos.Type.DeleteKey)
+        .setClientId(UUID.randomUUID().toString()).build();
+  }
+
+}

+ 225 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRenameRequest.java

@@ -0,0 +1,225 @@
+/**
+ * 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 java.util.UUID;
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+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
+    .OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .KeyArgs;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .RenameKeyRequest;
+
+/**
+ * Tests RenameKey request.
+ */
+public class TestOMKeyRenameRequest extends TestOMKeyRequest {
+
+  @Test
+  public void testPreExecute() throws Exception {
+    doPreExecute(createRenameKeyRequest(UUID.randomUUID().toString()));
+  }
+
+  @Test
+  public void testValidateAndUpdateCache() throws Exception {
+    String toKeyName = UUID.randomUUID().toString();
+    OMRequest modifiedOmRequest =
+        doPreExecute(createRenameKeyRequest(toKeyName));
+
+    TestOMRequestUtils.addKeyToTable(false, volumeName, bucketName, keyName,
+        clientID, replicationType, replicationFactor, omMetadataManager);
+
+    OMKeyRenameRequest omKeyRenameRequest =
+        new OMKeyRenameRequest(modifiedOmRequest);
+
+    OMClientResponse omKeyRenameResponse =
+        omKeyRenameRequest.validateAndUpdateCache(ozoneManager, 100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.OK,
+        omKeyRenameResponse.getOMResponse().getStatus());
+
+    String key = omMetadataManager.getOzoneKey(volumeName, bucketName, keyName);
+    // Original key should be deleted, toKey should exist.
+    OmKeyInfo omKeyInfo = omMetadataManager.getKeyTable().get(key);
+
+    Assert.assertNull(omKeyInfo);
+
+    omKeyInfo =
+        omMetadataManager.getKeyTable().get(omMetadataManager.getOzoneKey(
+            volumeName, bucketName, toKeyName));
+
+    Assert.assertNotNull(omKeyInfo);
+
+    // For new key modification time should be updated.
+
+    KeyArgs keyArgs = modifiedOmRequest.getRenameKeyRequest().getKeyArgs();
+
+    Assert.assertEquals(keyArgs.getModificationTime(),
+        omKeyInfo.getModificationTime());
+
+    // KeyName should be updated in OmKeyInfo to toKeyName.
+    Assert.assertEquals(omKeyInfo.getKeyName(), toKeyName);
+
+  }
+
+
+  @Test
+  public void testValidateAndUpdateCacheWithKeyNotFound() throws Exception {
+    String toKeyName = UUID.randomUUID().toString();
+    OMRequest modifiedOmRequest =
+        doPreExecute(createRenameKeyRequest(toKeyName));
+
+    // Add only volume and bucket entry to DB.
+
+    // In actual implementation we don't check for bucket/volume exists
+    // during delete key.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+        omMetadataManager);
+
+    OMKeyRenameRequest omKeyRenameRequest =
+        new OMKeyRenameRequest(modifiedOmRequest);
+
+    OMClientResponse omKeyRenameResponse =
+        omKeyRenameRequest.validateAndUpdateCache(ozoneManager, 100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.KEY_NOT_FOUND,
+        omKeyRenameResponse.getOMResponse().getStatus());
+
+  }
+
+
+  @Test
+  public void testValidateAndUpdateCacheWithOutVolumeAndBucket()
+      throws Exception {
+    String toKeyName = UUID.randomUUID().toString();
+    OMRequest modifiedOmRequest =
+        doPreExecute(createRenameKeyRequest(toKeyName));
+
+    // In actual implementation we don't check for bucket/volume exists
+    // during delete key. So it should still return error KEY_NOT_FOUND
+
+    OMKeyRenameRequest omKeyRenameRequest =
+        new OMKeyRenameRequest(modifiedOmRequest);
+
+    OMClientResponse omKeyRenameResponse =
+        omKeyRenameRequest.validateAndUpdateCache(ozoneManager, 100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.KEY_NOT_FOUND,
+        omKeyRenameResponse.getOMResponse().getStatus());
+
+  }
+
+  @Test
+  public void testValidateAndUpdateCacheWithToKeyInvalid() throws Exception {
+    String toKeyName = "";
+    OMRequest modifiedOmRequest =
+        doPreExecute(createRenameKeyRequest(toKeyName));
+
+    // Add only volume and bucket entry to DB.
+
+    // In actual implementation we don't check for bucket/volume exists
+    // during delete key.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+        omMetadataManager);
+
+    OMKeyRenameRequest omKeyRenameRequest =
+        new OMKeyRenameRequest(modifiedOmRequest);
+
+    OMClientResponse omKeyRenameResponse =
+        omKeyRenameRequest.validateAndUpdateCache(ozoneManager, 100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.INVALID_KEY_NAME,
+        omKeyRenameResponse.getOMResponse().getStatus());
+
+  }
+
+  @Test
+  public void testValidateAndUpdateCacheWithFromKeyInvalid() throws Exception {
+    String toKeyName = UUID.randomUUID().toString();
+    keyName = "";
+    OMRequest modifiedOmRequest =
+        doPreExecute(createRenameKeyRequest(toKeyName));
+
+    // Add only volume and bucket entry to DB.
+
+    // In actual implementation we don't check for bucket/volume exists
+    // during delete key.
+    TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+        omMetadataManager);
+
+    OMKeyRenameRequest omKeyRenameRequest =
+        new OMKeyRenameRequest(modifiedOmRequest);
+
+    OMClientResponse omKeyRenameResponse =
+        omKeyRenameRequest.validateAndUpdateCache(ozoneManager, 100L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.INVALID_KEY_NAME,
+        omKeyRenameResponse.getOMResponse().getStatus());
+
+  }
+
+
+  /**
+   * This method calls preExecute and verify the modified request.
+   * @param originalOmRequest
+   * @return OMRequest - modified request returned from preExecute.
+   * @throws Exception
+   */
+
+  private OMRequest doPreExecute(OMRequest originalOmRequest) throws Exception {
+    OMKeyRenameRequest omKeyRenameRequest =
+        new OMKeyRenameRequest(originalOmRequest);
+
+    OMRequest modifiedOmRequest = omKeyRenameRequest.preExecute(ozoneManager);
+
+    // Will not be equal, as UserInfo will be set and modification time is
+    // set in KeyArgs.
+    Assert.assertNotEquals(originalOmRequest, modifiedOmRequest);
+
+    Assert.assertTrue(modifiedOmRequest.getRenameKeyRequest()
+        .getKeyArgs().getModificationTime() > 0);
+
+    return modifiedOmRequest;
+  }
+
+  /**
+   * Create OMRequest which encapsulates RenameKeyRequest.
+   * @return OMRequest
+   */
+  private OMRequest createRenameKeyRequest(String toKeyName) {
+    KeyArgs keyArgs = KeyArgs.newBuilder().setKeyName(keyName)
+        .setVolumeName(volumeName).setBucketName(bucketName).build();
+
+    RenameKeyRequest renameKeyRequest = RenameKeyRequest.newBuilder()
+            .setKeyArgs(keyArgs).setToKeyName(toKeyName).build();
+
+    return OMRequest.newBuilder()
+        .setClientId(UUID.randomUUID().toString())
+        .setRenameKeyRequest(renameKeyRequest)
+        .setCmdType(OzoneManagerProtocolProtos.Type.RenameKey).build();
+  }
+
+}

+ 151 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java

@@ -0,0 +1,151 @@
+/**
+ * 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 java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.hdds.client.ContainerBlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
+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.ScmClient;
+import org.apache.hadoop.ozone.security.OzoneBlockTokenSecretManager;
+import org.apache.hadoop.util.Time;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.when;
+
+/**
+ * Base test class for key request.
+ */
+@SuppressWarnings("visibilitymodifier")
+public class TestOMKeyRequest {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  protected OzoneManager ozoneManager;
+  protected OMMetrics omMetrics;
+  protected OMMetadataManager omMetadataManager;
+  protected AuditLogger auditLogger;
+
+  protected ScmClient scmClient;
+  protected OzoneBlockTokenSecretManager ozoneBlockTokenSecretManager;
+  protected ScmBlockLocationProtocol scmBlockLocationProtocol;
+
+  protected final long containerID = 1000L;
+  protected final long localID = 100L;
+
+  protected String volumeName;
+  protected String bucketName;
+  protected String keyName;
+  protected HddsProtos.ReplicationType replicationType;
+  protected HddsProtos.ReplicationFactor replicationFactor;
+  protected long clientID;
+  protected long scmBlockSize = 1000L;
+  protected long dataSize;
+
+
+  @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));
+
+    scmClient = Mockito.mock(ScmClient.class);
+    ozoneBlockTokenSecretManager =
+        Mockito.mock(OzoneBlockTokenSecretManager.class);
+    scmBlockLocationProtocol = Mockito.mock(ScmBlockLocationProtocol.class);
+    when(ozoneManager.getScmClient()).thenReturn(scmClient);
+    when(ozoneManager.getBlockTokenSecretManager())
+        .thenReturn(ozoneBlockTokenSecretManager);
+    when(ozoneManager.getScmBlockSize()).thenReturn(scmBlockSize);
+    when(ozoneManager.getPreallocateBlocksMax()).thenReturn(2);
+    when(ozoneManager.isGrpcBlockTokenEnabled()).thenReturn(false);
+    when(ozoneManager.getOMNodeId()).thenReturn(UUID.randomUUID().toString());
+    when(scmClient.getBlockClient()).thenReturn(scmBlockLocationProtocol);
+
+    Pipeline pipeline = Pipeline.newBuilder()
+        .setState(Pipeline.PipelineState.OPEN)
+        .setId(PipelineID.randomId())
+        .setType(HddsProtos.ReplicationType.STAND_ALONE)
+        .setFactor(HddsProtos.ReplicationFactor.ONE)
+        .setNodes(new ArrayList<>())
+        .build();
+
+    AllocatedBlock allocatedBlock =
+        new AllocatedBlock.Builder()
+            .setContainerBlockID(new ContainerBlockID(containerID, localID))
+            .setPipeline(pipeline).build();
+
+    List<AllocatedBlock> allocatedBlocks = new ArrayList<>();
+
+    allocatedBlocks.add(allocatedBlock);
+
+    when(scmBlockLocationProtocol.allocateBlock(anyLong(), anyInt(),
+        any(HddsProtos.ReplicationType.class),
+        any(HddsProtos.ReplicationFactor.class),
+        anyString(), any(ExcludeList.class))).thenReturn(allocatedBlocks);
+
+
+    volumeName = UUID.randomUUID().toString();
+    bucketName = UUID.randomUUID().toString();
+    keyName = UUID.randomUUID().toString();
+    replicationFactor = HddsProtos.ReplicationFactor.ONE;
+    replicationType = HddsProtos.ReplicationType.RATIS;
+    clientID = Time.now();
+    dataSize = 1000L;
+
+  }
+
+  @After
+  public void stop() {
+    omMetrics.unRegister();
+    Mockito.framework().clearInlineMocks();
+  }
+}

+ 23 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/package-info.java

@@ -0,0 +1,23 @@
+/*
+ * 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 contains test classes for key requests.
+ */
+package org.apache.hadoop.ozone.om.request.key;

+ 93 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMAllocateBlockResponse.java

@@ -0,0 +1,93 @@
+/**
+ * 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.key;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .AllocateBlockResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMResponse;
+
+/**
+ * Tests OMAllocateBlockResponse.
+ */
+public class TestOMAllocateBlockResponse extends TestOMKeyResponse {
+
+  @Test
+  public void testAddToDBBatch() throws Exception {
+
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
+        bucketName, keyName, replicationType, replicationFactor);
+
+    OMResponse omResponse = OMResponse.newBuilder()
+        .setAllocateBlockResponse(
+            AllocateBlockResponse.getDefaultInstance())
+        .setStatus(OzoneManagerProtocolProtos.Status.OK)
+        .setCmdType(OzoneManagerProtocolProtos.Type.AllocateBlock)
+        .build();
+    OMAllocateBlockResponse omAllocateBlockResponse =
+        new OMAllocateBlockResponse(omKeyInfo, clientID, omResponse);
+
+    String openKey = omMetadataManager.getOpenKey(volumeName, bucketName,
+        keyName, clientID);
+
+    // Not adding key entry before to test whether commit is successful or not.
+    Assert.assertFalse(omMetadataManager.getOpenKeyTable().isExist(openKey));
+    omAllocateBlockResponse.addToDBBatch(omMetadataManager, batchOperation);
+
+    // Do manual commit and see whether addToBatch is successful or not.
+    omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+    Assert.assertTrue(omMetadataManager.getOpenKeyTable().isExist(openKey));
+  }
+
+  @Test
+  public void testAddToDBBatchWithErrorResponse() throws Exception {
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
+        bucketName, keyName, replicationType, replicationFactor);
+
+    OMResponse omResponse = OMResponse.newBuilder()
+        .setAllocateBlockResponse(
+            AllocateBlockResponse.getDefaultInstance())
+        .setStatus(OzoneManagerProtocolProtos.Status.KEY_NOT_FOUND)
+        .setCmdType(OzoneManagerProtocolProtos.Type.AllocateBlock)
+        .build();
+    OMAllocateBlockResponse omAllocateBlockResponse =
+        new OMAllocateBlockResponse(omKeyInfo, clientID, omResponse);
+
+    // Before calling addToDBBatch
+    String openKey = omMetadataManager.getOpenKey(volumeName, bucketName,
+        keyName, clientID);
+    Assert.assertFalse(omMetadataManager.getOpenKeyTable().isExist(openKey));
+
+    omAllocateBlockResponse.addToDBBatch(omMetadataManager, batchOperation);
+
+    // Do manual commit and see whether addToBatch is successful or not.
+    omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+    // As omResponse is error it is a no-op.
+    Assert.assertFalse(omMetadataManager.getOpenKeyTable().isExist(openKey));
+
+  }
+}

+ 108 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponse.java

@@ -0,0 +1,108 @@
+/**
+ * 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.key;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+/**
+ * Tests OMKeyCommitResponse.
+ */
+public class TestOMKeyCommitResponse extends TestOMKeyResponse {
+
+  @Test
+  public void testAddToDBBatch() throws Exception {
+
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
+        bucketName, keyName, replicationType, replicationFactor);
+
+    OzoneManagerProtocolProtos.OMResponse omResponse =
+        OzoneManagerProtocolProtos.OMResponse.newBuilder().setCommitKeyResponse(
+            OzoneManagerProtocolProtos.CommitKeyResponse.getDefaultInstance())
+            .setStatus(OzoneManagerProtocolProtos.Status.OK)
+            .setCmdType(OzoneManagerProtocolProtos.Type.CommitKey)
+            .build();
+
+    // As during commit Key, entry will be already there in openKeyTable.
+    // Adding it here.
+    TestOMRequestUtils.addKeyToTable(true, volumeName, bucketName, keyName,
+        clientID, replicationType, replicationFactor, omMetadataManager);
+
+    String openKey = omMetadataManager.getOpenKey(volumeName, bucketName,
+        keyName, clientID);
+    Assert.assertTrue(omMetadataManager.getOpenKeyTable().isExist(openKey));
+
+    OMKeyCommitResponse omKeyCommitResponse =
+        new OMKeyCommitResponse(omKeyInfo, clientID, omResponse);
+
+    omKeyCommitResponse.addToDBBatch(omMetadataManager, batchOperation);
+
+    // Do manual commit and see whether addToBatch is successful or not.
+    omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+    // When key commit key is deleted from openKey table and added to keyTable.
+    Assert.assertFalse(omMetadataManager.getOpenKeyTable().isExist(openKey));
+    Assert.assertTrue(omMetadataManager.getKeyTable().isExist(
+        omMetadataManager.getOzoneKey(volumeName, bucketName, keyName)));
+  }
+
+  @Test
+  public void testAddToDBBatchNoOp() throws Exception {
+
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
+        bucketName, keyName, replicationType, replicationFactor);
+
+    OzoneManagerProtocolProtos.OMResponse omResponse =
+        OzoneManagerProtocolProtos.OMResponse.newBuilder().setCommitKeyResponse(
+            OzoneManagerProtocolProtos.CommitKeyResponse.getDefaultInstance())
+            .setStatus(OzoneManagerProtocolProtos.Status.KEY_NOT_FOUND)
+            .setCmdType(OzoneManagerProtocolProtos.Type.CommitKey)
+            .build();
+
+
+    OMKeyCommitResponse omKeyCommitResponse =
+        new OMKeyCommitResponse(omKeyInfo, clientID, omResponse);
+
+    // As during commit Key, entry will be already there in openKeyTable.
+    // Adding it here.
+    TestOMRequestUtils.addKeyToTable(true, volumeName, bucketName, keyName,
+        clientID, replicationType, replicationFactor, omMetadataManager);
+
+    String openKey = omMetadataManager.getOpenKey(volumeName, bucketName,
+        keyName, clientID);
+    Assert.assertTrue(omMetadataManager.getOpenKeyTable().isExist(openKey));
+
+
+    omKeyCommitResponse.addToDBBatch(omMetadataManager, batchOperation);
+
+    // Do manual commit and see whether addToBatch is successful or not.
+    omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+
+    // As omResponse is error it is a no-op. So, entry should still be in
+    // openKey table.
+    Assert.assertTrue(omMetadataManager.getOpenKeyTable().isExist(openKey));
+    Assert.assertFalse(omMetadataManager.getKeyTable().isExist(
+        omMetadataManager.getOzoneKey(volumeName, bucketName, keyName)));
+  }
+}

+ 92 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCreateResponse.java

@@ -0,0 +1,92 @@
+/**
+ * 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.key;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .CreateKeyResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMResponse;
+
+
+/**
+ * Tests MKeyCreateResponse.
+ */
+public class TestOMKeyCreateResponse extends TestOMKeyResponse {
+
+  @Test
+  public void testAddToDBBatch() throws Exception {
+
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
+        bucketName, keyName, replicationType, replicationFactor);
+
+    OMResponse omResponse = OMResponse.newBuilder().setCreateKeyResponse(
+                CreateKeyResponse.getDefaultInstance())
+            .setStatus(OzoneManagerProtocolProtos.Status.OK)
+            .setCmdType(OzoneManagerProtocolProtos.Type.CreateKey)
+            .build();
+
+    OMKeyCreateResponse omKeyCreateResponse =
+        new OMKeyCreateResponse(omKeyInfo, clientID, omResponse);
+
+    String openKey = omMetadataManager.getOpenKey(volumeName, bucketName,
+        keyName, clientID);
+    Assert.assertFalse(omMetadataManager.getOpenKeyTable().isExist(openKey));
+    omKeyCreateResponse.addToDBBatch(omMetadataManager, batchOperation);
+
+    // Do manual commit and see whether addToBatch is successful or not.
+    omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+    Assert.assertTrue(omMetadataManager.getOpenKeyTable().isExist(openKey));
+  }
+
+  @Test
+  public void testAddToDBBatchWithErrorResponse() throws Exception {
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
+        bucketName, keyName, replicationType, replicationFactor);
+
+    OMResponse omResponse = OMResponse.newBuilder().setCreateKeyResponse(
+        CreateKeyResponse.getDefaultInstance())
+        .setStatus(OzoneManagerProtocolProtos.Status.KEY_NOT_FOUND)
+        .setCmdType(OzoneManagerProtocolProtos.Type.CreateKey)
+        .build();
+
+    OMKeyCreateResponse omKeyCreateResponse =
+        new OMKeyCreateResponse(omKeyInfo, clientID, omResponse);
+
+    // Before calling addToDBBatch
+    String openKey = omMetadataManager.getOpenKey(volumeName, bucketName,
+        keyName, clientID);
+    Assert.assertFalse(omMetadataManager.getOpenKeyTable().isExist(openKey));
+
+    omKeyCreateResponse.addToDBBatch(omMetadataManager, batchOperation);
+
+    // Do manual commit and see whether addToBatch is successful or not.
+    omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+    // As omResponse is error it is a no-op.
+    Assert.assertFalse(omMetadataManager.getOpenKeyTable().isExist(openKey));
+
+  }
+}

+ 163 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyDeleteResponse.java

@@ -0,0 +1,163 @@
+/**
+ * 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.key;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests OMKeyDeleteResponse.
+ */
+public class TestOMKeyDeleteResponse extends TestOMKeyResponse {
+
+  @Test
+  public void testAddToDBBatch() throws Exception {
+
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
+        bucketName, keyName, replicationType, replicationFactor);
+
+    OzoneManagerProtocolProtos.OMResponse omResponse =
+        OzoneManagerProtocolProtos.OMResponse.newBuilder().setDeleteKeyResponse(
+            OzoneManagerProtocolProtos.DeleteKeyResponse.getDefaultInstance())
+            .setStatus(OzoneManagerProtocolProtos.Status.OK)
+            .setCmdType(OzoneManagerProtocolProtos.Type.DeleteKey)
+            .build();
+
+    OMKeyDeleteResponse omKeyDeleteResponse =
+        new OMKeyDeleteResponse(omKeyInfo, omResponse);
+
+    String ozoneKey = omMetadataManager.getOzoneKey(volumeName, bucketName,
+        keyName);
+
+    TestOMRequestUtils.addKeyToTable(false, volumeName, bucketName, keyName,
+        clientID, replicationType, replicationFactor, omMetadataManager);
+
+    Assert.assertTrue(omMetadataManager.getKeyTable().isExist(ozoneKey));
+    omKeyDeleteResponse.addToDBBatch(omMetadataManager, batchOperation);
+
+    // Do manual commit and see whether addToBatch is successful or not.
+    omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+    Assert.assertFalse(omMetadataManager.getKeyTable().isExist(ozoneKey));
+
+    // As default key entry does not have any blocks, it should not be in
+    // deletedKeyTable.
+    Assert.assertFalse(omMetadataManager.getDeletedTable().isExist(ozoneKey));
+  }
+
+  @Test
+  public void testAddToDBBatchWithNonEmptyBlocks() throws Exception {
+
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
+        bucketName, keyName, replicationType, replicationFactor);
+
+    // Add block to key.
+    List<OmKeyLocationInfo> omKeyLocationInfoList = new ArrayList<>();
+
+    Pipeline pipeline = Pipeline.newBuilder()
+        .setState(Pipeline.PipelineState.OPEN)
+        .setId(PipelineID.randomId())
+        .setType(replicationType)
+        .setFactor(replicationFactor)
+        .setNodes(new ArrayList<>())
+        .build();
+
+    OmKeyLocationInfo omKeyLocationInfo =
+        new OmKeyLocationInfo.Builder().setBlockID(
+            new BlockID(100L, 1000L))
+            .setOffset(0).setLength(100L).setPipeline(pipeline).build();
+
+
+    omKeyLocationInfoList.add(omKeyLocationInfo);
+
+    omKeyInfo.appendNewBlocks(omKeyLocationInfoList, false);
+
+    String ozoneKey = omMetadataManager.getOzoneKey(volumeName, bucketName,
+        keyName);
+
+    omMetadataManager.getKeyTable().put(ozoneKey, omKeyInfo);
+
+    OzoneManagerProtocolProtos.OMResponse omResponse =
+        OzoneManagerProtocolProtos.OMResponse.newBuilder().setDeleteKeyResponse(
+            OzoneManagerProtocolProtos.DeleteKeyResponse.getDefaultInstance())
+            .setStatus(OzoneManagerProtocolProtos.Status.OK)
+            .setCmdType(OzoneManagerProtocolProtos.Type.DeleteKey)
+            .build();
+
+    OMKeyDeleteResponse omKeyDeleteResponse =
+        new OMKeyDeleteResponse(omKeyInfo, omResponse);
+
+    Assert.assertTrue(omMetadataManager.getKeyTable().isExist(ozoneKey));
+    omKeyDeleteResponse.addToDBBatch(omMetadataManager, batchOperation);
+
+    // Do manual commit and see whether addToBatch is successful or not.
+    omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+    Assert.assertFalse(omMetadataManager.getKeyTable().isExist(ozoneKey));
+
+    // Key has blocks, it should not be in deletedKeyTable.
+    Assert.assertTrue(omMetadataManager.getDeletedTable().isExist(ozoneKey));
+  }
+
+
+  @Test
+  public void testAddToDBBatchWithErrorResponse() throws Exception {
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
+        bucketName, keyName, replicationType, replicationFactor);
+
+    OzoneManagerProtocolProtos.OMResponse omResponse =
+        OzoneManagerProtocolProtos.OMResponse.newBuilder().setDeleteKeyResponse(
+            OzoneManagerProtocolProtos.DeleteKeyResponse.getDefaultInstance())
+            .setStatus(OzoneManagerProtocolProtos.Status.KEY_NOT_FOUND)
+            .setCmdType(OzoneManagerProtocolProtos.Type.DeleteKey)
+            .build();
+
+    OMKeyDeleteResponse omKeyDeleteResponse =
+        new OMKeyDeleteResponse(omKeyInfo, omResponse);
+
+    String ozoneKey = omMetadataManager.getOzoneKey(volumeName, bucketName,
+        keyName);
+
+    TestOMRequestUtils.addKeyToTable(false, volumeName, bucketName, keyName,
+        clientID, replicationType, replicationFactor, omMetadataManager);
+
+    Assert.assertTrue(omMetadataManager.getKeyTable().isExist(ozoneKey));
+
+    omKeyDeleteResponse.addToDBBatch(omMetadataManager, batchOperation);
+
+    // Do manual commit and see whether addToBatch is successful or not.
+    omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+    // As omResponse is error it is a no-op. So, entry should be still in the
+    // keyTable.
+    Assert.assertTrue(omMetadataManager.getKeyTable().isExist(ozoneKey));
+
+  }
+}

+ 148 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyRenameResponse.java

@@ -0,0 +1,148 @@
+/**
+ * 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.key;
+
+import java.util.UUID;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+/**
+ * Tests OMKeyRenameResponse.
+ */
+public class TestOMKeyRenameResponse extends TestOMKeyResponse {
+  @Test
+  public void testAddToDBBatch() throws Exception {
+
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
+        bucketName, keyName, replicationType, replicationFactor);
+
+    OzoneManagerProtocolProtos.OMResponse omResponse =
+        OzoneManagerProtocolProtos.OMResponse.newBuilder().setRenameKeyResponse(
+            OzoneManagerProtocolProtos.RenameKeyResponse.getDefaultInstance())
+            .setStatus(OzoneManagerProtocolProtos.Status.OK)
+            .setCmdType(OzoneManagerProtocolProtos.Type.RenameKey)
+            .build();
+
+    String toKeyName = UUID.randomUUID().toString();
+
+    OMKeyRenameResponse omKeyRenameResponse =
+        new OMKeyRenameResponse(omKeyInfo, toKeyName, keyName, omResponse);
+
+    String ozoneFromKey = omMetadataManager.getOzoneKey(volumeName, bucketName,
+        keyName);
+
+    String ozoneToKey = omMetadataManager.getOzoneKey(volumeName, bucketName,
+        toKeyName);
+
+    TestOMRequestUtils.addKeyToTable(false, volumeName, bucketName, keyName,
+        clientID, replicationType, replicationFactor, omMetadataManager);
+
+    Assert.assertTrue(omMetadataManager.getKeyTable().isExist(ozoneFromKey));
+    Assert.assertFalse(omMetadataManager.getKeyTable().isExist(ozoneToKey));
+
+    omKeyRenameResponse.addToDBBatch(omMetadataManager, batchOperation);
+
+    // Do manual commit and see whether addToBatch is successful or not.
+    omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+    Assert.assertFalse(omMetadataManager.getKeyTable().isExist(ozoneFromKey));
+    Assert.assertTrue(omMetadataManager.getKeyTable().isExist(ozoneToKey));
+  }
+
+  @Test
+  public void testAddToDBBatchWithErrorResponse() throws Exception {
+
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
+        bucketName, keyName, replicationType, replicationFactor);
+
+    OzoneManagerProtocolProtos.OMResponse omResponse =
+        OzoneManagerProtocolProtos.OMResponse.newBuilder().setRenameKeyResponse(
+            OzoneManagerProtocolProtos.RenameKeyResponse.getDefaultInstance())
+            .setStatus(OzoneManagerProtocolProtos.Status.KEY_NOT_FOUND)
+            .setCmdType(OzoneManagerProtocolProtos.Type.RenameKey)
+            .build();
+
+    String toKeyName = UUID.randomUUID().toString();
+
+    OMKeyRenameResponse omKeyRenameResponse =
+        new OMKeyRenameResponse(omKeyInfo, toKeyName, keyName, omResponse);
+
+    String ozoneFromKey = omMetadataManager.getOzoneKey(volumeName, bucketName,
+        keyName);
+
+    String ozoneToKey = omMetadataManager.getOzoneKey(volumeName, bucketName,
+        toKeyName);
+
+    TestOMRequestUtils.addKeyToTable(false, volumeName, bucketName, keyName,
+        clientID, replicationType, replicationFactor, omMetadataManager);
+
+    Assert.assertTrue(omMetadataManager.getKeyTable().isExist(ozoneFromKey));
+    Assert.assertFalse(omMetadataManager.getKeyTable().isExist(ozoneToKey));
+
+    omKeyRenameResponse.addToDBBatch(omMetadataManager, batchOperation);
+
+    // Do manual commit and see whether addToBatch is successful or not.
+    omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+    // As omResponse has error, it is a no-op. So, no changes should happen.
+    Assert.assertTrue(omMetadataManager.getKeyTable().isExist(ozoneFromKey));
+    Assert.assertFalse(omMetadataManager.getKeyTable().isExist(ozoneToKey));
+
+  }
+
+  @Test
+  public void testAddToDBBatchWithSameKeyName() throws Exception {
+
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
+        bucketName, keyName, replicationType, replicationFactor);
+
+    OzoneManagerProtocolProtos.OMResponse omResponse =
+        OzoneManagerProtocolProtos.OMResponse.newBuilder().setRenameKeyResponse(
+            OzoneManagerProtocolProtos.RenameKeyResponse.getDefaultInstance())
+            .setStatus(OzoneManagerProtocolProtos.Status.KEY_NOT_FOUND)
+            .setCmdType(OzoneManagerProtocolProtos.Type.RenameKey)
+            .build();
+
+
+    // Passing toKeyName also same as KeyName.
+    OMKeyRenameResponse omKeyRenameResponse =
+        new OMKeyRenameResponse(omKeyInfo, keyName, keyName, omResponse);
+
+    String ozoneFromKey = omMetadataManager.getOzoneKey(volumeName, bucketName,
+        keyName);
+
+    TestOMRequestUtils.addKeyToTable(false, volumeName, bucketName, keyName,
+        clientID, replicationType, replicationFactor, omMetadataManager);
+
+    Assert.assertTrue(omMetadataManager.getKeyTable().isExist(ozoneFromKey));
+
+    omKeyRenameResponse.addToDBBatch(omMetadataManager, batchOperation);
+
+    // Do manual commit and see whether addToBatch is successful or not.
+    omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+    Assert.assertTrue(omMetadataManager.getKeyTable().isExist(ozoneFromKey));
+
+  }
+}

+ 75 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyResponse.java

@@ -0,0 +1,75 @@
+/**
+ * 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.key;
+
+import java.util.UUID;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.Mockito;
+
+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.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.utils.db.BatchOperation;
+
+/**
+ * Base test class for key response.
+ */
+@SuppressWarnings("visibilitymodifier")
+public class TestOMKeyResponse {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  protected OMMetadataManager omMetadataManager;
+  protected BatchOperation batchOperation;
+
+  protected String volumeName;
+  protected String bucketName;
+  protected String keyName;
+  protected HddsProtos.ReplicationFactor replicationFactor;
+  protected HddsProtos.ReplicationType replicationType;
+  protected long clientID;
+
+  @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();
+
+    volumeName = UUID.randomUUID().toString();
+    bucketName = UUID.randomUUID().toString();
+    keyName = UUID.randomUUID().toString();
+    replicationFactor = HddsProtos.ReplicationFactor.ONE;
+    replicationType = HddsProtos.ReplicationType.RATIS;
+    clientID = 1000L;
+  }
+
+  @After
+  public void stop() {
+    Mockito.framework().clearInlineMocks();
+  }
+
+}

+ 23 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/package-info.java

@@ -0,0 +1,23 @@
+/*
+ * 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 contains test classes for key responses.
+ */
+package org.apache.hadoop.ozone.om.response.key;