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/08/30 01:57:05 UTC

[GitHub] [ozone] GeorgeJahad opened a new pull request, #3729: HDDS-6855. Path Based Access for Ozone Snapshots.

GeorgeJahad opened a new pull request, #3729:
URL: https://github.com/apache/ozone/pull/3729

   ## What changes were proposed in this pull request?
   ### OmSnapshot
   This PR builds on the OmMReader PR https://github.com/apache/ozone/pull/3653  to create the new OmSnapshot abstraction which is an OmMReader that reads a snapshots metadata.
   
   It's basically identical to the ozoneManager OmMReader with two exceptions:
   1. Its keymanager and prefix manager contain an OmMetadataManager that reads from a snapshot instead of the active filesystem.
   2. It normalizes/denormalizes each request as it comes in to remove/replace the ".snapshot/snapshotName" prefix.
   
   ### OmLock
   The OmMetadataManager use locks to prevent the active filesystem from being corrupted.  These locks are unnecessary for snapshotting, but I didn't want to modify all the metadatamanager code to check whether it is looking at a snapshot or not.  Instead I created the OmLock interface with two implementations, the current one, and a new "read-only" lock for snapshots.
   
   The new read only lock isn't a lock at all.  It just returns true for aquiring read locks and false for write locks.
   
   ### OmSnapshotManager, (OSM)
   The OmSnapshotManager contains the LRU cache of OmSnapshots.
   
   When the OM gets a read request, it passes it to the OSM to see if it is a snapshot request.  If so, the OSM retrieves the snapshot from the cache, (reading it in first if necessary,) and returns that snapshots OmMReader to the OM to complete the request.
   
   If itt is not a snapshot, the OM just uses the standard OmMReader for the active filesystem.
   
   ### IOmMetadataReader interface
   I also added this interface, (which is implemented by the existing OmMetadataReader.  The OzoneManager and OmSnapshot also implement it and delegate the requests to their respective OmMetadataReader instances.)  It wasn't strictly needed but seems cleaner to do it this way.
   
   Let me know if you think it is overkill.
   
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/HDDS-6855
   
   
   ## How was this patch tested?
   
   There are many existing tests that exercise reading of the metadata.  I wanted to leverage those as much as possible.  So my basic approach has been to copy them into this PR and modify them just enough to support snapshots.  The tests were originally structured something like this:
   
   ```
   1. create some data
   2. read back the data to confirm it was correctly created.
   3. (possibly repeat the above steps to create more data.)
   ```
   
   Between steps 1 and 2, I've added two more:
   ```
   1a. create a snapshot
   1b. delete the data from the active filesystem, (to be sure the snapshot code isn't inadvertently reading from there.)
   ```
   
   In addition I've had to modify the read requests in the tests to use the snapshot prefix and read from the snapshot.
   
   The original tests were taken from the TestOzoneFileSystem, TestOzoneRpcClientAbstract, and TestObjectStoreWithFSO test classes.
   
   To be sure I've only changed the minimum required to support snapshotting, I've created a python script to generate the diffs between those original tests and my changes.  Those diffs are [here](https://github.com/GeorgeJahad/ozone/compare/a27cc3729..977923847?w=1).
   
   
   Where possible I've parameterized the tests so that they excercise all 3 bucket types.
   
   ## TODO
   1. This PR handles most snapshot read operations with the exception of ACL's.  That will be handled in the next PR.
   2. We need to prevent users from writing to keys that are prefixed with ".snapshot", (because they won't be able to read them back out again).  I added a [ticket](https://issues.apache.org/jira/browse/HDDS-7168) for it.
   
   ## UI
   With this PR, the following types of commands now work:
   ```
   ozone sh key get testgbj2/bucket1/.snapshot/snapshot1/k1 k2
   ozone fs -ls  /testgbj2/bucket1/.snapshot/snapshot1
   ozone fs -cp /testgbj2/bucket1/.snapshot/snapshot1/k1  /testgbj2/bucket1/k3
   ```
   
   These commands will init the snapshot so that the commands above can operate on it:
   ```
   ozone sh volume create testgbj2
   ozone sh bucket create testgbj2/bucket1
   echo k1 > k1.orig
   ozone sh key put testgbj2/bucket1/k1 k1.orig
   ozone sh snapshot create testgbj2/bucket1 snapshot1
   ```
   


-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java:
##########
@@ -286,6 +289,25 @@ protected OmMetadataManagerImpl() {
     this.omEpoch = 0;
   }
 
+  //  metadata constructor for snapshots
+  private OmMetadataManagerImpl(OzoneConfiguration conf, String snapshotName)
+      throws IOException {
+    lock = new OmReadOnlyLock();
+    omEpoch = 0;
+    String snapshotDir = OMStorage.getOmDbDir(conf).toString() +
+        OM_KEY_PREFIX + OM_SNAPSHOT_DIR;
+    setStore(loadDB(conf, new File(snapshotDir),
+        OM_DB_NAME + snapshotName, true));

Review Comment:
   ok



-- 
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] prashantpogde commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/IOmMetadataReader.java:
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Protocol for OmMetadataReader's.
+ */
+public interface IOmMetadataReader {
+  /**
+   * Look up for the container of an existing key.
+   *
+   * @param args the args of the key.
+   * @return OmKeyInfo instance that client uses to talk to container.
+   */
+  OmKeyInfo lookupKey(OmKeyArgs args) throws IOException;
+
+  /**
+   * List the status for a file or a directory and its contents.
+   *
+   * @param args    Key args
+   * @param recursive  For a directory if true all the descendants of a
+   *                   particular directory are listed
+   * @param startKey   Key from which listing needs to start. If startKey exists
+   *                   its status is included in the final list.
+   * @param numEntries Number of entries to list from the start key
+   * @param allowPartialPrefixes if partial prefixes should be allowed,
+   *                             this is needed in context of ListKeys
+   * @return list of file status
+   */
+  List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,
+                                   String startKey, long numEntries,
+                                   boolean allowPartialPrefixes)
+      throws IOException;
+
+  default List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,
+      String startKey, long numEntries)
+      throws IOException {
+    return listStatus(args, recursive, startKey, numEntries, false);
+  }
+
+  /**
+   * OzoneFS api to get file status for an entry.
+   *
+   * @param keyArgs Key args
+   * @throws OMException if file does not exist
+   *                     if bucket does not exist
+   * @throws IOException if there is error in the db
+   *                     invalid arguments
+   */
+  OzoneFileStatus getFileStatus(OmKeyArgs keyArgs) throws IOException;
+
+
+  /**
+   * OzoneFS api to lookup for a file.
+   *
+   * @param args Key args
+   * @throws OMException if given key is not found or it is not a file
+   *                     if bucket does not exist
+   * @throws IOException if there is error in the db
+   *                     invalid arguments
+   */
+  OmKeyInfo lookupFile(OmKeyArgs args) throws IOException;
+
+  /**
+   * Returns a list of keys represented by {@link OmKeyInfo}
+   * in the given bucket. Argument volumeName, bucketName is required,
+   * others are optional.
+   *
+   * @param volumeName
+   *   the name of the volume.
+   * @param bucketName
+   *   the name of the bucket.
+   * @param startKey
+   *   the start key name, only the keys whose name is
+   *   after this value will be included in the result.
+   * @param keyPrefix
+   *   key name prefix, only the keys whose name has
+   *   this prefix will be included in the result.
+   * @param maxKeys
+   *   the maximum number of keys to return. It ensures
+   *   the size of the result will not exceed this limit.
+   * @return a list of keys.
+   */
+  List<OmKeyInfo> listKeys(String volumeName, String bucketName,

Review Comment:
   We can track it under https://issues.apache.org/jira/browse/HDDS-7277. I will resolve this comment for this PR.



-- 
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] prashantpogde merged pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/IOmMetadataReader.java:
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Protocol for OmMetadataReader's.
+ */
+public interface IOmMetadataReader {
+  /**
+   * Look up for the container of an existing key.
+   *
+   * @param args the args of the key.
+   * @return OmKeyInfo instance that client uses to talk to container.
+   */
+  OmKeyInfo lookupKey(OmKeyArgs args) throws IOException;
+
+  /**
+   * List the status for a file or a directory and its contents.
+   *
+   * @param args    Key args
+   * @param recursive  For a directory if true all the descendants of a
+   *                   particular directory are listed
+   * @param startKey   Key from which listing needs to start. If startKey exists
+   *                   its status is included in the final list.
+   * @param numEntries Number of entries to list from the start key
+   * @param allowPartialPrefixes if partial prefixes should be allowed,
+   *                             this is needed in context of ListKeys
+   * @return list of file status
+   */
+  List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,

Review Comment:
   Actually, after responding to @smengcl 's comment above about about the listKeys() keyPrefix, I realized that this command does work, but has to be invoked with a keyPrefix.  So, instead of this:
   ```
   ozone sh key list /vol1/bucket1/.snapshot/snap1
   ```
   It works like this:
   ```
   ozone sh key list /vol1/bucket1/ -p .snapshot/snap1
   ```
   
   I admit that is not a great user interface, and should probably be fixed.  But at least you can see the functionality is there.
   



-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshotFileSystem.java:
##########
@@ -0,0 +1,577 @@
+/**
+ * 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.commons.io.IOUtils;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.ozone.OzoneFileSystem;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.fs.ozone.Constants.LISTING_PAGE_SIZE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_SCHEME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * OmSnapshot file system tests.
+ */
+@RunWith(Parameterized.class)
+public class TestOmSnapshotFileSystem {
+  private static MiniOzoneCluster cluster = null;
+  private static OzoneConfiguration conf;
+  private static String volumeName;
+  private static String bucketName;
+  private static FileSystem fs;
+  private static OzoneFileSystem o3fs;
+  private static OzoneManagerProtocol writeClient;
+  private static BucketLayout bucketLayout;
+  private static boolean enabledFileSystemPaths;
+  private static File metaDir;
+  private static OzoneManager ozoneManager;
+  private static String keyPrefix;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOmSnapshot.class);
+
+
+
+  @Rule
+  public Timeout timeout = new Timeout(1200000);

Review Comment:
   ah I see.  Yeh it was just a cut and paste from here: https://github.com/apache/ozone/blob/master/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreWithFSO.java#L93
   
   I didn't even notice it.  I'll update.



-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OmLock.java:
##########
@@ -0,0 +1,98 @@
+/**
+ * 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.lock;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Interface for OM Metadata locks.
+ */
+public interface OmLock {

Review Comment:
   will do



##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OmReadOnlyLock.java:
##########
@@ -0,0 +1,167 @@
+/**
+ * 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.lock;
+
+/**
+ * Read only "lock" for snapshots
+ * Uses no lock.  Always returns true when aquiring
+ * read lock and false for write locks
+ */
+public class OmReadOnlyLock implements OmLock {
+  @Override
+  public boolean acquireLock(OzoneManagerLock.Resource resource,
+                             String... resources) {
+    return false;
+  }
+
+  @Override
+  public boolean acquireReadLock(OzoneManagerLock.Resource resource,
+                                 String... resources) {
+    return true;
+  }
+
+  @Override
+  public boolean acquireReadHashedLock(OzoneManagerLock.Resource resource,
+                                 String resourceName) {
+    return true;
+  }
+
+  @Override
+  public boolean acquireWriteLock(OzoneManagerLock.Resource resource,
+                                  String... resources) {
+    return false;
+  }
+
+  @Override
+  public boolean acquireWriteHashedLock(OzoneManagerLock.Resource resource,
+                                        String resourceName) {
+    return false;
+  }
+
+  @Override
+  public String generateResourceName(OzoneManagerLock.Resource resource,
+                                     String... resources) {
+    return "";
+  }
+
+  @Override
+  public boolean acquireMultiUserLock(String firstUser, String secondUser) {
+    return false;
+  }
+
+  @Override
+  public void releaseMultiUserLock(String firstUser, String secondUser) {
+

Review Comment:
   will do



-- 
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] prashantpogde commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotMetrics.java:
##########
@@ -0,0 +1,147 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+
+/**
+ * This class is for maintaining Snapshot Manager statistics.
+ */
+@InterfaceAudience.Private
+@Metrics(about = "Snapshot Manager Metrics", context = "dfs")
+public final class OmSnapshotMetrics implements OmMetadataReaderMetrics {

Review Comment:
   I am seeing these metrics aggregated for all snapshots. We should perhaps file a separate Jira ticket to see these metrics aggregated for all snapshots as well as ability to see usage per snapshot. 



-- 
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] smengcl commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java:
##########
@@ -286,6 +289,25 @@ protected OmMetadataManagerImpl() {
     this.omEpoch = 0;
   }
 
+  //  metadata constructor for snapshots
+  private OmMetadataManagerImpl(OzoneConfiguration conf, String snapshotName)
+      throws IOException {
+    lock = new OmReadOnlyLock();
+    omEpoch = 0;
+    String snapshotDir = OMStorage.getOmDbDir(conf).toString() +
+        OM_KEY_PREFIX + OM_SNAPSHOT_DIR;
+    setStore(loadDB(conf, new File(snapshotDir),
+        OM_DB_NAME + snapshotName, true));

Review Comment:
   Probably rename `snapshotName` to `snapshotUUID` to avoid confusion with the user input snapshot name?



##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java:
##########
@@ -812,4 +813,9 @@ public static String getOMAddressListPrintString(List<OMNodeDetails> omList) {
     printString.append("]");
     return printString.toString();
   }
+
+  // Key points to entire bucket's snapshot
+  public static boolean isBucketSnapshotIndicator(String key) {
+    return key.startsWith(OM_SNAPSHOT_INDICATOR) && key.split("/").length == 2;

Review Comment:
   Regarding `key.split("/").length == 2`, would you give an example of the expected input `key` here?
   
   I also wonder if the `key` is guaranteed sanitized when this method is invoked (e.g. maybe some input has extra `/` at the end?).



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -0,0 +1,421 @@
+/**
+ * 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.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.HddsWhiteboxTestUtils;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.AfterClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.client.ReplicationFactor.ONE;
+import static org.apache.hadoop.hdds.client.ReplicationType.RATIS;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIR;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.helpers.BucketLayout.FILE_SYSTEM_OPTIMIZED;
+import static org.apache.hadoop.ozone.om.helpers.BucketLayout.OBJECT_STORE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+
+/**
+ * Test OmSnapshot bucket interface.
+ */
+@RunWith(Parameterized.class)
+public class TestOmSnapshot {
+  private static MiniOzoneCluster cluster = null;
+  private static String volumeName;
+  private static String bucketName;
+  private static OzoneManagerProtocol writeClient;
+  private static BucketLayout bucketLayout = BucketLayout.LEGACY;
+  private static boolean enabledFileSystemPaths;
+  private static ObjectStore store;
+  private static File metaDir;
+  private static OzoneManager ozoneManager;
+
+
+
+  @Rule
+  public Timeout timeout = new Timeout(1200000);

Review Comment:
   It took 147s to run through all 3 sets of params on my Mac. Another run took 124s. Not sure if my dev env is slowing things down, but each `testListKey` in a set of param seemingly took over 30s (which might have included the prep phase).
   
   GitHub CI might time out with 180s here. If that is the case we could further bump the timeout to maybe 300s. (On the other hand, we could try to reduce the overall runtime of the test if possible.)



##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/IOmMetadataReader.java:
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Protocol for OmMetadataReader's.
+ */
+public interface IOmMetadataReader {
+  /**
+   * Look up for the container of an existing key.
+   *
+   * @param args the args of the key.
+   * @return OmKeyInfo instance that client uses to talk to container.
+   */
+  OmKeyInfo lookupKey(OmKeyArgs args) throws IOException;
+
+  /**
+   * List the status for a file or a directory and its contents.
+   *
+   * @param args    Key args
+   * @param recursive  For a directory if true all the descendants of a
+   *                   particular directory are listed
+   * @param startKey   Key from which listing needs to start. If startKey exists
+   *                   its status is included in the final list.
+   * @param numEntries Number of entries to list from the start key
+   * @param allowPartialPrefixes if partial prefixes should be allowed,
+   *                             this is needed in context of ListKeys
+   * @return list of file status
+   */
+  List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,
+                                   String startKey, long numEntries,
+                                   boolean allowPartialPrefixes)
+      throws IOException;
+
+  default List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,
+      String startKey, long numEntries)
+      throws IOException {
+    return listStatus(args, recursive, startKey, numEntries, false);
+  }
+
+  /**
+   * OzoneFS api to get file status for an entry.
+   *
+   * @param keyArgs Key args
+   * @throws OMException if file does not exist
+   *                     if bucket does not exist
+   * @throws IOException if there is error in the db
+   *                     invalid arguments
+   */
+  OzoneFileStatus getFileStatus(OmKeyArgs keyArgs) throws IOException;
+
+

Review Comment:
   nit: extra line
   
   ```suggestion
   
   ```



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshotFileSystem.java:
##########
@@ -0,0 +1,577 @@
+/**
+ * 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.commons.io.IOUtils;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.ozone.OzoneFileSystem;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.fs.ozone.Constants.LISTING_PAGE_SIZE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_SCHEME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * OmSnapshot file system tests.
+ */
+@RunWith(Parameterized.class)
+public class TestOmSnapshotFileSystem {
+  private static MiniOzoneCluster cluster = null;
+  private static OzoneConfiguration conf;
+  private static String volumeName;
+  private static String bucketName;
+  private static FileSystem fs;
+  private static OzoneFileSystem o3fs;
+  private static OzoneManagerProtocol writeClient;
+  private static BucketLayout bucketLayout;
+  private static boolean enabledFileSystemPaths;
+  private static File metaDir;
+  private static OzoneManager ozoneManager;
+  private static String keyPrefix;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOmSnapshot.class);
+
+
+
+  @Rule
+  public Timeout timeout = new Timeout(1200000);

Review Comment:
   Same here.
   
   fyi it took 78s for me to run this. 120s timeout is probably fine for GH CI.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java:
##########
@@ -286,6 +289,25 @@ protected OmMetadataManagerImpl() {
     this.omEpoch = 0;
   }
 
+  //  metadata constructor for snapshots
+  private OmMetadataManagerImpl(OzoneConfiguration conf, String snapshotName)
+      throws IOException {
+    lock = new OmReadOnlyLock();
+    omEpoch = 0;
+    String snapshotDir = OMStorage.getOmDbDir(conf).toString() +

Review Comment:
   nit
   ```suggestion
       String snapshotDir = OMStorage.getOmDbDir(conf) +
   ```



##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OmReadOnlyLock.java:
##########
@@ -0,0 +1,167 @@
+/**
+ * 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.lock;
+
+/**
+ * Read only "lock" for snapshots
+ * Uses no lock.  Always returns true when aquiring
+ * read lock and false for write locks
+ */
+public class OmReadOnlyLock implements OmLock {
+  @Override
+  public boolean acquireLock(OzoneManagerLock.Resource resource,
+                             String... resources) {
+    return false;
+  }
+
+  @Override
+  public boolean acquireReadLock(OzoneManagerLock.Resource resource,
+                                 String... resources) {
+    return true;
+  }
+
+  @Override
+  public boolean acquireReadHashedLock(OzoneManagerLock.Resource resource,
+                                 String resourceName) {
+    return true;
+  }
+
+  @Override
+  public boolean acquireWriteLock(OzoneManagerLock.Resource resource,
+                                  String... resources) {
+    return false;
+  }
+
+  @Override
+  public boolean acquireWriteHashedLock(OzoneManagerLock.Resource resource,
+                                        String resourceName) {
+    return false;
+  }
+
+  @Override
+  public String generateResourceName(OzoneManagerLock.Resource resource,
+                                     String... resources) {
+    return "";
+  }
+
+  @Override
+  public boolean acquireMultiUserLock(String firstUser, String secondUser) {
+    return false;
+  }
+
+  @Override
+  public void releaseMultiUserLock(String firstUser, String secondUser) {
+

Review Comment:
   How about adding a single-line comment in each empty block?
   
   ```suggestion
       // Intentionally empty
   ```



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java:
##########
@@ -286,6 +289,25 @@ protected OmMetadataManagerImpl() {
     this.omEpoch = 0;
   }
 
+  //  metadata constructor for snapshots

Review Comment:
   nit
   ```suggestion
     // metadata constructor for snapshots
   ```



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.
+ */
+
+package org.apache.hadoop.ozone.om;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import org.apache.hadoop.hdds.utils.db.DBCheckpoint;
+import org.apache.hadoop.hdds.utils.db.RDBStore;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_INDICATOR;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_KEY_NAME;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+
+
+/**
+ * This class is used to manage/create OM snapshots.
+ */
+public final class OmSnapshotManager {
+  private final OzoneManager ozoneManager;
+  private final LoadingCache<String, OmSnapshot> snapshotCache;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OmSnapshotManager.class);
+
+  OmSnapshotManager(OzoneManager ozoneManager) {
+    this.ozoneManager = ozoneManager;
+
+    // size of lru cache
+    int cacheSize = ozoneManager.getConfiguration().getInt(
+        OzoneConfigKeys.OZONE_OM_SNAPSHOT_CACHE_MAX_SIZE,
+        OzoneConfigKeys.OZONE_OM_SNAPSHOT_CACHE_MAX_SIZE_DEFAULT);
+
+    CacheLoader<String, OmSnapshot> loader;
+    loader = new CacheLoader<String, OmSnapshot>() {
+      @Override
+
+      // load the snapshot into the cache if not already there
+      @Nonnull
+      public OmSnapshot load(@Nonnull String snapshotTableKey)
+          throws IOException {
+        SnapshotInfo snapshotInfo;
+        // see if the snapshot exists
+        snapshotInfo = getSnapshotInfo(snapshotTableKey);
+
+        // read in the snapshot
+        OzoneConfiguration conf = ozoneManager.getConfiguration();
+        OMMetadataManager snapshotMetadataManager;
+
+        // Create the snapshot metadata manager by finding the corresponding
+        // RocksDB instance, creating an OmMetadataManagerImpl instance based on
+        // that
+        try {
+          snapshotMetadataManager = OmMetadataManagerImpl
+              .createSnapshotMetadataManager(
+              conf, snapshotInfo.getCheckpointDirName());
+        } catch (IOException e) {
+          LOG.error("Failed to retrieve snapshot: {}, {}", snapshotTableKey, e);
+          throw e;
+        }
+
+        // create the other manager instances based on snapshot metadataManager
+        PrefixManagerImpl pm = new PrefixManagerImpl(snapshotMetadataManager,
+            false);
+        KeyManagerImpl km = new KeyManagerImpl(null,
+            ozoneManager.getScmClient(), snapshotMetadataManager, conf, null,
+            ozoneManager.getBlockTokenSecretManager(),
+            ozoneManager.getKmsProvider(), pm);
+
+        return new OmSnapshot(km, pm, ozoneManager,
+            snapshotInfo.getVolumeName(),
+            snapshotInfo.getBucketName(),
+            snapshotInfo.getName());
+      }
+    };
+
+    // init LRU cache
+    snapshotCache = CacheBuilder.newBuilder()
+        .maximumSize(cacheSize).build(loader);
+    
+  }
+  /**

Review Comment:
   nit
   
   ```suggestion
     }
   
     /**
   ```



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshotFileSystem.java:
##########
@@ -0,0 +1,577 @@
+/**
+ * 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.commons.io.IOUtils;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.ozone.OzoneFileSystem;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.fs.ozone.Constants.LISTING_PAGE_SIZE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_SCHEME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * OmSnapshot file system tests.
+ */
+@RunWith(Parameterized.class)
+public class TestOmSnapshotFileSystem {
+  private static MiniOzoneCluster cluster = null;
+  private static OzoneConfiguration conf;
+  private static String volumeName;
+  private static String bucketName;
+  private static FileSystem fs;
+  private static OzoneFileSystem o3fs;
+  private static OzoneManagerProtocol writeClient;
+  private static BucketLayout bucketLayout;
+  private static boolean enabledFileSystemPaths;
+  private static File metaDir;
+  private static OzoneManager ozoneManager;
+  private static String keyPrefix;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOmSnapshot.class);
+
+
+
+  @Rule
+  public Timeout timeout = new Timeout(1200000);
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    return Arrays.asList(
+        new Object[]{BucketLayout.FILE_SYSTEM_OPTIMIZED, false},
+        new Object[]{BucketLayout.LEGACY, true});
+  }
+
+  public TestOmSnapshotFileSystem(BucketLayout newBucketLayout,
+      boolean newEnableFileSystemPaths) throws Exception {
+    // Checking whether 'newBucketLayout' and
+    // 'newEnableFileSystemPaths' flags represents next parameter
+    // index values. This is to ensure that initialize init() function
+    // will be invoked only at the beginning of every new set of
+    // Parameterized.Parameters.
+    if (TestOmSnapshotFileSystem.enabledFileSystemPaths !=
+        newEnableFileSystemPaths ||
+        TestOmSnapshotFileSystem.bucketLayout != newBucketLayout) {
+      setConfig(newBucketLayout, newEnableFileSystemPaths);
+      tearDown();
+      init();
+    }
+  }
+  private static void setConfig(BucketLayout newBucketLayout,
+      boolean newEnableFileSystemPaths) {
+    TestOmSnapshotFileSystem.enabledFileSystemPaths = newEnableFileSystemPaths;
+    TestOmSnapshotFileSystem.bucketLayout = newBucketLayout;
+  }
+  /**
+   * Create a MiniDFSCluster for testing.
+   * <p>
+   *

Review Comment:
   nit: unfinished javadoc?



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -0,0 +1,421 @@
+/**
+ * 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.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.HddsWhiteboxTestUtils;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.AfterClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.client.ReplicationFactor.ONE;
+import static org.apache.hadoop.hdds.client.ReplicationType.RATIS;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIR;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.helpers.BucketLayout.FILE_SYSTEM_OPTIMIZED;
+import static org.apache.hadoop.ozone.om.helpers.BucketLayout.OBJECT_STORE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+
+/**
+ * Test OmSnapshot bucket interface.
+ */
+@RunWith(Parameterized.class)
+public class TestOmSnapshot {
+  private static MiniOzoneCluster cluster = null;
+  private static String volumeName;
+  private static String bucketName;
+  private static OzoneManagerProtocol writeClient;
+  private static BucketLayout bucketLayout = BucketLayout.LEGACY;
+  private static boolean enabledFileSystemPaths;
+  private static ObjectStore store;
+  private static File metaDir;
+  private static OzoneManager ozoneManager;
+
+
+
+  @Rule
+  public Timeout timeout = new Timeout(1200000);

Review Comment:
   ```suggestion
     @Rule
     public Timeout timeout = new Timeout(180, TimeUnit.SECONDS);
   ```



##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -3287,4 +3287,13 @@
       will create intermediate directories.
     </description>
   </property>
+  <property>
+    <name>ozone.om.snapshot.cache.max.size</name>
+    <value>200</value>
+    <tag>OZONE, OM</tag>
+    <description>
+      Size of the OM Snapshot LRU cache.  This is the maximum number of open OM Snapshot RocksDb instances
+      that will be held in memory at any one time.

Review Comment:
   nit
   ```suggestion
         that will be held in memory at any time.
   ```



##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OmLock.java:
##########
@@ -0,0 +1,98 @@
+/**
+ * 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.lock;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Interface for OM Metadata locks.
+ */
+public interface OmLock {

Review Comment:
   What about `IOzoneManagerLock` ?



-- 
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] sadanand48 commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.
+ */
+
+package org.apache.hadoop.ozone.om;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import org.apache.hadoop.hdds.utils.db.DBCheckpoint;
+import org.apache.hadoop.hdds.utils.db.RDBStore;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_INDICATOR;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_KEY_NAME;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+
+
+/**
+ * This class is used to manage/create OM snapshots.
+ */
+public final class OmSnapshotManager {
+  private final OzoneManager ozoneManager;
+  private final LoadingCache<String, OmSnapshot> snapshotCache;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OmSnapshotManager.class);
+
+  OmSnapshotManager(OzoneManager ozoneManager) {
+    this.ozoneManager = ozoneManager;
+
+    // size of lru cache
+    int cacheSize = ozoneManager.getConfiguration().getInt(
+        OzoneConfigKeys.OZONE_OM_SNAPSHOT_CACHE_MAX_SIZE,
+        OzoneConfigKeys.OZONE_OM_SNAPSHOT_CACHE_MAX_SIZE_DEFAULT);
+
+    CacheLoader<String, OmSnapshot> loader;
+    loader = new CacheLoader<String, OmSnapshot>() {
+      @Override
+
+      // load the snapshot into the cache if not already there
+      @Nonnull
+      public OmSnapshot load(@Nonnull String snapshotTableKey)
+          throws IOException {
+        SnapshotInfo snapshotInfo;
+        // see if the snapshot exists
+        snapshotInfo = getSnapshotInfo(snapshotTableKey);
+
+        // read in the snapshot
+        OzoneConfiguration conf = ozoneManager.getConfiguration();
+        OMMetadataManager snapshotMetadataManager;
+
+        // Create the snapshot metadata manager by finding the corresponding
+        // RocksDB instance, creating an OmMetadataManagerImpl instance based on
+        // that
+        try {
+          snapshotMetadataManager = OmMetadataManagerImpl
+              .createSnapshotMetadataManager(

Review Comment:
   We should also close the open RocksDB instances during cache eviction.



-- 
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] prashantpogde commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/IOmMetadataReader.java:
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Protocol for OmMetadataReader's.
+ */
+public interface IOmMetadataReader {
+  /**
+   * Look up for the container of an existing key.
+   *
+   * @param args the args of the key.
+   * @return OmKeyInfo instance that client uses to talk to container.
+   */
+  OmKeyInfo lookupKey(OmKeyArgs args) throws IOException;
+
+  /**
+   * List the status for a file or a directory and its contents.
+   *
+   * @param args    Key args
+   * @param recursive  For a directory if true all the descendants of a
+   *                   particular directory are listed
+   * @param startKey   Key from which listing needs to start. If startKey exists
+   *                   its status is included in the final list.
+   * @param numEntries Number of entries to list from the start key
+   * @param allowPartialPrefixes if partial prefixes should be allowed,
+   *                             this is needed in context of ListKeys
+   * @return list of file status
+   */
+  List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,

Review Comment:
   We can do both 1 & 2. Anyone who wants to us ".snapshot" syntax can do that. Optionally, bucket/Key level commands can provide --snapshot parameter too. This could be just a UI side fix. On the server side, we should get the snapshot name as a proto parameter. 
   



-- 
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] prashantpogde commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/IOmMetadataReader.java:
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Protocol for OmMetadataReader's.
+ */
+public interface IOmMetadataReader {
+  /**
+   * Look up for the container of an existing key.
+   *
+   * @param args the args of the key.
+   * @return OmKeyInfo instance that client uses to talk to container.
+   */
+  OmKeyInfo lookupKey(OmKeyArgs args) throws IOException;
+
+  /**
+   * List the status for a file or a directory and its contents.
+   *
+   * @param args    Key args
+   * @param recursive  For a directory if true all the descendants of a
+   *                   particular directory are listed
+   * @param startKey   Key from which listing needs to start. If startKey exists
+   *                   its status is included in the final list.
+   * @param numEntries Number of entries to list from the start key
+   * @param allowPartialPrefixes if partial prefixes should be allowed,
+   *                             this is needed in context of ListKeys
+   * @return list of file status
+   */
+  List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,
+                                   String startKey, long numEntries,
+                                   boolean allowPartialPrefixes)
+      throws IOException;
+
+  default List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,
+      String startKey, long numEntries)
+      throws IOException {
+    return listStatus(args, recursive, startKey, numEntries, false);
+  }
+
+  /**
+   * OzoneFS api to get file status for an entry.
+   *
+   * @param keyArgs Key args
+   * @throws OMException if file does not exist
+   *                     if bucket does not exist
+   * @throws IOException if there is error in the db
+   *                     invalid arguments
+   */
+  OzoneFileStatus getFileStatus(OmKeyArgs keyArgs) throws IOException;
+
+
+  /**
+   * OzoneFS api to lookup for a file.
+   *
+   * @param args Key args
+   * @throws OMException if given key is not found or it is not a file
+   *                     if bucket does not exist
+   * @throws IOException if there is error in the db
+   *                     invalid arguments
+   */
+  OmKeyInfo lookupFile(OmKeyArgs args) throws IOException;
+
+  /**
+   * Returns a list of keys represented by {@link OmKeyInfo}
+   * in the given bucket. Argument volumeName, bucketName is required,
+   * others are optional.
+   *
+   * @param volumeName
+   *   the name of the volume.
+   * @param bucketName
+   *   the name of the bucket.
+   * @param startKey
+   *   the start key name, only the keys whose name is
+   *   after this value will be included in the result.
+   * @param keyPrefix
+   *   key name prefix, only the keys whose name has
+   *   this prefix will be included in the result.
+   * @param maxKeys
+   *   the maximum number of keys to return. It ensures
+   *   the size of the result will not exceed this limit.
+   * @return a list of keys.
+   */
+  List<OmKeyInfo> listKeys(String volumeName, String bucketName,

Review Comment:
   bash-4.2$ ozone sh key list /vol1/bucket1/.snapshot/snap1
   Invalid value for positional parameter at index 0 (<value>): cannot convert '/vol1/bucket1/.snapshot/snap1' to OzoneAddress (org.apache.hadoop.ozone.client.OzoneClientException: Invalid bucket name. Delimiters (/) not allowed in bucket name)



-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotMetrics.java:
##########
@@ -0,0 +1,147 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+
+/**
+ * This class is for maintaining Snapshot Manager statistics.
+ */
+@InterfaceAudience.Private
+@Metrics(about = "Snapshot Manager Metrics", context = "dfs")
+public final class OmSnapshotMetrics implements OmMetadataReaderMetrics {

Review Comment:
   https://issues.apache.org/jira/browse/HDDS-7246



-- 
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] prashantpogde commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -3287,4 +3287,13 @@
       will create intermediate directories.
     </description>
   </property>
+  <property>
+    <name>ozone.om.snapshot.cache.max.size</name>
+    <value>200</value>
+    <tag>OZONE, OM</tag>
+    <description>
+      Size of the OM Snapshot LRU cache.  This is the maximum number of open OM Snapshot RocksDb instances

Review Comment:
   We can use rocksDB option when opening rocksDB instance corresponding to a snapshot. We can track this task under https://issues.apache.org/jira/browse/HDDS-7276. I will resolve this comment 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] prashantpogde commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java:
##########
@@ -413,15 +435,16 @@ public void start(OzoneConfiguration configuration) throws IOException {
 
   public static DBStore loadDB(OzoneConfiguration configuration, File metaDir)
       throws IOException {
-    return loadDB(configuration, metaDir, OM_DB_NAME);
+    return loadDB(configuration, metaDir, OM_DB_NAME, false);
   }
 
   public static DBStore loadDB(OzoneConfiguration configuration, File metaDir,
-      String dbName) throws IOException {
+      String dbName, boolean readOnly) throws IOException {
     RocksDBConfiguration rocksDBConfiguration =
         configuration.getObject(RocksDBConfiguration.class);
     DBStoreBuilder dbStoreBuilder = DBStoreBuilder.newBuilder(configuration,

Review Comment:
   We should limit the memory used for Snapshot rocksDB instances.



-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.
+ */
+
+package org.apache.hadoop.ozone.om;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import org.apache.hadoop.hdds.utils.db.DBCheckpoint;
+import org.apache.hadoop.hdds.utils.db.RDBStore;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_INDICATOR;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_KEY_NAME;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+
+
+/**
+ * This class is used to manage/create OM snapshots.
+ */
+public final class OmSnapshotManager {
+  private final OzoneManager ozoneManager;
+  private final LoadingCache<String, OmSnapshot> snapshotCache;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OmSnapshotManager.class);
+
+  OmSnapshotManager(OzoneManager ozoneManager) {
+    this.ozoneManager = ozoneManager;
+
+    // size of lru cache
+    int cacheSize = ozoneManager.getConfiguration().getInt(
+        OzoneConfigKeys.OZONE_OM_SNAPSHOT_CACHE_MAX_SIZE,
+        OzoneConfigKeys.OZONE_OM_SNAPSHOT_CACHE_MAX_SIZE_DEFAULT);
+
+    CacheLoader<String, OmSnapshot> loader;
+    loader = new CacheLoader<String, OmSnapshot>() {
+      @Override
+
+      // load the snapshot into the cache if not already there
+      @Nonnull
+      public OmSnapshot load(@Nonnull String snapshotTableKey)
+          throws IOException {
+        SnapshotInfo snapshotInfo;
+        // see if the snapshot exists
+        snapshotInfo = getSnapshotInfo(snapshotTableKey);
+
+        // read in the snapshot
+        OzoneConfiguration conf = ozoneManager.getConfiguration();
+        OMMetadataManager snapshotMetadataManager;
+
+        // Create the snapshot metadata manager by finding the corresponding
+        // RocksDB instance, creating an OmMetadataManagerImpl instance based on
+        // that
+        try {
+          snapshotMetadataManager = OmMetadataManagerImpl
+              .createSnapshotMetadataManager(

Review Comment:
   @sadanand48 fixed



-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java:
##########
@@ -812,4 +813,9 @@ public static String getOMAddressListPrintString(List<OMNodeDetails> omList) {
     printString.append("]");
     return printString.toString();
   }
+
+  // Key points to entire bucket's snapshot
+  public static boolean isBucketSnapshotIndicator(String key) {
+    return key.startsWith(OM_SNAPSHOT_INDICATOR) && key.split("/").length == 2;

Review Comment:
   The OzoneBucket.listKeys() method has a keyPrefix parameter.  The parameter can be null/empty, and invoked like so:
   ```
      bucket.listKeys("");
   ```
   If that paramter is null or empty, it is not supposed to be included in the listKeys result.  [This is how the code looked prior to me adding snapshot reads](https://github.com/apache/ozone/blob/master/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java#L1370-L1375)
   
   The isBucketSnapshotIndicator() method tests to see if we have the snapshot equivalent of an empty keyPrefix. The indicator looks like so: ".snapshot/snapshot1", so the empty listKeys() invocation for snapshots would look like:
   ```
      bucket.listKeys(".snapshot/snapshot1");
   ```
   
   
   
   WRT sanitizing, I'll could go ahead and remove any trailing slashes, inside this method, if that is preferable.
   



-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java:
##########
@@ -812,4 +813,9 @@ public static String getOMAddressListPrintString(List<OMNodeDetails> omList) {
     printString.append("]");
     return printString.toString();
   }
+
+  // Key points to entire bucket's snapshot
+  public static boolean isBucketSnapshotIndicator(String key) {
+    return key.startsWith(OM_SNAPSHOT_INDICATOR) && key.split("/").length == 2;

Review Comment:
   The OzoneBucket.listKeys() method has a keyPrefix parameter.  The parameter can be null/empty, and invoked like so:
   ```
      bucket.listKeys("");
   ```
   If that paramter is null or empty, it is not supposed to be included in the listKeys result.  [This is how the code looked prior to me adding snapshot reads](https://github.com/apache/ozone/blob/master/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java#L1370-L1375)
   
   The isBucketSnapshotIndicator() method tests to see if we have the snapshot equivalent of an empty keyPrefix. The indicator looks like so: ".snapshot/snapshot1", so the empty listKeys() invocation for snapshots would look like:
   ```
      bucket.listKeys(".snapshot/snapshot1");
   ```
   
   
   
   WRT "sanitizing", I was not concerned about extra "/"'s at the end of the string, because the code doesn't sanitize the empty string "".  Is that a reasonable assumption?
   
   



-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshotFileSystem.java:
##########
@@ -0,0 +1,577 @@
+/**
+ * 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.commons.io.IOUtils;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.ozone.OzoneFileSystem;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.fs.ozone.Constants.LISTING_PAGE_SIZE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_SCHEME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * OmSnapshot file system tests.
+ */
+@RunWith(Parameterized.class)
+public class TestOmSnapshotFileSystem {
+  private static MiniOzoneCluster cluster = null;
+  private static OzoneConfiguration conf;
+  private static String volumeName;
+  private static String bucketName;
+  private static FileSystem fs;
+  private static OzoneFileSystem o3fs;
+  private static OzoneManagerProtocol writeClient;
+  private static BucketLayout bucketLayout;
+  private static boolean enabledFileSystemPaths;
+  private static File metaDir;
+  private static OzoneManager ozoneManager;
+  private static String keyPrefix;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOmSnapshot.class);
+
+
+
+  @Rule
+  public Timeout timeout = new Timeout(1200000);

Review Comment:
   FWIW I haven't seen either of these timeout on GH CI.



-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/IOmMetadataReader.java:
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Protocol for OmMetadataReader's.
+ */
+public interface IOmMetadataReader {
+  /**
+   * Look up for the container of an existing key.
+   *
+   * @param args the args of the key.
+   * @return OmKeyInfo instance that client uses to talk to container.
+   */
+  OmKeyInfo lookupKey(OmKeyArgs args) throws IOException;
+
+  /**
+   * List the status for a file or a directory and its contents.
+   *
+   * @param args    Key args
+   * @param recursive  For a directory if true all the descendants of a
+   *                   particular directory are listed
+   * @param startKey   Key from which listing needs to start. If startKey exists
+   *                   its status is included in the final list.
+   * @param numEntries Number of entries to list from the start key
+   * @param allowPartialPrefixes if partial prefixes should be allowed,
+   *                             this is needed in context of ListKeys
+   * @return list of file status
+   */
+  List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,

Review Comment:
   When you mentioned this issue earlier today, I didn't realize you meant this command.  I have seen it before. It is not really a snapshot issue but a UI issue.  I was going to add a separate jira ticket for it.
   
   This is actually a bucket level command and requires the parameter to be a valid bucket name, (as you can see from the error message: "Invalid bucket name. Delimiters (/) not allowed in bucket name").  It is easy enough to fix, we just have to decide what the UI should be.
   
   Two possible options are:
   1. Change the client side bucket name validation to allow snapshot names, (which seems a bit ugly to me.)
   2. Allow as separate optional --snapshotName parameter for this UI.
   
   



-- 
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] prashantpogde commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -3287,4 +3287,13 @@
       will create intermediate directories.
     </description>
   </property>
+  <property>
+    <name>ozone.om.snapshot.cache.max.size</name>
+    <value>200</value>
+    <tag>OZONE, OM</tag>
+    <description>
+      Size of the OM Snapshot LRU cache.  This is the maximum number of open OM Snapshot RocksDb instances

Review Comment:
   We should also tune down the default memory for each snapshot rocksDB instance. It may be OK if we give lower priority to reads from Snapshots compared to active Object store. We should run some experiments to check the balance between assigned memory per rocksDB vs read performance.



-- 
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] prashantpogde commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java:
##########
@@ -812,4 +813,9 @@ public static String getOMAddressListPrintString(List<OMNodeDetails> omList) {
     printString.append("]");
     return printString.toString();
   }
+
+  // Key points to entire bucket's snapshot
+  public static boolean isBucketSnapshotIndicator(String key) {
+    return key.startsWith(OM_SNAPSHOT_INDICATOR) && key.split("/").length == 2;

Review Comment:
   +1



-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/IOmMetadataReader.java:
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Protocol for OmMetadataReader's.
+ */
+public interface IOmMetadataReader {
+  /**
+   * Look up for the container of an existing key.
+   *
+   * @param args the args of the key.
+   * @return OmKeyInfo instance that client uses to talk to container.
+   */
+  OmKeyInfo lookupKey(OmKeyArgs args) throws IOException;
+
+  /**
+   * List the status for a file or a directory and its contents.
+   *
+   * @param args    Key args
+   * @param recursive  For a directory if true all the descendants of a
+   *                   particular directory are listed
+   * @param startKey   Key from which listing needs to start. If startKey exists
+   *                   its status is included in the final list.
+   * @param numEntries Number of entries to list from the start key
+   * @param allowPartialPrefixes if partial prefixes should be allowed,
+   *                             this is needed in context of ListKeys
+   * @return list of file status
+   */
+  List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,

Review Comment:
   When you mentioned this issue earlier today, I didn't realize you meant this command.  I have seen it before. It is not really a snapshot issue but a UI issue.  I was going to add a separate jira ticket for it.
   
   This is actually a bucket level command and requires the parameter to be a valid bucket name, (as you can see from the error message: "Invalid bucket name. Delimiters (/) not allowed in bucket name").  It is easy enough to fix, we just have to decide what the UI should be.
   
   Two possible options are:
   1. Change the client side bucket name validation to allow snapshot names, (which seems a bit ugly to me.)
   2. Allow a separate optional --snapshotName parameter for the "ozone sh key list command".
   
   



-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java:
##########
@@ -812,4 +813,9 @@ public static String getOMAddressListPrintString(List<OMNodeDetails> omList) {
     printString.append("]");
     return printString.toString();
   }
+
+  // Key points to entire bucket's snapshot
+  public static boolean isBucketSnapshotIndicator(String key) {
+    return key.startsWith(OM_SNAPSHOT_INDICATOR) && key.split("/").length == 2;

Review Comment:
   The OzoneBucket.listKeys() method has a keyPrefix parameter.  The parameter can be null/empty, and invoked like so:
   ```
      bucket.listKeys("");
   ```
   If that paramter is null or empty, it is not supposed to be included in the listKeys result.  [This is how the code looked prior to me adding snapshot reads](https://github.com/apache/ozone/blob/master/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java#L1370-L1375)
   
   The isBucketSnapshotIndicator() method tests to see if we have the snapshot equivalent of an empty keyPrefix. The indicator looks like so: ".snapshot/snapshot1", so the empty listKeys() invocation for snapshots would look like:
   ```
      bucket.listKeys(".snapshot/snapshot1");
   ```
   
   
   
   WRT "sanitizing", I was not concerned about extra "/"'s at the end of the string, because the code doesn't sanitize the empty string "".  Is that a reasonable assumption?
   



-- 
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] prashantpogde commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -3287,4 +3287,13 @@
       will create intermediate directories.
     </description>
   </property>
+  <property>
+    <name>ozone.om.snapshot.cache.max.size</name>
+    <value>200</value>

Review Comment:
   200 could be a lot of open rocksDB instances. I guess a cache of last 10-20 accessed snapshots should be enough. If anyone is making excessive use of concurrently open/actively read snapshots, they can tune it.



-- 
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] smengcl commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java:
##########
@@ -286,6 +289,25 @@ protected OmMetadataManagerImpl() {
     this.omEpoch = 0;
   }
 
+  //  metadata constructor for snapshots
+  private OmMetadataManagerImpl(OzoneConfiguration conf, String snapshotName)
+      throws IOException {
+    lock = new OmReadOnlyLock();
+    omEpoch = 0;
+    String snapshotDir = OMStorage.getOmDbDir(conf).toString() +
+        OM_KEY_PREFIX + OM_SNAPSHOT_DIR;
+    setStore(loadDB(conf, new File(snapshotDir),
+        OM_DB_NAME + snapshotName, true));

Review Comment:
   Yup, your suggestion sounds good to me. We can add a comment above the definition of `snapshotDirName` saying it consists of a UUID.



-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java:
##########
@@ -286,6 +289,25 @@ protected OmMetadataManagerImpl() {
     this.omEpoch = 0;
   }
 
+  //  metadata constructor for snapshots
+  private OmMetadataManagerImpl(OzoneConfiguration conf, String snapshotName)
+      throws IOException {
+    lock = new OmReadOnlyLock();
+    omEpoch = 0;
+    String snapshotDir = OMStorage.getOmDbDir(conf).toString() +
+        OM_KEY_PREFIX + OM_SNAPSHOT_DIR;
+    setStore(loadDB(conf, new File(snapshotDir),
+        OM_DB_NAME + snapshotName, true));

Review Comment:
   Since the UUID is an implementation detail, I'd prefer to call it snapshotDirName instead of snapshotUUID.  Is that acceptable?



-- 
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] smengcl commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshotFileSystem.java:
##########
@@ -0,0 +1,577 @@
+/**
+ * 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.commons.io.IOUtils;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.ozone.OzoneFileSystem;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.fs.ozone.Constants.LISTING_PAGE_SIZE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_SCHEME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * OmSnapshot file system tests.
+ */
+@RunWith(Parameterized.class)
+public class TestOmSnapshotFileSystem {
+  private static MiniOzoneCluster cluster = null;
+  private static OzoneConfiguration conf;
+  private static String volumeName;
+  private static String bucketName;
+  private static FileSystem fs;
+  private static OzoneFileSystem o3fs;
+  private static OzoneManagerProtocol writeClient;
+  private static BucketLayout bucketLayout;
+  private static boolean enabledFileSystemPaths;
+  private static File metaDir;
+  private static OzoneManager ozoneManager;
+  private static String keyPrefix;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOmSnapshot.class);
+
+
+
+  @Rule
+  public Timeout timeout = new Timeout(1200000);

Review Comment:
   Yes because the current value is too large IMO. 1200000 ms = 1200 s = 20 m . We don't need 20 minutes to run this test.
   
   ```suggestion
   
   
     @Rule
     public Timeout timeout = new Timeout(120, TimeUnit.SECONDS);
   ```



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshotFileSystem.java:
##########
@@ -0,0 +1,577 @@
+/**
+ * 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.commons.io.IOUtils;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.ozone.OzoneFileSystem;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.fs.ozone.Constants.LISTING_PAGE_SIZE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_SCHEME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * OmSnapshot file system tests.
+ */
+@RunWith(Parameterized.class)
+public class TestOmSnapshotFileSystem {
+  private static MiniOzoneCluster cluster = null;
+  private static OzoneConfiguration conf;
+  private static String volumeName;
+  private static String bucketName;
+  private static FileSystem fs;
+  private static OzoneFileSystem o3fs;
+  private static OzoneManagerProtocol writeClient;
+  private static BucketLayout bucketLayout;
+  private static boolean enabledFileSystemPaths;
+  private static File metaDir;
+  private static OzoneManager ozoneManager;
+  private static String keyPrefix;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOmSnapshot.class);
+
+
+
+  @Rule
+  public Timeout timeout = new Timeout(1200000);

Review Comment:
   Yes because the current timeout is too large IMO. 1200000 ms = 1200 s = 20 m . We don't need 20 minutes to run this test.
   
   ```suggestion
   
   
     @Rule
     public Timeout timeout = new Timeout(120, TimeUnit.SECONDS);
   ```



-- 
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] prashantpogde commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -3287,4 +3287,13 @@
       will create intermediate directories.
     </description>
   </property>
+  <property>
+    <name>ozone.om.snapshot.cache.max.size</name>
+    <value>200</value>

Review Comment:
   thank you for tuning it down to 10



-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/IOmMetadataReader.java:
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Protocol for OmMetadataReader's.
+ */
+public interface IOmMetadataReader {
+  /**
+   * Look up for the container of an existing key.
+   *
+   * @param args the args of the key.
+   * @return OmKeyInfo instance that client uses to talk to container.
+   */
+  OmKeyInfo lookupKey(OmKeyArgs args) throws IOException;
+
+  /**
+   * List the status for a file or a directory and its contents.
+   *
+   * @param args    Key args
+   * @param recursive  For a directory if true all the descendants of a
+   *                   particular directory are listed
+   * @param startKey   Key from which listing needs to start. If startKey exists
+   *                   its status is included in the final list.
+   * @param numEntries Number of entries to list from the start key
+   * @param allowPartialPrefixes if partial prefixes should be allowed,
+   *                             this is needed in context of ListKeys
+   * @return list of file status
+   */
+  List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,

Review Comment:
   When you mentioned this issue earlier today, I didn't realize you meant this command.  I have seen it before. It is not really a snapshot issue but a UI issue.  I was going to add a separate jira ticket for it.
   
   This is actually a bucket level command and requires the parameter to be a valid bucket name, (as you can see from the error message: "Invalid bucket name. Delimiters (/) not allowed in bucket name").  It is easy enough to fix, we just have to decide what the UI should be.
   
   Two possible options are:
   1. Change the client side bucket name validation to allow snapshot names, (which seems a bit ugly to me.)
   2. Allow a separate optional --snapshotName parameter for the "ozone sh key list" command.
   
   



-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -3287,4 +3287,13 @@
       will create intermediate directories.
     </description>
   </property>
+  <property>
+    <name>ozone.om.snapshot.cache.max.size</name>
+    <value>200</value>
+    <tag>OZONE, OM</tag>
+    <description>
+      Size of the OM Snapshot LRU cache.  This is the maximum number of open OM Snapshot RocksDb instances

Review Comment:
   Not quite following.  How would we "tune the default memory"?



-- 
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] prashantpogde commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/IOmMetadataReader.java:
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Protocol for OmMetadataReader's.
+ */
+public interface IOmMetadataReader {
+  /**
+   * Look up for the container of an existing key.
+   *
+   * @param args the args of the key.
+   * @return OmKeyInfo instance that client uses to talk to container.
+   */
+  OmKeyInfo lookupKey(OmKeyArgs args) throws IOException;
+
+  /**
+   * List the status for a file or a directory and its contents.
+   *
+   * @param args    Key args
+   * @param recursive  For a directory if true all the descendants of a
+   *                   particular directory are listed
+   * @param startKey   Key from which listing needs to start. If startKey exists
+   *                   its status is included in the final list.
+   * @param numEntries Number of entries to list from the start key
+   * @param allowPartialPrefixes if partial prefixes should be allowed,
+   *                             this is needed in context of ListKeys
+   * @return list of file status
+   */
+  List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,
+                                   String startKey, long numEntries,
+                                   boolean allowPartialPrefixes)
+      throws IOException;
+
+  default List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,
+      String startKey, long numEntries)
+      throws IOException {
+    return listStatus(args, recursive, startKey, numEntries, false);
+  }
+
+  /**
+   * OzoneFS api to get file status for an entry.
+   *
+   * @param keyArgs Key args
+   * @throws OMException if file does not exist
+   *                     if bucket does not exist
+   * @throws IOException if there is error in the db
+   *                     invalid arguments
+   */
+  OzoneFileStatus getFileStatus(OmKeyArgs keyArgs) throws IOException;
+
+
+  /**
+   * OzoneFS api to lookup for a file.
+   *
+   * @param args Key args
+   * @throws OMException if given key is not found or it is not a file
+   *                     if bucket does not exist
+   * @throws IOException if there is error in the db
+   *                     invalid arguments
+   */
+  OmKeyInfo lookupFile(OmKeyArgs args) throws IOException;
+
+  /**
+   * Returns a list of keys represented by {@link OmKeyInfo}
+   * in the given bucket. Argument volumeName, bucketName is required,
+   * others are optional.
+   *
+   * @param volumeName
+   *   the name of the volume.
+   * @param bucketName
+   *   the name of the bucket.
+   * @param startKey
+   *   the start key name, only the keys whose name is
+   *   after this value will be included in the result.
+   * @param keyPrefix
+   *   key name prefix, only the keys whose name has
+   *   this prefix will be included in the result.
+   * @param maxKeys
+   *   the maximum number of keys to return. It ensures
+   *   the size of the result will not exceed this limit.
+   * @return a list of keys.
+   */
+  List<OmKeyInfo> listKeys(String volumeName, String bucketName,

Review Comment:
   bash-4.2$ ozone sh key list /vol1/bucket1/.snapshot/snap1
   Invalid value for positional parameter at index 0 (<value>): cannot convert '/vol1/bucket1/.snapshot/snap1' to OzoneAddress (org.apache.hadoop.ozone.client.OzoneClientException: Invalid bucket name. Delimiters (/) not allowed in bucket name)
   
   Perhaps some name parsing issue.



-- 
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] prashantpogde commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/IOmMetadataReader.java:
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Protocol for OmMetadataReader's.
+ */
+public interface IOmMetadataReader {
+  /**
+   * Look up for the container of an existing key.
+   *
+   * @param args the args of the key.
+   * @return OmKeyInfo instance that client uses to talk to container.
+   */
+  OmKeyInfo lookupKey(OmKeyArgs args) throws IOException;
+
+  /**
+   * List the status for a file or a directory and its contents.
+   *
+   * @param args    Key args
+   * @param recursive  For a directory if true all the descendants of a
+   *                   particular directory are listed
+   * @param startKey   Key from which listing needs to start. If startKey exists
+   *                   its status is included in the final list.
+   * @param numEntries Number of entries to list from the start key
+   * @param allowPartialPrefixes if partial prefixes should be allowed,
+   *                             this is needed in context of ListKeys
+   * @return list of file status
+   */
+  List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,

Review Comment:
   it works for the FS interface, but doesnt seem to work for object store interface. This is what I see on my setup
   
   bash-4.2$ ozone fs -ls /vol1/bucket1/.snapshot/snap1
   Found 1 items
   -rw-rw-rw-   1 hadoop hadoop       3811 2022-09-07 21:05 /vol1/bucket1/.snapshot/snap1/k1
   
   bash-4.2$ ozone sh key list /vol1/bucket1/.snapshot/snap1
   Invalid value for positional parameter at index 0 (<value>): cannot convert '/vol1/bucket1/.snapshot/snap1' to OzoneAddress (org.apache.hadoop.ozone.client.OzoneClientException: Invalid bucket name. Delimiters (/) not allowed in bucket name)



-- 
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] prashantpogde commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -3287,4 +3287,13 @@
       will create intermediate directories.
     </description>
   </property>
+  <property>
+    <name>ozone.om.snapshot.cache.max.size</name>
+    <value>200</value>

Review Comment:
   200 could be a lot of open rocksDB instances. I guess a cache of about 10 or  so snapshots should be enough. If anyone is making excessive use of concurrently open/actively read snapshots, they can tune it.



-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java:
##########
@@ -812,4 +813,9 @@ public static String getOMAddressListPrintString(List<OMNodeDetails> omList) {
     printString.append("]");
     return printString.toString();
   }
+
+  // Key points to entire bucket's snapshot
+  public static boolean isBucketSnapshotIndicator(String key) {
+    return key.startsWith(OM_SNAPSHOT_INDICATOR) && key.split("/").length == 2;

Review Comment:
   The OzoneBucket.listKeys() method has a keyPrefix parameter.  The parameter can be null/empty, and invoked like so:
   ```
      bucket.listKeys("");
   ```
   If that paramter is null or empty, it is not supposed to be included in the listKeys result.  [This is how the code looked prior to me adding snapshot reads](https://github.com/apache/ozone/blob/master/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java#L1370-L1375)
   
   The isBucketSnapshotIndicator() method tests to see if we have the snapshot equivalent of an empty keyPrefix. The indicator looks like so: ".snapshot/snapshot1", so the empty listKeys() invocation for snapshots would look like:
   ```
      bucket.listKeys(".snapshot/snapshot1");
   ```
   
   The isBucketSnapshotIndicator() method is only used by ozoneBucket.listKeys() [here](https://github.com/apache/ozone/blob/acc7e5e0e870e2f6a6aa1b1570cb95c3a73ddf3d/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java#L1376).
   
   So maybe it would be clearer/better, if I just made it a private method in that class, and called it "isSnapshotEmptyPrefix()".  Would that be clearer?
   
   WRT "sanitizing", I was not concerned about extra "/"'s at the end of the string, because the code doesn't sanitize the empty string "".  Is that a reasonable assumption?
   



-- 
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] prashantpogde commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java:
##########
@@ -413,15 +435,16 @@ public void start(OzoneConfiguration configuration) throws IOException {
 
   public static DBStore loadDB(OzoneConfiguration configuration, File metaDir)
       throws IOException {
-    return loadDB(configuration, metaDir, OM_DB_NAME);
+    return loadDB(configuration, metaDir, OM_DB_NAME, false);
   }
 
   public static DBStore loadDB(OzoneConfiguration configuration, File metaDir,
-      String dbName) throws IOException {
+      String dbName, boolean readOnly) throws IOException {
     RocksDBConfiguration rocksDBConfiguration =
         configuration.getObject(RocksDBConfiguration.class);
     DBStoreBuilder dbStoreBuilder = DBStoreBuilder.newBuilder(configuration,

Review Comment:
   Created https://issues.apache.org/jira/browse/HDDS-7276 to track this.



-- 
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] prashantpogde commented on a diff in pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java:
##########
@@ -812,4 +813,9 @@ public static String getOMAddressListPrintString(List<OMNodeDetails> omList) {
     printString.append("]");
     return printString.toString();
   }
+
+  // Key points to entire bucket's snapshot
+  public static boolean isBucketSnapshotIndicator(String key) {
+    return key.startsWith(OM_SNAPSHOT_INDICATOR) && key.split("/").length == 2;

Review Comment:
   Resolving this comment here. If we find bugs around this, we can issue it in a separate PR.



-- 
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] prashantpogde commented on pull request #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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

   @GeorgeJahad Thank you for working on this. Any additional problems found on the snapshot read path can be addressed under new PRs.


-- 
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 #3729: HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots.

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.
+ */
+
+package org.apache.hadoop.ozone.om;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import org.apache.hadoop.hdds.utils.db.DBCheckpoint;
+import org.apache.hadoop.hdds.utils.db.RDBStore;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_INDICATOR;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_KEY_NAME;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+
+
+/**
+ * This class is used to manage/create OM snapshots.
+ */
+public final class OmSnapshotManager {
+  private final OzoneManager ozoneManager;
+  private final LoadingCache<String, OmSnapshot> snapshotCache;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OmSnapshotManager.class);
+
+  OmSnapshotManager(OzoneManager ozoneManager) {
+    this.ozoneManager = ozoneManager;
+
+    // size of lru cache
+    int cacheSize = ozoneManager.getConfiguration().getInt(
+        OzoneConfigKeys.OZONE_OM_SNAPSHOT_CACHE_MAX_SIZE,
+        OzoneConfigKeys.OZONE_OM_SNAPSHOT_CACHE_MAX_SIZE_DEFAULT);
+
+    CacheLoader<String, OmSnapshot> loader;
+    loader = new CacheLoader<String, OmSnapshot>() {
+      @Override
+
+      // load the snapshot into the cache if not already there
+      @Nonnull
+      public OmSnapshot load(@Nonnull String snapshotTableKey)
+          throws IOException {
+        SnapshotInfo snapshotInfo;
+        // see if the snapshot exists
+        snapshotInfo = getSnapshotInfo(snapshotTableKey);
+
+        // read in the snapshot
+        OzoneConfiguration conf = ozoneManager.getConfiguration();
+        OMMetadataManager snapshotMetadataManager;
+
+        // Create the snapshot metadata manager by finding the corresponding
+        // RocksDB instance, creating an OmMetadataManagerImpl instance based on
+        // that
+        try {
+          snapshotMetadataManager = OmMetadataManagerImpl
+              .createSnapshotMetadataManager(

Review Comment:
   Good Catch!  I'll fix, thanks @sadanand48 !



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