You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by "hemantk-12 (via GitHub)" <gi...@apache.org> on 2023/02/02 19:57:31 UTC

[GitHub] [ozone] hemantk-12 commented on a diff in pull request #4134: HDDS-7513. [Snapshot] Support list snapshots using fs -ls

hemantk-12 commented on code in PR #4134:
URL: https://github.com/apache/ozone/pull/4134#discussion_r1094958692


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsSnapshot.java:
##########
@@ -65,259 +77,229 @@ public static void initClass() throws Exception {
     ozoneManager = cluster.getOzoneManager();
   }
 
-  @Before
-  public void init() {
+  @BeforeEach
+  public void init() throws Exception {
     String hostPrefix = OZONE_OFS_URI_SCHEME + "://" + OM_SERVICE_ID;
-    clientConf = new OzoneConfiguration(cluster.getConf());
+    OzoneConfiguration clientConf =
+        new OzoneConfiguration(cluster.getConf());
     clientConf.set(FS_DEFAULT_NAME_KEY, hostPrefix);
+
+    shell = new OzoneFsShell(clientConf);
+
+    this.volume = "vol-" + RandomStringUtils.randomNumeric(5);
+    this.bucket = "buck-" + RandomStringUtils.randomNumeric(5);
+    this.key = "key-" + RandomStringUtils.randomNumeric(5);
+
+    this.bucketPath = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
+    this.bucketWithSnapshotIndicatorPath = bucketPath +
+        OM_KEY_PREFIX + OM_SNAPSHOT_INDICATOR;
+
+    String keyPath = bucketPath + OM_KEY_PREFIX + key;
+    createVolBuckKey(bucketPath, keyPath);
+  }
+
+  @AfterEach
+  public void cleanup() throws IOException {
+    shell.close();
   }
 
-  @AfterClass
+  @AfterAll
   public static void shutdown() {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
 
   private void createVolBuckKey(String testVolBucket, String testKey)
-          throws Exception {
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      // Create volume and bucket
-      int res = ToolRunner.run(shell,
-              new String[]{"-mkdir", "-p", testVolBucket});
-      assertEquals(0, res);
-      // Create key
-      res = ToolRunner.run(shell, new String[]{"-touch", testKey});
-      assertEquals(0, res);
-      // List the bucket to make sure that bucket exists.
-      res = ToolRunner.run(shell, new String[]{"-ls", testVolBucket});
-      assertEquals(0, res);
-    } finally {
-      shell.close();
-    }
-  }
-
-  @Test
-  public void testCreateSnapshot() throws Exception {
-    String volume = "vol1";
-    String bucket = "bucket1";
-    String testVolBucket = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
-    String snapshotName = "snap1";
-    String testKey = testVolBucket + "/key1";
+      throws Exception {
+
+    // Create volume and bucket
+    int res = ToolRunner.run(shell,
+        new String[]{"-mkdir", "-p", testVolBucket});
+    Assertions.assertEquals(0, res);
+    // Create key
+    res = ToolRunner.run(shell, new String[]{"-touch", testKey});
+    Assertions.assertEquals(0, res);
+    // List the bucket to make sure that bucket exists.
+    res = ToolRunner.run(shell, new String[]{"-ls", testVolBucket});
+    Assertions.assertEquals(0, res);
 
-    createVolBuckKey(testVolBucket, testKey);
-
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      int res = ToolRunner.run(shell,
-          new String[]{"-createSnapshot", testVolBucket, snapshotName});
-      // Asserts that create request succeeded
-      assertEquals(0, res);
-
-      SnapshotInfo snapshotInfo = ozoneManager
-          .getMetadataManager()
-          .getSnapshotInfoTable()
-          .get(SnapshotInfo.getTableKey(volume, bucket, snapshotName));
-
-      // Assert that snapshot exists in RocksDB.
-      // We can't use list or valid if snapshot directory exists because DB
-      // transaction might not be flushed by the time.
-      Assert.assertNotNull(snapshotInfo);
-    } finally {
-      shell.close();
-    }
   }
 
   @Test
   public void testCreateSnapshotDuplicateName() throws Exception {
-    String volume = "vol-" + RandomStringUtils.randomNumeric(5);
-    String bucket = "buc-" + RandomStringUtils.randomNumeric(5);
-    String key = "key-" + RandomStringUtils.randomNumeric(5);
     String snapshotName = "snap-" + RandomStringUtils.randomNumeric(5);
 
-    String testVolBucket = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
-    String testKey = testVolBucket + OM_KEY_PREFIX + key;
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", bucketPath, snapshotName});
+    // Asserts that create request succeeded
+    Assertions.assertEquals(0, res);
 
-    createVolBuckKey(testVolBucket, testKey);
+    res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", bucketPath, snapshotName});
+    // Asserts that create request fails since snapshot name provided twice
+    Assertions.assertEquals(1, res);
+  }
 
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      int res = ToolRunner.run(shell,
-              new String[]{"-createSnapshot", testVolBucket, snapshotName});
-      // Asserts that create request succeeded
-      assertEquals(0, res);
-
-      res = ToolRunner.run(shell,
-              new String[]{"-createSnapshot", testVolBucket, snapshotName});
-      // Asserts that create request fails since snapshot name provided twice
-      assertEquals(1, res);
-    } finally {
-      shell.close();
-    }
+  /**
+   * Create snapshot should succeed.
+   * 1st case: valid snapshot name
+   * 2nd case: snapshot name length is less than 64 chars
+   */
+  @ParameterizedTest
+  @ValueSource(strings = {"snap-1",
+      "snap75795657617173401188448010125899089001363595171500499231286"})
+  public void testCreateSnapshotSuccess(String snapshotName)
+      throws Exception {
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", bucketPath, snapshotName});
+    // Asserts that create request succeeded
+    Assertions.assertEquals(0, res);
+
+    SnapshotInfo snapshotInfo = ozoneManager
+        .getMetadataManager()
+        .getSnapshotInfoTable()
+        .get(SnapshotInfo.getTableKey(volume, bucket, snapshotName));
+
+    // Assert that snapshot exists in RocksDB.
+    // We can't use list or valid if snapshot directory exists because DB
+    // transaction might not be flushed by the time.
+    Assertions.assertNotNull(snapshotInfo);
   }
 
-  @Test
-  public void testCreateSnapshotInvalidName() throws Exception {
-    String volume = "vol-" + RandomStringUtils.randomNumeric(5);
-    String bucket = "buc-" + RandomStringUtils.randomNumeric(5);
-    String key = "key-" + RandomStringUtils.randomNumeric(5);
-    String snapshotName = "snapa?b";
+  /**

Review Comment:
   How case 3 is getting test? You are not passing or setting `bucketPath` it to invalid bucket? I'll suggest something like following:
   
   ```suggestion
     private static Stream<Arguments> createSnapshotFailureScenarios() {
       return Stream.of(
           Arguments.of("1st case: snapshot name contains invalid char",
               validBucketPath,
               "snapa?b"),
           Arguments.of("2nd case: snapshot name consists only of numbers",
               validBucketPath,
               "1234"),
           Arguments.of("3rd case: bucket path is invalid",
               invalidBucketPath,
               "validSnapshotName12"),
           Arguments.of("4th case: snapshot name length is more than 64 chars",
               validBucketPath,
               "snap156808943643007724443266605711479126926050896107709081166294")
       );
     }
   
     @ParameterizedTest(name = "{0}")
     @MethodSource("createSnapshotFailureScenarios")
     public void testCreateSnapshotFailure(String description,
                                           String bucketPath,
                                           String snapshotName)
         throws Exception {
   
       int res = ToolRunner.run(shell,
           new String[]{"-createSnapshot", bucketPath, snapshotName});
       Assertions.assertEquals(1, res);
     }
   ```
   
   Also is there a way to validate the response from OFS. Because we are just validating that request fails but not verify the reason. We should verify that to make sure that reason is correct.



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsSnapshot.java:
##########
@@ -65,259 +77,229 @@ public static void initClass() throws Exception {
     ozoneManager = cluster.getOzoneManager();
   }
 
-  @Before
-  public void init() {
+  @BeforeEach
+  public void init() throws Exception {
     String hostPrefix = OZONE_OFS_URI_SCHEME + "://" + OM_SERVICE_ID;
-    clientConf = new OzoneConfiguration(cluster.getConf());
+    OzoneConfiguration clientConf =
+        new OzoneConfiguration(cluster.getConf());
     clientConf.set(FS_DEFAULT_NAME_KEY, hostPrefix);
+
+    shell = new OzoneFsShell(clientConf);
+
+    this.volume = "vol-" + RandomStringUtils.randomNumeric(5);
+    this.bucket = "buck-" + RandomStringUtils.randomNumeric(5);
+    this.key = "key-" + RandomStringUtils.randomNumeric(5);
+
+    this.bucketPath = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
+    this.bucketWithSnapshotIndicatorPath = bucketPath +
+        OM_KEY_PREFIX + OM_SNAPSHOT_INDICATOR;
+
+    String keyPath = bucketPath + OM_KEY_PREFIX + key;
+    createVolBuckKey(bucketPath, keyPath);
+  }
+
+  @AfterEach
+  public void cleanup() throws IOException {
+    shell.close();
   }
 
-  @AfterClass
+  @AfterAll
   public static void shutdown() {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
 
   private void createVolBuckKey(String testVolBucket, String testKey)
-          throws Exception {
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      // Create volume and bucket
-      int res = ToolRunner.run(shell,
-              new String[]{"-mkdir", "-p", testVolBucket});
-      assertEquals(0, res);
-      // Create key
-      res = ToolRunner.run(shell, new String[]{"-touch", testKey});
-      assertEquals(0, res);
-      // List the bucket to make sure that bucket exists.
-      res = ToolRunner.run(shell, new String[]{"-ls", testVolBucket});
-      assertEquals(0, res);
-    } finally {
-      shell.close();
-    }
-  }
-
-  @Test
-  public void testCreateSnapshot() throws Exception {
-    String volume = "vol1";
-    String bucket = "bucket1";
-    String testVolBucket = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
-    String snapshotName = "snap1";
-    String testKey = testVolBucket + "/key1";
+      throws Exception {
+
+    // Create volume and bucket
+    int res = ToolRunner.run(shell,
+        new String[]{"-mkdir", "-p", testVolBucket});
+    Assertions.assertEquals(0, res);
+    // Create key
+    res = ToolRunner.run(shell, new String[]{"-touch", testKey});
+    Assertions.assertEquals(0, res);
+    // List the bucket to make sure that bucket exists.
+    res = ToolRunner.run(shell, new String[]{"-ls", testVolBucket});
+    Assertions.assertEquals(0, res);
 
-    createVolBuckKey(testVolBucket, testKey);
-
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      int res = ToolRunner.run(shell,
-          new String[]{"-createSnapshot", testVolBucket, snapshotName});
-      // Asserts that create request succeeded
-      assertEquals(0, res);
-
-      SnapshotInfo snapshotInfo = ozoneManager
-          .getMetadataManager()
-          .getSnapshotInfoTable()
-          .get(SnapshotInfo.getTableKey(volume, bucket, snapshotName));
-
-      // Assert that snapshot exists in RocksDB.
-      // We can't use list or valid if snapshot directory exists because DB
-      // transaction might not be flushed by the time.
-      Assert.assertNotNull(snapshotInfo);
-    } finally {
-      shell.close();
-    }
   }
 
   @Test
   public void testCreateSnapshotDuplicateName() throws Exception {
-    String volume = "vol-" + RandomStringUtils.randomNumeric(5);
-    String bucket = "buc-" + RandomStringUtils.randomNumeric(5);
-    String key = "key-" + RandomStringUtils.randomNumeric(5);
     String snapshotName = "snap-" + RandomStringUtils.randomNumeric(5);
 
-    String testVolBucket = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
-    String testKey = testVolBucket + OM_KEY_PREFIX + key;
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", bucketPath, snapshotName});
+    // Asserts that create request succeeded
+    Assertions.assertEquals(0, res);
 
-    createVolBuckKey(testVolBucket, testKey);
+    res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", bucketPath, snapshotName});
+    // Asserts that create request fails since snapshot name provided twice
+    Assertions.assertEquals(1, res);
+  }
 
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      int res = ToolRunner.run(shell,
-              new String[]{"-createSnapshot", testVolBucket, snapshotName});
-      // Asserts that create request succeeded
-      assertEquals(0, res);
-
-      res = ToolRunner.run(shell,
-              new String[]{"-createSnapshot", testVolBucket, snapshotName});
-      // Asserts that create request fails since snapshot name provided twice
-      assertEquals(1, res);
-    } finally {
-      shell.close();
-    }
+  /**
+   * Create snapshot should succeed.
+   * 1st case: valid snapshot name
+   * 2nd case: snapshot name length is less than 64 chars
+   */
+  @ParameterizedTest
+  @ValueSource(strings = {"snap-1",
+      "snap75795657617173401188448010125899089001363595171500499231286"})
+  public void testCreateSnapshotSuccess(String snapshotName)
+      throws Exception {
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", bucketPath, snapshotName});
+    // Asserts that create request succeeded
+    Assertions.assertEquals(0, res);
+
+    SnapshotInfo snapshotInfo = ozoneManager
+        .getMetadataManager()
+        .getSnapshotInfoTable()
+        .get(SnapshotInfo.getTableKey(volume, bucket, snapshotName));
+
+    // Assert that snapshot exists in RocksDB.
+    // We can't use list or valid if snapshot directory exists because DB
+    // transaction might not be flushed by the time.
+    Assertions.assertNotNull(snapshotInfo);
   }
 
-  @Test
-  public void testCreateSnapshotInvalidName() throws Exception {
-    String volume = "vol-" + RandomStringUtils.randomNumeric(5);
-    String bucket = "buc-" + RandomStringUtils.randomNumeric(5);
-    String key = "key-" + RandomStringUtils.randomNumeric(5);
-    String snapshotName = "snapa?b";
+  /**
+   * Create snapshot should fail.
+   * 1st case: snapshot name contains invalid char
+   * 2nd case: snapshot name consists only of numbers
+   * 3rd case: bucket path is invalid
+   * 4th case: snapshot name length is more than 64 chars
+   */
+  @ParameterizedTest
+  @ValueSource(strings = {"snapa?b", "1234",
+      "snap156808943643007724443266605711479126926050896107709081166294"})
+  public void testCreateSnapshotFailure(String snapshotName)
+      throws Exception {
+
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", bucketPath, snapshotName});
+    // Asserts that create request failed
+    Assertions.assertEquals(1, res);
+  }
 
-    String testVolBucket = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
-    String testKey = testVolBucket + OM_KEY_PREFIX + key;
+  @Test
+  public void testCreateSnapshotParameterMissing() throws Exception {

Review Comment:
   What if we pass empty string for URI? Would it be wrong bucket path or bucket path is missing.
   
   Just thinking if all the failure scenarios can be group to single parameterized test. 



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsSnapshot.java:
##########
@@ -65,259 +77,229 @@ public static void initClass() throws Exception {
     ozoneManager = cluster.getOzoneManager();
   }
 
-  @Before
-  public void init() {
+  @BeforeEach
+  public void init() throws Exception {
     String hostPrefix = OZONE_OFS_URI_SCHEME + "://" + OM_SERVICE_ID;
-    clientConf = new OzoneConfiguration(cluster.getConf());
+    OzoneConfiguration clientConf =
+        new OzoneConfiguration(cluster.getConf());
     clientConf.set(FS_DEFAULT_NAME_KEY, hostPrefix);
+
+    shell = new OzoneFsShell(clientConf);
+
+    this.volume = "vol-" + RandomStringUtils.randomNumeric(5);
+    this.bucket = "buck-" + RandomStringUtils.randomNumeric(5);
+    this.key = "key-" + RandomStringUtils.randomNumeric(5);
+
+    this.bucketPath = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
+    this.bucketWithSnapshotIndicatorPath = bucketPath +
+        OM_KEY_PREFIX + OM_SNAPSHOT_INDICATOR;
+
+    String keyPath = bucketPath + OM_KEY_PREFIX + key;
+    createVolBuckKey(bucketPath, keyPath);
+  }
+
+  @AfterEach
+  public void cleanup() throws IOException {
+    shell.close();
   }
 
-  @AfterClass
+  @AfterAll
   public static void shutdown() {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
 
   private void createVolBuckKey(String testVolBucket, String testKey)
-          throws Exception {
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      // Create volume and bucket
-      int res = ToolRunner.run(shell,
-              new String[]{"-mkdir", "-p", testVolBucket});
-      assertEquals(0, res);
-      // Create key
-      res = ToolRunner.run(shell, new String[]{"-touch", testKey});
-      assertEquals(0, res);
-      // List the bucket to make sure that bucket exists.
-      res = ToolRunner.run(shell, new String[]{"-ls", testVolBucket});
-      assertEquals(0, res);
-    } finally {
-      shell.close();
-    }
-  }
-
-  @Test
-  public void testCreateSnapshot() throws Exception {
-    String volume = "vol1";
-    String bucket = "bucket1";
-    String testVolBucket = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
-    String snapshotName = "snap1";
-    String testKey = testVolBucket + "/key1";
+      throws Exception {
+
+    // Create volume and bucket
+    int res = ToolRunner.run(shell,
+        new String[]{"-mkdir", "-p", testVolBucket});
+    Assertions.assertEquals(0, res);
+    // Create key
+    res = ToolRunner.run(shell, new String[]{"-touch", testKey});
+    Assertions.assertEquals(0, res);
+    // List the bucket to make sure that bucket exists.
+    res = ToolRunner.run(shell, new String[]{"-ls", testVolBucket});
+    Assertions.assertEquals(0, res);
 
-    createVolBuckKey(testVolBucket, testKey);
-
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      int res = ToolRunner.run(shell,
-          new String[]{"-createSnapshot", testVolBucket, snapshotName});
-      // Asserts that create request succeeded
-      assertEquals(0, res);
-
-      SnapshotInfo snapshotInfo = ozoneManager
-          .getMetadataManager()
-          .getSnapshotInfoTable()
-          .get(SnapshotInfo.getTableKey(volume, bucket, snapshotName));
-
-      // Assert that snapshot exists in RocksDB.
-      // We can't use list or valid if snapshot directory exists because DB
-      // transaction might not be flushed by the time.
-      Assert.assertNotNull(snapshotInfo);
-    } finally {
-      shell.close();
-    }
   }
 
   @Test
   public void testCreateSnapshotDuplicateName() throws Exception {
-    String volume = "vol-" + RandomStringUtils.randomNumeric(5);
-    String bucket = "buc-" + RandomStringUtils.randomNumeric(5);
-    String key = "key-" + RandomStringUtils.randomNumeric(5);
     String snapshotName = "snap-" + RandomStringUtils.randomNumeric(5);
 
-    String testVolBucket = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
-    String testKey = testVolBucket + OM_KEY_PREFIX + key;
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", bucketPath, snapshotName});
+    // Asserts that create request succeeded
+    Assertions.assertEquals(0, res);
 
-    createVolBuckKey(testVolBucket, testKey);
+    res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", bucketPath, snapshotName});
+    // Asserts that create request fails since snapshot name provided twice
+    Assertions.assertEquals(1, res);
+  }
 
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      int res = ToolRunner.run(shell,
-              new String[]{"-createSnapshot", testVolBucket, snapshotName});
-      // Asserts that create request succeeded
-      assertEquals(0, res);
-
-      res = ToolRunner.run(shell,
-              new String[]{"-createSnapshot", testVolBucket, snapshotName});
-      // Asserts that create request fails since snapshot name provided twice
-      assertEquals(1, res);
-    } finally {
-      shell.close();
-    }
+  /**
+   * Create snapshot should succeed.
+   * 1st case: valid snapshot name
+   * 2nd case: snapshot name length is less than 64 chars
+   */
+  @ParameterizedTest
+  @ValueSource(strings = {"snap-1",
+      "snap75795657617173401188448010125899089001363595171500499231286"})
+  public void testCreateSnapshotSuccess(String snapshotName)
+      throws Exception {
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", bucketPath, snapshotName});
+    // Asserts that create request succeeded
+    Assertions.assertEquals(0, res);
+
+    SnapshotInfo snapshotInfo = ozoneManager
+        .getMetadataManager()
+        .getSnapshotInfoTable()
+        .get(SnapshotInfo.getTableKey(volume, bucket, snapshotName));
+
+    // Assert that snapshot exists in RocksDB.
+    // We can't use list or valid if snapshot directory exists because DB
+    // transaction might not be flushed by the time.
+    Assertions.assertNotNull(snapshotInfo);
   }
 
-  @Test
-  public void testCreateSnapshotInvalidName() throws Exception {
-    String volume = "vol-" + RandomStringUtils.randomNumeric(5);
-    String bucket = "buc-" + RandomStringUtils.randomNumeric(5);
-    String key = "key-" + RandomStringUtils.randomNumeric(5);
-    String snapshotName = "snapa?b";
+  /**
+   * Create snapshot should fail.
+   * 1st case: snapshot name contains invalid char
+   * 2nd case: snapshot name consists only of numbers
+   * 3rd case: bucket path is invalid
+   * 4th case: snapshot name length is more than 64 chars
+   */
+  @ParameterizedTest
+  @ValueSource(strings = {"snapa?b", "1234",
+      "snap156808943643007724443266605711479126926050896107709081166294"})
+  public void testCreateSnapshotFailure(String snapshotName)
+      throws Exception {
+
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", bucketPath, snapshotName});
+    // Asserts that create request failed
+    Assertions.assertEquals(1, res);
+  }
 
-    String testVolBucket = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
-    String testKey = testVolBucket + OM_KEY_PREFIX + key;
+  @Test
+  public void testCreateSnapshotParameterMissing() throws Exception {
 
-    createVolBuckKey(testVolBucket, testKey);
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot"});
+    // Asserts that create request failed since mandatory params not passed
+    Assertions.assertEquals(-1, res);
+  }
 
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      int res = ToolRunner.run(shell,
-              new String[]{"-createSnapshot", testVolBucket, snapshotName});
-      // Asserts that create request failed since invalid name passed
-      assertEquals(1, res);
+  @Test
+  public void testCreateSnapshotInvalidURI() throws Exception {

Review Comment:
   This won't be needed if you create tests as suggested in previews comment.



##########
hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java:
##########
@@ -635,19 +637,36 @@ public FileStatusAdapter getFileStatus(String path, URI uri,
       Path qualifiedPath, String userName) throws IOException {
     incrementCounter(Statistic.OBJECTS_QUERY, 1);
     OFSPath ofsPath = new OFSPath(path, config);
-    String key = ofsPath.getKeyName();
     if (ofsPath.isRoot()) {
       return getFileStatusAdapterForRoot(uri);
-    }
-    if (ofsPath.isVolume()) {
+    } else if (ofsPath.isVolume()) {
       OzoneVolume volume = objectStore.getVolume(ofsPath.getVolumeName());
       return getFileStatusAdapterForVolume(volume, uri);
+    } else {
+      return getFileStatusForKeyOrSnapshot(
+          ofsPath, uri, qualifiedPath, userName);
     }
+  }
+
+  /**
+   * Check OFSPath to determine whether we are on a bucket path
+   * or a snapshot path and return FileStatusAdapter.
+   */
+  private FileStatusAdapter getFileStatusForKeyOrSnapshot(
+      OFSPath ofsPath, URI uri, Path qualifiedPath, String userName)
+      throws IOException {
+    String key = ofsPath.getKeyName();
     try {
       OzoneBucket bucket = getBucket(ofsPath, false);
-      OzoneFileStatus status = bucket.getFileStatus(key);
-      return toFileStatusAdapter(status, userName, uri, qualifiedPath,
-          ofsPath.getNonKeyPath());
+      if (ofsPath.isSnapshotPath()) {
+        OzoneVolume volume = objectStore.getVolume(ofsPath.getVolumeName());
+        return getFileStatusAdapterWithSnapshotIndicator(

Review Comment:
   nit: Should we use`toFileStatusAdapterWithSnapshotIndicator` just to be consistent with `toFileStatusAdapter`?



##########
hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java:
##########
@@ -635,19 +637,36 @@ public FileStatusAdapter getFileStatus(String path, URI uri,
       Path qualifiedPath, String userName) throws IOException {
     incrementCounter(Statistic.OBJECTS_QUERY, 1);
     OFSPath ofsPath = new OFSPath(path, config);
-    String key = ofsPath.getKeyName();
     if (ofsPath.isRoot()) {
       return getFileStatusAdapterForRoot(uri);
-    }
-    if (ofsPath.isVolume()) {
+    } else if (ofsPath.isVolume()) {
       OzoneVolume volume = objectStore.getVolume(ofsPath.getVolumeName());
       return getFileStatusAdapterForVolume(volume, uri);
+    } else {
+      return getFileStatusForKeyOrSnapshot(
+          ofsPath, uri, qualifiedPath, userName);
     }
+  }
+
+  /**

Review Comment:
   nit:
   ```suggestion
     /**
      * Return FileStatusAdapter based on OFSPath is a valid bucket path
      * or valid snapshot path.
      * Throws exception in case of failure.
      */
   ```   



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsSnapshot.java:
##########
@@ -65,259 +77,229 @@ public static void initClass() throws Exception {
     ozoneManager = cluster.getOzoneManager();
   }
 
-  @Before
-  public void init() {
+  @BeforeEach
+  public void init() throws Exception {
     String hostPrefix = OZONE_OFS_URI_SCHEME + "://" + OM_SERVICE_ID;
-    clientConf = new OzoneConfiguration(cluster.getConf());
+    OzoneConfiguration clientConf =
+        new OzoneConfiguration(cluster.getConf());
     clientConf.set(FS_DEFAULT_NAME_KEY, hostPrefix);
+
+    shell = new OzoneFsShell(clientConf);
+
+    this.volume = "vol-" + RandomStringUtils.randomNumeric(5);
+    this.bucket = "buck-" + RandomStringUtils.randomNumeric(5);
+    this.key = "key-" + RandomStringUtils.randomNumeric(5);
+
+    this.bucketPath = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
+    this.bucketWithSnapshotIndicatorPath = bucketPath +
+        OM_KEY_PREFIX + OM_SNAPSHOT_INDICATOR;
+
+    String keyPath = bucketPath + OM_KEY_PREFIX + key;
+    createVolBuckKey(bucketPath, keyPath);
+  }
+
+  @AfterEach
+  public void cleanup() throws IOException {
+    shell.close();
   }
 
-  @AfterClass
+  @AfterAll
   public static void shutdown() {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
 
   private void createVolBuckKey(String testVolBucket, String testKey)
-          throws Exception {
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      // Create volume and bucket
-      int res = ToolRunner.run(shell,
-              new String[]{"-mkdir", "-p", testVolBucket});
-      assertEquals(0, res);
-      // Create key
-      res = ToolRunner.run(shell, new String[]{"-touch", testKey});
-      assertEquals(0, res);
-      // List the bucket to make sure that bucket exists.
-      res = ToolRunner.run(shell, new String[]{"-ls", testVolBucket});
-      assertEquals(0, res);
-    } finally {
-      shell.close();
-    }
-  }
-
-  @Test
-  public void testCreateSnapshot() throws Exception {
-    String volume = "vol1";
-    String bucket = "bucket1";
-    String testVolBucket = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
-    String snapshotName = "snap1";
-    String testKey = testVolBucket + "/key1";
+      throws Exception {
+
+    // Create volume and bucket
+    int res = ToolRunner.run(shell,
+        new String[]{"-mkdir", "-p", testVolBucket});
+    Assertions.assertEquals(0, res);
+    // Create key
+    res = ToolRunner.run(shell, new String[]{"-touch", testKey});
+    Assertions.assertEquals(0, res);
+    // List the bucket to make sure that bucket exists.
+    res = ToolRunner.run(shell, new String[]{"-ls", testVolBucket});
+    Assertions.assertEquals(0, res);
 
-    createVolBuckKey(testVolBucket, testKey);
-
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      int res = ToolRunner.run(shell,
-          new String[]{"-createSnapshot", testVolBucket, snapshotName});
-      // Asserts that create request succeeded
-      assertEquals(0, res);
-
-      SnapshotInfo snapshotInfo = ozoneManager
-          .getMetadataManager()
-          .getSnapshotInfoTable()
-          .get(SnapshotInfo.getTableKey(volume, bucket, snapshotName));
-
-      // Assert that snapshot exists in RocksDB.
-      // We can't use list or valid if snapshot directory exists because DB
-      // transaction might not be flushed by the time.
-      Assert.assertNotNull(snapshotInfo);
-    } finally {
-      shell.close();
-    }
   }
 
   @Test
   public void testCreateSnapshotDuplicateName() throws Exception {
-    String volume = "vol-" + RandomStringUtils.randomNumeric(5);
-    String bucket = "buc-" + RandomStringUtils.randomNumeric(5);
-    String key = "key-" + RandomStringUtils.randomNumeric(5);
     String snapshotName = "snap-" + RandomStringUtils.randomNumeric(5);
 
-    String testVolBucket = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
-    String testKey = testVolBucket + OM_KEY_PREFIX + key;
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", bucketPath, snapshotName});
+    // Asserts that create request succeeded
+    Assertions.assertEquals(0, res);
 
-    createVolBuckKey(testVolBucket, testKey);
+    res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", bucketPath, snapshotName});
+    // Asserts that create request fails since snapshot name provided twice
+    Assertions.assertEquals(1, res);
+  }
 
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      int res = ToolRunner.run(shell,
-              new String[]{"-createSnapshot", testVolBucket, snapshotName});
-      // Asserts that create request succeeded
-      assertEquals(0, res);
-
-      res = ToolRunner.run(shell,
-              new String[]{"-createSnapshot", testVolBucket, snapshotName});
-      // Asserts that create request fails since snapshot name provided twice
-      assertEquals(1, res);
-    } finally {
-      shell.close();
-    }
+  /**
+   * Create snapshot should succeed.
+   * 1st case: valid snapshot name
+   * 2nd case: snapshot name length is less than 64 chars
+   */
+  @ParameterizedTest
+  @ValueSource(strings = {"snap-1",
+      "snap75795657617173401188448010125899089001363595171500499231286"})
+  public void testCreateSnapshotSuccess(String snapshotName)
+      throws Exception {
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", bucketPath, snapshotName});
+    // Asserts that create request succeeded
+    Assertions.assertEquals(0, res);
+
+    SnapshotInfo snapshotInfo = ozoneManager
+        .getMetadataManager()
+        .getSnapshotInfoTable()
+        .get(SnapshotInfo.getTableKey(volume, bucket, snapshotName));
+
+    // Assert that snapshot exists in RocksDB.
+    // We can't use list or valid if snapshot directory exists because DB
+    // transaction might not be flushed by the time.
+    Assertions.assertNotNull(snapshotInfo);
   }
 
-  @Test
-  public void testCreateSnapshotInvalidName() throws Exception {
-    String volume = "vol-" + RandomStringUtils.randomNumeric(5);
-    String bucket = "buc-" + RandomStringUtils.randomNumeric(5);
-    String key = "key-" + RandomStringUtils.randomNumeric(5);
-    String snapshotName = "snapa?b";
+  /**
+   * Create snapshot should fail.
+   * 1st case: snapshot name contains invalid char
+   * 2nd case: snapshot name consists only of numbers
+   * 3rd case: bucket path is invalid
+   * 4th case: snapshot name length is more than 64 chars
+   */
+  @ParameterizedTest
+  @ValueSource(strings = {"snapa?b", "1234",
+      "snap156808943643007724443266605711479126926050896107709081166294"})
+  public void testCreateSnapshotFailure(String snapshotName)
+      throws Exception {
+
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", bucketPath, snapshotName});
+    // Asserts that create request failed
+    Assertions.assertEquals(1, res);
+  }
 
-    String testVolBucket = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
-    String testKey = testVolBucket + OM_KEY_PREFIX + key;
+  @Test
+  public void testCreateSnapshotParameterMissing() throws Exception {
 
-    createVolBuckKey(testVolBucket, testKey);
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot"});
+    // Asserts that create request failed since mandatory params not passed
+    Assertions.assertEquals(-1, res);
+  }
 
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      int res = ToolRunner.run(shell,
-              new String[]{"-createSnapshot", testVolBucket, snapshotName});
-      // Asserts that create request failed since invalid name passed
-      assertEquals(1, res);
+  @Test
+  public void testCreateSnapshotInvalidURI() throws Exception {
 
-    } finally {
-      shell.close();
-    }
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", "invalidURI"});
+    // Asserts that create request failed since
+    // invalid volume-bucket URI passed
+    Assertions.assertEquals(1, res);
   }
 
+  /**
+   * Test list snapshots with "ozone fs -ls".
+   */
   @Test
-  public void testCreateSnapshotOnlyNumericName() throws Exception {
-    String volume = "vol-" + RandomStringUtils.randomNumeric(5);
-    String bucket = "buc-" + RandomStringUtils.randomNumeric(5);
-    String key = "key-" + RandomStringUtils.randomNumeric(5);
-    String snapshotName = "1234";
-
-    String testVolBucket = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
-    String testKey = testVolBucket + OM_KEY_PREFIX + key;
+  public void testFsLsSnapshot() throws Exception {

Review Comment:
   nit: I think `testFsLsSnapshot` and `testFsLsSnapshotKeys` can be grouped.



##########
hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java:
##########
@@ -1123,6 +1169,75 @@ private static FileStatusAdapter getFileStatusAdapterForBucket(
                     HddsProtos.ReplicationType.EC);
   }
 
+  /**
+   * Generate a FileStatusAdapter for a snapshot under a bucket.
+   * @param ozoneBucket OzoneBucket object.
+   * @param ozoneSnapshot OzoneSnapshot object.
+   * @param uri Full URI to OFS root.
+   * @param owner Owner of the parent volume of the bucket.
+   * @param group Group of the parent volume of the bucket.
+   * @return FileStatusAdapter for a snapshot.
+   */
+  private static FileStatusAdapter getFileStatusAdapterForBucketSnapshot(
+      OzoneBucket ozoneBucket, OzoneSnapshot ozoneSnapshot,
+      URI uri, String owner, String group) {
+    String pathStr = uri.toString() +
+        OZONE_URI_DELIMITER + ozoneSnapshot.getVolumeName() +
+        OZONE_URI_DELIMITER + ozoneSnapshot.getBucketName() +
+        OZONE_URI_DELIMITER + OM_SNAPSHOT_INDICATOR +
+        OZONE_URI_DELIMITER + ozoneSnapshot.getName();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("getFileStatusAdapterForBucketSnapshot: " +
+              "ozoneSnapshot={}, pathStr={}",
+          ozoneSnapshot.getName(), pathStr);
+    }
+    Path path = new Path(pathStr);
+    return new FileStatusAdapter(0L, 0L, path, true, (short)0, 0L,
+        ozoneSnapshot.getCreationTime(), 0L,
+        FsPermission.getDirDefault().toShort(),
+        owner, group, null, new BlockLocation[0],
+        !StringUtils.isEmpty(ozoneBucket.getEncryptionKeyName()),
+        ozoneBucket.getReplicationConfig() != null &&
+            ozoneBucket.getReplicationConfig().getReplicationType() ==
+                HddsProtos.ReplicationType.EC);
+  }
+
+  /**
+   * Generate a FileStatusAdapter for a bucket
+   * followed by a snapshot indicator.
+   * @param ozoneVolume OzoneVolume object.
+   * @param ozoneBucket OzoneBucket object.
+   * @param uri Full URI to OFS root.
+   * @return FileStatusAdapter for a snapshot indicator.
+   */
+  private static FileStatusAdapter getFileStatusAdapterWithSnapshotIndicator(

Review Comment:
   What's difference between `getFileStatusAdapterWithSnapshotIndicator` and `getFileStatusAdapterForBucketSnapshot`? Also `getFileStatusAdapterWithSnapshotIndicator`'s description says `Generate a FileStatusAdapter for a bucket` but it is for a snapshot bucket or at least snapshot path.



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