You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by si...@apache.org on 2023/02/11 04:43:09 UTC

[ozone] branch master updated: HDDS-7513. [Snapshot] Support list snapshots using fs -ls (#4134)

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

siyao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new a8d20de041 HDDS-7513. [Snapshot] Support list snapshots using fs -ls (#4134)
a8d20de041 is described below

commit a8d20de041baaf2b6345d833797484389740b609
Author: Christos Bisias <ch...@gmail.com>
AuthorDate: Sat Feb 11 06:43:03 2023 +0200

    HDDS-7513. [Snapshot] Support list snapshots using fs -ls (#4134)
---
 .../main/java/org/apache/hadoop/ozone/OFSPath.java |  18 +
 hadoop-ozone/dist/src/main/compose/ozone/test.sh   |   4 +-
 .../dist/src/main/smoketest/snapshot/basic.robot   |  56 ---
 .../src/main/smoketest/snapshot/snapshot-fs.robot  |  60 +++
 .../main/smoketest/snapshot/snapshot-setup.robot   |  63 +++
 .../src/main/smoketest/snapshot/snapshot-sh.robot  |  57 +++
 .../hadoop/fs/ozone/TestOzoneFsSnapshot.java       | 491 +++++++++++----------
 .../ozone/BasicRootedOzoneClientAdapterImpl.java   | 126 +++++-
 8 files changed, 576 insertions(+), 299 deletions(-)

diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OFSPath.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OFSPath.java
index 3c6bd1fa10..22c865b10e 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OFSPath.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OFSPath.java
@@ -40,6 +40,7 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ENABLE_OFS_SHARED
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ENABLE_OFS_SHARED_TMP_DIR_DEFAULT;
 import static org.apache.hadoop.ozone.OzoneConsts.OZONE_OFS_URI_SCHEME;
 import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_INDICATOR;
 
 /**
  * Utility class for Rooted Ozone Filesystem (OFS) path processing.
@@ -277,6 +278,23 @@ public class OFSPath {
         !this.getVolumeName().isEmpty();
   }
 
+  /**
+   * If volume and bucket names are not empty and the key name
+   * only contains the snapshot indicator, then return true.
+   * e.g. /vol/bucket/.snapshot is a snapshot path.
+   */
+  public boolean isSnapshotPath() {
+    if (keyName.startsWith(OM_SNAPSHOT_INDICATOR)) {
+      String[] keyNames = keyName.split(OZONE_URI_DELIMITER);
+
+      if (keyNames.length == 1) {
+        return  !bucketName.isEmpty() &&
+                !volumeName.isEmpty();
+      }
+    }
+    return false;
+  }
+
   /**
    * If key name is not empty, the given path is a key.
    * e.g. /volume1/bucket2/key3 is a key.
diff --git a/hadoop-ozone/dist/src/main/compose/ozone/test.sh b/hadoop-ozone/dist/src/main/compose/ozone/test.sh
index 8e330742ae..5015913f7d 100755
--- a/hadoop-ozone/dist/src/main/compose/ozone/test.sh
+++ b/hadoop-ozone/dist/src/main/compose/ozone/test.sh
@@ -50,8 +50,6 @@ execute_robot_test scm recon
 
 execute_robot_test scm om-ratis
 
-execute_robot_test scm snapshot/basic.robot
-
 execute_robot_test scm freon
 
 execute_robot_test scm cli
@@ -64,6 +62,8 @@ execute_robot_test scm -v SCHEME:o3fs -v BUCKET_TYPE:bucket -N ozonefs-o3fs-buck
 
 execute_robot_test scm ec/basic.robot
 
+execute_robot_test scm snapshot
+
 stop_docker_env
 
 generate_report
diff --git a/hadoop-ozone/dist/src/main/smoketest/snapshot/basic.robot b/hadoop-ozone/dist/src/main/smoketest/snapshot/basic.robot
deleted file mode 100644
index 70ac33f6fa..0000000000
--- a/hadoop-ozone/dist/src/main/smoketest/snapshot/basic.robot
+++ /dev/null
@@ -1,56 +0,0 @@
-# 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.
-
-*** Settings ***
-Documentation       Test Snapshot Commands
-Library             OperatingSystem
-Resource            ../commonlib.robot
-Resource            ../ec/lib.resource
-Suite Setup         Prepare For Tests
-
-*** Variables ***
-${volume}       snapvolume
-${bucket}       snapbucket
-
-*** Test Cases ***
-Snapshot Creation
-    ${result} =     Execute             ozone sh volume create /${volume}
-                    Should not contain  ${result}       Failed
-    ${result} =     Execute             ozone sh bucket create /${volume}/${bucket}
-                    Should not contain  ${result}       Failed
-                    Execute             ozone sh key put /${volume}/${bucket}/key1 /tmp/1mb
-    ${result} =     Execute             ozone sh snapshot create /${volume}/${bucket} snapshot1
-                    Should not contain  ${result}       Failed
-
-Snapshot List
-
-    ${result} =     Execute             ozone sh snapshot ls /${volume}/${bucket}
-                    Should contain  ${result}       snapshot1
-                    Should contain  ${result}       SNAPSHOT_ACTIVE
-
-
-Snapshot Diff
-                    Execute             ozone sh key put /${volume}/${bucket}/key2 /tmp/2mb
-                    Execute             ozone sh key put /${volume}/${bucket}/key3 /tmp/2mb
-                    Execute             ozone sh snapshot create /${volume}/${bucket} snapshot2
-    ${result} =     Execute             ozone sh snapshot snapshotDiff /${volume}/${bucket} snapshot1 snapshot2
-                    Should contain  ${result}       +    key2
-                    Should contain  ${result}       +    key3
-
-Read Snapshot
-                    Key Should Match Local File         /${volume}/${bucket}/.snapshot/snapshot1/key1      /tmp/1mb
-                    Key Should Match Local File         /${volume}/${bucket}/.snapshot/snapshot2/key2      /tmp/2mb
-                    Key Should Match Local File         /${volume}/${bucket}/.snapshot/snapshot2/key3      /tmp/2mb
-
diff --git a/hadoop-ozone/dist/src/main/smoketest/snapshot/snapshot-fs.robot b/hadoop-ozone/dist/src/main/smoketest/snapshot/snapshot-fs.robot
new file mode 100644
index 0000000000..30630f2095
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/snapshot/snapshot-fs.robot
@@ -0,0 +1,60 @@
+# 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.
+
+*** Settings ***
+Documentation       Test for using fs commands with snapshots.
+Library             OperatingSystem
+Resource            snapshot-setup.robot
+Test Timeout        5 minutes
+
+*** Variables ***
+${KEY_ONE}
+${KEY_TWO}
+${SNAPSHOT_ONE}
+
+*** Keywords ***
+Keys creation
+    ${key_one} =            Create key          ${VOLUME}       ${BUCKET}       /etc/hosts
+    Set Suite Variable      ${KEY_ONE}          ${key_one}
+    ${key_two} =            Create key          ${VOLUME}       ${BUCKET}       /etc/os-release
+    Set Suite Variable      ${KEY_TWO}          ${key_two}
+
+Snapshot creation
+    ${snapshot_one} =       Create snapshot     ${VOLUME}       ${BUCKET}
+    Set Suite Variable      ${SNAPSHOT_ONE}     ${snapshot_one}
+
+List snapshots with fs -ls
+    ${result} =     Execute                 ozone fs -ls /${VOLUME}/${BUCKET}/${SNAPSHOT_INDICATOR}
+                    Should contain          ${result}       /${VOLUME}/${BUCKET}/${SNAPSHOT_INDICATOR}/${SNAPSHOT_ONE}
+
+List snapshot keys with fs -ls
+    ${result} =     Execute                 ozone fs -ls /${VOLUME}/${BUCKET}/${SNAPSHOT_INDICATOR}/${SNAPSHOT_ONE}
+                    Should contain          ${result}       /${VOLUME}/${BUCKET}/${SNAPSHOT_INDICATOR}/${SNAPSHOT_ONE}/${KEY_ONE}
+                    Should contain          ${result}       /${VOLUME}/${BUCKET}/${SNAPSHOT_INDICATOR}/${SNAPSHOT_ONE}/${KEY_TWO}
+    ${result} =     Execute                 ozone fs -ls /${VOLUME}/${BUCKET}/${SNAPSHOT_INDICATOR}/${SNAPSHOT_ONE}/${KEY_ONE}
+                    Should contain          ${result}       /${VOLUME}/${BUCKET}/${SNAPSHOT_INDICATOR}/${SNAPSHOT_ONE}/${KEY_ONE}
+                    Should not contain      ${result}       /${VOLUME}/${BUCKET}/${SNAPSHOT_INDICATOR}/${SNAPSHOT_ONE}/${KEY_TWO}
+
+*** Test Cases ***
+Test set up paths and snapshot
+    Setup volume and bucket
+    Keys creation
+    Snapshot creation
+
+Test list snapshots with fs -ls
+    List snapshots with fs -ls
+
+Test list snapshot keys with fs -ls
+    List snapshot keys with fs -ls
diff --git a/hadoop-ozone/dist/src/main/smoketest/snapshot/snapshot-setup.robot b/hadoop-ozone/dist/src/main/smoketest/snapshot/snapshot-setup.robot
new file mode 100644
index 0000000000..a5145be0ac
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/snapshot/snapshot-setup.robot
@@ -0,0 +1,63 @@
+# 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.
+
+*** Settings ***
+Documentation       Test for using fs commands with snapshots.
+Library             OperatingSystem
+Library             String
+Library             BuiltIn
+Resource            ../commonlib.robot
+
+*** Variables ***
+${SNAPSHOT_INDICATOR}      .snapshot
+${VOLUME}
+${BUCKET}
+
+*** Keywords ***
+Create volume
+    ${random} =     Generate Random String  5  [LOWER]
+    ${volume} =     Set Variable        vol-${random}
+    ${result} =     Execute             ozone sh volume create /${volume}
+                    Should not contain  ${result}       Failed
+    [Return]        ${volume}
+
+Create bucket
+    [Arguments]     ${volume}           ${bucket_layout}
+    ${random} =     Generate Random String  5  [LOWER]
+    ${bucket} =     Set Variable        buc-${random}
+    ${result} =     Execute             ozone sh bucket create -l ${bucket_layout} /${volume}/${bucket}
+                    Should not contain  ${result}       Failed
+    [Return]        ${bucket}
+
+Create key
+    [Arguments]     ${volume}           ${bucket}       ${file}
+    ${random} =     Generate Random String  5  [LOWER]
+    ${key} =        Set Variable        key-${random}
+    ${result} =     Execute             ozone sh key put /${volume}/${bucket}/${key} ${file}
+    [Return]        ${key}
+
+Create snapshot
+    [Arguments]     ${volume}           ${bucket}
+    ${random} =     Generate Random String  5  [LOWER]
+    ${snapshot} =   Set Variable        snap-${random}
+    ${result} =     Execute             ozone sh snapshot create /${volume}/${bucket} ${snapshot}
+                    Should not contain  ${result}       Failed
+    [Return]        ${snapshot}
+
+Setup volume and bucket
+    ${volume} =             Create Volume
+    Set Suite Variable      ${VOLUME}           ${volume}
+    ${bucket} =             Create bucket       ${VOLUME}       FILE_SYSTEM_OPTIMIZED
+    Set Suite Variable      ${BUCKET}           ${bucket}
diff --git a/hadoop-ozone/dist/src/main/smoketest/snapshot/snapshot-sh.robot b/hadoop-ozone/dist/src/main/smoketest/snapshot/snapshot-sh.robot
new file mode 100644
index 0000000000..03807ff7d7
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/snapshot/snapshot-sh.robot
@@ -0,0 +1,57 @@
+# 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.
+
+*** Settings ***
+Documentation       Test for using sh commands with snapshots.
+Library             OperatingSystem
+Resource            ../ozone-lib/shell.robot
+Resource            snapshot-setup.robot
+Test Timeout        5 minutes
+
+*** Variables ***
+${SNAPSHOT_ONE}
+${SNAPSHOT_TWO}
+${KEY_ONE}
+${KEY_TWO}
+${KEY_THREE}
+
+*** Test Cases ***
+Snapshot Creation
+    Setup volume and bucket
+    ${key_one} =            snapshot-setup.Create key           ${VOLUME}       ${BUCKET}       /etc/hosts
+    Set Suite Variable      ${KEY_ONE}          ${key_one}
+    ${snapshot_one} =       Create snapshot     ${VOLUME}       ${BUCKET}
+    Set Suite Variable      ${SNAPSHOT_ONE}     ${snapshot_one}
+
+Snapshot List
+    ${result} =     Execute             ozone sh snapshot ls /${VOLUME}/${BUCKET}
+                    Should contain      ${result}       ${SNAPSHOT_ONE}
+                    Should contain      ${result}       SNAPSHOT_ACTIVE
+
+Snapshot Diff
+    ${key_two} =            snapshot-setup.Create key           ${VOLUME}       ${BUCKET}       /etc/passwd
+    Set Suite Variable      ${KEY_TWO}          ${key_two}
+    ${key_three} =          snapshot-setup.Create key           ${VOLUME}       ${BUCKET}       /etc/group
+    Set Suite Variable      ${KEY_THREE}        ${key_three}
+    ${snapshot_two} =       Create snapshot     ${VOLUME}       ${BUCKET}
+    Set Suite Variable      ${SNAPSHOT_TWO}     ${snapshot_two}
+    ${result} =     Execute             ozone sh snapshot snapshotDiff /${VOLUME}/${BUCKET} ${SNAPSHOT_ONE} ${SNAPSHOT_TWO}
+                    Should contain      ${result}       +    ${KEY_TWO}
+                    Should contain      ${result}       +    ${KEY_THREE}
+
+Read Snapshot
+    Key Should Match Local File         /${VOLUME}/${BUCKET}/${SNAPSHOT_INDICATOR}/${SNAPSHOT_ONE}/${KEY_ONE}       /etc/hosts
+    Key Should Match Local File         /${VOLUME}/${BUCKET}/${SNAPSHOT_INDICATOR}/${SNAPSHOT_TWO}/${KEY_TWO}       /etc/passwd
+    Key Should Match Local File         /${VOLUME}/${BUCKET}/${SNAPSHOT_INDICATOR}/${SNAPSHOT_TWO}/${KEY_THREE}     /etc/group
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsSnapshot.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsSnapshot.java
index 14ebd7fa00..dcbdebf9bf 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsSnapshot.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsSnapshot.java
@@ -16,41 +16,66 @@
  */
 package org.apache.hadoop.fs.ozone;
 
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
 import java.util.UUID;
+import java.util.stream.Stream;
+
+import com.google.common.base.Strings;
 
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.om.OMStorage;
 import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
 import org.apache.hadoop.util.ToolRunner;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.junit.jupiter.params.provider.ValueSource;
 
 import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY;
+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_OFS_URI_SCHEME;
-import static org.junit.Assert.assertEquals;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_INDICATOR;
 
 /**
  * Test client-side CRUD snapshot operations with Ozone Manager.
+ * Setting a timeout for every test method to 300 seconds.
  */
+@Timeout(value = 300)
 public class TestOzoneFsSnapshot {
-  // Set the timeout for every test.
-  @Rule
-  public Timeout testTimeout = Timeout.seconds(300);
 
   private static MiniOzoneCluster cluster;
   private static final String OM_SERVICE_ID = "om-service-test1";
-  private OzoneConfiguration clientConf;
   private static OzoneManager ozoneManager;
-
-  @BeforeClass
+  private static OzoneFsShell shell;
+  private static final String VOLUME =
+      "vol-" + RandomStringUtils.randomNumeric(5);;
+  private static final String BUCKET =
+      "buck-" + RandomStringUtils.randomNumeric(5);
+  private static final String KEY =
+      "key-" + RandomStringUtils.randomNumeric(5);
+  private static final String BUCKET_PATH =
+      OM_KEY_PREFIX + VOLUME + OM_KEY_PREFIX + BUCKET;
+  private static final String BUCKET_WITH_SNAPSHOT_INDICATOR_PATH =
+      BUCKET_PATH + OM_KEY_PREFIX + OM_SNAPSHOT_INDICATOR;
+  private static final String KEY_PATH =
+      BUCKET_PATH + OM_KEY_PREFIX + KEY;
+
+  @BeforeAll
   public static void initClass() throws Exception {
     OzoneConfiguration conf = new OzoneConfiguration();
 
@@ -63,261 +88,257 @@ public class TestOzoneFsSnapshot {
         .build();
     cluster.waitForClusterToBeReady();
     ozoneManager = cluster.getOzoneManager();
-  }
 
-  @Before
-  public void init() {
     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);
+
+    createVolBuckKey();
   }
 
-  @AfterClass
-  public static void shutdown() {
+  @AfterAll
+  public static void shutdown() throws IOException {
+    shell.close();
     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();
-    }
-  }
+  private static void createVolBuckKey()
+      throws Exception {
+    // Create volume and bucket
+    int res = ToolRunner.run(shell,
+        new String[]{"-mkdir", "-p", BUCKET_PATH});
+    Assertions.assertEquals(0, res);
+    // Create key
+    res = ToolRunner.run(shell, new String[]{"-touch", KEY_PATH});
+    Assertions.assertEquals(0, res);
+    // List the bucket to make sure that bucket exists.
+    res = ToolRunner.run(shell, new String[]{"-ls", BUCKET_PATH});
+    Assertions.assertEquals(0, res);
 
-  @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";
-
-    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;
-
-    createVolBuckKey(testVolBucket, testKey);
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", BUCKET_PATH, snapshotName});
+    // Asserts that create request succeeded
+    Assertions.assertEquals(0, 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();
-    }
+    res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", BUCKET_PATH, snapshotName});
+    // Asserts that create request fails since snapshot name provided twice
+    Assertions.assertEquals(1, res);
   }
 
-  @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";
-
-    String testVolBucket = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
-    String testKey = testVolBucket + OM_KEY_PREFIX + key;
-
-    createVolBuckKey(testVolBucket, testKey);
-
-    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);
-
-    } 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", BUCKET_PATH, 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 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;
-
-    createVolBuckKey(testVolBucket, testKey);
-
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      int res = ToolRunner.run(shell,
-              new String[]{"-createSnapshot", testVolBucket, snapshotName});
-      // Asserts that create request failed since only numeric name passed
-      assertEquals(1, res);
-
-    } finally {
-      shell.close();
-    }
+  private static Stream<Arguments> createSnapshotFailureScenarios() {
+    String invalidBucketPath = "/invalid/uri";
+    return Stream.of(
+        Arguments.of("1st case: snapshot name contains invalid char",
+            BUCKET_PATH,
+            "snapa?b",
+            "Invalid snapshot name",
+            1),
+        Arguments.of("2nd case: snapshot name consists only of numbers",
+            BUCKET_PATH,
+            "1234",
+            "Invalid snapshot name",
+            1),
+        Arguments.of("3rd case: bucket path is invalid",
+            invalidBucketPath,
+            "validSnapshotName12",
+            "No such file or directory",
+            1),
+        Arguments.of("4th case: snapshot name length is more than 64 chars",
+            BUCKET_PATH,
+            "snap156808943643007724443266605711479126926050896107709081166294",
+            "Invalid snapshot name",
+            1),
+        Arguments.of("5th case: all parameters are missing",
+            "",
+            "",
+            "Can not create a Path from an empty string",
+            -1)
+    );
   }
 
-  @Test
-  public void testCreateSnapshotInvalidURI() throws Exception {
-
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-
-      int res = ToolRunner.run(shell,
-              new String[]{"-createSnapshot", "invalidURI"});
-      // Asserts that create request failed since
-      // invalid volume-bucket URI passed
-      assertEquals(1, res);
-
-    } finally {
-      shell.close();
-    }
+  @ParameterizedTest(name = "{0}")
+  @MethodSource("createSnapshotFailureScenarios")
+  public void testCreateSnapshotFailure(String description,
+                                        String paramBucketPath,
+                                        String snapshotName,
+                                        String expectedMessage,
+                                        int expectedResponse)
+      throws Exception {
+    String errorMessage = execShellCommandAndGetOutput(expectedResponse,
+        new String[]{"-createSnapshot", paramBucketPath, snapshotName});
+
+    Assertions.assertTrue(errorMessage
+        .contains(expectedMessage));
   }
 
+  /**
+   * Test list snapshot and snapshot keys with "ozone fs -ls".
+   */
   @Test
-  public void testCreateSnapshotNameLength() throws Exception {
-    String volume = "vol-" + RandomStringUtils.randomNumeric(5);
-    String bucket = "buc-" + RandomStringUtils.randomNumeric(5);
-    String key = "key-" + RandomStringUtils.randomNumeric(5);
-    String name63 =
-            "snap75795657617173401188448010125899089001363595171500499231286";
-    String name64 =
-            "snap156808943643007724443266605711479126926050896107709081166294";
-
-    String testVolBucket = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
-    String testKey = testVolBucket + OM_KEY_PREFIX + key;
-
-    createVolBuckKey(testVolBucket, testKey);
-
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-      int res = ToolRunner.run(shell,
-              new String[]{"-createSnapshot", testVolBucket, name63});
-      // Asserts that create request succeeded since namelength
-      // less than 64 char
-      assertEquals(0, res);
-
-      res = ToolRunner.run(shell,
-              new String[]{"-createSnapshot", testVolBucket, name64});
-      // Asserts that create request fails since namelength
-      // more than 64 char
-      assertEquals(1, res);
-
-      SnapshotInfo snapshotInfo = ozoneManager
-              .getMetadataManager()
-              .getSnapshotInfoTable()
-              .get(SnapshotInfo.getTableKey(volume, bucket, name63));
-
-      Assert.assertNotNull(snapshotInfo);
-
-    } finally {
-      shell.close();
-    }
+  public void testFsLsSnapshot() throws Exception {
+    String newKey = "key-" + RandomStringUtils.randomNumeric(5);
+    String newKeyPath = BUCKET_PATH + OM_KEY_PREFIX + newKey;
+
+    // Create new key, while the old one
+    // might be deleted from a previous test.
+    execShellCommandAndGetOutput(0,
+        new String[]{"-touch", newKeyPath});
+
+    // Create snapshot
+    String snapshotName = createSnapshot();
+    // Setup snapshot paths
+    String snapshotPath = BUCKET_WITH_SNAPSHOT_INDICATOR_PATH +
+        OM_KEY_PREFIX + snapshotName;
+    String snapshotKeyPath = snapshotPath + OM_KEY_PREFIX + newKey;
+
+    // Check for snapshot with "ozone fs -ls"
+    String listSnapOut = execShellCommandAndGetOutput(0,
+        new String[]{"-ls", BUCKET_WITH_SNAPSHOT_INDICATOR_PATH});
+
+    // Assert that output contains above snapshotName
+    Assertions.assertTrue(listSnapOut
+        .contains(snapshotPath));
+
+    // Check for snapshot keys with "ozone fs -ls"
+    String listSnapKeyOut = execShellCommandAndGetOutput(0,
+        new String[]{"-ls", snapshotPath});
+
+    // Assert that output contains the snapshot key
+    Assertions.assertTrue(listSnapKeyOut
+        .contains(snapshotKeyPath));
   }
 
   @Test
-  public void testCreateSnapshotParameterMissing() throws Exception {
-
-    OzoneFsShell shell = new OzoneFsShell(clientConf);
-    try {
-
-      int res = ToolRunner.run(shell,
-              new String[]{"-createSnapshot"});
-      // Asserts that create request failed since mandatory params not passed
-      assertEquals(-1, res);
+  public void testDeleteBucketWithSnapshot() throws Exception {
+    String snapshotName = createSnapshot();
+
+    String snapshotPath = BUCKET_WITH_SNAPSHOT_INDICATOR_PATH
+        + OM_KEY_PREFIX + snapshotName;
+    String snapshotKeyPath = snapshotPath + OM_KEY_PREFIX + KEY;
+
+    // Delete bucket key should succeed
+    String deleteKeyOut = execShellCommandAndGetOutput(0,
+        new String[]{"-rm", "-r", "-skipTrash", KEY_PATH});
+
+    Assertions.assertTrue(deleteKeyOut
+        .contains("Deleted " + BUCKET_PATH));
+
+    // Delete bucket should fail due to existing snapshot
+    String deleteBucketOut = execShellCommandAndGetOutput(1,
+        new String[]{"-rm", "-r", "-skipTrash", BUCKET_PATH});
+    Assertions.assertTrue(deleteBucketOut
+          .contains(BUCKET + " can't be deleted when it has snapshots"));
+
+    // Key shouldn't exist under bucket
+    String listKeyOut = execShellCommandAndGetOutput(0,
+        new String[]{"-ls", BUCKET_PATH});
+    Assertions.assertTrue(Strings.isNullOrEmpty(listKeyOut));
+
+    // Key should still exist under snapshot
+    String listSnapKeyOut = execShellCommandAndGetOutput(0,
+        new String[]{"-ls", snapshotPath});
+    Assertions.assertTrue(listSnapKeyOut.contains(snapshotKeyPath));
+  }
 
-    } finally {
-      shell.close();
-    }
+  /**
+   * Execute a shell command with provided arguments
+   * and return a string of the output.
+   */
+  private String execShellCommandAndGetOutput(
+      int response, String[] args) throws Exception {
+    ByteArrayOutputStream successBytes = new ByteArrayOutputStream();
+    ByteArrayOutputStream errorBytes = new ByteArrayOutputStream();
+
+    // Setup output streams
+    System.setOut(new PrintStream(
+        successBytes, false, StandardCharsets.UTF_8.name()));
+    System.setErr(new PrintStream(
+        errorBytes, false, StandardCharsets.UTF_8.name()));
+
+    // Execute command
+    int res = ToolRunner.run(shell, args);
+    Assertions.assertEquals(response, res);
+
+    // Store command output to a string,
+    // if command should succeed then
+    // get successBytes else get errorBytes
+    String output = response == 0 ?
+        successBytes.toString(StandardCharsets.UTF_8.name()) :
+        errorBytes.toString(StandardCharsets.UTF_8.name());
+
+    // Flush byte array streams
+    successBytes.flush();
+    errorBytes.flush();
+
+    // Restore output streams
+    System.setOut(new PrintStream(
+        successBytes, false, StandardCharsets.UTF_8.name()));
+    System.setErr(new PrintStream(
+        errorBytes, false, StandardCharsets.UTF_8.name()));
+
+    return output;
   }
 
-  @Test
-  public void testDeleteBucketWithSnapshot() throws Exception {
-    String volume = "vol-" + RandomStringUtils.randomNumeric(5);
-    String bucket = "buc-" + RandomStringUtils.randomNumeric(5);
-    String key = "key-" + RandomStringUtils.randomNumeric(5);
+  private String createSnapshot() throws Exception {
     String snapshotName = "snap-" + RandomStringUtils.randomNumeric(5);
 
-    String testVolBucket = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket;
-    String testKey = testVolBucket + OM_KEY_PREFIX + key;
-
-    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);
-
-      res = ToolRunner.run(shell,
-              new String[]{"-rm", "-r", "-skipTrash", testKey});
-      assertEquals(0, res);
-
-      res = ToolRunner.run(shell,
-              new String[]{"-rm", "-r", "-skipTrash", testVolBucket});
-      assertEquals(1, res);
-
-      res = ToolRunner.run(shell,
-              new String[]{"-ls", testVolBucket});
-      assertEquals(0, res);
-
-      String snapshotPath = testVolBucket + OM_KEY_PREFIX + ".snapshot"
-              + OM_KEY_PREFIX + snapshotName + OM_KEY_PREFIX;
-      res = ToolRunner.run(shell,
-              new String[]{"-ls", snapshotPath});
-      assertEquals(0, res);
-
-    } finally {
-      shell.close();
-    }
+    // Create snapshot
+    int res = ToolRunner.run(shell,
+        new String[]{"-createSnapshot", BUCKET_PATH, snapshotName});
+    // Asserts that create request succeeded
+    Assertions.assertEquals(0, res);
+
+    File metaDir = OMStorage
+        .getOmDbDir(ozoneManager.getConfiguration());
+
+    // wait till the snapshot directory exists
+    SnapshotInfo snapshotInfo = ozoneManager.getMetadataManager()
+        .getSnapshotInfoTable()
+        .get(SnapshotInfo.getTableKey(VOLUME, BUCKET, snapshotName));
+    String snapshotDirName = metaDir + OM_KEY_PREFIX +
+        OM_SNAPSHOT_DIR + OM_KEY_PREFIX + OM_DB_NAME +
+        snapshotInfo.getCheckpointDirName() + OM_KEY_PREFIX + "CURRENT";
+    GenericTestUtils.waitFor(() -> new File(snapshotDirName).exists(),
+        1000, 100000);
+
+    return snapshotName;
   }
 }
diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java
index c1d882b2e4..0374b9ee27 100644
--- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java
+++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientFactory;
 import org.apache.hadoop.ozone.client.OzoneKey;
 import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.OzoneSnapshot;
 import org.apache.hadoop.ozone.client.io.OzoneDataStreamOutput;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
@@ -81,6 +82,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_INDICATOR;
 import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
 import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes
     .BUCKET_ALREADY_EXISTS;
@@ -635,19 +637,37 @@ public class BasicRootedOzoneClientAdapterImpl
       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);
     }
+  }
+
+  /**
+   * Return FileStatusAdapter based on OFSPath being a
+   * valid bucket path or valid snapshot path.
+   * Throws exception in case of failure.
+   */
+  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(
+            volume, bucket, uri);
+      } else {
+        OzoneFileStatus status = bucket.getFileStatus(key);
+        return toFileStatusAdapter(status, userName, uri, qualifiedPath,
+            ofsPath.getNonKeyPath());
+      }
     } catch (OMException e) {
       if (e.getResult() == OMException.ResultCodes.FILE_NOT_FOUND) {
         throw new FileNotFoundException(key + ": No such file or directory!");
@@ -784,6 +804,26 @@ public class BasicRootedOzoneClientAdapterImpl
     return res;
   }
 
+  /**
+   * Helper for OFS listStatus on a bucket to get all snapshots.
+   */
+  private List<FileStatusAdapter> listStatusBucketSnapshot(
+      String volumeName, String bucketName, URI uri) throws IOException {
+    List<OzoneSnapshot> snapshotList =
+        objectStore.listSnapshot(volumeName, bucketName);
+
+    OzoneBucket ozoneBucket = getBucket(volumeName, bucketName, false);
+    UserGroupInformation ugi =
+        UserGroupInformation.createRemoteUser(ozoneBucket.getOwner());
+    String owner = ugi.getShortUserName();
+    String group = getGroupName(ugi);
+
+    return snapshotList.stream()
+        .map(ozoneSnapshot -> getFileStatusAdapterForBucketSnapshot(
+            ozoneBucket, ozoneSnapshot, uri, owner, group))
+        .collect(Collectors.toList());
+  }
+
   /**
    * OFS listStatus implementation.
    *
@@ -838,6 +878,11 @@ public class BasicRootedOzoneClientAdapterImpl
           recursive, startBucket, numEntries, uri, workingDir, username);
     }
 
+    if (ofsPath.isSnapshotPath()) {
+      return listStatusBucketSnapshot(ofsPath.getVolumeName(),
+          ofsPath.getBucketName(), uri);
+    }
+
     String keyName = ofsPath.getKeyName();
     // Internally we need startKey to be passed into bucket.listStatus
     String startKey = ofsStartPath.getKeyName();
@@ -1123,6 +1168,75 @@ public class BasicRootedOzoneClientAdapterImpl
                     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(
+      OzoneVolume ozoneVolume, OzoneBucket ozoneBucket, URI uri) {
+    UserGroupInformation ugi =
+        UserGroupInformation.createRemoteUser(ozoneVolume.getOwner());
+    String owner = ugi.getShortUserName();
+    String group = getGroupName(ugi);
+
+    String pathStr = uri.toString() +
+        OZONE_URI_DELIMITER + ozoneBucket.getVolumeName() +
+        OZONE_URI_DELIMITER + ozoneBucket.getName() +
+        OZONE_URI_DELIMITER + OM_SNAPSHOT_INDICATOR;
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("getFileStatusAdapterWithSnapshotIndicator: " +
+              "ozoneBucket={}, pathStr={}",
+          ozoneBucket.getVolumeName() + OZONE_URI_DELIMITER +
+              ozoneBucket.getName(), pathStr);
+    }
+    Path path = new Path(pathStr);
+    return new FileStatusAdapter(0L, 0L, path, true, (short)0, 0L,
+        ozoneBucket.getCreationTime().getEpochSecond() * 1000, 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 OFS root.
    * @param uri Full URI to OFS root.


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