You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by ms...@apache.org on 2021/02/09 16:59:23 UTC

[ozone] branch HDDS-2823 updated: HDDS-4782.Merge SCMRatisSnapshotInfo and OMRatisSnapshotInfo into a single class (#1880)

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

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


The following commit(s) were added to refs/heads/HDDS-2823 by this push:
     new 4771661  HDDS-4782.Merge SCMRatisSnapshotInfo and OMRatisSnapshotInfo into a single class (#1880)
4771661 is described below

commit 47716613c0ff8f09cfc84cfc794027e554b8ae90
Author: bshashikant <sh...@apache.org>
AuthorDate: Tue Feb 9 22:28:46 2021 +0530

    HDDS-4782.Merge SCMRatisSnapshotInfo and OMRatisSnapshotInfo into a single class (#1880)
---
 .../ozone/common/ha/ratis/RatisSnapshotInfo.java   | 17 ++++--
 .../hadoop/ozone/common/ha/ratis/package-info.java | 18 ++++++
 .../hadoop/hdds/scm/ha/SCMRatisSnapshotInfo.java   | 71 ----------------------
 .../hadoop/hdds/scm/ha/SCMTransactionInfo.java     |  3 +-
 .../org/apache/hadoop/ozone/om/OzoneManager.java   |  8 +--
 .../ozone/om/ratis/OzoneManagerStateMachine.java   |  3 +-
 .../om/ratis/TestOzoneManagerRatisServer.java      |  3 +-
 .../om/ratis/TestOzoneManagerStateMachine.java     |  3 +-
 8 files changed, 42 insertions(+), 84 deletions(-)

diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OMRatisSnapshotInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ha/ratis/RatisSnapshotInfo.java
similarity index 84%
rename from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OMRatisSnapshotInfo.java
rename to hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ha/ratis/RatisSnapshotInfo.java
index 91c388e..a9de892 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OMRatisSnapshotInfo.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ha/ratis/RatisSnapshotInfo.java
@@ -15,7 +15,7 @@
  * the License.
  */
 
-package org.apache.hadoop.ozone.om.ratis;
+package org.apache.hadoop.ozone.common.ha.ratis;
 
 import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.FileInfo;
@@ -29,14 +29,14 @@ import static org.apache.hadoop.ozone.OzoneConsts.TRANSACTION_INFO_SPLIT_KEY;
 
 /**
  * This class captures the snapshotIndex and term of the latest snapshot in
- * the OM.
+ * the server.
  * Ratis server loads the snapshotInfo during startup and updates the
- * lastApplied index to this snapshotIndex. OM SnapshotInfo does not contain
+ * lastApplied index to this snapshotIndex. SnapshotInfo does not contain
  * any files. It is used only to store/ update the last applied index and term.
  */
-public class OMRatisSnapshotInfo implements SnapshotInfo {
+public class RatisSnapshotInfo implements SnapshotInfo {
 
-  static final Logger LOG = LoggerFactory.getLogger(OMRatisSnapshotInfo.class);
+  static final Logger LOG = LoggerFactory.getLogger(RatisSnapshotInfo.class);
 
   private volatile long term = 0;
   private volatile long snapshotIndex = -1;
@@ -50,6 +50,13 @@ public class OMRatisSnapshotInfo implements SnapshotInfo {
     this.snapshotIndex = newIndex;
   }
 
+  public RatisSnapshotInfo() {}
+
+  public RatisSnapshotInfo(long term, long index) {
+    this.term = term;
+    this.snapshotIndex = index;
+  }
+
   @Override
   public TermIndex getTermIndex() {
     return TermIndex.valueOf(term, snapshotIndex);
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ha/ratis/package-info.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ha/ratis/package-info.java
new file mode 100644
index 0000000..f01aef4
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ha/ratis/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * 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.common.ha.ratis;
\ No newline at end of file
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMRatisSnapshotInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMRatisSnapshotInfo.java
deleted file mode 100644
index b2f2ed4..0000000
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMRatisSnapshotInfo.java
+++ /dev/null
@@ -1,71 +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>
- * <p>http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * <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.hdds.scm.ha;
-
-import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.server.storage.FileInfo;
-import org.apache.ratis.statemachine.SnapshotInfo;
-
-import java.util.List;
-
-import static org.apache.hadoop.ozone.OzoneConsts.TRANSACTION_INFO_SPLIT_KEY;
-
-/**
- * This class captures the snapshotIndex and term of the latest snapshot in
- * the SCM
- * Ratis server loads the snapshotInfo during startup and updates the
- * lastApplied index to this snapshotIndex. SCM SnapshotInfo does not contain
- * any files. It is used only to store/ update the last applied index and term.
- */
-public class SCMRatisSnapshotInfo implements SnapshotInfo {
-  private final long term;
-  private final long snapshotIndex;
-
-  public SCMRatisSnapshotInfo(long term, long index) {
-    this.term = term;
-    this.snapshotIndex = index;
-  }
-
-  @Override
-  public TermIndex getTermIndex() {
-    return TermIndex.valueOf(term, snapshotIndex);
-  }
-
-  @Override
-  public long getTerm() {
-    return term;
-  }
-
-  @Override
-  public long getIndex() {
-    return snapshotIndex;
-  }
-
-  @Override
-  public List<FileInfo> getFiles() {
-    return null;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder stringBuilder = new StringBuilder();
-    stringBuilder.append(term);
-    stringBuilder.append(TRANSACTION_INFO_SPLIT_KEY);
-    stringBuilder.append(snapshotIndex);
-    return stringBuilder.toString();
-  }
-}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMTransactionInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMTransactionInfo.java
index ca1235e..7946178 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMTransactionInfo.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMTransactionInfo.java
@@ -18,6 +18,7 @@ package org.apache.hadoop.hdds.scm.ha;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.StringUtils;
+import org.apache.hadoop.ozone.common.ha.ratis.RatisSnapshotInfo;
 import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.statemachine.SnapshotInfo;
 
@@ -99,7 +100,7 @@ public final class SCMTransactionInfo {
   }
 
   public SnapshotInfo toSnapshotInfo() {
-    return new SCMRatisSnapshotInfo(term, transactionIndex);
+    return new RatisSnapshotInfo(term, transactionIndex);
   }
 
   public static Builder builder() {
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index 9a5c200..fd0b7ed 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -136,7 +136,7 @@ import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfoEx;
 import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
 import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB;
-import org.apache.hadoop.ozone.om.ratis.OMRatisSnapshotInfo;
+import org.apache.hadoop.ozone.common.ha.ratis.RatisSnapshotInfo;
 import org.apache.hadoop.ozone.om.ratis.OMTransactionInfo;
 import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
@@ -310,7 +310,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   private OMNodeDetails omNodeDetails;
   private List<OMNodeDetails> peerNodes;
   private File omRatisSnapshotDir;
-  private final OMRatisSnapshotInfo omRatisSnapshotInfo;
+  private final RatisSnapshotInfo omRatisSnapshotInfo;
   private final Map<String, RatisDropwizardExports> ratisMetricsMap =
       new ConcurrentHashMap<>();
 
@@ -445,7 +445,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     // Create special volume s3v which is required for S3G.
     addS3GVolumeToDB();
 
-    this.omRatisSnapshotInfo = new OMRatisSnapshotInfo();
+    this.omRatisSnapshotInfo = new RatisSnapshotInfo();
 
     if (isRatisEnabled) {
       // Create Ratis storage dir
@@ -1337,7 +1337,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     return omTransactionInfo.getTransactionIndex();
   }
 
-  public OMRatisSnapshotInfo getSnapshotInfo() {
+  public RatisSnapshotInfo getSnapshotInfo() {
     return omRatisSnapshotInfo;
   }
 
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java
index a81d5d4..99085e6 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java
@@ -32,6 +32,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.hadoop.ozone.common.ha.ratis.RatisSnapshotInfo;
 import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
@@ -80,7 +81,7 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
   private RequestHandler handler;
   private RaftGroupId raftGroupId;
   private OzoneManagerDoubleBuffer ozoneManagerDoubleBuffer;
-  private final OMRatisSnapshotInfo snapshotInfo;
+  private final RatisSnapshotInfo snapshotInfo;
   private final ExecutorService executorService;
   private final ExecutorService installSnapshotExecutor;
   private final boolean isTracingEnabled;
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java
index 5be7109..cec29be 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OmUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.common.ha.ratis.RatisSnapshotInfo;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.ha.OMNodeDetails;
@@ -104,7 +105,7 @@ public class TestOzoneManagerRatisServer {
     omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration);
     when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager);
     initialTermIndex = TermIndex.valueOf(0, 0);
-    OMRatisSnapshotInfo omRatisSnapshotInfo = new OMRatisSnapshotInfo();
+    RatisSnapshotInfo omRatisSnapshotInfo = new RatisSnapshotInfo();
     when(ozoneManager.getSnapshotInfo()).thenReturn(omRatisSnapshotInfo);
     omRatisServer = OzoneManagerRatisServer.newOMRatisServer(conf, ozoneManager,
       omNodeDetails, Collections.emptyList());
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerStateMachine.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerStateMachine.java
index 285c992..3368cff 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerStateMachine.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerStateMachine.java
@@ -17,6 +17,7 @@
 package org.apache.hadoop.ozone.om.ratis;
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.common.ha.ratis.RatisSnapshotInfo;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
@@ -58,7 +59,7 @@ public class TestOzoneManagerStateMachine {
 
     when(ozoneManagerRatisServer.getOzoneManager()).thenReturn(ozoneManager);
     when(ozoneManager.getSnapshotInfo()).thenReturn(
-        Mockito.mock(OMRatisSnapshotInfo.class));
+        Mockito.mock(RatisSnapshotInfo.class));
     ozoneManagerStateMachine =
         new OzoneManagerStateMachine(ozoneManagerRatisServer, false);
     ozoneManagerStateMachine.notifyTermIndexUpdated(0, 0);


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