You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/09/16 16:46:16 UTC

[GitHub] [ozone] sadanand48 commented on a diff in pull request #3658: HDDS-6983. Snapshot Chain - list of snapshots per snapshottable bucket

sadanand48 commented on code in PR #3658:
URL: https://github.com/apache/ozone/pull/3658#discussion_r973083050


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -0,0 +1,446 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.LinkedHashMap;
+import java.util.TreeMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is used for creating and accessing Snapshot Chains.
+ *
+ * The snapshot chain maintains the in-memory sequence of snapshots
+ * created in chronological order.  There are two such snapshots maintained
+ * i.) Path based snapshot chain, sequence of snapshots created for a
+ * given /volume/bucket
+ * ii.) Global snapshot chain, sequence of all snapshots created in order

Review Comment:
   What’s the use case for maintaining global snapshot order? Is it for supporting snapshots other than at the bucket level?



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -0,0 +1,446 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.LinkedHashMap;
+import java.util.TreeMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is used for creating and accessing Snapshot Chains.
+ *
+ * The snapshot chain maintains the in-memory sequence of snapshots
+ * created in chronological order.  There are two such snapshots maintained
+ * i.) Path based snapshot chain, sequence of snapshots created for a
+ * given /volume/bucket
+ * ii.) Global snapshot chain, sequence of all snapshots created in order
+ *
+ * On start, the snapshot chains are initialized from the on disk
+ * SnapshotInfoTable from the om RocksDB.
+ */
+public class SnapshotChainManager {
+  private LinkedHashMap<String, SnapshotChainInfo>  snapshotChainGlobal;
+  private Map<String, LinkedHashMap<String, SnapshotChainInfo>>
+      snapshotChainPath;
+  private Map<String, String> latestPathSnapshotID;
+  private String latestGlobalSnapshotID;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotChainManager.class);
+
+  public SnapshotChainManager(OMMetadataManager metadataManager)
+          throws IOException {
+    snapshotChainGlobal = new LinkedHashMap<>();
+    snapshotChainPath = new HashMap<>();
+    latestPathSnapshotID = new HashMap<>();
+    latestGlobalSnapshotID = null;
+    loadFromSnapshotInfoTable(metadataManager);
+  }
+
+  private void addSnapshotGlobal(String snapshotID,
+                            String prevGlobalID) throws IOException {
+    // set previous snapshotID to null if it is "" for
+    // internal in-mem structure
+    if (prevGlobalID != null && prevGlobalID.equals("")) {
+      prevGlobalID = null;
+    }
+    // on add snapshot; set previous snapshot entry nextSnapshotID =
+    // snapshotID
+    if (prevGlobalID != null &&
+        snapshotChainGlobal.containsKey(prevGlobalID)) {
+      snapshotChainGlobal
+          .get(prevGlobalID)
+          .setNextSnapshotID(snapshotID);
+    }
+    if (prevGlobalID != null &&
+        !snapshotChainGlobal.containsKey(prevGlobalID)) {
+      throw new IOException("Snapshot Chain corruption: "
+          + " previous snapshotID given but no associated snapshot "
+      + "found in snapshot chain: SnapshotID "
+      + prevGlobalID);
+    }
+    snapshotChainGlobal.put(snapshotID,
+            new SnapshotChainInfo(snapshotID, prevGlobalID, null));
+
+    // set state variable latestGlobal snapshot entry to this snapshotID
+    latestGlobalSnapshotID = snapshotID;

Review Comment:
   Shouldn't the variables `latestGlobalSnapshotID` & `latestPathSnapshotID` be thread-safe? 



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -0,0 +1,446 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.LinkedHashMap;
+import java.util.TreeMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is used for creating and accessing Snapshot Chains.
+ *
+ * The snapshot chain maintains the in-memory sequence of snapshots
+ * created in chronological order.  There are two such snapshots maintained
+ * i.) Path based snapshot chain, sequence of snapshots created for a
+ * given /volume/bucket
+ * ii.) Global snapshot chain, sequence of all snapshots created in order
+ *
+ * On start, the snapshot chains are initialized from the on disk
+ * SnapshotInfoTable from the om RocksDB.
+ */
+public class SnapshotChainManager {
+  private LinkedHashMap<String, SnapshotChainInfo>  snapshotChainGlobal;
+  private Map<String, LinkedHashMap<String, SnapshotChainInfo>>
+      snapshotChainPath;
+  private Map<String, String> latestPathSnapshotID;
+  private String latestGlobalSnapshotID;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotChainManager.class);
+
+  public SnapshotChainManager(OMMetadataManager metadataManager)
+          throws IOException {
+    snapshotChainGlobal = new LinkedHashMap<>();
+    snapshotChainPath = new HashMap<>();
+    latestPathSnapshotID = new HashMap<>();
+    latestGlobalSnapshotID = null;
+    loadFromSnapshotInfoTable(metadataManager);
+  }
+
+  private void addSnapshotGlobal(String snapshotID,
+                            String prevGlobalID) throws IOException {
+    // set previous snapshotID to null if it is "" for
+    // internal in-mem structure
+    if (prevGlobalID != null && prevGlobalID.equals("")) {
+      prevGlobalID = null;
+    }
+    // on add snapshot; set previous snapshot entry nextSnapshotID =
+    // snapshotID
+    if (prevGlobalID != null &&
+        snapshotChainGlobal.containsKey(prevGlobalID)) {
+      snapshotChainGlobal
+          .get(prevGlobalID)
+          .setNextSnapshotID(snapshotID);
+    }
+    if (prevGlobalID != null &&
+        !snapshotChainGlobal.containsKey(prevGlobalID)) {
+      throw new IOException("Snapshot Chain corruption: "
+          + " previous snapshotID given but no associated snapshot "
+      + "found in snapshot chain: SnapshotID "
+      + prevGlobalID);
+    }
+    snapshotChainGlobal.put(snapshotID,
+            new SnapshotChainInfo(snapshotID, prevGlobalID, null));
+
+    // set state variable latestGlobal snapshot entry to this snapshotID
+    latestGlobalSnapshotID = snapshotID;
+  };
+
+  private void addSnapshotPath(String snapshotPath,
+                               String snapshotID,
+                                String prevPathID) throws IOException {
+    // set previous snapshotID to null if it is "" for
+    // internal in-mem structure
+    if (prevPathID != null && prevPathID.equals("")) {
+      prevPathID = null;
+    }
+
+    // on add snapshot; set previous snapshot entry nextSnapshotID =
+    // snapshotID
+    if (prevPathID != null &&
+        ((!snapshotChainPath
+            .containsKey(snapshotPath)) ||
+        (!snapshotChainPath
+            .get(snapshotPath)
+            .containsKey(prevPathID)))) {
+      throw new IOException("Snapshot Chain corruption: "
+          + "previous snapshotID given but no associated snapshot "
+          + "found in snapshot chain: SnapshotID "
+          + prevPathID);
+    }
+
+    if (prevPathID != null &&
+            snapshotChainPath.containsKey(snapshotPath)) {
+      snapshotChainPath
+              .get(snapshotPath)
+              .get(prevPathID)
+              .setNextSnapshotID(snapshotID);
+    }
+
+    if (!snapshotChainPath.containsKey(snapshotPath)) {
+      snapshotChainPath.put(snapshotPath, new LinkedHashMap<>());
+    }
+
+    snapshotChainPath
+        .get(snapshotPath)
+        .put(snapshotID,
+                new SnapshotChainInfo(snapshotID, prevPathID, null));
+
+    // set state variable latestPath snapshot entry to this snapshotID
+    latestPathSnapshotID.put(snapshotPath, snapshotID);
+  };
+
+  private boolean deleteSnapshotGlobal(String snapshotID) throws IOException {
+    boolean status = true;
+    if (snapshotChainGlobal.containsKey(snapshotID)) {
+      // reset prev and next snapshot entries in chain ordered list
+      // for node removal
+      String next = snapshotChainGlobal.get(snapshotID).getNextSnapshotID();
+      String prev = snapshotChainGlobal.get(snapshotID).getPreviousSnapshotID();
+
+      if (prev != null && !snapshotChainGlobal.containsKey(prev)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to be "
+                + "deleted has prev node element not found in snapshot chain: "
+                + "SnapshotID " + prev);
+      }
+      if (next != null && !snapshotChainGlobal.containsKey(next)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to be "
+                + "deleted has next node element not found in snapshot chain: "
+                + "SnapshotID " + next);
+      }
+      snapshotChainGlobal.remove(snapshotID);
+      if (next != null) {
+        snapshotChainGlobal.get(next).setPreviousSnapshotID(prev);
+      }
+      if (prev != null) {
+        snapshotChainGlobal.get(prev).setNextSnapshotID(next);
+      }
+      // remove from latest list if necessary
+      if (latestGlobalSnapshotID.equals(snapshotID)) {
+        latestGlobalSnapshotID = prev;
+      }
+    } else {
+      // snapshotID not found in snapshot chain, log warning and return
+      LOG.warn("Snapshot chain: snapshotID not found: SnapshotID {}",
+          snapshotID);
+    }
+
+    return status;
+  }
+
+  private boolean deleteSnapshotPath(String snapshotPath,
+                                     String snapshotID) throws IOException {
+    boolean status = true;
+    if (snapshotChainPath.containsKey(snapshotPath) &&
+            snapshotChainPath
+                    .get(snapshotPath)
+                    .containsKey(snapshotID)) {
+      // reset prev and next snapshot entries in chain ordered list
+      // for node removal
+      String next = snapshotChainPath
+          .get(snapshotPath)
+          .get(snapshotID)
+          .getNextSnapshotID();
+      String prev = snapshotChainPath
+          .get(snapshotPath)
+          .get(snapshotID)
+          .getPreviousSnapshotID();
+
+      if (prev != null &&
+          !snapshotChainPath
+              .get(snapshotPath)
+              .containsKey(prev)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to "
+                + "be deleted has prev node element not found in snapshot "
+                + "chain: Snapshot path " + snapshotPath + ", SnapshotID "
+                + prev);
+      }
+      if (next != null && !snapshotChainPath
+          .get(snapshotPath)
+          .containsKey(next)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to "
+                + "be deleted has next node element not found in snapshot "
+                + "chain:  Snapshot path " + snapshotPath + ", SnapshotID "
+                + next);
+      }
+      snapshotChainPath
+          .get(snapshotPath)
+          .remove(snapshotID);
+      if (next != null) {
+        snapshotChainPath
+            .get(snapshotPath)
+            .get(next)
+            .setPreviousSnapshotID(prev);
+      }
+      if (prev != null) {
+        snapshotChainPath
+            .get(snapshotPath)
+            .get(prev)
+            .setNextSnapshotID(next);
+      }
+      // remove from latest list if necessary
+      if (latestPathSnapshotID.get(snapshotPath).equals(snapshotID)) {
+        latestPathSnapshotID.remove(snapshotPath);
+        if (prev != null) {
+          latestPathSnapshotID.put(snapshotPath, prev);
+        }
+      }
+
+    } else {
+      // snapshotID not found in snapshot chain, log warning and return
+      LOG.warn("Snapshot chain: snapshotID not found: Snapshot path {}," +
+              " SnapshotID {}",
+          snapshotPath, snapshotID);
+    }
+
+    return status;
+  }
+
+  private void loadFromSnapshotInfoTable(OMMetadataManager metadataManager)
+          throws IOException {
+    // read from snapshotInfo table to populate
+    // snapshot chains - both global and local path
+    TableIterator<String, ? extends Table.KeyValue<String, SnapshotInfo>>
+            keyIter = metadataManager.getSnapshotInfoTable().iterator();
+    Map<Long, SnapshotInfo> snaps = new TreeMap<>();
+    Table.KeyValue< String, SnapshotInfo > kv;
+    snapshotChainGlobal.clear();
+    snapshotChainPath.clear();
+
+    while (keyIter.hasNext()) {
+      kv = keyIter.next();
+      snaps.put(kv.getValue().getCreationTime(), kv.getValue());
+    }
+    for (SnapshotInfo sinfo : snaps.values()) {
+      addSnapshot(sinfo);
+    }
+  }
+
+  public void addSnapshot(SnapshotInfo sinfo) throws IOException {
+    addSnapshotGlobal(sinfo.getSnapshotID(),
+        sinfo.getGlobalPreviousSnapshotID());
+    addSnapshotPath(sinfo.getSnapshotPath(),
+        sinfo.getSnapshotID(),
+        sinfo.getPathPreviousSnapshotID());
+  }
+
+  public boolean deleteSnapshot(SnapshotInfo sinfo) throws IOException {
+    boolean status;
+
+    status = deleteSnapshotGlobal(sinfo.getSnapshotID());
+    return status && deleteSnapshotPath(sinfo.getSnapshotPath(),
+        sinfo.getSnapshotID());
+  }
+
+  public String getLatestGlobalSnapshot() {
+    return latestGlobalSnapshotID;
+  }
+
+  public String getLatestPathSnapshot(String snapshotPath) {
+    return latestPathSnapshotID.get(snapshotPath);
+  }
+
+  public boolean hasNextGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    boolean hasNext = false;
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getNextSnapshotID() != null) {
+      hasNext = true;
+    }
+    return hasNext;
+  }
+
+  public String nextGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getNextSnapshotID() == null) {

Review Comment:
   The above `hasNextGlobalSnapshot(snapshotID)` could be used here  instead



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -0,0 +1,446 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.LinkedHashMap;
+import java.util.TreeMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is used for creating and accessing Snapshot Chains.
+ *
+ * The snapshot chain maintains the in-memory sequence of snapshots
+ * created in chronological order.  There are two such snapshots maintained
+ * i.) Path based snapshot chain, sequence of snapshots created for a
+ * given /volume/bucket
+ * ii.) Global snapshot chain, sequence of all snapshots created in order
+ *
+ * On start, the snapshot chains are initialized from the on disk
+ * SnapshotInfoTable from the om RocksDB.
+ */
+public class SnapshotChainManager {
+  private LinkedHashMap<String, SnapshotChainInfo>  snapshotChainGlobal;
+  private Map<String, LinkedHashMap<String, SnapshotChainInfo>>
+      snapshotChainPath;
+  private Map<String, String> latestPathSnapshotID;
+  private String latestGlobalSnapshotID;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotChainManager.class);
+
+  public SnapshotChainManager(OMMetadataManager metadataManager)
+          throws IOException {
+    snapshotChainGlobal = new LinkedHashMap<>();
+    snapshotChainPath = new HashMap<>();
+    latestPathSnapshotID = new HashMap<>();
+    latestGlobalSnapshotID = null;
+    loadFromSnapshotInfoTable(metadataManager);
+  }
+
+  private void addSnapshotGlobal(String snapshotID,
+                            String prevGlobalID) throws IOException {
+    // set previous snapshotID to null if it is "" for
+    // internal in-mem structure
+    if (prevGlobalID != null && prevGlobalID.equals("")) {
+      prevGlobalID = null;
+    }
+    // on add snapshot; set previous snapshot entry nextSnapshotID =
+    // snapshotID
+    if (prevGlobalID != null &&
+        snapshotChainGlobal.containsKey(prevGlobalID)) {
+      snapshotChainGlobal
+          .get(prevGlobalID)
+          .setNextSnapshotID(snapshotID);
+    }
+    if (prevGlobalID != null &&
+        !snapshotChainGlobal.containsKey(prevGlobalID)) {
+      throw new IOException("Snapshot Chain corruption: "
+          + " previous snapshotID given but no associated snapshot "
+      + "found in snapshot chain: SnapshotID "
+      + prevGlobalID);
+    }
+    snapshotChainGlobal.put(snapshotID,
+            new SnapshotChainInfo(snapshotID, prevGlobalID, null));
+
+    // set state variable latestGlobal snapshot entry to this snapshotID
+    latestGlobalSnapshotID = snapshotID;
+  };
+
+  private void addSnapshotPath(String snapshotPath,
+                               String snapshotID,
+                                String prevPathID) throws IOException {
+    // set previous snapshotID to null if it is "" for
+    // internal in-mem structure
+    if (prevPathID != null && prevPathID.equals("")) {
+      prevPathID = null;
+    }
+
+    // on add snapshot; set previous snapshot entry nextSnapshotID =
+    // snapshotID
+    if (prevPathID != null &&
+        ((!snapshotChainPath
+            .containsKey(snapshotPath)) ||
+        (!snapshotChainPath
+            .get(snapshotPath)
+            .containsKey(prevPathID)))) {
+      throw new IOException("Snapshot Chain corruption: "
+          + "previous snapshotID given but no associated snapshot "
+          + "found in snapshot chain: SnapshotID "
+          + prevPathID);
+    }
+
+    if (prevPathID != null &&
+            snapshotChainPath.containsKey(snapshotPath)) {
+      snapshotChainPath
+              .get(snapshotPath)
+              .get(prevPathID)
+              .setNextSnapshotID(snapshotID);
+    }
+
+    if (!snapshotChainPath.containsKey(snapshotPath)) {
+      snapshotChainPath.put(snapshotPath, new LinkedHashMap<>());
+    }
+
+    snapshotChainPath
+        .get(snapshotPath)
+        .put(snapshotID,
+                new SnapshotChainInfo(snapshotID, prevPathID, null));
+
+    // set state variable latestPath snapshot entry to this snapshotID
+    latestPathSnapshotID.put(snapshotPath, snapshotID);
+  };
+
+  private boolean deleteSnapshotGlobal(String snapshotID) throws IOException {
+    boolean status = true;
+    if (snapshotChainGlobal.containsKey(snapshotID)) {
+      // reset prev and next snapshot entries in chain ordered list
+      // for node removal
+      String next = snapshotChainGlobal.get(snapshotID).getNextSnapshotID();
+      String prev = snapshotChainGlobal.get(snapshotID).getPreviousSnapshotID();
+
+      if (prev != null && !snapshotChainGlobal.containsKey(prev)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to be "
+                + "deleted has prev node element not found in snapshot chain: "
+                + "SnapshotID " + prev);
+      }
+      if (next != null && !snapshotChainGlobal.containsKey(next)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to be "
+                + "deleted has next node element not found in snapshot chain: "
+                + "SnapshotID " + next);
+      }
+      snapshotChainGlobal.remove(snapshotID);
+      if (next != null) {
+        snapshotChainGlobal.get(next).setPreviousSnapshotID(prev);
+      }
+      if (prev != null) {
+        snapshotChainGlobal.get(prev).setNextSnapshotID(next);
+      }
+      // remove from latest list if necessary
+      if (latestGlobalSnapshotID.equals(snapshotID)) {
+        latestGlobalSnapshotID = prev;
+      }
+    } else {
+      // snapshotID not found in snapshot chain, log warning and return
+      LOG.warn("Snapshot chain: snapshotID not found: SnapshotID {}",
+          snapshotID);
+    }
+
+    return status;
+  }
+
+  private boolean deleteSnapshotPath(String snapshotPath,
+                                     String snapshotID) throws IOException {
+    boolean status = true;
+    if (snapshotChainPath.containsKey(snapshotPath) &&
+            snapshotChainPath
+                    .get(snapshotPath)
+                    .containsKey(snapshotID)) {
+      // reset prev and next snapshot entries in chain ordered list
+      // for node removal
+      String next = snapshotChainPath
+          .get(snapshotPath)
+          .get(snapshotID)
+          .getNextSnapshotID();
+      String prev = snapshotChainPath
+          .get(snapshotPath)
+          .get(snapshotID)
+          .getPreviousSnapshotID();
+
+      if (prev != null &&
+          !snapshotChainPath
+              .get(snapshotPath)
+              .containsKey(prev)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to "
+                + "be deleted has prev node element not found in snapshot "
+                + "chain: Snapshot path " + snapshotPath + ", SnapshotID "
+                + prev);
+      }
+      if (next != null && !snapshotChainPath
+          .get(snapshotPath)
+          .containsKey(next)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to "
+                + "be deleted has next node element not found in snapshot "
+                + "chain:  Snapshot path " + snapshotPath + ", SnapshotID "
+                + next);
+      }
+      snapshotChainPath
+          .get(snapshotPath)
+          .remove(snapshotID);
+      if (next != null) {
+        snapshotChainPath
+            .get(snapshotPath)
+            .get(next)
+            .setPreviousSnapshotID(prev);
+      }
+      if (prev != null) {
+        snapshotChainPath
+            .get(snapshotPath)
+            .get(prev)
+            .setNextSnapshotID(next);
+      }
+      // remove from latest list if necessary
+      if (latestPathSnapshotID.get(snapshotPath).equals(snapshotID)) {
+        latestPathSnapshotID.remove(snapshotPath);
+        if (prev != null) {
+          latestPathSnapshotID.put(snapshotPath, prev);
+        }
+      }
+
+    } else {
+      // snapshotID not found in snapshot chain, log warning and return
+      LOG.warn("Snapshot chain: snapshotID not found: Snapshot path {}," +
+              " SnapshotID {}",
+          snapshotPath, snapshotID);
+    }
+
+    return status;
+  }
+
+  private void loadFromSnapshotInfoTable(OMMetadataManager metadataManager)
+          throws IOException {
+    // read from snapshotInfo table to populate
+    // snapshot chains - both global and local path
+    TableIterator<String, ? extends Table.KeyValue<String, SnapshotInfo>>
+            keyIter = metadataManager.getSnapshotInfoTable().iterator();
+    Map<Long, SnapshotInfo> snaps = new TreeMap<>();
+    Table.KeyValue< String, SnapshotInfo > kv;
+    snapshotChainGlobal.clear();
+    snapshotChainPath.clear();
+
+    while (keyIter.hasNext()) {
+      kv = keyIter.next();
+      snaps.put(kv.getValue().getCreationTime(), kv.getValue());
+    }
+    for (SnapshotInfo sinfo : snaps.values()) {
+      addSnapshot(sinfo);
+    }
+  }
+
+  public void addSnapshot(SnapshotInfo sinfo) throws IOException {
+    addSnapshotGlobal(sinfo.getSnapshotID(),
+        sinfo.getGlobalPreviousSnapshotID());
+    addSnapshotPath(sinfo.getSnapshotPath(),
+        sinfo.getSnapshotID(),
+        sinfo.getPathPreviousSnapshotID());
+  }
+
+  public boolean deleteSnapshot(SnapshotInfo sinfo) throws IOException {
+    boolean status;
+
+    status = deleteSnapshotGlobal(sinfo.getSnapshotID());
+    return status && deleteSnapshotPath(sinfo.getSnapshotPath(),
+        sinfo.getSnapshotID());
+  }
+
+  public String getLatestGlobalSnapshot() {
+    return latestGlobalSnapshotID;
+  }
+
+  public String getLatestPathSnapshot(String snapshotPath) {
+    return latestPathSnapshotID.get(snapshotPath);
+  }
+
+  public boolean hasNextGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    boolean hasNext = false;
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getNextSnapshotID() != null) {
+      hasNext = true;
+    }
+    return hasNext;
+  }
+
+  public String nextGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getNextSnapshotID() == null) {
+      LOG.error("no following snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("no following snapshot from: "
+              + snapshotID);
+    }
+    return snapshotChainGlobal
+            .get(snapshotID)
+            .getNextSnapshotID();
+  }
+
+  public boolean hasPreviousGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    boolean hasPrevious = false;
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getPreviousSnapshotID() != null) {
+      hasPrevious = true;
+    }
+    return hasPrevious;
+  }
+
+  public String previousGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getPreviousSnapshotID() == null) {
+      LOG.error("no preceeding snapshot for provided snapshotID {}",
+              snapshotID);
+      throw new NoSuchElementException("No preceeding snapshot from: "
+      + snapshotID);
+    }
+    return snapshotChainGlobal
+            .get(snapshotID)
+            .getPreviousSnapshotID();
+  }
+
+  public boolean hasNextPathSnapshot(String snapshotMask, String snapshotID)
+          throws NoSuchElementException {
+    boolean hasNext = false;
+    if (!snapshotChainPath.containsKey(snapshotMask) ||
+            !snapshotChainPath.get(snapshotMask).containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotMask {} and "
+              + " snapshotID {}", snapshotMask, snapshotID);
+      throw new NoSuchElementException("No such snapshot: "
+      + snapshotID + "for path: " + snapshotMask);
+    }
+    if (snapshotChainPath
+            .get(snapshotMask)
+            .get(snapshotID)
+            .getNextSnapshotID() != null) {
+      hasNext = true;
+    }
+    return hasNext;
+  }
+
+  public boolean hasPreviousPathSnapshot(String snapshotMask, String snapshotID)
+          throws NoSuchElementException {
+    boolean hasPrevious = false;
+    if (!snapshotChainPath.containsKey(snapshotMask) ||
+            !snapshotChainPath.get(snapshotMask).containsKey(snapshotID)) {

Review Comment:
   This can throw NPE for 2nd condition



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -0,0 +1,446 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.LinkedHashMap;
+import java.util.TreeMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is used for creating and accessing Snapshot Chains.
+ *
+ * The snapshot chain maintains the in-memory sequence of snapshots
+ * created in chronological order.  There are two such snapshots maintained
+ * i.) Path based snapshot chain, sequence of snapshots created for a
+ * given /volume/bucket
+ * ii.) Global snapshot chain, sequence of all snapshots created in order
+ *
+ * On start, the snapshot chains are initialized from the on disk
+ * SnapshotInfoTable from the om RocksDB.
+ */
+public class SnapshotChainManager {
+  private LinkedHashMap<String, SnapshotChainInfo>  snapshotChainGlobal;
+  private Map<String, LinkedHashMap<String, SnapshotChainInfo>>
+      snapshotChainPath;
+  private Map<String, String> latestPathSnapshotID;
+  private String latestGlobalSnapshotID;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotChainManager.class);
+
+  public SnapshotChainManager(OMMetadataManager metadataManager)
+          throws IOException {
+    snapshotChainGlobal = new LinkedHashMap<>();
+    snapshotChainPath = new HashMap<>();
+    latestPathSnapshotID = new HashMap<>();
+    latestGlobalSnapshotID = null;
+    loadFromSnapshotInfoTable(metadataManager);
+  }
+
+  private void addSnapshotGlobal(String snapshotID,
+                            String prevGlobalID) throws IOException {
+    // set previous snapshotID to null if it is "" for
+    // internal in-mem structure
+    if (prevGlobalID != null && prevGlobalID.equals("")) {
+      prevGlobalID = null;
+    }
+    // on add snapshot; set previous snapshot entry nextSnapshotID =
+    // snapshotID
+    if (prevGlobalID != null &&
+        snapshotChainGlobal.containsKey(prevGlobalID)) {
+      snapshotChainGlobal
+          .get(prevGlobalID)
+          .setNextSnapshotID(snapshotID);
+    }
+    if (prevGlobalID != null &&
+        !snapshotChainGlobal.containsKey(prevGlobalID)) {
+      throw new IOException("Snapshot Chain corruption: "
+          + " previous snapshotID given but no associated snapshot "
+      + "found in snapshot chain: SnapshotID "
+      + prevGlobalID);
+    }
+    snapshotChainGlobal.put(snapshotID,
+            new SnapshotChainInfo(snapshotID, prevGlobalID, null));
+
+    // set state variable latestGlobal snapshot entry to this snapshotID
+    latestGlobalSnapshotID = snapshotID;
+  };
+
+  private void addSnapshotPath(String snapshotPath,
+                               String snapshotID,
+                                String prevPathID) throws IOException {
+    // set previous snapshotID to null if it is "" for
+    // internal in-mem structure
+    if (prevPathID != null && prevPathID.equals("")) {
+      prevPathID = null;
+    }
+
+    // on add snapshot; set previous snapshot entry nextSnapshotID =
+    // snapshotID
+    if (prevPathID != null &&
+        ((!snapshotChainPath
+            .containsKey(snapshotPath)) ||
+        (!snapshotChainPath
+            .get(snapshotPath)
+            .containsKey(prevPathID)))) {
+      throw new IOException("Snapshot Chain corruption: "
+          + "previous snapshotID given but no associated snapshot "
+          + "found in snapshot chain: SnapshotID "
+          + prevPathID);
+    }
+
+    if (prevPathID != null &&
+            snapshotChainPath.containsKey(snapshotPath)) {
+      snapshotChainPath
+              .get(snapshotPath)
+              .get(prevPathID)
+              .setNextSnapshotID(snapshotID);
+    }
+
+    if (!snapshotChainPath.containsKey(snapshotPath)) {
+      snapshotChainPath.put(snapshotPath, new LinkedHashMap<>());
+    }
+
+    snapshotChainPath
+        .get(snapshotPath)
+        .put(snapshotID,
+                new SnapshotChainInfo(snapshotID, prevPathID, null));
+
+    // set state variable latestPath snapshot entry to this snapshotID
+    latestPathSnapshotID.put(snapshotPath, snapshotID);
+  };
+
+  private boolean deleteSnapshotGlobal(String snapshotID) throws IOException {
+    boolean status = true;
+    if (snapshotChainGlobal.containsKey(snapshotID)) {
+      // reset prev and next snapshot entries in chain ordered list
+      // for node removal
+      String next = snapshotChainGlobal.get(snapshotID).getNextSnapshotID();
+      String prev = snapshotChainGlobal.get(snapshotID).getPreviousSnapshotID();
+
+      if (prev != null && !snapshotChainGlobal.containsKey(prev)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to be "
+                + "deleted has prev node element not found in snapshot chain: "
+                + "SnapshotID " + prev);
+      }
+      if (next != null && !snapshotChainGlobal.containsKey(next)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to be "
+                + "deleted has next node element not found in snapshot chain: "
+                + "SnapshotID " + next);
+      }
+      snapshotChainGlobal.remove(snapshotID);
+      if (next != null) {
+        snapshotChainGlobal.get(next).setPreviousSnapshotID(prev);
+      }
+      if (prev != null) {
+        snapshotChainGlobal.get(prev).setNextSnapshotID(next);
+      }
+      // remove from latest list if necessary
+      if (latestGlobalSnapshotID.equals(snapshotID)) {
+        latestGlobalSnapshotID = prev;
+      }
+    } else {
+      // snapshotID not found in snapshot chain, log warning and return
+      LOG.warn("Snapshot chain: snapshotID not found: SnapshotID {}",
+          snapshotID);
+    }
+
+    return status;
+  }
+
+  private boolean deleteSnapshotPath(String snapshotPath,
+                                     String snapshotID) throws IOException {
+    boolean status = true;
+    if (snapshotChainPath.containsKey(snapshotPath) &&
+            snapshotChainPath
+                    .get(snapshotPath)
+                    .containsKey(snapshotID)) {
+      // reset prev and next snapshot entries in chain ordered list
+      // for node removal
+      String next = snapshotChainPath
+          .get(snapshotPath)
+          .get(snapshotID)
+          .getNextSnapshotID();
+      String prev = snapshotChainPath
+          .get(snapshotPath)
+          .get(snapshotID)
+          .getPreviousSnapshotID();
+
+      if (prev != null &&
+          !snapshotChainPath
+              .get(snapshotPath)
+              .containsKey(prev)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to "
+                + "be deleted has prev node element not found in snapshot "
+                + "chain: Snapshot path " + snapshotPath + ", SnapshotID "
+                + prev);
+      }
+      if (next != null && !snapshotChainPath
+          .get(snapshotPath)
+          .containsKey(next)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to "
+                + "be deleted has next node element not found in snapshot "
+                + "chain:  Snapshot path " + snapshotPath + ", SnapshotID "
+                + next);
+      }
+      snapshotChainPath
+          .get(snapshotPath)
+          .remove(snapshotID);
+      if (next != null) {
+        snapshotChainPath
+            .get(snapshotPath)
+            .get(next)
+            .setPreviousSnapshotID(prev);
+      }
+      if (prev != null) {
+        snapshotChainPath
+            .get(snapshotPath)
+            .get(prev)
+            .setNextSnapshotID(next);
+      }
+      // remove from latest list if necessary
+      if (latestPathSnapshotID.get(snapshotPath).equals(snapshotID)) {
+        latestPathSnapshotID.remove(snapshotPath);
+        if (prev != null) {
+          latestPathSnapshotID.put(snapshotPath, prev);
+        }
+      }
+
+    } else {
+      // snapshotID not found in snapshot chain, log warning and return
+      LOG.warn("Snapshot chain: snapshotID not found: Snapshot path {}," +
+              " SnapshotID {}",
+          snapshotPath, snapshotID);
+    }
+
+    return status;
+  }
+
+  private void loadFromSnapshotInfoTable(OMMetadataManager metadataManager)
+          throws IOException {
+    // read from snapshotInfo table to populate
+    // snapshot chains - both global and local path
+    TableIterator<String, ? extends Table.KeyValue<String, SnapshotInfo>>
+            keyIter = metadataManager.getSnapshotInfoTable().iterator();
+    Map<Long, SnapshotInfo> snaps = new TreeMap<>();
+    Table.KeyValue< String, SnapshotInfo > kv;
+    snapshotChainGlobal.clear();
+    snapshotChainPath.clear();
+
+    while (keyIter.hasNext()) {
+      kv = keyIter.next();
+      snaps.put(kv.getValue().getCreationTime(), kv.getValue());
+    }
+    for (SnapshotInfo sinfo : snaps.values()) {
+      addSnapshot(sinfo);
+    }
+  }
+
+  public void addSnapshot(SnapshotInfo sinfo) throws IOException {
+    addSnapshotGlobal(sinfo.getSnapshotID(),
+        sinfo.getGlobalPreviousSnapshotID());
+    addSnapshotPath(sinfo.getSnapshotPath(),
+        sinfo.getSnapshotID(),
+        sinfo.getPathPreviousSnapshotID());
+  }
+
+  public boolean deleteSnapshot(SnapshotInfo sinfo) throws IOException {
+    boolean status;
+
+    status = deleteSnapshotGlobal(sinfo.getSnapshotID());
+    return status && deleteSnapshotPath(sinfo.getSnapshotPath(),
+        sinfo.getSnapshotID());
+  }
+
+  public String getLatestGlobalSnapshot() {
+    return latestGlobalSnapshotID;
+  }
+
+  public String getLatestPathSnapshot(String snapshotPath) {
+    return latestPathSnapshotID.get(snapshotPath);
+  }
+
+  public boolean hasNextGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    boolean hasNext = false;
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getNextSnapshotID() != null) {
+      hasNext = true;
+    }
+    return hasNext;
+  }
+
+  public String nextGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getNextSnapshotID() == null) {
+      LOG.error("no following snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("no following snapshot from: "
+              + snapshotID);
+    }
+    return snapshotChainGlobal
+            .get(snapshotID)
+            .getNextSnapshotID();
+  }
+
+  public boolean hasPreviousGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    boolean hasPrevious = false;
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getPreviousSnapshotID() != null) {
+      hasPrevious = true;
+    }
+    return hasPrevious;
+  }
+
+  public String previousGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getPreviousSnapshotID() == null) {
+      LOG.error("no preceeding snapshot for provided snapshotID {}",
+              snapshotID);
+      throw new NoSuchElementException("No preceeding snapshot from: "
+      + snapshotID);
+    }
+    return snapshotChainGlobal
+            .get(snapshotID)
+            .getPreviousSnapshotID();
+  }
+
+  public boolean hasNextPathSnapshot(String snapshotMask, String snapshotID)

Review Comment:
   Minor comment : Let's use a common  naming `snapshotPath`  instead of `snapshotMask` as we have used it in most places?



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -0,0 +1,446 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.LinkedHashMap;
+import java.util.TreeMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is used for creating and accessing Snapshot Chains.
+ *
+ * The snapshot chain maintains the in-memory sequence of snapshots
+ * created in chronological order.  There are two such snapshots maintained
+ * i.) Path based snapshot chain, sequence of snapshots created for a
+ * given /volume/bucket
+ * ii.) Global snapshot chain, sequence of all snapshots created in order
+ *
+ * On start, the snapshot chains are initialized from the on disk
+ * SnapshotInfoTable from the om RocksDB.
+ */
+public class SnapshotChainManager {
+  private LinkedHashMap<String, SnapshotChainInfo>  snapshotChainGlobal;
+  private Map<String, LinkedHashMap<String, SnapshotChainInfo>>
+      snapshotChainPath;
+  private Map<String, String> latestPathSnapshotID;
+  private String latestGlobalSnapshotID;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotChainManager.class);
+
+  public SnapshotChainManager(OMMetadataManager metadataManager)
+          throws IOException {
+    snapshotChainGlobal = new LinkedHashMap<>();
+    snapshotChainPath = new HashMap<>();
+    latestPathSnapshotID = new HashMap<>();
+    latestGlobalSnapshotID = null;
+    loadFromSnapshotInfoTable(metadataManager);
+  }
+
+  private void addSnapshotGlobal(String snapshotID,
+                            String prevGlobalID) throws IOException {
+    // set previous snapshotID to null if it is "" for
+    // internal in-mem structure
+    if (prevGlobalID != null && prevGlobalID.equals("")) {
+      prevGlobalID = null;
+    }
+    // on add snapshot; set previous snapshot entry nextSnapshotID =
+    // snapshotID
+    if (prevGlobalID != null &&
+        snapshotChainGlobal.containsKey(prevGlobalID)) {
+      snapshotChainGlobal
+          .get(prevGlobalID)
+          .setNextSnapshotID(snapshotID);
+    }
+    if (prevGlobalID != null &&
+        !snapshotChainGlobal.containsKey(prevGlobalID)) {
+      throw new IOException("Snapshot Chain corruption: "
+          + " previous snapshotID given but no associated snapshot "
+      + "found in snapshot chain: SnapshotID "
+      + prevGlobalID);
+    }
+    snapshotChainGlobal.put(snapshotID,
+            new SnapshotChainInfo(snapshotID, prevGlobalID, null));
+
+    // set state variable latestGlobal snapshot entry to this snapshotID
+    latestGlobalSnapshotID = snapshotID;
+  };
+
+  private void addSnapshotPath(String snapshotPath,
+                               String snapshotID,
+                                String prevPathID) throws IOException {
+    // set previous snapshotID to null if it is "" for
+    // internal in-mem structure
+    if (prevPathID != null && prevPathID.equals("")) {
+      prevPathID = null;
+    }
+
+    // on add snapshot; set previous snapshot entry nextSnapshotID =
+    // snapshotID
+    if (prevPathID != null &&
+        ((!snapshotChainPath
+            .containsKey(snapshotPath)) ||
+        (!snapshotChainPath
+            .get(snapshotPath)
+            .containsKey(prevPathID)))) {
+      throw new IOException("Snapshot Chain corruption: "
+          + "previous snapshotID given but no associated snapshot "
+          + "found in snapshot chain: SnapshotID "
+          + prevPathID);
+    }
+
+    if (prevPathID != null &&
+            snapshotChainPath.containsKey(snapshotPath)) {
+      snapshotChainPath
+              .get(snapshotPath)
+              .get(prevPathID)
+              .setNextSnapshotID(snapshotID);
+    }
+
+    if (!snapshotChainPath.containsKey(snapshotPath)) {
+      snapshotChainPath.put(snapshotPath, new LinkedHashMap<>());
+    }
+
+    snapshotChainPath
+        .get(snapshotPath)
+        .put(snapshotID,
+                new SnapshotChainInfo(snapshotID, prevPathID, null));
+
+    // set state variable latestPath snapshot entry to this snapshotID
+    latestPathSnapshotID.put(snapshotPath, snapshotID);
+  };
+
+  private boolean deleteSnapshotGlobal(String snapshotID) throws IOException {
+    boolean status = true;
+    if (snapshotChainGlobal.containsKey(snapshotID)) {
+      // reset prev and next snapshot entries in chain ordered list
+      // for node removal
+      String next = snapshotChainGlobal.get(snapshotID).getNextSnapshotID();
+      String prev = snapshotChainGlobal.get(snapshotID).getPreviousSnapshotID();
+
+      if (prev != null && !snapshotChainGlobal.containsKey(prev)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to be "
+                + "deleted has prev node element not found in snapshot chain: "
+                + "SnapshotID " + prev);
+      }
+      if (next != null && !snapshotChainGlobal.containsKey(next)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to be "
+                + "deleted has next node element not found in snapshot chain: "
+                + "SnapshotID " + next);
+      }
+      snapshotChainGlobal.remove(snapshotID);
+      if (next != null) {
+        snapshotChainGlobal.get(next).setPreviousSnapshotID(prev);
+      }
+      if (prev != null) {
+        snapshotChainGlobal.get(prev).setNextSnapshotID(next);
+      }
+      // remove from latest list if necessary
+      if (latestGlobalSnapshotID.equals(snapshotID)) {
+        latestGlobalSnapshotID = prev;
+      }
+    } else {
+      // snapshotID not found in snapshot chain, log warning and return
+      LOG.warn("Snapshot chain: snapshotID not found: SnapshotID {}",
+          snapshotID);
+    }
+
+    return status;
+  }
+
+  private boolean deleteSnapshotPath(String snapshotPath,
+                                     String snapshotID) throws IOException {
+    boolean status = true;
+    if (snapshotChainPath.containsKey(snapshotPath) &&
+            snapshotChainPath
+                    .get(snapshotPath)
+                    .containsKey(snapshotID)) {
+      // reset prev and next snapshot entries in chain ordered list
+      // for node removal
+      String next = snapshotChainPath
+          .get(snapshotPath)
+          .get(snapshotID)
+          .getNextSnapshotID();
+      String prev = snapshotChainPath
+          .get(snapshotPath)
+          .get(snapshotID)
+          .getPreviousSnapshotID();
+
+      if (prev != null &&
+          !snapshotChainPath
+              .get(snapshotPath)
+              .containsKey(prev)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to "
+                + "be deleted has prev node element not found in snapshot "
+                + "chain: Snapshot path " + snapshotPath + ", SnapshotID "
+                + prev);
+      }
+      if (next != null && !snapshotChainPath
+          .get(snapshotPath)
+          .containsKey(next)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to "
+                + "be deleted has next node element not found in snapshot "
+                + "chain:  Snapshot path " + snapshotPath + ", SnapshotID "
+                + next);
+      }
+      snapshotChainPath
+          .get(snapshotPath)
+          .remove(snapshotID);
+      if (next != null) {
+        snapshotChainPath
+            .get(snapshotPath)
+            .get(next)
+            .setPreviousSnapshotID(prev);
+      }
+      if (prev != null) {
+        snapshotChainPath
+            .get(snapshotPath)
+            .get(prev)
+            .setNextSnapshotID(next);
+      }
+      // remove from latest list if necessary
+      if (latestPathSnapshotID.get(snapshotPath).equals(snapshotID)) {
+        latestPathSnapshotID.remove(snapshotPath);
+        if (prev != null) {
+          latestPathSnapshotID.put(snapshotPath, prev);
+        }
+      }
+
+    } else {
+      // snapshotID not found in snapshot chain, log warning and return
+      LOG.warn("Snapshot chain: snapshotID not found: Snapshot path {}," +
+              " SnapshotID {}",
+          snapshotPath, snapshotID);
+    }
+
+    return status;
+  }
+
+  private void loadFromSnapshotInfoTable(OMMetadataManager metadataManager)
+          throws IOException {
+    // read from snapshotInfo table to populate
+    // snapshot chains - both global and local path
+    TableIterator<String, ? extends Table.KeyValue<String, SnapshotInfo>>
+            keyIter = metadataManager.getSnapshotInfoTable().iterator();
+    Map<Long, SnapshotInfo> snaps = new TreeMap<>();
+    Table.KeyValue< String, SnapshotInfo > kv;
+    snapshotChainGlobal.clear();
+    snapshotChainPath.clear();
+
+    while (keyIter.hasNext()) {
+      kv = keyIter.next();
+      snaps.put(kv.getValue().getCreationTime(), kv.getValue());
+    }
+    for (SnapshotInfo sinfo : snaps.values()) {
+      addSnapshot(sinfo);
+    }
+  }
+
+  public void addSnapshot(SnapshotInfo sinfo) throws IOException {
+    addSnapshotGlobal(sinfo.getSnapshotID(),
+        sinfo.getGlobalPreviousSnapshotID());
+    addSnapshotPath(sinfo.getSnapshotPath(),
+        sinfo.getSnapshotID(),
+        sinfo.getPathPreviousSnapshotID());
+  }
+
+  public boolean deleteSnapshot(SnapshotInfo sinfo) throws IOException {
+    boolean status;
+
+    status = deleteSnapshotGlobal(sinfo.getSnapshotID());
+    return status && deleteSnapshotPath(sinfo.getSnapshotPath(),
+        sinfo.getSnapshotID());
+  }
+
+  public String getLatestGlobalSnapshot() {
+    return latestGlobalSnapshotID;
+  }
+
+  public String getLatestPathSnapshot(String snapshotPath) {
+    return latestPathSnapshotID.get(snapshotPath);
+  }
+
+  public boolean hasNextGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    boolean hasNext = false;
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getNextSnapshotID() != null) {
+      hasNext = true;
+    }
+    return hasNext;
+  }
+
+  public String nextGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getNextSnapshotID() == null) {
+      LOG.error("no following snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("no following snapshot from: "
+              + snapshotID);
+    }
+    return snapshotChainGlobal
+            .get(snapshotID)
+            .getNextSnapshotID();
+  }
+
+  public boolean hasPreviousGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    boolean hasPrevious = false;
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getPreviousSnapshotID() != null) {
+      hasPrevious = true;
+    }
+    return hasPrevious;
+  }
+
+  public String previousGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)

Review Comment:
   `hasPreviousGlobalSnapshot` could be used here instead



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -0,0 +1,446 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.LinkedHashMap;
+import java.util.TreeMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is used for creating and accessing Snapshot Chains.
+ *
+ * The snapshot chain maintains the in-memory sequence of snapshots
+ * created in chronological order.  There are two such snapshots maintained
+ * i.) Path based snapshot chain, sequence of snapshots created for a
+ * given /volume/bucket
+ * ii.) Global snapshot chain, sequence of all snapshots created in order
+ *
+ * On start, the snapshot chains are initialized from the on disk
+ * SnapshotInfoTable from the om RocksDB.
+ */
+public class SnapshotChainManager {
+  private LinkedHashMap<String, SnapshotChainInfo>  snapshotChainGlobal;
+  private Map<String, LinkedHashMap<String, SnapshotChainInfo>>
+      snapshotChainPath;
+  private Map<String, String> latestPathSnapshotID;
+  private String latestGlobalSnapshotID;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotChainManager.class);
+
+  public SnapshotChainManager(OMMetadataManager metadataManager)
+          throws IOException {
+    snapshotChainGlobal = new LinkedHashMap<>();
+    snapshotChainPath = new HashMap<>();
+    latestPathSnapshotID = new HashMap<>();
+    latestGlobalSnapshotID = null;
+    loadFromSnapshotInfoTable(metadataManager);
+  }
+
+  private void addSnapshotGlobal(String snapshotID,
+                            String prevGlobalID) throws IOException {
+    // set previous snapshotID to null if it is "" for
+    // internal in-mem structure
+    if (prevGlobalID != null && prevGlobalID.equals("")) {
+      prevGlobalID = null;
+    }
+    // on add snapshot; set previous snapshot entry nextSnapshotID =
+    // snapshotID
+    if (prevGlobalID != null &&
+        snapshotChainGlobal.containsKey(prevGlobalID)) {
+      snapshotChainGlobal
+          .get(prevGlobalID)
+          .setNextSnapshotID(snapshotID);
+    }
+    if (prevGlobalID != null &&
+        !snapshotChainGlobal.containsKey(prevGlobalID)) {
+      throw new IOException("Snapshot Chain corruption: "
+          + " previous snapshotID given but no associated snapshot "
+      + "found in snapshot chain: SnapshotID "
+      + prevGlobalID);
+    }
+    snapshotChainGlobal.put(snapshotID,
+            new SnapshotChainInfo(snapshotID, prevGlobalID, null));
+
+    // set state variable latestGlobal snapshot entry to this snapshotID
+    latestGlobalSnapshotID = snapshotID;
+  };
+
+  private void addSnapshotPath(String snapshotPath,
+                               String snapshotID,
+                                String prevPathID) throws IOException {
+    // set previous snapshotID to null if it is "" for
+    // internal in-mem structure
+    if (prevPathID != null && prevPathID.equals("")) {
+      prevPathID = null;
+    }
+
+    // on add snapshot; set previous snapshot entry nextSnapshotID =
+    // snapshotID
+    if (prevPathID != null &&
+        ((!snapshotChainPath
+            .containsKey(snapshotPath)) ||
+        (!snapshotChainPath
+            .get(snapshotPath)
+            .containsKey(prevPathID)))) {
+      throw new IOException("Snapshot Chain corruption: "
+          + "previous snapshotID given but no associated snapshot "
+          + "found in snapshot chain: SnapshotID "
+          + prevPathID);
+    }
+
+    if (prevPathID != null &&
+            snapshotChainPath.containsKey(snapshotPath)) {
+      snapshotChainPath
+              .get(snapshotPath)
+              .get(prevPathID)
+              .setNextSnapshotID(snapshotID);
+    }
+
+    if (!snapshotChainPath.containsKey(snapshotPath)) {
+      snapshotChainPath.put(snapshotPath, new LinkedHashMap<>());
+    }
+
+    snapshotChainPath
+        .get(snapshotPath)
+        .put(snapshotID,
+                new SnapshotChainInfo(snapshotID, prevPathID, null));
+
+    // set state variable latestPath snapshot entry to this snapshotID
+    latestPathSnapshotID.put(snapshotPath, snapshotID);
+  };
+
+  private boolean deleteSnapshotGlobal(String snapshotID) throws IOException {
+    boolean status = true;
+    if (snapshotChainGlobal.containsKey(snapshotID)) {
+      // reset prev and next snapshot entries in chain ordered list
+      // for node removal
+      String next = snapshotChainGlobal.get(snapshotID).getNextSnapshotID();
+      String prev = snapshotChainGlobal.get(snapshotID).getPreviousSnapshotID();
+
+      if (prev != null && !snapshotChainGlobal.containsKey(prev)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to be "
+                + "deleted has prev node element not found in snapshot chain: "
+                + "SnapshotID " + prev);
+      }
+      if (next != null && !snapshotChainGlobal.containsKey(next)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to be "
+                + "deleted has next node element not found in snapshot chain: "
+                + "SnapshotID " + next);
+      }
+      snapshotChainGlobal.remove(snapshotID);
+      if (next != null) {
+        snapshotChainGlobal.get(next).setPreviousSnapshotID(prev);
+      }
+      if (prev != null) {
+        snapshotChainGlobal.get(prev).setNextSnapshotID(next);
+      }
+      // remove from latest list if necessary
+      if (latestGlobalSnapshotID.equals(snapshotID)) {
+        latestGlobalSnapshotID = prev;
+      }
+    } else {
+      // snapshotID not found in snapshot chain, log warning and return
+      LOG.warn("Snapshot chain: snapshotID not found: SnapshotID {}",
+          snapshotID);
+    }
+
+    return status;
+  }
+
+  private boolean deleteSnapshotPath(String snapshotPath,
+                                     String snapshotID) throws IOException {
+    boolean status = true;
+    if (snapshotChainPath.containsKey(snapshotPath) &&
+            snapshotChainPath
+                    .get(snapshotPath)
+                    .containsKey(snapshotID)) {
+      // reset prev and next snapshot entries in chain ordered list
+      // for node removal
+      String next = snapshotChainPath
+          .get(snapshotPath)
+          .get(snapshotID)
+          .getNextSnapshotID();
+      String prev = snapshotChainPath
+          .get(snapshotPath)
+          .get(snapshotID)
+          .getPreviousSnapshotID();
+
+      if (prev != null &&
+          !snapshotChainPath
+              .get(snapshotPath)
+              .containsKey(prev)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to "
+                + "be deleted has prev node element not found in snapshot "
+                + "chain: Snapshot path " + snapshotPath + ", SnapshotID "
+                + prev);
+      }
+      if (next != null && !snapshotChainPath
+          .get(snapshotPath)
+          .containsKey(next)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to "
+                + "be deleted has next node element not found in snapshot "
+                + "chain:  Snapshot path " + snapshotPath + ", SnapshotID "
+                + next);
+      }
+      snapshotChainPath
+          .get(snapshotPath)
+          .remove(snapshotID);
+      if (next != null) {
+        snapshotChainPath
+            .get(snapshotPath)
+            .get(next)
+            .setPreviousSnapshotID(prev);
+      }
+      if (prev != null) {
+        snapshotChainPath
+            .get(snapshotPath)
+            .get(prev)
+            .setNextSnapshotID(next);
+      }

Review Comment:
   We can cleanup the outerMap if innerMap has no entries
   ```
   if (snapshotChainPath.get(snapshotPath).isEmpty()){
       snapshotChainPath.remove(snapshotPath);
   }
   ```



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -0,0 +1,446 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.LinkedHashMap;
+import java.util.TreeMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is used for creating and accessing Snapshot Chains.
+ *
+ * The snapshot chain maintains the in-memory sequence of snapshots
+ * created in chronological order.  There are two such snapshots maintained
+ * i.) Path based snapshot chain, sequence of snapshots created for a
+ * given /volume/bucket
+ * ii.) Global snapshot chain, sequence of all snapshots created in order
+ *
+ * On start, the snapshot chains are initialized from the on disk
+ * SnapshotInfoTable from the om RocksDB.
+ */
+public class SnapshotChainManager {
+  private LinkedHashMap<String, SnapshotChainInfo>  snapshotChainGlobal;
+  private Map<String, LinkedHashMap<String, SnapshotChainInfo>>
+      snapshotChainPath;
+  private Map<String, String> latestPathSnapshotID;
+  private String latestGlobalSnapshotID;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotChainManager.class);
+
+  public SnapshotChainManager(OMMetadataManager metadataManager)
+          throws IOException {
+    snapshotChainGlobal = new LinkedHashMap<>();
+    snapshotChainPath = new HashMap<>();
+    latestPathSnapshotID = new HashMap<>();
+    latestGlobalSnapshotID = null;
+    loadFromSnapshotInfoTable(metadataManager);
+  }
+
+  private void addSnapshotGlobal(String snapshotID,
+                            String prevGlobalID) throws IOException {
+    // set previous snapshotID to null if it is "" for
+    // internal in-mem structure
+    if (prevGlobalID != null && prevGlobalID.equals("")) {
+      prevGlobalID = null;
+    }
+    // on add snapshot; set previous snapshot entry nextSnapshotID =
+    // snapshotID
+    if (prevGlobalID != null &&
+        snapshotChainGlobal.containsKey(prevGlobalID)) {
+      snapshotChainGlobal
+          .get(prevGlobalID)
+          .setNextSnapshotID(snapshotID);
+    }
+    if (prevGlobalID != null &&
+        !snapshotChainGlobal.containsKey(prevGlobalID)) {
+      throw new IOException("Snapshot Chain corruption: "
+          + " previous snapshotID given but no associated snapshot "
+      + "found in snapshot chain: SnapshotID "
+      + prevGlobalID);
+    }
+    snapshotChainGlobal.put(snapshotID,
+            new SnapshotChainInfo(snapshotID, prevGlobalID, null));
+
+    // set state variable latestGlobal snapshot entry to this snapshotID
+    latestGlobalSnapshotID = snapshotID;
+  };
+
+  private void addSnapshotPath(String snapshotPath,
+                               String snapshotID,
+                                String prevPathID) throws IOException {
+    // set previous snapshotID to null if it is "" for
+    // internal in-mem structure
+    if (prevPathID != null && prevPathID.equals("")) {
+      prevPathID = null;
+    }
+
+    // on add snapshot; set previous snapshot entry nextSnapshotID =
+    // snapshotID
+    if (prevPathID != null &&

Review Comment:
   `((!snapshotChainPath
               .containsKey(snapshotPath)) ||
           (!snapshotChainPath
               .get(snapshotPath)
               .containsKey(prevPathID))`
   This can give NPE for 2nd condition if key with a snapshotPath doesn't exist. 



##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotChain.java:
##########
@@ -0,0 +1,305 @@
+/*
+ * 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;
+
+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.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_DB_DIRS;
+
+/**
+ * Tests SnapshotChain that stores in chronological order
+ * Ozone object storage snapshots.  There exist two types
+ * of chains provided by the SnapshotChainManager
+ * i.) path based snapshots - a list of snapshots taken for a given bucket
+ * ii.) global snapshots - a list of every snapshot taken in chrono order
+ */
+public class TestSnapshotChain {
+  private OMMetadataManager omMetadataManager;
+  private Map<String, SnapshotInfo> sinfos;
+  private SnapshotChainManager chainManager;
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  @Before
+  public void setup() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    conf.set(OZONE_OM_DB_DIRS,
+        folder.getRoot().getAbsolutePath());
+    omMetadataManager = new OmMetadataManagerImpl(conf);
+    sinfos = new HashMap<>();
+    chainManager = new SnapshotChainManager(omMetadataManager);
+  }
+
+  private SnapshotInfo createSnapshotInfo(String volName,
+                                          String bucketName,
+                                          String snapshotName,
+                                          String snapshotID,
+                                          String pathPrevID,
+                                          String globalPrevID) {
+    return new SnapshotInfo.Builder()
+        .setSnapshotID(snapshotID)
+        .setName(snapshotName)
+        .setVolumeName(volName)
+        .setBucketName(bucketName)
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE)
+        .setCreationTime(Time.now())
+        .setDeletionTime(-1L)
+        .setPathPreviousSnapshotID(pathPrevID)
+        .setGlobalPreviousSnapshotID(globalPrevID)
+        .setSnapshotPath(String.join("/", volName, bucketName))
+        .setCheckpointDir("checkpoint.testdir")
+        .build();
+  }
+
+  private void deleteSnapshot(String snapshotID) throws IOException {
+    SnapshotInfo sinfo = null;
+    final String snapshotPath = "vol1/bucket1";
+    // reset the next snapshotInfo.globalPreviousSnapshotID
+    // to the previous in the entry to be deleted.
+    if (chainManager.hasNextGlobalSnapshot(snapshotID)) {
+      sinfo = sinfos
+              .get(chainManager.nextGlobalSnapshot(snapshotID));

Review Comment:
   I guess updating the  next pointers for SnapshotInfo  should also be done in case of addSnapshots  similar to what is done here for delete . Also I guess we could have separate methods define for doing this (updating pointers for SnapshotInfo ) to provide some abstraction.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -0,0 +1,446 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.LinkedHashMap;
+import java.util.TreeMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is used for creating and accessing Snapshot Chains.
+ *
+ * The snapshot chain maintains the in-memory sequence of snapshots
+ * created in chronological order.  There are two such snapshots maintained
+ * i.) Path based snapshot chain, sequence of snapshots created for a
+ * given /volume/bucket
+ * ii.) Global snapshot chain, sequence of all snapshots created in order
+ *
+ * On start, the snapshot chains are initialized from the on disk
+ * SnapshotInfoTable from the om RocksDB.
+ */
+public class SnapshotChainManager {
+  private LinkedHashMap<String, SnapshotChainInfo>  snapshotChainGlobal;
+  private Map<String, LinkedHashMap<String, SnapshotChainInfo>>
+      snapshotChainPath;
+  private Map<String, String> latestPathSnapshotID;
+  private String latestGlobalSnapshotID;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotChainManager.class);
+
+  public SnapshotChainManager(OMMetadataManager metadataManager)
+          throws IOException {
+    snapshotChainGlobal = new LinkedHashMap<>();
+    snapshotChainPath = new HashMap<>();
+    latestPathSnapshotID = new HashMap<>();
+    latestGlobalSnapshotID = null;
+    loadFromSnapshotInfoTable(metadataManager);
+  }
+
+  private void addSnapshotGlobal(String snapshotID,
+                            String prevGlobalID) throws IOException {
+    // set previous snapshotID to null if it is "" for
+    // internal in-mem structure
+    if (prevGlobalID != null && prevGlobalID.equals("")) {
+      prevGlobalID = null;
+    }
+    // on add snapshot; set previous snapshot entry nextSnapshotID =
+    // snapshotID
+    if (prevGlobalID != null &&
+        snapshotChainGlobal.containsKey(prevGlobalID)) {
+      snapshotChainGlobal
+          .get(prevGlobalID)
+          .setNextSnapshotID(snapshotID);
+    }
+    if (prevGlobalID != null &&
+        !snapshotChainGlobal.containsKey(prevGlobalID)) {
+      throw new IOException("Snapshot Chain corruption: "
+          + " previous snapshotID given but no associated snapshot "
+      + "found in snapshot chain: SnapshotID "
+      + prevGlobalID);
+    }
+    snapshotChainGlobal.put(snapshotID,
+            new SnapshotChainInfo(snapshotID, prevGlobalID, null));
+
+    // set state variable latestGlobal snapshot entry to this snapshotID
+    latestGlobalSnapshotID = snapshotID;
+  };
+
+  private void addSnapshotPath(String snapshotPath,
+                               String snapshotID,
+                                String prevPathID) throws IOException {
+    // set previous snapshotID to null if it is "" for
+    // internal in-mem structure
+    if (prevPathID != null && prevPathID.equals("")) {
+      prevPathID = null;
+    }
+
+    // on add snapshot; set previous snapshot entry nextSnapshotID =
+    // snapshotID
+    if (prevPathID != null &&
+        ((!snapshotChainPath
+            .containsKey(snapshotPath)) ||
+        (!snapshotChainPath
+            .get(snapshotPath)
+            .containsKey(prevPathID)))) {
+      throw new IOException("Snapshot Chain corruption: "
+          + "previous snapshotID given but no associated snapshot "
+          + "found in snapshot chain: SnapshotID "
+          + prevPathID);
+    }
+
+    if (prevPathID != null &&
+            snapshotChainPath.containsKey(snapshotPath)) {
+      snapshotChainPath
+              .get(snapshotPath)
+              .get(prevPathID)
+              .setNextSnapshotID(snapshotID);
+    }
+
+    if (!snapshotChainPath.containsKey(snapshotPath)) {
+      snapshotChainPath.put(snapshotPath, new LinkedHashMap<>());
+    }
+
+    snapshotChainPath
+        .get(snapshotPath)
+        .put(snapshotID,
+                new SnapshotChainInfo(snapshotID, prevPathID, null));
+
+    // set state variable latestPath snapshot entry to this snapshotID
+    latestPathSnapshotID.put(snapshotPath, snapshotID);
+  };
+
+  private boolean deleteSnapshotGlobal(String snapshotID) throws IOException {
+    boolean status = true;
+    if (snapshotChainGlobal.containsKey(snapshotID)) {
+      // reset prev and next snapshot entries in chain ordered list
+      // for node removal
+      String next = snapshotChainGlobal.get(snapshotID).getNextSnapshotID();
+      String prev = snapshotChainGlobal.get(snapshotID).getPreviousSnapshotID();
+
+      if (prev != null && !snapshotChainGlobal.containsKey(prev)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to be "
+                + "deleted has prev node element not found in snapshot chain: "
+                + "SnapshotID " + prev);
+      }
+      if (next != null && !snapshotChainGlobal.containsKey(next)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to be "
+                + "deleted has next node element not found in snapshot chain: "
+                + "SnapshotID " + next);
+      }
+      snapshotChainGlobal.remove(snapshotID);
+      if (next != null) {
+        snapshotChainGlobal.get(next).setPreviousSnapshotID(prev);
+      }
+      if (prev != null) {
+        snapshotChainGlobal.get(prev).setNextSnapshotID(next);
+      }
+      // remove from latest list if necessary
+      if (latestGlobalSnapshotID.equals(snapshotID)) {
+        latestGlobalSnapshotID = prev;
+      }
+    } else {
+      // snapshotID not found in snapshot chain, log warning and return
+      LOG.warn("Snapshot chain: snapshotID not found: SnapshotID {}",
+          snapshotID);
+    }
+
+    return status;
+  }
+
+  private boolean deleteSnapshotPath(String snapshotPath,
+                                     String snapshotID) throws IOException {
+    boolean status = true;
+    if (snapshotChainPath.containsKey(snapshotPath) &&
+            snapshotChainPath
+                    .get(snapshotPath)
+                    .containsKey(snapshotID)) {
+      // reset prev and next snapshot entries in chain ordered list
+      // for node removal
+      String next = snapshotChainPath
+          .get(snapshotPath)
+          .get(snapshotID)
+          .getNextSnapshotID();
+      String prev = snapshotChainPath
+          .get(snapshotPath)
+          .get(snapshotID)
+          .getPreviousSnapshotID();
+
+      if (prev != null &&
+          !snapshotChainPath
+              .get(snapshotPath)
+              .containsKey(prev)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to "
+                + "be deleted has prev node element not found in snapshot "
+                + "chain: Snapshot path " + snapshotPath + ", SnapshotID "
+                + prev);
+      }
+      if (next != null && !snapshotChainPath
+          .get(snapshotPath)
+          .containsKey(next)) {
+        throw new IOException("Snapshot chain corruption: snapshot node to "
+                + "be deleted has next node element not found in snapshot "
+                + "chain:  Snapshot path " + snapshotPath + ", SnapshotID "
+                + next);
+      }
+      snapshotChainPath
+          .get(snapshotPath)
+          .remove(snapshotID);
+      if (next != null) {
+        snapshotChainPath
+            .get(snapshotPath)
+            .get(next)
+            .setPreviousSnapshotID(prev);
+      }
+      if (prev != null) {
+        snapshotChainPath
+            .get(snapshotPath)
+            .get(prev)
+            .setNextSnapshotID(next);
+      }
+      // remove from latest list if necessary
+      if (latestPathSnapshotID.get(snapshotPath).equals(snapshotID)) {
+        latestPathSnapshotID.remove(snapshotPath);
+        if (prev != null) {
+          latestPathSnapshotID.put(snapshotPath, prev);
+        }
+      }
+
+    } else {
+      // snapshotID not found in snapshot chain, log warning and return
+      LOG.warn("Snapshot chain: snapshotID not found: Snapshot path {}," +
+              " SnapshotID {}",
+          snapshotPath, snapshotID);
+    }
+
+    return status;
+  }
+
+  private void loadFromSnapshotInfoTable(OMMetadataManager metadataManager)
+          throws IOException {
+    // read from snapshotInfo table to populate
+    // snapshot chains - both global and local path
+    TableIterator<String, ? extends Table.KeyValue<String, SnapshotInfo>>
+            keyIter = metadataManager.getSnapshotInfoTable().iterator();
+    Map<Long, SnapshotInfo> snaps = new TreeMap<>();
+    Table.KeyValue< String, SnapshotInfo > kv;
+    snapshotChainGlobal.clear();
+    snapshotChainPath.clear();
+
+    while (keyIter.hasNext()) {
+      kv = keyIter.next();
+      snaps.put(kv.getValue().getCreationTime(), kv.getValue());
+    }
+    for (SnapshotInfo sinfo : snaps.values()) {
+      addSnapshot(sinfo);
+    }
+  }
+
+  public void addSnapshot(SnapshotInfo sinfo) throws IOException {
+    addSnapshotGlobal(sinfo.getSnapshotID(),
+        sinfo.getGlobalPreviousSnapshotID());
+    addSnapshotPath(sinfo.getSnapshotPath(),
+        sinfo.getSnapshotID(),
+        sinfo.getPathPreviousSnapshotID());
+  }
+
+  public boolean deleteSnapshot(SnapshotInfo sinfo) throws IOException {
+    boolean status;
+
+    status = deleteSnapshotGlobal(sinfo.getSnapshotID());
+    return status && deleteSnapshotPath(sinfo.getSnapshotPath(),
+        sinfo.getSnapshotID());
+  }
+
+  public String getLatestGlobalSnapshot() {
+    return latestGlobalSnapshotID;
+  }
+
+  public String getLatestPathSnapshot(String snapshotPath) {
+    return latestPathSnapshotID.get(snapshotPath);
+  }
+
+  public boolean hasNextGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    boolean hasNext = false;
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getNextSnapshotID() != null) {
+      hasNext = true;
+    }
+    return hasNext;
+  }
+
+  public String nextGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getNextSnapshotID() == null) {
+      LOG.error("no following snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("no following snapshot from: "
+              + snapshotID);
+    }
+    return snapshotChainGlobal
+            .get(snapshotID)
+            .getNextSnapshotID();
+  }
+
+  public boolean hasPreviousGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    boolean hasPrevious = false;
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getPreviousSnapshotID() != null) {
+      hasPrevious = true;
+    }
+    return hasPrevious;
+  }
+
+  public String previousGlobalSnapshot(String snapshotID)
+          throws NoSuchElementException {
+    if (!snapshotChainGlobal.containsKey(snapshotID)) {
+      LOG.error("no snapshot for provided snapshotID {}", snapshotID);
+      throw new NoSuchElementException("No snapshot: " + snapshotID);
+    }
+    if (snapshotChainGlobal
+            .get(snapshotID)
+            .getPreviousSnapshotID() == null) {
+      LOG.error("no preceeding snapshot for provided snapshotID {}",
+              snapshotID);
+      throw new NoSuchElementException("No preceeding snapshot from: "
+      + snapshotID);
+    }
+    return snapshotChainGlobal
+            .get(snapshotID)
+            .getPreviousSnapshotID();
+  }
+
+  public boolean hasNextPathSnapshot(String snapshotMask, String snapshotID)
+          throws NoSuchElementException {
+    boolean hasNext = false;
+    if (!snapshotChainPath.containsKey(snapshotMask) ||
+            !snapshotChainPath.get(snapshotMask).containsKey(snapshotID)) {

Review Comment:
   This can throw NPE for 2nd condition



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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