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

hemant 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 1d79d07673 HDDS-9215. Snapshot Chain corruption because snapshot chain 
need not be created in increasing order of CreatedTime (#5223)
1d79d07673 is described below

commit 1d79d076739882edee63878e686266ce9930694d
Author: Swaminathan Balachandran <[email protected]>
AuthorDate: Thu Aug 31 10:23:52 2023 -0700

    HDDS-9215. Snapshot Chain corruption because snapshot chain need not be 
created in increasing order of CreatedTime (#5223)
---
 .../apache/hadoop/ozone/om/SnapshotChainInfo.java  |   8 ++
 .../hadoop/ozone/om/SnapshotChainManager.java      | 115 ++++++++++++++++-----
 .../request/snapshot/OMSnapshotCreateRequest.java  |   8 +-
 .../apache/hadoop/ozone/om/TestSnapshotChain.java  | 110 ++++++++++++++++++--
 4 files changed, 202 insertions(+), 39 deletions(-)

diff --git 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainInfo.java
 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainInfo.java
index 239ed9be64..74fec97218 100644
--- 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainInfo.java
+++ 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainInfo.java
@@ -53,6 +53,14 @@ public class SnapshotChainInfo {
     return nextSnapshotId;
   }
 
+  public boolean hasNextSnapshotId() {
+    return Objects.nonNull(getNextSnapshotId());
+  }
+
+  public boolean hasPreviousSnapshotId() {
+    return Objects.nonNull(getPreviousSnapshotId());
+  }
+
   public UUID getPreviousSnapshotId() {
     return previousSnapshotId;
   }
diff --git 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java
 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java
index b47d851519..f87383756e 100644
--- 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java
+++ 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java
@@ -25,10 +25,10 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.NoSuchElementException;
-import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -68,20 +68,42 @@ public class SnapshotChainManager {
   /**
    * Add snapshot to global snapshot chain.
    */
-  private void addSnapshotGlobal(UUID snapshotID,
-                                 UUID prevGlobalID) throws IOException {
-    // On add snapshot, set previous snapshot entry nextSnapshotID = snapshotID
-    if (prevGlobalID != null && globalSnapshotChain.containsKey(prevGlobalID)) 
{
-      globalSnapshotChain.get(prevGlobalID).setNextSnapshotId(snapshotID);
+  private void addSnapshotGlobal(UUID snapshotID, UUID prevGlobalID) {
+    if (globalSnapshotChain.containsKey(snapshotID)) {
+      throw new IllegalStateException(String.format(
+          "Global Snapshot chain corruption. Snapshot with snapshotId: %s is " 
+
+              "already present in the chain.", snapshotID));
+    }
+    if (globalSnapshotChain.size() > 0 && prevGlobalID == null) {
+      throw new IllegalStateException(String.format("Snapshot chain " +
+          "corruption. Adding snapshot %s as head node while there are %d " +
+              "snapshots in the global snapshot chain.", snapshotID,
+          globalSnapshotChain.size()));
     }
 
     if (prevGlobalID != null &&
         !globalSnapshotChain.containsKey(prevGlobalID)) {
-      throw new IOException(String.format("Snapshot chain corruption. " +
-              "Previous snapshotId: %s is set for snapshotId: %s but no " +
-              "associated snapshot found in snapshot chain.", prevGlobalID,
-          snapshotID));
+      throw new IllegalStateException(String.format(
+          "Global Snapshot chain corruption. Previous snapshotId: %s is " +
+              "set for snapshotId: %s but no associated snapshot found in " +
+              "snapshot chain.", prevGlobalID, snapshotID));
+    }
+
+    if (prevGlobalID != null) {
+      if (globalSnapshotChain.get(prevGlobalID).hasNextSnapshotId()) {
+        throw new IllegalStateException(String.format(
+            "Global Snapshot chain corruption. Snapshot with snapshotId: %s " +
+                "already has the next snapshotId: %s. Adding snapshot %s " +
+                "with prevSnapshotId: %s will make the chain non linear.",
+            prevGlobalID,
+            globalSnapshotChain.get(prevGlobalID).getNextSnapshotId(),
+            snapshotID, prevGlobalID));
+      }
+      // On add snapshot, set previous snapshot entry nextSnapshotID =
+      // snapshotID
+      globalSnapshotChain.get(prevGlobalID).setNextSnapshotId(snapshotID);
     }
+
     globalSnapshotChain.put(snapshotID,
         new SnapshotChainInfo(snapshotID, prevGlobalID, null));
 
@@ -92,20 +114,38 @@ public class SnapshotChainManager {
   /**
    * Add snapshot to bucket snapshot chain(path based).
    */
-  private void addSnapshotPath(String snapshotPath,
-                               UUID snapshotID,
-                               UUID prevPathID) throws IOException {
+  private void addSnapshotPath(String snapshotPath, UUID snapshotID,
+                               UUID prevPathID) {
     // On add snapshot, set previous snapshot entry nextSnapshotId = snapshotId
     if (prevPathID != null &&
         ((!snapshotChainByPath.containsKey(snapshotPath)) ||
             (!snapshotChainByPath.get(snapshotPath).containsKey(prevPathID)))) 
{
-      throw new IOException(String.format("Snapshot chain corruption. " +
-              "Previous snapshotId: %s is set for snapshotId: %s but no " +
-              "associated snapshot found in snapshot chain.", prevPathID,
-          snapshotID));
+      throw new IllegalStateException(String.format(
+          "Path Snapshot chain corruption. Previous snapshotId: %s is set " +
+              "for snapshotId: %s but no associated snapshot found in " +
+              "snapshot chain.", prevPathID, snapshotID));
+    }
+
+    if (prevPathID == null && snapshotChainByPath.containsKey(snapshotPath) &&
+        !snapshotChainByPath.get(snapshotPath).isEmpty()) {
+      throw new IllegalStateException(String.format(
+          "Path Snapshot chain corruption. Error while adding snapshot with " +
+              "snapshotId %s with as the first snapshot in snapshot path: " +
+              "%s which already has %d snapshots.", snapshotID, snapshotPath,
+          snapshotChainByPath.get(snapshotPath).size()));
     }
 
     if (prevPathID != null && snapshotChainByPath.containsKey(snapshotPath)) {
+      if (snapshotChainByPath.get(snapshotPath).get(prevPathID)
+          .hasNextSnapshotId()) {
+        throw new IllegalStateException(String.format(
+            "Path Snapshot chain corruption. Next snapshotId: %s is already " +
+                "set for snapshotId: %s. Adding snapshotId: %s with " +
+                "prevSnapshotId: %s will make the chain non linear.",
+            snapshotChainByPath.get(snapshotPath).get(prevPathID)
+                .getNextSnapshotId(), prevPathID,
+            snapshotID, prevPathID));
+      }
       snapshotChainByPath
           .get(snapshotPath)
           .get(prevPathID)
@@ -234,7 +274,10 @@ public class SnapshotChainManager {
     // snapshot chains - both global and local path
     try (TableIterator<String, ? extends Table.KeyValue<String, SnapshotInfo>>
              keyIter = metadataManager.getSnapshotInfoTable().iterator()) {
-      Map<Long, SnapshotInfo> snaps = new TreeMap<>();
+      Map<UUID, SnapshotInfo> snaps = new HashMap<>();
+      // Forward Linked list for snapshot chain.
+      Map<UUID, UUID> snapshotToNextSnapshotMap = new HashMap<>();
+      UUID head = null;
       Table.KeyValue<String, SnapshotInfo> kv;
       globalSnapshotChain.clear();
       snapshotChainByPath.clear();
@@ -243,10 +286,37 @@ public class SnapshotChainManager {
 
       while (keyIter.hasNext()) {
         kv = keyIter.next();
-        snaps.put(kv.getValue().getCreationTime(), kv.getValue());
+        SnapshotInfo snapshotInfo = kv.getValue();
+        snaps.put(kv.getValue().getSnapshotId(), snapshotInfo);
+        if (snapshotInfo.getGlobalPreviousSnapshotId() != null) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Global Snapshot chain link {} -> {}",
+                snapshotInfo.getGlobalPreviousSnapshotId(),
+                snapshotInfo.getSnapshotId());
+          }
+          // Adding edge to the linked list. prevGlobalSnapId -> snapId
+          snapshotToNextSnapshotMap.put(
+              snapshotInfo.getGlobalPreviousSnapshotId(),
+              snapshotInfo.getSnapshotId());
+        } else {
+          head = snapshotInfo.getSnapshotId();
+        }
       }
-      for (SnapshotInfo snapshotInfo : snaps.values()) {
-        addSnapshot(snapshotInfo);
+      int size = 0;
+      UUID prev = null;
+      while (head != null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Adding Snapshot Info: {}", snaps.get(head));
+        }
+        addSnapshot(snaps.get(head));
+        size += 1;
+        prev = head;
+        head = snapshotToNextSnapshotMap.get(head);
+      }
+      if (size != snaps.size()) {
+        throw new IllegalStateException(String.format("Snapshot chain " +
+            "corruption. All snapshots have not been added to the " +
+            "snapshot chain. Last snapshot added to chain : %s", prev));
       }
     } catch (IOException ioException) {
       // TODO: [SNAPSHOT] Fail gracefully.
@@ -257,8 +327,7 @@ public class SnapshotChainManager {
   /**
    * Add snapshot to snapshot chain.
    */
-  public synchronized void addSnapshot(SnapshotInfo snapshotInfo)
-      throws IOException {
+  public synchronized void addSnapshot(SnapshotInfo snapshotInfo) {
     addSnapshotGlobal(snapshotInfo.getSnapshotId(),
         snapshotInfo.getGlobalPreviousSnapshotId());
     addSnapshotPath(snapshotInfo.getSnapshotPath(),
diff --git 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java
 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java
index 95c78e2be1..d5d261c7f0 100644
--- 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java
+++ 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java
@@ -220,9 +220,7 @@ public class OMSnapshotCreateRequest extends 
OMClientRequest {
    * it was removed at T-5.
    */
   private void addSnapshotInfoToSnapshotChainAndCache(
-      OmMetadataManagerImpl omMetadataManager,
-      long transactionLogIndex
-  ) throws IOException {
+      OmMetadataManagerImpl omMetadataManager, long transactionLogIndex)  {
     // It is synchronized on SnapshotChainManager object so that this block is
     // synchronized with OMSnapshotPurgeResponse#cleanupSnapshotChain and only
     // one of these two operation gets executed at a time otherwise we could be
@@ -245,7 +243,7 @@ public class OMSnapshotCreateRequest extends 
OMClientRequest {
         omMetadataManager.getSnapshotInfoTable()
             .addCacheEntry(new CacheKey<>(snapshotInfo.getTableKey()),
                 CacheValue.get(transactionLogIndex, snapshotInfo));
-      } catch (IOException ioException) {
+      } catch (IllegalStateException illegalStateException) {
         // Remove snapshot from the SnapshotChainManager in case of any 
failure.
         // It is possible that createSnapshot request fails after snapshot gets
         // added to snapshot chain manager because couldn't add it to cache/DB.
@@ -258,7 +256,7 @@ public class OMSnapshotCreateRequest extends 
OMClientRequest {
         // added to the SnapshotInfo table.
         removeSnapshotInfoFromSnapshotChainManager(snapshotChainManager,
             snapshotInfo);
-        throw ioException;
+        throw illegalStateException;
       }
     }
   }
diff --git 
a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotChain.java
 
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotChain.java
index 53527a3f37..fb473fb700 100644
--- 
a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotChain.java
+++ 
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotChain.java
@@ -17,21 +17,32 @@
  */
 package org.apache.hadoop.ozone.om;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.utils.db.Table;
 import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
-import org.apache.hadoop.util.Time;
+import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Named;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.io.TempDir;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.junit.jupiter.params.provider.ValueSource;
 
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
 
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_DB_DIRS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -65,14 +76,15 @@ public class TestSnapshotChain {
 
   private SnapshotInfo createSnapshotInfo(UUID snapshotID,
                                           UUID pathPrevID,
-                                          UUID globalPrevID) {
+                                          UUID globalPrevID,
+                                          long creationTime) {
     return new SnapshotInfo.Builder()
         .setSnapshotId(snapshotID)
         .setName("test")
         .setVolumeName("vol1")
         .setBucketName("bucket1")
         .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE)
-        .setCreationTime(Time.now())
+        .setCreationTime(creationTime)
         .setDeletionTime(-1L)
         .setPathPreviousSnapshotId(pathPrevID)
         .setGlobalPreviousSnapshotId(globalPrevID)
@@ -141,13 +153,14 @@ public class TestSnapshotChain {
     snapshotIDs.add(snapshotID3);
 
     UUID prevSnapshotID = null;
-
+    List<Integer> times = Arrays.asList(1, 2, 0);
+    int timeIdx = 0;
     // add 3 snapshots
     for (UUID snapshotID : snapshotIDs) {
       chainManager.addSnapshot(createSnapshotInfo(
           snapshotID,
           prevSnapshotID,
-          prevSnapshotID));
+          prevSnapshotID, times.get(timeIdx++)));
       prevSnapshotID = snapshotID;
     }
 
@@ -203,14 +216,15 @@ public class TestSnapshotChain {
     snapshotIDs.add(snapshotID3);
 
     UUID prevSnapshotID = null;
-
+    List<Integer> times = Arrays.asList(1, 2, 0);
+    int timeIdx = 0;
     // add 3 snapshots
     for (UUID snapshotID : snapshotIDs) {
       snapshotIdToSnapshotInfoMap.put(snapshotID,
           createSnapshotInfo(
               snapshotID,
               prevSnapshotID,
-              prevSnapshotID));
+              prevSnapshotID, times.get(timeIdx++)));
 
       chainManager.addSnapshot(snapshotIdToSnapshotInfoMap.get(snapshotID));
       prevSnapshotID = snapshotID;
@@ -239,8 +253,10 @@ public class TestSnapshotChain {
     assertEquals(snapshotID3, 
chainManager.previousGlobalSnapshot(snapshotID2));
   }
 
-  @Test
-  public void testChainFromLoadFromTable() throws Exception {
+  @ParameterizedTest
+  @ValueSource(booleans = {true, false})
+  public void testChainFromLoadFromTable(boolean increasingTIme)
+      throws Exception {
     Table<String, SnapshotInfo> snapshotInfo =
             omMetadataManager.getSnapshotInfoTable();
 
@@ -253,11 +269,12 @@ public class TestSnapshotChain {
     snapshotIDs.add(snapshotID2);
 
     UUID prevSnapshotID = null;
-
+    long time = System.currentTimeMillis();
     // add 3 snapshots
     for (UUID snapshotID : snapshotIDs) {
       snapshotInfo.put(snapshotID.toString(),
-          createSnapshotInfo(snapshotID, prevSnapshotID, prevSnapshotID));
+          createSnapshotInfo(snapshotID, prevSnapshotID, prevSnapshotID,
+          increasingTIme ? time++ : time--));
       prevSnapshotID = snapshotID;
     }
 
@@ -273,4 +290,75 @@ public class TestSnapshotChain {
         () -> chainManager.previousPathSnapshot(String
             .join("/", "vol1", "bucket1"), snapshotID1));
   }
+
+  private static Stream<? extends Arguments> invalidSnapshotChain() {
+    List<UUID> nodes = IntStream.range(0, 5)
+        .mapToObj(i -> UUID.randomUUID())
+        .collect(Collectors.toList());
+    return Stream.of(
+        Arguments.of(nodes, Named.of("Disconnected Snapshot Chain",
+            ImmutableMap.of(
+                nodes.get(1), nodes.get(0),
+                nodes.get(2), nodes.get(1),
+                nodes.get(4), nodes.get(3)))),
+        Arguments.of(nodes, Named.of("Complete Cyclic Snapshot Chain",
+            ImmutableMap.of(
+                nodes.get(0), nodes.get(4),
+                nodes.get(1), nodes.get(0),
+                nodes.get(2), nodes.get(1),
+                nodes.get(3), nodes.get(2),
+                nodes.get(4), nodes.get(3)))),
+        Arguments.of(nodes, Named.of("Partial Cyclic Snapshot Chain",
+            ImmutableMap.of(
+                nodes.get(0), nodes.get(3),
+                nodes.get(1), nodes.get(0),
+                nodes.get(2), nodes.get(1),
+                nodes.get(3), nodes.get(2),
+                nodes.get(4), nodes.get(3)))),
+        Arguments.of(nodes, Named.of("Diverged Snapshot Chain",
+            ImmutableMap.of(nodes.get(1), nodes.get(0),
+                nodes.get(2), nodes.get(1),
+                nodes.get(3), nodes.get(2),
+                nodes.get(4), nodes.get(2))))
+    );
+  }
+
+  @ParameterizedTest
+  @MethodSource("invalidSnapshotChain")
+  public void testInvalidGlobalChainFromLoadFromTable(
+      List<UUID> snapshotIDs, Map<UUID, UUID> snapshotChain) throws Exception {
+    Table<String, SnapshotInfo> snapshotInfo =
+        omMetadataManager.getSnapshotInfoTable();
+    for (UUID snapshotID : snapshotIDs) {
+      snapshotInfo.put(snapshotID.toString(),
+          createSnapshotInfo(snapshotID, snapshotChain.get(snapshotID),
+              snapshotChain.get(snapshotID), System.currentTimeMillis()));
+    }
+    IllegalStateException exception = Assertions.assertThrows(
+        IllegalStateException.class,
+        () -> new SnapshotChainManager(omMetadataManager));
+    Assertions.assertTrue(exception.getMessage()
+        .startsWith("Snapshot chain corruption. All snapshots have not been " +
+            "added to the snapshot chain."));
+  }
+
+  @ParameterizedTest
+  @MethodSource("invalidSnapshotChain")
+  public void testInvalidChainFromLoadFromTable(List<UUID> snapshotIDs,
+      Map<UUID, UUID> snapshotChain) throws Exception {
+    Table<String, SnapshotInfo> snapshotInfo =
+        omMetadataManager.getSnapshotInfoTable();
+    UUID prevSnapshotId = null;
+    for (UUID snapshotID : snapshotIDs) {
+      snapshotInfo.put(snapshotID.toString(),
+          createSnapshotInfo(snapshotID, snapshotChain.get(snapshotID),
+              prevSnapshotId, System.currentTimeMillis()));
+      prevSnapshotId = snapshotID;
+    }
+    IllegalStateException exception = Assertions.assertThrows(
+        IllegalStateException.class,
+        () -> new SnapshotChainManager(omMetadataManager));
+    Assertions.assertTrue(exception.getMessage()
+        .startsWith("Path Snapshot chain corruption."));
+  }
 }


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

Reply via email to