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

[hadoop-ozone] 01/02: HDDS-2876. Consolidate ObjectID and UpdateID from Info objects into one class

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

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

commit 860d79874ad837f182617da84b23540e50411551
Author: Hanisha Koneru <ha...@apache.org>
AuthorDate: Mon Feb 10 11:52:48 2020 +0100

    HDDS-2876. Consolidate ObjectID and UpdateID from Info objects into one class
    
    Closes #451
---
 .../hadoop/ozone/om/helpers/OmBucketInfo.java      | 64 +++------------
 .../apache/hadoop/ozone/om/helpers/OmKeyInfo.java  | 61 ++++----------
 .../ozone/om/helpers/OmMultipartKeyInfo.java       | 35 ++------
 .../hadoop/ozone/om/helpers/OmVolumeArgs.java      | 59 +++-----------
 .../hadoop/ozone/om/helpers/WithObjectID.java      | 92 ++++++++++++++++++++++
 .../hadoop/ozone/om/request/OMClientRequest.java   | 10 ++-
 .../om/request/bucket/OMBucketCreateRequest.java   |  3 +-
 .../om/request/bucket/OMBucketDeleteRequest.java   |  3 +-
 .../request/bucket/OMBucketSetPropertyRequest.java |  3 +-
 .../om/request/bucket/acl/OMBucketAclRequest.java  |  3 +-
 .../ozone/om/request/file/OMFileCreateRequest.java |  2 +-
 .../ozone/om/request/key/OMKeyCreateRequest.java   |  3 +-
 .../ozone/om/request/key/OMKeyDeleteRequest.java   |  3 +-
 .../ozone/om/request/key/OMKeyRenameRequest.java   |  3 +-
 .../request/s3/bucket/S3BucketCreateRequest.java   |  3 +-
 .../request/s3/bucket/S3BucketDeleteRequest.java   |  3 +-
 .../om/request/volume/OMVolumeCreateRequest.java   |  3 +-
 .../om/request/volume/OMVolumeDeleteRequest.java   |  3 +-
 .../om/request/volume/OMVolumeSetOwnerRequest.java |  3 +-
 .../om/request/volume/OMVolumeSetQuotaRequest.java |  3 +-
 .../bucket/TestOMBucketSetPropertyRequest.java     |  4 +-
 21 files changed, 159 insertions(+), 207 deletions(-)

diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
index f4ecb5f..5dec9c6 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
@@ -40,7 +40,7 @@ import com.google.common.base.Preconditions;
 /**
  * A class that encapsulates Bucket Info.
  */
-public final class OmBucketInfo extends WithMetadata implements Auditable {
+public final class OmBucketInfo extends WithObjectID implements Auditable {
   /**
    * Name of the volume in which the bucket belongs to.
    */
@@ -66,16 +66,6 @@ public final class OmBucketInfo extends WithMetadata implements Auditable {
    * Creation time of bucket.
    */
   private final long creationTime;
-  /**
-   * ObjectIDs are unique and immutable identifier for each object in the
-   * System.
-   */
-  private long objectID;
-  /**
-   * UpdateIDs are monotonically increasing values which are updated
-   * each time there is an update.
-   */
-  private long updateID;
 
   /**
    * Bucket encryption key info if encryption is enabled.
@@ -195,22 +185,6 @@ public final class OmBucketInfo extends WithMetadata implements Auditable {
   }
 
   /**
-   * Returns objectID.
-   * @return long
-   */
-  public long getObjectID() {
-    return objectID;
-  }
-
-  /**
-   * Returns updateID.
-   * @return long
-   */
-  public long getUpdateID() {
-    return updateID;
-  }
-
-  /**
    * Returns bucket encryption key info.
    * @return bucket encryption key info
    */
@@ -219,31 +193,6 @@ public final class OmBucketInfo extends WithMetadata implements Auditable {
   }
 
   /**
-   * Set the Object ID. If this value is already set then this function throws.
-   * There is a reason why we cannot use the final here. The OMBucketInfo is
-   * deserialized from the protobuf in many places in code. We need to set
-   * this object ID, after it is deserialized.
-   *
-   * @param obId - long
-   */
-  public void setObjectID(long obId) {
-    if(this.objectID != 0) {
-      throw new UnsupportedOperationException("Attempt to modify object ID " +
-          "which is not zero. Current Object ID is " + this.objectID);
-    }
-    this.objectID = obId;
-  }
-
-  /**
-   * Sets the update ID. For each modification of this object, we will set
-   * this to a value greater than the current value.
-   * @param updateID  long
-   */
-  public void setUpdateID(long updateID) {
-    this.updateID = updateID;
-  }
-
-  /**
    * Returns new builder class that builds a OmBucketInfo.
    *
    * @return Builder
@@ -453,6 +402,17 @@ public final class OmBucketInfo extends WithMetadata implements Auditable {
   }
 
   @Override
+  public String getObjectInfo() {
+    return "OMBucketInfo{" +
+        "volume='" + volumeName + '\'' +
+        ", bucket='" + bucketName + '\'' +
+        ", isVersionEnabled='" + isVersionEnabled + '\'' +
+        ", storageType='" + storageType + '\'' +
+        ", creationTime='" + creationTime + '\'' +
+        '}';
+  }
+
+  @Override
   public boolean equals(Object o) {
     if (this == o) {
       return true;
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
index 9e79129..9dc2072 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
@@ -40,7 +40,7 @@ import com.google.common.base.Preconditions;
  * This is returned from OM to client, and client use class to talk to
  * datanode. Also, this is the metadata written to om.db on server side.
  */
-public final class OmKeyInfo extends WithMetadata {
+public final class OmKeyInfo extends WithObjectID {
   private final String volumeName;
   private final String bucketName;
   // name of key client specified
@@ -52,8 +52,6 @@ public final class OmKeyInfo extends WithMetadata {
   private HddsProtos.ReplicationType type;
   private HddsProtos.ReplicationFactor factor;
   private FileEncryptionInfo encInfo;
-  private long objectID;
-  private long updateID;
 
   /**
    * ACL Information.
@@ -142,50 +140,6 @@ public final class OmKeyInfo extends WithMetadata {
   }
 
   /**
-   * Set the Object ID. If this value is already set then this function throws.
-   * There is a reason why we cannot use the final here. The OMKeyInfo is
-   * deserialized from the protobuf in many places in code. We need to set
-   * this object ID, after it is deserialized.
-   *
-   * @param obId - long
-   */
-  public void setObjectID(long obId) {
-    if(this.objectID != 0) {
-      throw new UnsupportedOperationException("Attempt to modify object ID " +
-          "which is not zero. Current Object ID is " + this.objectID);
-    }
-    this.objectID = obId;
-  }
-
-  /**
-   * Sets the update ID. For each modification of this object, we will set
-   * this to a value greater than the current value.
-   * @param updateId  long
-   */
-  public void setUpdateID(long updateId) {
-    Preconditions.checkArgument(updateId > this.updateID,
-        "Trying to set updateID to a value ({}) which is less than the " +
-            "current value ({}) for ()", updateId, this.updateID, this);
-    this.updateID = updateId;
-  }
-
-  /**
-   * Returns objectID.
-   * @return long
-   */
-  public long getObjectID() {
-    return objectID;
-  }
-
-  /**
-   * Returns updateID.
-   * @return long
-   */
-  public long getUpdateID() {
-    return updateID;
-  }
-
-  /**
    * updates the length of the each block in the list given.
    * This will be called when the key is being committed to OzoneManager.
    *
@@ -480,6 +434,19 @@ public final class OmKeyInfo extends WithMetadata {
   }
 
   @Override
+  public String getObjectInfo() {
+    return "OMKeyInfo{" +
+        "volume='" + volumeName + '\'' +
+        ", bucket='" + bucketName + '\'' +
+        ", key='" + keyName + '\'' +
+        ", dataSize='" + dataSize + '\'' +
+        ", creationTime='" + creationTime + '\'' +
+        ", type='" + type + '\'' +
+        ", factor='" + factor + '\'' +
+        '}';
+  }
+
+  @Override
   public boolean equals(Object o) {
     if (this == o) {
       return true;
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmMultipartKeyInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmMultipartKeyInfo.java
index 66fe853..df8751c 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmMultipartKeyInfo.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmMultipartKeyInfo.java
@@ -29,14 +29,12 @@ import java.util.TreeMap;
  * This class represents multipart upload information for a key, which holds
  * upload part information of the key.
  */
-public class OmMultipartKeyInfo {
+public class OmMultipartKeyInfo extends WithObjectID {
   private final String uploadID;
   private final long creationTime;
   private final ReplicationType replicationType;
   private final ReplicationFactor replicationFactor;
   private TreeMap<Integer, PartKeyInfo> partKeyInfoList;
-  private long objectID;
-  private long updateID;
 
   /**
    * Construct OmMultipartKeyInfo object which holds multipart upload
@@ -55,32 +53,6 @@ public class OmMultipartKeyInfo {
   }
 
   /**
-   * Sets the update ID. For each modification of this object, we will set
-   * this to a value greater than the current value.
-   * @param updateID  long
-   */
-  public void setUpdateID(long updateID) {
-    this.updateID = updateID;
-  }
-
-  /**
-   * Returns objectID.
-   * @return long
-   */
-  public long getObjectID() {
-    return objectID;
-  }
-
-  /**
-   * Returns updateID.
-   * @return long
-   */
-  public long getUpdateID() {
-    return updateID;
-  }
-
-
-  /**
    * Returns the uploadID for this multi part upload of a key.
    * @return uploadID
    */
@@ -211,6 +183,11 @@ public class OmMultipartKeyInfo {
   }
 
   @Override
+  public String getObjectInfo() {
+    return getProto().toString();
+  }
+
+  @Override
   public boolean equals(Object other) {
     if (this == other) {
       return true;
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
index a1fea2e..26d0b23 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
@@ -37,57 +37,13 @@ import com.google.common.base.Preconditions;
 /**
  * A class that encapsulates the OmVolumeArgs Args.
  */
-public final class OmVolumeArgs extends WithMetadata implements Auditable {
+public final class OmVolumeArgs extends WithObjectID implements Auditable {
   private final String adminName;
   private String ownerName;
   private final String volume;
   private long creationTime;
   private long quotaInBytes;
   private final OmOzoneAclMap aclMap;
-  private long objectID;
-  private long updateID;
-
-  /**
-   * Set the Object ID. If this value is already set then this function throws.
-   * There is a reason why we cannot use the final here. The OmVolumeArgs is
-   * deserialized from the protobuf in many places in code. We need to set
-   * this object ID, after it is deserialized.
-   *
-   * @param obId - long
-   */
-  public void setObjectID(long obId) {
-    if(this.objectID != 0) {
-      throw new UnsupportedOperationException("Attempt to modify object ID " +
-          "which is not zero. Current Object ID is " + this.objectID);
-    }
-    this.objectID = obId;
-  }
-
-  /**
-   * Returns a monotonically increasing ID, that denotes the last update.
-   * Each time an update happens, this ID is incremented.
-   * @return long
-   */
-  public long getUpdateID() {
-    return updateID;
-  }
-
-  /**
-   * Sets the update ID. For each modification of this object, we will set
-   * this to a value greater than the current value.
-   * @param updateID  long
-   */
-  public void setUpdateID(long updateID) {
-    this.updateID = updateID;
-  }
-
-  /**
-   * A immutable identity field for this object.
-   * @return  long.
-   */
-  public long getObjectID() {
-    return objectID;
-  }
 
   /**
    * Private constructor, constructed via builder.
@@ -261,8 +217,6 @@ public final class OmVolumeArgs extends WithMetadata implements Auditable {
       return this;
     }
 
-
-
     /**
      * Constructs a builder.
      */
@@ -359,6 +313,17 @@ public final class OmVolumeArgs extends WithMetadata implements Auditable {
         volInfo.getUpdateID());
   }
 
+  @Override
+  public String getObjectInfo() {
+    return "OMVolumeArgs{" +
+        "volume='" + volume + '\'' +
+        ", admin='" + adminName + '\'' +
+        ", owner='" + ownerName + '\'' +
+        ", creationTime='" + creationTime + '\'' +
+        ", quota='" + quotaInBytes + '\'' +
+        '}';
+  }
+
   /**
    * Return a new copy of the object.
    */
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/WithObjectID.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/WithObjectID.java
new file mode 100644
index 0000000..34463ae
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/WithObjectID.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.helpers;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Mixin class to handle ObjectID and UpdateID.
+ */
+public class WithObjectID extends WithMetadata {
+
+  /**
+   * ObjectIDs are unique and immutable identifier for each object in the
+   * System.
+   */
+  @SuppressWarnings("visibilitymodifier")
+  protected long objectID;
+  /**
+   * UpdateIDs are monotonically increasing values which are updated
+   * each time there is an update.
+   */
+  @SuppressWarnings("visibilitymodifier")
+  protected long updateID;
+
+  /**
+   * Returns objectID.
+   * @return long
+   */
+  public long getObjectID() {
+    return objectID;
+  }
+
+  /**
+   * Returns updateID.
+   * @return long
+   */
+  public long getUpdateID() {
+    return updateID;
+  }
+
+  /**
+   * Set the Object ID. If this value is already set then this function throws.
+   * There is a reason why we cannot use the final here. The object
+   * ({@link OmVolumeArgs}/ {@link OmBucketInfo}/ {@link OmKeyInfo}) is
+   * deserialized from the protobuf in many places in code. We need to set
+   * this object ID, after it is deserialized.
+   *
+   * @param obId - long
+   */
+  public void setObjectID(long obId) {
+    if(this.objectID != 0) {
+      throw new UnsupportedOperationException("Attempt to modify object ID " +
+          "which is not zero. Current Object ID is " + this.objectID);
+    }
+    this.objectID = obId;
+  }
+
+  /**
+   * Sets the update ID. For each modification of this object, we will set
+   * this to a value greater than the current value.
+   * @param updateId  long
+   */
+  public void setUpdateID(long updateId) {
+    Preconditions.checkArgument(updateId > this.updateID, String.format(
+        "Trying to set updateID to %d which is not greater than the current " +
+            "value of %d for %s", updateId, this.updateID, getObjectInfo()));
+    this.updateID = updateId;
+  }
+
+  public boolean isUpdateIDset() {
+    return this.updateID > 0;
+  }
+
+  public String getObjectInfo() {
+    return this.toString();
+  }
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/OMClientRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/OMClientRequest.java
index bffb099..1a34f8e 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/OMClientRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/OMClientRequest.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.ozone.audit.AuditLogger;
 import org.apache.hadoop.ozone.audit.AuditMessage;
 import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.WithObjectID;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
@@ -247,13 +248,16 @@ public abstract class OMClientRequest implements RequestAuditor {
 
   /**
    * Check if the transaction is a replay.
-   * @param updateID last updateID of the entity in the DB
+   * @param ozoneObj OMVolumeArgs or OMBucketInfo or OMKeyInfo object whose 
+   *                 updateID needs to be compared with
    * @param transactionID the current transaction ID
    * @return true if transactionID is less than or equal to updateID, false
    * otherwise.
    */
-  public boolean isReplay(OzoneManager om, long updateID, long transactionID) {
-    return om.isRatisEnabled() && transactionID <= updateID;
+  protected boolean isReplay(OzoneManager om, WithObjectID ozoneObj,
+      long transactionID) {
+    return om.isRatisEnabled() && ozoneObj.isUpdateIDset() &&
+        transactionID <= ozoneObj.getUpdateID();
   }
 
   /**
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
index 9cf9741..3516a0a 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
@@ -169,8 +169,7 @@ public class OMBucketCreateRequest extends OMClientRequest {
           .get(bucketKey);
       if (dbBucketInfo != null) {
         // Check if this transaction is a replay of ratis logs.
-        if (isReplay(ozoneManager, dbBucketInfo.getUpdateID(),
-            transactionLogIndex)) {
+        if (isReplay(ozoneManager, dbBucketInfo, transactionLogIndex)) {
           // Replay implies the response has already been returned to
           // the client. So take no further action and return a dummy
           // OMClientResponse.
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketDeleteRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketDeleteRequest.java
index 93d5cb3..400c9d5 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketDeleteRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketDeleteRequest.java
@@ -127,8 +127,7 @@ public class OMBucketDeleteRequest extends OMClientRequest {
       // If this is a replay, then the response has already been returned to
       // the client. So take no further action and return a dummy
       // OMClientResponse.
-      if (isReplay(ozoneManager, omBucketInfo.getUpdateID(),
-          transactionLogIndex)) {
+      if (isReplay(ozoneManager, omBucketInfo, transactionLogIndex)) {
         LOG.debug("Replayed Transaction {} ignored. Request: {}",
             transactionLogIndex, deleteBucketRequest);
         return new OMBucketDeleteResponse(createReplayOMResponse(omResponse));
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java
index a0b12ae..f099d29 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java
@@ -124,8 +124,7 @@ public class OMBucketSetPropertyRequest extends OMClientRequest {
       // Check if this transaction is a replay of ratis logs.
       // If a replay, then the response has already been returned to the
       // client. So take no further action and return a dummy OMClientResponse.
-      if (isReplay(ozoneManager, dbBucketInfo.getUpdateID(),
-          transactionLogIndex)) {
+      if (isReplay(ozoneManager, dbBucketInfo, transactionLogIndex)) {
         LOG.debug("Replayed Transaction {} ignored. Request: {}",
             transactionLogIndex, setBucketPropertyRequest);
         return new OMBucketSetPropertyResponse(
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketAclRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketAclRequest.java
index a14e4b0..322ea4c 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketAclRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/acl/OMBucketAclRequest.java
@@ -110,8 +110,7 @@ public abstract class OMBucketAclRequest extends OMClientRequest {
       // If this is a replay, then the response has already been returned to
       // the client. So take no further action and return a dummy
       // OMClientResponse.
-      if (isReplay(ozoneManager, omBucketInfo.getUpdateID(),
-          transactionLogIndex)) {
+      if (isReplay(ozoneManager, omBucketInfo, transactionLogIndex)) {
         LOG.debug("Replayed Transaction {} ignored. Request: {}",
             transactionLogIndex, getOmRequest());
         return new OMBucketAclResponse(createReplayOMResponse(omResponse));
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java
index 4745932..ed867a7 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java
@@ -226,7 +226,7 @@ public class OMFileCreateRequest extends OMKeyRequest {
         // the openKey table would eventually reach the same state.
         // The reason we do not check the OpenKey table is to avoid a DB read
         // in regular non-replay scenario.
-        if (isReplay(ozoneManager, dbKeyInfo.getUpdateID(), trxnLogIndex)) {
+        if (isReplay(ozoneManager, dbKeyInfo, trxnLogIndex)) {
           // Replay implies the response has already been returned to
           // the client. So take no further action and return a dummy response.
           throw new OMReplayException();
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java
index d6ce620..8352cea 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java
@@ -202,8 +202,7 @@ public class OMKeyCreateRequest extends OMKeyRequest {
         // the openKey table would eventually reach the same state.
         // The reason we do not check the OpenKey table is to avoid a DB read
         // in regular non-replay scenario.
-        if (isReplay(ozoneManager, dbKeyInfo.getUpdateID(),
-            trxnLogIndex)) {
+        if (isReplay(ozoneManager, dbKeyInfo, trxnLogIndex)) {
           // Replay implies the response has already been returned to
           // the client. So take no further action and return a dummy
           // OMClientResponse.
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyDeleteRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyDeleteRequest.java
index e30f16b..517d89d 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyDeleteRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyDeleteRequest.java
@@ -136,8 +136,7 @@ public class OMKeyDeleteRequest extends OMKeyRequest {
       }
 
       // Check if this transaction is a replay of ratis logs.
-      if (isReplay(ozoneManager, omKeyInfo.getUpdateID(),
-          trxnLogIndex)) {
+      if (isReplay(ozoneManager, omKeyInfo, trxnLogIndex)) {
         // Replay implies the response has already been returned to
         // the client. So take no further action and return a dummy
         // OMClientResponse.
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequest.java
index 52f9310..9ec3edb 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequest.java
@@ -152,8 +152,7 @@ public class OMKeyRenameRequest extends OMKeyRequest {
       if (toKeyValue != null) {
 
         // Check if this transaction is a replay of ratis logs.
-        if (isReplay(ozoneManager, toKeyValue.getUpdateID(),
-            trxnLogIndex)) {
+        if (isReplay(ozoneManager, toKeyValue, trxnLogIndex)) {
 
           // Check if fromKey is still in the DB and created before this
           // replay.
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/S3BucketCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/S3BucketCreateRequest.java
index e50203c..faf48df 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/S3BucketCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/S3BucketCreateRequest.java
@@ -156,8 +156,7 @@ public class S3BucketCreateRequest extends OMVolumeRequest {
             .get(omBucketKey);
         if (dbBucketInfo != null) {
           // Check if this transaction is a replay of ratis logs.
-          if (isReplay(ozoneManager, dbBucketInfo.getUpdateID(),
-              trxnLogIndex)) {
+          if (isReplay(ozoneManager, dbBucketInfo, trxnLogIndex)) {
             // Replay implies the response has already been returned to
             // the client. So take no further action and return a dummy
             // OMClientResponse.
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/S3BucketDeleteRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/S3BucketDeleteRequest.java
index 7dc4d96..ca5af6c 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/S3BucketDeleteRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/S3BucketDeleteRequest.java
@@ -133,8 +133,7 @@ public class S3BucketDeleteRequest extends OMVolumeRequest {
           // If this is a replay, then the response has already been returned to
           // the client. So take no further action and return a dummy
           // OMClientResponse.
-          if (isReplay(ozoneManager, dbBucketInfo.getUpdateID(),
-              transactionLogIndex)) {
+          if (isReplay(ozoneManager, dbBucketInfo, transactionLogIndex)) {
             LOG.debug("Replayed Transaction {} ignored. Request: {}",
                 transactionLogIndex, s3DeleteBucketRequest);
             return new S3BucketDeleteResponse(
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeCreateRequest.java
index e02297b..de1b18a 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeCreateRequest.java
@@ -160,8 +160,7 @@ public class OMVolumeCreateRequest extends OMVolumeRequest {
         LOG.debug("volume:{} successfully created", omVolumeArgs.getVolume());
       } else {
         // Check if this transaction is a replay of ratis logs.
-        if (isReplay(ozoneManager, dbVolumeArgs.getUpdateID(),
-            transactionLogIndex)) {
+        if (isReplay(ozoneManager, dbVolumeArgs, transactionLogIndex)) {
           // Replay implies the response has already been returned to
           // the client. So take no further action and return a dummy
           // OMClientResponse.
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeDeleteRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeDeleteRequest.java
index 8861325..e6f5646 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeDeleteRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeDeleteRequest.java
@@ -103,8 +103,7 @@ public class OMVolumeDeleteRequest extends OMVolumeRequest {
       // If this is a replay, then the response has already been returned to
       // the client. So take no further action and return a dummy
       // OMClientResponse.
-      if (isReplay(ozoneManager, omVolumeArgs.getUpdateID(),
-          transactionLogIndex)) {
+      if (isReplay(ozoneManager, omVolumeArgs, transactionLogIndex)) {
         LOG.debug("Replayed Transaction {} ignored. Request: {}",
             transactionLogIndex, deleteVolumeRequest);
         return new OMVolumeDeleteResponse(createReplayOMResponse(omResponse));
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetOwnerRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetOwnerRequest.java
index 10e8daf..a31c9a6 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetOwnerRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetOwnerRequest.java
@@ -135,8 +135,7 @@ public class OMVolumeSetOwnerRequest extends OMVolumeRequest {
       // If this is a replay, then the response has already been returned to
       // the client. So take no further action and return a dummy
       // OMClientResponse.
-      if (isReplay(ozoneManager, omVolumeArgs.getUpdateID(),
-          transactionLogIndex)) {
+      if (isReplay(ozoneManager, omVolumeArgs, transactionLogIndex)) {
         LOG.debug("Replayed Transaction {} ignored. Request: {}",
             transactionLogIndex, setVolumePropertyRequest);
         return new OMVolumeSetOwnerResponse(createReplayOMResponse(omResponse));
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetQuotaRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetQuotaRequest.java
index 12d91fa..cfb4b1b 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetQuotaRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetQuotaRequest.java
@@ -124,8 +124,7 @@ public class OMVolumeSetQuotaRequest extends OMVolumeRequest {
       // If this is a replay, then the response has already been returned to
       // the client. So take no further action and return a dummy
       // OMClientResponse.
-      if (isReplay(ozoneManager, omVolumeArgs.getUpdateID(),
-          transactionLogIndex)) {
+      if (isReplay(ozoneManager, omVolumeArgs, transactionLogIndex)) {
         LOG.debug("Replayed Transaction {} ignored. Request: {}",
             transactionLogIndex, setVolumePropertyRequest);
         return new OMVolumeSetQuotaResponse(createReplayOMResponse(omResponse));
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/bucket/TestOMBucketSetPropertyRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/bucket/TestOMBucketSetPropertyRequest.java
index 9ff126a..0670c3e 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/bucket/TestOMBucketSetPropertyRequest.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/bucket/TestOMBucketSetPropertyRequest.java
@@ -140,8 +140,8 @@ public class TestOMBucketSetPropertyRequest extends TestBucketRequest {
         ozoneManagerDoubleBufferHelper);
 
     // Replay the transaction - Execute the same request again
-    OMClientResponse omClientResponse =
-        omBucketSetPropertyRequest.validateAndUpdateCache(ozoneManager, 1,
+    OMClientResponse omClientResponse = omBucketSetPropertyRequest
+        .validateAndUpdateCache(ozoneManager, 1,
             ozoneManagerDoubleBufferHelper);
 
     // Replay should result in Replay response


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