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/07/06 00:50:02 UTC

[GitHub] [ozone] neils-dev opened a new pull request, #3582: HDDS-6849. OM changes for tracking Snapshot Metadata

neils-dev opened a new pull request, #3582:
URL: https://github.com/apache/ozone/pull/3582

   ## What changes were proposed in this pull request?
   
   Adding support for snapshot metadata in ozone manager.  Snapshot metadata is to be stored in the ozone manager RocksDB database as an additional `SnapshotInfo` column family.  Metadata for each snapshot created is captured in a `SnapshotInfo` entry (schema as defined in `CheckpointBasedDesign` doc) and inserted into the RocksDB snapshot table.
   
   Included in this is the schema `SnapshotInfo` entry builder, protobuf definition, `OmMetadataManager` integration and unit tests testing `SnapshotInfo` entry accessors, and `RocksDB` table creation, Snapshot entry additions and deletions.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-6849
   
   ## How was this patch tested?
   Unit tests:
   i.) testing `SnapshotInfo` metadata data structure holding state info for object store snapshots. - `TestOmSnapshotInfo`
   ii.) testing `SnapshotInfo` `RocksDB` om database table for object store snapshots - `TestSnapshotInfo`
   
   `hadoop-ozone/common$ mvn -Dtest=TestOmSnapshotInfo test`
   [INFO] Tests run: 3, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.047 s - in org.apache.hadoop.ozone.om.helpers.TestOmSnapshotInfo
   [INFO] 
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 3, Failures: 0, Errors: 0, Skipped: 0
   
   `hadoop-ozone/ozone-manager$ mvn -Dtest=TestSnapshotInfo test`
   [INFO] Running org.apache.hadoop.ozone.om.TestSnapshotInfo
   [INFO] Tests run: 3, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 6.859 s - in org.apache.hadoop.ozone.om.TestSnapshotInfo
   [INFO] 
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 3, Failures: 0, Errors: 0, Skipped: 0
   
   
   


-- 
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


[GitHub] [ozone] neils-dev commented on a diff in pull request #3582: HDDS-6849. OM changes for tracking Snapshot Metadata

Posted by GitBox <gi...@apache.org>.
neils-dev commented on code in PR #3582:
URL: https://github.com/apache/ozone/pull/3582#discussion_r922905427


##########
hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/codec/OmDBSnapshotInfoCodec.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.codec;
+
+import org.apache.hadoop.hdds.utils.db.Codec;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotInfoEntry;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Codec to encode SnapshotInfo as byte array.
+ */
+public class OmDBSnapshotInfoCodec implements Codec<SnapshotInfo> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OmDBSnapshotInfoCodec.class);
+
+  @Override
+  public byte[] toPersistedFormat(SnapshotInfo object) throws IOException {
+    checkNotNull(object, "Null object can't be converted to byte array.");
+    return object.getProtobuf().toByteArray();
+  }
+
+  @Override
+  public SnapshotInfo fromPersistedFormat(byte[] rawData)
+      throws IOException {
+    checkNotNull(rawData, "Null byte array can't be converted to " +
+        "real object.");
+    return SnapshotInfo.getFromProtobuf(
+        SnapshotInfoEntry.parseFrom(rawData));
+  }
+
+  @Override
+  public SnapshotInfo copyObject(SnapshotInfo object) {
+    // Note: Not really a "copy". from OMDBSnapshotInfoCodec
+    return object;

Review Comment:
   Thought here was that we could do an implicit copy as there was no need for a 'deep' copy for the data structure as the `SnapshotInfo` consists of longs, ints and strings.  In the case of `OmKeyInfo` and `OmBucketInfo `objects, they include data structures within that warrant a deep copy, however this is not the case for `SnapshotInfo`, `OmDBTenantState`, `OmDBUserPrincipalInfo`.
   
   I can create an explicit copy for the object.  Let me know if you think it is necessary.



-- 
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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3582: HDDS-6849. OM changes for tracking Snapshot Metadata

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3582:
URL: https://github.com/apache/ozone/pull/3582#discussion_r917521489


##########
hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/codec/OmDBSnapshotInfoCodec.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.codec;
+
+import org.apache.hadoop.hdds.utils.db.Codec;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotInfoEntry;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Codec to encode SnapshotInfo as byte array.
+ */
+public class OmDBSnapshotInfoCodec implements Codec<SnapshotInfo> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OmDBSnapshotInfoCodec.class);
+
+  @Override
+  public byte[] toPersistedFormat(SnapshotInfo object) throws IOException {
+    checkNotNull(object, "Null object can't be converted to byte array.");
+    return object.getProtobuf().toByteArray();
+  }
+
+  @Override
+  public SnapshotInfo fromPersistedFormat(byte[] rawData)
+      throws IOException {
+    checkNotNull(rawData, "Null byte array can't be converted to " +
+        "real object.");
+    return SnapshotInfo.getFromProtobuf(
+        SnapshotInfoEntry.parseFrom(rawData));
+  }
+
+  @Override
+  public SnapshotInfo copyObject(SnapshotInfo object) {
+    // Note: Not really a "copy". from OMDBSnapshotInfoCodec
+    return object;

Review Comment:
   Seems odd that this is not a real copy.  OmBucketInfoCodec does it:
   
   https://github.com/apache/ozone/blob/79a9d39da7f33e71bc00183e280105562354cca4/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/codec/OmBucketInfoCodec.java#L54-L57



-- 
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


[GitHub] [ozone] nandakumar131 commented on pull request #3582: HDDS-6849. OM changes for tracking Snapshot Metadata

Posted by GitBox <gi...@apache.org>.
nandakumar131 commented on PR #3582:
URL: https://github.com/apache/ozone/pull/3582#issuecomment-1200771248

   @adoroszlai, this is kind of a POC here. Once the we get detailed design doc out, we will move development to the master branch.


-- 
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


[GitHub] [ozone] neils-dev commented on a diff in pull request #3582: HDDS-6849. OM changes for tracking Snapshot Metadata

Posted by GitBox <gi...@apache.org>.
neils-dev commented on code in PR #3582:
URL: https://github.com/apache/ozone/pull/3582#discussion_r922905516


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/SnapshotInfo.java:
##########
@@ -0,0 +1,321 @@
+package org.apache.hadoop.ozone.om.helpers;
+
+/*
+ * 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
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  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.
+ */
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotInfoEntry;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotStatusProto;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * This class is used for storing info related to Snapshots.
+ *
+ * Each snapshot created has an associated SnapshotInfo entry
+ * containing the snapshotid, snapshot path,
+ * snapshot checkpoint directory, previous snapshotid
+ * for the snapshot path & global amongst other necessary fields.
+ */
+public final class SnapshotInfo {
+
+  /**
+   * SnapshotStatus enum composed of
+   * active, deleted and reclaimed statues.
+   */
+  public enum SnapshotStatus {
+    SNAPSHOT_ACTIVE,
+    SNAPSHOT_DELETED,
+    SNAPSHOT_RECLAIMED;
+
+    public static final SnapshotStatus DEFAULT = SNAPSHOT_ACTIVE;
+
+    public SnapshotStatusProto toProto() {
+      switch (this) {
+      case SNAPSHOT_ACTIVE:
+        return SnapshotStatusProto.SNAPSHOT_ACTIVE;
+      case SNAPSHOT_DELETED:
+        return SnapshotStatusProto.SNAPSHOT_DELETED;
+      case SNAPSHOT_RECLAIMED:
+        return SnapshotStatusProto.SNAPSHOT_RECLAIMED;
+      default:
+        throw new IllegalStateException(
+            "BUG: missing valid SnapshotStatus, found status=" + this);
+      }
+    }
+
+    public static SnapshotStatus valueOf(SnapshotStatusProto status) {
+      switch (status) {
+      case SNAPSHOT_ACTIVE:
+        return SNAPSHOT_ACTIVE;
+      case SNAPSHOT_DELETED:
+        return SNAPSHOT_DELETED;
+      case SNAPSHOT_RECLAIMED:
+        return SNAPSHOT_RECLAIMED;
+      default:
+        throw new IllegalStateException(
+            "BUG: missing valid SnapshotStatus, found status=" + status);
+      }
+    }
+  };
+
+  private final String snapshotID;  // UUID
+  private String name;
+  private SnapshotStatus snapshotStatus;
+  private final long creationTime;
+  private long deletionTime;
+  private String pathPreviousSnapshotID;
+  private String globalPreviousSnapshotID;
+  private String snapshotPath; // snapshot mask
+  private String checkpointDir;
+
+  /**
+   * Private constructor, constructed via builder.
+   * @param snapshotID - Snapshot UUID.
+   * @param name - snapshot name.
+   * @param snapshotStatus - status: SNAPSHOT_ACTIVE, SNAPSHOT_DELETED,
+   *                      SNAPSHOT_RECLAIMED
+   * @param creationTime - Snapshot creation time.
+   * @param deletionTime - Snapshot deletion time.
+   * @param pathPreviousSnapshotID - Snapshot path previous snapshot id.
+   * @param globalPreviousSnapshotID - Snapshot global previous snapshot id.
+   * @param snapshotPath - Snapshot path, bucket .snapshot path.
+   * @param checkpointDir - Snapshot checkpoint directory.
+   */
+  @SuppressWarnings("checkstyle:ParameterNumber")
+  private SnapshotInfo(String snapshotID,
+                       String name,
+                       SnapshotStatus snapshotStatus,
+                       long creationTime,
+                       long deletionTime,
+                       String pathPreviousSnapshotID,
+                       String globalPreviousSnapshotID,
+                       String snapshotPath,
+                       String checkpointDir) {
+    this.snapshotID = snapshotID;
+    this.name = name;
+    this.snapshotStatus = snapshotStatus;
+    this.creationTime = creationTime;
+    this.deletionTime = deletionTime;
+    this.pathPreviousSnapshotID = pathPreviousSnapshotID;
+    this.globalPreviousSnapshotID = globalPreviousSnapshotID;
+    this.snapshotPath = snapshotPath;
+    this.checkpointDir = checkpointDir;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public void setSnapshotStatus(SnapshotStatus snapshotStatus) {
+    this.snapshotStatus = snapshotStatus;
+  }
+
+  public void setDeletionTime(long delTime) {
+    this.deletionTime = delTime;
+  }
+
+  public void setPathPreviousSnapshotID(String pathPreviousSnapshotID) {
+    this.pathPreviousSnapshotID = pathPreviousSnapshotID;
+  }
+
+  public void setGlobalPreviousSnapshotID(String globalPreviousSnapshotID) {
+    this.globalPreviousSnapshotID = globalPreviousSnapshotID;
+  }
+
+  public void setSnapshotPath(String snapshotPath) {
+    this.snapshotPath = snapshotPath;
+  }
+
+  public void setCheckpointDir(String checkpointDir) {
+    this.checkpointDir = checkpointDir;
+  }
+
+  public String getSnapshotID() {
+    return snapshotID;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public SnapshotStatus getSnapshotStatus() {
+    return snapshotStatus;
+  }
+
+  public long getCreationTime() {
+    return creationTime;
+  }
+
+  public long getDeletionTime() {
+    return deletionTime;
+  }
+
+  public String getPathPreviousSnapshotID() {
+    return pathPreviousSnapshotID;
+  }
+
+  public String getGlobalPreviousSnapshotID() {
+    return globalPreviousSnapshotID;
+  }
+
+  public String getSnapshotPath() {
+    return snapshotPath;
+  }
+
+  public String getCheckpointDir() {
+    return checkpointDir;
+  }
+
+  public static SnapshotInfo.Builder newBuilder() {
+    return new SnapshotInfo.Builder();
+  }
+
+  public SnapshotInfo.Builder toBuilder() {
+    return new SnapshotInfo.Builder()
+        .setSnapshotID(snapshotID)
+        .setSnapshotStatus(snapshotStatus)
+        .setCreationTime(creationTime)
+        .setDeletionTime(deletionTime)
+        .setPathPreviousSnapshotID(pathPreviousSnapshotID)
+        .setGlobalPreviousSnapshotID(globalPreviousSnapshotID)
+        .setSnapshotPath(snapshotPath)
+        .setCheckpointDir(checkpointDir);
+  }
+
+  /**
+   * Builder of SnapshotInfo.
+   */
+  public static class Builder {
+    private String snapshotID;
+    private String name;
+    private SnapshotStatus snapshotStatus;
+    private long creationTime;
+    private long deletionTime;
+    private String pathPreviousSnapshotID;
+    private String globalPreviousSnapshotID;
+    private String snapshotPath;
+    private String checkpointDir;
+
+    public Builder() {
+      // default values
+      this.snapshotStatus = SnapshotStatus.DEFAULT;
+    }
+
+    public Builder setSnapshotID(String snapshotID) {
+      this.snapshotID = snapshotID;
+      return this;
+    }
+
+    public Builder setName(String name) {
+      this.name = name;
+      return this;
+    }
+
+    public Builder setSnapshotStatus(SnapshotStatus snapshotStatus) {
+      this.snapshotStatus = snapshotStatus;
+      return this;
+    }
+
+    public Builder setCreationTime(long crTime) {
+      this.creationTime = crTime;
+      return this;
+    }
+
+    public Builder setDeletionTime(long delTime) {
+      this.deletionTime = delTime;
+      return this;
+    }
+
+    public Builder setPathPreviousSnapshotID(String pathPreviousSnapshotID) {
+      this.pathPreviousSnapshotID = pathPreviousSnapshotID;
+      return this;
+    }
+
+    public Builder setGlobalPreviousSnapshotID(
+        String globalPreviousSnapshotID) {
+      this.globalPreviousSnapshotID = globalPreviousSnapshotID;
+      return this;
+    }
+
+    public Builder setSnapshotPath(String snapshotPath) {
+      this.snapshotPath = snapshotPath;
+      return this;
+    }
+
+    public Builder setCheckpointDir(String checkpointDir) {
+      this.checkpointDir = checkpointDir;
+      return this;
+    }
+
+    public SnapshotInfo build() {
+      Preconditions.checkNotNull(name);
+      return new SnapshotInfo(
+          snapshotID,
+          name,
+          snapshotStatus,
+          creationTime,
+          deletionTime,
+          pathPreviousSnapshotID,
+          globalPreviousSnapshotID,
+          snapshotPath,
+          checkpointDir
+      );
+    }
+  }
+
+  /**
+   * Creates SnapshotInfo protobuf from OmBucketInfo.

Review Comment:
   Thanks @nandakumar131  for your review and comments.  I have fixed the typo.  Update in latest commit.



-- 
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


[GitHub] [ozone] nandakumar131 commented on a diff in pull request #3582: HDDS-6849. OM changes for tracking Snapshot Metadata

Posted by GitBox <gi...@apache.org>.
nandakumar131 commented on code in PR #3582:
URL: https://github.com/apache/ozone/pull/3582#discussion_r920729842


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -711,6 +711,31 @@ message PrefixInfo {
     optional uint64 updateID = 5;
 }
 
+/**
+ * SnapshotStatus - snapshot states.
+ * Snapshot in one of : active, deleted, or reclaimed state
+ */
+enum SnapshotStatusProto {
+  SNAPSHOT_ACTIVE = 1;
+  SNAPSHOT_DELETED = 2;
+  SNAPSHOT_RECLAIMED = 3;
+}
+
+/**
+ * SnapshotInfo table entry for Bucket/Volume snapshot metadata
+ */
+message SnapshotInfoEntry {

Review Comment:
   Suggestion: Can we have just _SnapshotInfo_ instead of _SnapshotInfoEntry_



##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/SnapshotInfo.java:
##########
@@ -0,0 +1,321 @@
+package org.apache.hadoop.ozone.om.helpers;
+
+/*
+ * 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
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  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.
+ */
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotInfoEntry;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotStatusProto;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * This class is used for storing info related to Snapshots.
+ *
+ * Each snapshot created has an associated SnapshotInfo entry
+ * containing the snapshotid, snapshot path,
+ * snapshot checkpoint directory, previous snapshotid
+ * for the snapshot path & global amongst other necessary fields.
+ */
+public final class SnapshotInfo {
+
+  /**
+   * SnapshotStatus enum composed of
+   * active, deleted and reclaimed statues.
+   */
+  public enum SnapshotStatus {
+    SNAPSHOT_ACTIVE,
+    SNAPSHOT_DELETED,
+    SNAPSHOT_RECLAIMED;
+
+    public static final SnapshotStatus DEFAULT = SNAPSHOT_ACTIVE;
+
+    public SnapshotStatusProto toProto() {
+      switch (this) {
+      case SNAPSHOT_ACTIVE:
+        return SnapshotStatusProto.SNAPSHOT_ACTIVE;
+      case SNAPSHOT_DELETED:
+        return SnapshotStatusProto.SNAPSHOT_DELETED;
+      case SNAPSHOT_RECLAIMED:
+        return SnapshotStatusProto.SNAPSHOT_RECLAIMED;
+      default:
+        throw new IllegalStateException(
+            "BUG: missing valid SnapshotStatus, found status=" + this);
+      }
+    }
+
+    public static SnapshotStatus valueOf(SnapshotStatusProto status) {
+      switch (status) {
+      case SNAPSHOT_ACTIVE:
+        return SNAPSHOT_ACTIVE;
+      case SNAPSHOT_DELETED:
+        return SNAPSHOT_DELETED;
+      case SNAPSHOT_RECLAIMED:
+        return SNAPSHOT_RECLAIMED;
+      default:
+        throw new IllegalStateException(
+            "BUG: missing valid SnapshotStatus, found status=" + status);
+      }
+    }
+  };
+
+  private final String snapshotID;  // UUID
+  private String name;
+  private SnapshotStatus snapshotStatus;
+  private final long creationTime;
+  private long deletionTime;
+  private String pathPreviousSnapshotID;
+  private String globalPreviousSnapshotID;
+  private String snapshotPath; // snapshot mask
+  private String checkpointDir;
+
+  /**
+   * Private constructor, constructed via builder.
+   * @param snapshotID - Snapshot UUID.
+   * @param name - snapshot name.
+   * @param snapshotStatus - status: SNAPSHOT_ACTIVE, SNAPSHOT_DELETED,
+   *                      SNAPSHOT_RECLAIMED
+   * @param creationTime - Snapshot creation time.
+   * @param deletionTime - Snapshot deletion time.
+   * @param pathPreviousSnapshotID - Snapshot path previous snapshot id.
+   * @param globalPreviousSnapshotID - Snapshot global previous snapshot id.
+   * @param snapshotPath - Snapshot path, bucket .snapshot path.
+   * @param checkpointDir - Snapshot checkpoint directory.
+   */
+  @SuppressWarnings("checkstyle:ParameterNumber")
+  private SnapshotInfo(String snapshotID,
+                       String name,
+                       SnapshotStatus snapshotStatus,
+                       long creationTime,
+                       long deletionTime,
+                       String pathPreviousSnapshotID,
+                       String globalPreviousSnapshotID,
+                       String snapshotPath,
+                       String checkpointDir) {
+    this.snapshotID = snapshotID;
+    this.name = name;
+    this.snapshotStatus = snapshotStatus;
+    this.creationTime = creationTime;
+    this.deletionTime = deletionTime;
+    this.pathPreviousSnapshotID = pathPreviousSnapshotID;
+    this.globalPreviousSnapshotID = globalPreviousSnapshotID;
+    this.snapshotPath = snapshotPath;
+    this.checkpointDir = checkpointDir;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public void setSnapshotStatus(SnapshotStatus snapshotStatus) {
+    this.snapshotStatus = snapshotStatus;
+  }
+
+  public void setDeletionTime(long delTime) {
+    this.deletionTime = delTime;
+  }
+
+  public void setPathPreviousSnapshotID(String pathPreviousSnapshotID) {
+    this.pathPreviousSnapshotID = pathPreviousSnapshotID;
+  }
+
+  public void setGlobalPreviousSnapshotID(String globalPreviousSnapshotID) {
+    this.globalPreviousSnapshotID = globalPreviousSnapshotID;
+  }
+
+  public void setSnapshotPath(String snapshotPath) {
+    this.snapshotPath = snapshotPath;
+  }
+
+  public void setCheckpointDir(String checkpointDir) {
+    this.checkpointDir = checkpointDir;
+  }
+
+  public String getSnapshotID() {
+    return snapshotID;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public SnapshotStatus getSnapshotStatus() {
+    return snapshotStatus;
+  }
+
+  public long getCreationTime() {
+    return creationTime;
+  }
+
+  public long getDeletionTime() {
+    return deletionTime;
+  }
+
+  public String getPathPreviousSnapshotID() {
+    return pathPreviousSnapshotID;
+  }
+
+  public String getGlobalPreviousSnapshotID() {
+    return globalPreviousSnapshotID;
+  }
+
+  public String getSnapshotPath() {
+    return snapshotPath;
+  }
+
+  public String getCheckpointDir() {
+    return checkpointDir;
+  }
+
+  public static SnapshotInfo.Builder newBuilder() {
+    return new SnapshotInfo.Builder();
+  }
+
+  public SnapshotInfo.Builder toBuilder() {
+    return new SnapshotInfo.Builder()
+        .setSnapshotID(snapshotID)
+        .setSnapshotStatus(snapshotStatus)
+        .setCreationTime(creationTime)
+        .setDeletionTime(deletionTime)
+        .setPathPreviousSnapshotID(pathPreviousSnapshotID)
+        .setGlobalPreviousSnapshotID(globalPreviousSnapshotID)
+        .setSnapshotPath(snapshotPath)
+        .setCheckpointDir(checkpointDir);
+  }
+
+  /**
+   * Builder of SnapshotInfo.
+   */
+  public static class Builder {
+    private String snapshotID;
+    private String name;
+    private SnapshotStatus snapshotStatus;
+    private long creationTime;
+    private long deletionTime;
+    private String pathPreviousSnapshotID;
+    private String globalPreviousSnapshotID;
+    private String snapshotPath;
+    private String checkpointDir;
+
+    public Builder() {
+      // default values
+      this.snapshotStatus = SnapshotStatus.DEFAULT;
+    }
+
+    public Builder setSnapshotID(String snapshotID) {
+      this.snapshotID = snapshotID;
+      return this;
+    }
+
+    public Builder setName(String name) {
+      this.name = name;
+      return this;
+    }
+
+    public Builder setSnapshotStatus(SnapshotStatus snapshotStatus) {
+      this.snapshotStatus = snapshotStatus;
+      return this;
+    }
+
+    public Builder setCreationTime(long crTime) {
+      this.creationTime = crTime;
+      return this;
+    }
+
+    public Builder setDeletionTime(long delTime) {
+      this.deletionTime = delTime;
+      return this;
+    }
+
+    public Builder setPathPreviousSnapshotID(String pathPreviousSnapshotID) {
+      this.pathPreviousSnapshotID = pathPreviousSnapshotID;
+      return this;
+    }
+
+    public Builder setGlobalPreviousSnapshotID(
+        String globalPreviousSnapshotID) {
+      this.globalPreviousSnapshotID = globalPreviousSnapshotID;
+      return this;
+    }
+
+    public Builder setSnapshotPath(String snapshotPath) {
+      this.snapshotPath = snapshotPath;
+      return this;
+    }
+
+    public Builder setCheckpointDir(String checkpointDir) {
+      this.checkpointDir = checkpointDir;
+      return this;
+    }
+
+    public SnapshotInfo build() {
+      Preconditions.checkNotNull(name);
+      return new SnapshotInfo(
+          snapshotID,
+          name,
+          snapshotStatus,
+          creationTime,
+          deletionTime,
+          pathPreviousSnapshotID,
+          globalPreviousSnapshotID,
+          snapshotPath,
+          checkpointDir
+      );
+    }
+  }
+
+  /**
+   * Creates SnapshotInfo protobuf from OmBucketInfo.

Review Comment:
   Typo: _OmBucketInfo_ -> _SnapshotInfo_



-- 
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


[GitHub] [ozone] nandakumar131 commented on a diff in pull request #3582: HDDS-6849. OM changes for tracking Snapshot Metadata

Posted by GitBox <gi...@apache.org>.
nandakumar131 commented on code in PR #3582:
URL: https://github.com/apache/ozone/pull/3582#discussion_r920732754


##########
hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/codec/OmDBSnapshotInfoCodec.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.codec;
+
+import org.apache.hadoop.hdds.utils.db.Codec;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotInfoEntry;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Codec to encode SnapshotInfo as byte array.
+ */
+public class OmDBSnapshotInfoCodec implements Codec<SnapshotInfo> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OmDBSnapshotInfoCodec.class);
+
+  @Override
+  public byte[] toPersistedFormat(SnapshotInfo object) throws IOException {
+    checkNotNull(object, "Null object can't be converted to byte array.");
+    return object.getProtobuf().toByteArray();
+  }
+
+  @Override
+  public SnapshotInfo fromPersistedFormat(byte[] rawData)
+      throws IOException {
+    checkNotNull(rawData, "Null byte array can't be converted to " +
+        "real object.");
+    return SnapshotInfo.getFromProtobuf(
+        SnapshotInfoEntry.parseFrom(rawData));
+  }
+
+  @Override
+  public SnapshotInfo copyObject(SnapshotInfo object) {
+    // Note: Not really a "copy". from OMDBSnapshotInfoCodec
+    return object;

Review Comment:
   As George pointed out, we can do a real copy here.



-- 
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


[GitHub] [ozone] adoroszlai commented on pull request #3582: HDDS-6849. OM changes for tracking Snapshot Metadata

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on PR #3582:
URL: https://github.com/apache/ozone/pull/3582#issuecomment-1187528434

   We have recently [discussed](https://lists.apache.org/thread/r7nb7bsvt1w8lmmj1jn4zbkz3j29xkzo) avoiding long-lived branches for feature development.  Since this feature branch is still new (no commits ahead of `master`), can you please consider developing the snapshot feature on `master`?  This would eliminate the burden required for periodic merges from `master`, as well as the lengthy process required for merge into `master` when the feature is near completion.


-- 
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


[GitHub] [ozone] neils-dev commented on a diff in pull request #3582: HDDS-6849. OM changes for tracking Snapshot Metadata

Posted by GitBox <gi...@apache.org>.
neils-dev commented on code in PR #3582:
URL: https://github.com/apache/ozone/pull/3582#discussion_r922906312


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -711,6 +711,31 @@ message PrefixInfo {
     optional uint64 updateID = 5;
 }
 
+/**
+ * SnapshotStatus - snapshot states.
+ * Snapshot in one of : active, deleted, or reclaimed state
+ */
+enum SnapshotStatusProto {
+  SNAPSHOT_ACTIVE = 1;
+  SNAPSHOT_DELETED = 2;
+  SNAPSHOT_RECLAIMED = 3;
+}
+
+/**
+ * SnapshotInfo table entry for Bucket/Volume snapshot metadata
+ */
+message SnapshotInfoEntry {

Review Comment:
   > Can we also add Volume Name and Bucket Name to the Snapshot Info structure?
   
   Added both volume name and bucket name fields to our `SnapshotInfo` table.  Thanks @nandakumar131  



-- 
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


[GitHub] [ozone] nandakumar131 merged pull request #3582: HDDS-6849. OM changes for tracking Snapshot Metadata

Posted by GitBox <gi...@apache.org>.
nandakumar131 merged PR #3582:
URL: https://github.com/apache/ozone/pull/3582


-- 
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


[GitHub] [ozone] neils-dev commented on a diff in pull request #3582: HDDS-6849. OM changes for tracking Snapshot Metadata

Posted by GitBox <gi...@apache.org>.
neils-dev commented on code in PR #3582:
URL: https://github.com/apache/ozone/pull/3582#discussion_r922906141


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -711,6 +711,31 @@ message PrefixInfo {
     optional uint64 updateID = 5;
 }
 
+/**
+ * SnapshotStatus - snapshot states.
+ * Snapshot in one of : active, deleted, or reclaimed state
+ */
+enum SnapshotStatusProto {
+  SNAPSHOT_ACTIVE = 1;
+  SNAPSHOT_DELETED = 2;
+  SNAPSHOT_RECLAIMED = 3;
+}
+
+/**
+ * SnapshotInfo table entry for Bucket/Volume snapshot metadata
+ */
+message SnapshotInfoEntry {

Review Comment:
   By using `SnapshotInfo` both for the proto and the class, it makes some declarations ambiguous without fully qualifying the objects used.  I too wanted to use `SnapshotInfo` for the proto definition at first.  I avoided the ambiguity  by using a unique identifier (`SnapshotInfoEntry`) for the proto snapshotinfo definition.
   
   As you suggested, I renamed the proto definition to `SnapshotInfo` and fully qualified the use of the proto where needed.  In latest commit.  Thanks.



-- 
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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3582: HDDS-6849. OM changes for tracking Snapshot Metadata

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3582:
URL: https://github.com/apache/ozone/pull/3582#discussion_r922911289


##########
hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/codec/OmDBSnapshotInfoCodec.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.codec;
+
+import org.apache.hadoop.hdds.utils.db.Codec;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotInfoEntry;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Codec to encode SnapshotInfo as byte array.
+ */
+public class OmDBSnapshotInfoCodec implements Codec<SnapshotInfo> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OmDBSnapshotInfoCodec.class);
+
+  @Override
+  public byte[] toPersistedFormat(SnapshotInfo object) throws IOException {
+    checkNotNull(object, "Null object can't be converted to byte array.");
+    return object.getProtobuf().toByteArray();
+  }
+
+  @Override
+  public SnapshotInfo fromPersistedFormat(byte[] rawData)
+      throws IOException {
+    checkNotNull(rawData, "Null byte array can't be converted to " +
+        "real object.");
+    return SnapshotInfo.getFromProtobuf(
+        SnapshotInfoEntry.parseFrom(rawData));
+  }
+
+  @Override
+  public SnapshotInfo copyObject(SnapshotInfo object) {
+    // Note: Not really a "copy". from OMDBSnapshotInfoCodec
+    return object;

Review Comment:
   OmDBTenantState and OmDBUserPrincipalInfo both appear to be immutable, so it makes sense for those "copy's" to just return the original object.  SnapshotInfo isn't immutable, (even though its fields are.)
   



-- 
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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3582: HDDS-6849. OM changes for tracking Snapshot Metadata

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3582:
URL: https://github.com/apache/ozone/pull/3582#discussion_r922917044


##########
hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/codec/OmDBSnapshotInfoCodec.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.codec;
+
+import org.apache.hadoop.hdds.utils.db.Codec;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotInfoEntry;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Codec to encode SnapshotInfo as byte array.
+ */
+public class OmDBSnapshotInfoCodec implements Codec<SnapshotInfo> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OmDBSnapshotInfoCodec.class);
+
+  @Override
+  public byte[] toPersistedFormat(SnapshotInfo object) throws IOException {
+    checkNotNull(object, "Null object can't be converted to byte array.");
+    return object.getProtobuf().toByteArray();
+  }
+
+  @Override
+  public SnapshotInfo fromPersistedFormat(byte[] rawData)
+      throws IOException {
+    checkNotNull(rawData, "Null byte array can't be converted to " +
+        "real object.");
+    return SnapshotInfo.getFromProtobuf(
+        SnapshotInfoEntry.parseFrom(rawData));
+  }
+
+  @Override
+  public SnapshotInfo copyObject(SnapshotInfo object) {
+    // Note: Not really a "copy". from OMDBSnapshotInfoCodec
+    return object;

Review Comment:
   Actually OmDBUserPrincipalInfo is not immutable.  So I would say that one is wrong too.



-- 
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