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

sammichen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new 76267a3666 HDDS-8973. Ozone SCM HA should not allocates duplicate IDs 
when transferring leadership (#5018)
76267a3666 is described below

commit 76267a366613d1f8d4e2f58f0af3e1d9b32e8269
Author: XiChen <[email protected]>
AuthorDate: Thu Jul 6 10:18:39 2023 +0800

    HDDS-8973. Ozone SCM HA should not allocates duplicate IDs when 
transferring leadership (#5018)
---
 .../hadoop/hdds/scm/ha/SequenceIdGenerator.java    | 15 ++++--
 .../hdds/scm/ha/TestSequenceIDGenerator.java       | 56 ++++++++++++++++++++++
 2 files changed, 67 insertions(+), 4 deletions(-)

diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SequenceIdGenerator.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SequenceIdGenerator.java
index 94cbbc9297..02b9565dd7 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SequenceIdGenerator.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SequenceIdGenerator.java
@@ -99,7 +99,13 @@ public class SequenceIdGenerator {
         OZONE_SCM_SEQUENCE_ID_BATCH_SIZE_DEFAULT);
 
     Preconditions.checkNotNull(scmhaManager);
-    this.stateManager = new StateManagerImpl.Builder()
+    this.stateManager = createStateManager(scmhaManager, sequenceIdTable);
+  }
+
+  public StateManager createStateManager(SCMHAManager scmhaManager,
+      Table<String, Long> sequenceIdTable) {
+    Preconditions.checkNotNull(scmhaManager);
+    return new StateManagerImpl.Builder()
         .setRatisServer(scmhaManager.getRatisServer())
         .setDBTransactionBuffer(scmhaManager.getDBTransactionBuffer())
         .setSequenceIdTable(sequenceIdTable).build();
@@ -125,11 +131,12 @@ public class SequenceIdGenerator {
         batch.nextId = prevLastId + 1;
 
         Preconditions.checkArgument(Long.MAX_VALUE - batch.lastId >= 
batchSize);
-        batch.lastId += sequenceIdName.equals(ROOT_CERTIFICATE_ID) ?
-            1 : batchSize;
+        long nextLastId = batch.lastId +
+            (sequenceIdName.equals(ROOT_CERTIFICATE_ID) ? 1 : batchSize);
 
         if (stateManager.allocateBatch(sequenceIdName,
-            prevLastId, batch.lastId)) {
+            prevLastId, nextLastId)) {
+          batch.lastId = nextLastId;
           LOG.info("Allocate a batch for {}, change lastId from {} to {}.",
               sequenceIdName, prevLastId, batch.lastId);
           break;
diff --git 
a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/ha/TestSequenceIDGenerator.java
 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/ha/TestSequenceIDGenerator.java
index 119f9989b5..166c905ad2 100644
--- 
a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/ha/TestSequenceIDGenerator.java
+++ 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/ha/TestSequenceIDGenerator.java
@@ -16,15 +16,22 @@
  */
 package org.apache.hadoop.hdds.scm.ha;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStore;
 import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStoreImpl;
 import org.apache.hadoop.hdds.scm.metadata.SCMDBTransactionBufferImpl;
+import org.apache.hadoop.hdds.utils.db.Table;
 import org.apache.hadoop.ozone.container.common.SCMTestUtils;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 
 import static 
org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_SEQUENCE_ID_BATCH_SIZE;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
 
 /**
  * Tests for {@link SequenceIdGenerator}.
@@ -116,4 +123,53 @@ public class TestSequenceIDGenerator {
     Assertions.assertEquals(202, sequenceIdGen.getNextId("otherKey"));
     Assertions.assertEquals(203, sequenceIdGen.getNextId("otherKey"));
   }
+
+  @Test
+  public void testSequenceIDGenUponRatisWhenCurrentScmIsNotALeader()
+      throws Exception {
+    int batchSize = 100;
+    OzoneConfiguration conf = SCMTestUtils.getConf();
+    conf.setInt(OZONE_SCM_SEQUENCE_ID_BATCH_SIZE, batchSize);
+    SCMMetadataStore scmMetadataStore = new SCMMetadataStoreImpl(conf);
+    scmMetadataStore.start(conf);
+    SCMHAManager scmHAManager = SCMHAManagerStub
+        .getInstance(true, new SCMDBTransactionBufferImpl());
+
+    SequenceIdGenerator.StateManager stateManager =
+        spy(new SequenceIdGenerator.StateManagerImpl.Builder()
+            .setRatisServer(scmHAManager.getRatisServer())
+            .setDBTransactionBuffer(scmHAManager.getDBTransactionBuffer())
+            .setSequenceIdTable(scmMetadataStore.getSequenceIdTable())
+            .build());
+    SequenceIdGenerator sequenceIdGen = new SequenceIdGenerator(
+        conf, scmHAManager, scmMetadataStore.getSequenceIdTable()) {
+      @Override
+      public StateManager createStateManager(
+          SCMHAManager scmhaManager, Table<String, Long> sequenceIdTable) {
+        Preconditions.checkNotNull(scmhaManager);
+        return stateManager;
+      }
+    };
+
+    Assertions.assertEquals(1L, sequenceIdGen.getNextId("someKey"));
+
+    // Simulation currently this SCM is not a leader node,
+    // So this SCM can only allocate IDs within the current batch
+    // ([1, batchSize]), does not allow the allocation of IDs for the next 
batch
+    // ([batchSize + 1, batchSize * 2])
+    when(stateManager.allocateBatch(anyString(), anyLong(), anyLong()))
+        .thenThrow(new SCMException(SCMException.ResultCodes.SCM_NOT_LEADER));
+
+    for (int i = 0; i < batchSize * 3; i++) {
+      try {
+        long nextID = sequenceIdGen.getNextId("someKey");
+        if (nextID > batchSize) {
+          Assertions.fail("Should not allocate a blockID: " + nextID +
+              " that exceeds the current Batch: " + batchSize);
+        }
+      } catch (Exception e) {
+        // ignore
+      }
+    }
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to