You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by ra...@apache.org on 2021/04/07 03:25:48 UTC

[ozone] 21/29: HDDS-4490.[FSO]RenameAndDelete : make ofs#rename and ofs#delete an atomic operation. (#1965)

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

rakeshr pushed a commit to branch HDDS-2939
in repository https://gitbox.apache.org/repos/asf/ozone.git

commit 11f6c9ca24bddbd219b202c4fc80518a78603029
Author: Sadanand Shenoy <sa...@gmail.com>
AuthorDate: Tue Mar 2 19:41:23 2021 +0530

    HDDS-4490.[FSO]RenameAndDelete : make ofs#rename and ofs#delete an atomic operation. (#1965)
---
 .../hadoop/fs/ozone/TestRootedOzoneFileSystem.java | 224 +++++++++++++++++++--
 .../fs/ozone/TestRootedOzoneFileSystemV1.java      | 117 +++++++++++
 .../hadoop/ozone/client/rpc/TestReadRetries.java   |  20 +-
 .../org/apache/hadoop/ozone/om/KeyManagerImpl.java |   8 +-
 .../java/org/apache/hadoop/ozone/om/OMMetrics.java |  17 ++
 .../hadoop/ozone/om/TrashOzoneFileSystem.java      |   2 +
 .../fs/ozone/BasicRootedOzoneFileSystem.java       |  37 ++++
 7 files changed, 398 insertions(+), 27 deletions(-)

diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystem.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystem.java
index 3e12373..b5b7703 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystem.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystem.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.OMMetrics;
 import org.apache.hadoop.ozone.om.TrashPolicyOzone;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType;
 import org.apache.hadoop.ozone.security.acl.OzoneAclConfig;
@@ -61,7 +62,10 @@ 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.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -71,12 +75,12 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Optional;
+import java.util.Random;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.stream.Collectors;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
-import static org.apache.hadoop.fs.FileSystem.LOG;
 import static org.apache.hadoop.fs.FileSystem.TRASH_PREFIX;
 import static org.apache.hadoop.fs.ozone.Constants.LISTING_PAGE_SIZE;
 import static org.apache.hadoop.ozone.OzoneAcl.AclScope.ACCESS;
@@ -85,8 +89,10 @@ import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
 import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
 import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_FOUND;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Ozone file system tests that are not covered by contract tests.
@@ -95,6 +101,9 @@ import static org.junit.Assert.assertTrue;
 @RunWith(Parameterized.class)
 public class TestRootedOzoneFileSystem {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRootedOzoneFileSystem.class);
+
   @Parameterized.Parameters
   public static Collection<Object[]> data() {
     return Arrays.asList(
@@ -110,11 +119,20 @@ public class TestRootedOzoneFileSystem {
     omRatisEnabled = enableOMRatis;
   }
 
+  public static FileSystem getFs() {
+    return fs;
+  }
+
+  public static Path getBucketPath() {
+    return bucketPath;
+  }
+
   @Rule
   public Timeout globalTimeout = Timeout.seconds(300);;
 
   private static boolean enabledFileSystemPaths;
   private static boolean omRatisEnabled;
+  private static boolean isBucketFSOptimized = false;
 
   private static OzoneConfiguration conf;
   private static MiniOzoneCluster cluster = null;
@@ -136,8 +154,13 @@ public class TestRootedOzoneFileSystem {
     conf = new OzoneConfiguration();
     conf.setInt(FS_TRASH_INTERVAL_KEY, 1);
     conf.setBoolean(OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY, omRatisEnabled);
-    conf.setBoolean(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS,
-        enabledFileSystemPaths);
+    if (isBucketFSOptimized) {
+      TestOMRequestUtils.configureFSOptimizedPaths(conf,
+          true, OMConfigKeys.OZONE_OM_LAYOUT_VERSION_V1);
+    } else {
+      conf.setBoolean(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS,
+          enabledFileSystemPaths);
+    }
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(3)
         .build();
@@ -179,6 +202,10 @@ public class TestRootedOzoneFileSystem {
     return cluster.getOzoneManager().getMetrics();
   }
 
+  protected static void setIsBucketFSOptimized(boolean isBucketFSO) {
+    isBucketFSOptimized = isBucketFSO;
+  }
+
   @Test
   public void testOzoneFsServiceLoader() throws IOException {
     OzoneConfiguration confTestLoader = new OzoneConfiguration();
@@ -1226,6 +1253,11 @@ public class TestRootedOzoneFileSystem {
     long prevNumTrashRenames = getOMMetrics().getNumTrashRenames();
     long prevNumTrashFileRenames = getOMMetrics().getNumTrashFilesRenames();
 
+    long prevNumTrashAtomicDirDeletes = getOMMetrics()
+        .getNumTrashAtomicDirDeletes();
+    long prevNumTrashAtomicDirRenames = getOMMetrics()
+        .getNumTrashAtomicDirRenames();
+
     // Call moveToTrash. We can't call protected fs.rename() directly
     trash.moveToTrash(keyPath1);
     // for key in second bucket
@@ -1254,11 +1286,17 @@ public class TestRootedOzoneFileSystem {
       }
     }, 1000, 180000);
 
-    // This condition should pass after the checkpoint
-    Assert.assertTrue(getOMMetrics()
-        .getNumTrashRenames() > prevNumTrashRenames);
-    Assert.assertTrue(getOMMetrics()
-        .getNumTrashFilesRenames() > prevNumTrashFileRenames);
+    if (isBucketFSOptimized){
+      Assert.assertTrue(getOMMetrics()
+          .getNumTrashAtomicDirRenames() > prevNumTrashAtomicDirRenames);
+    } else {
+      // This condition should pass after the checkpoint
+      Assert.assertTrue(getOMMetrics()
+          .getNumTrashRenames() > prevNumTrashRenames);
+      // With new layout version, file renames wouldn't be counted
+      Assert.assertTrue(getOMMetrics()
+          .getNumTrashFilesRenames() > prevNumTrashFileRenames);
+    }
 
     // wait for deletion of checkpoint dir
     GenericTestUtils.waitFor(()-> {
@@ -1273,10 +1311,16 @@ public class TestRootedOzoneFileSystem {
     }, 1000, 120000);
 
     // This condition should succeed once the checkpoint directory is deleted
-    GenericTestUtils.waitFor(
-        () -> getOMMetrics().getNumTrashDeletes() > prevNumTrashDeletes
-            && getOMMetrics().getNumTrashFilesDeletes()
-            > prevNumTrashFileDeletes, 100, 180000);
+    if(isBucketFSOptimized){
+      GenericTestUtils.waitFor(
+          () -> getOMMetrics().getNumTrashAtomicDirDeletes() >
+              prevNumTrashAtomicDirDeletes, 100, 180000);
+    } else {
+      GenericTestUtils.waitFor(
+          () -> getOMMetrics().getNumTrashDeletes() > prevNumTrashDeletes
+              && getOMMetrics().getNumTrashFilesDeletes()
+              >= prevNumTrashFileDeletes, 100, 180000);
+    }
     // Cleanup
     ofs.delete(trashRoot, true);
     ofs.delete(trashRoot2, true);
@@ -1309,4 +1353,160 @@ public class TestRootedOzoneFileSystem {
     LambdaTestUtils.intercept(InvalidPathException.class, "Invalid path Name",
         () -> fs.create(path, false));
   }
+
+  @Test
+  public void testRenameDir() throws Exception {
+    final String dir = "dir1";
+    final Path source = new Path(getBucketPath(), dir);
+    final Path dest = new Path(source.toString() + ".renamed");
+    // Add a sub-dir to the directory to be moved.
+    final Path subdir = new Path(source, "sub_dir1");
+    getFs().mkdirs(subdir);
+    LOG.info("Created dir {}", subdir);
+    LOG.info("Will move {} to {}", source, dest);
+    getFs().rename(source, dest);
+    assertTrue("Directory rename failed", getFs().exists(dest));
+    // Verify that the subdir is also renamed i.e. keys corresponding to the
+    // sub-directories of the renamed directory have also been renamed.
+    assertTrue("Keys under the renamed directory not renamed",
+        getFs().exists(new Path(dest, "sub_dir1")));
+    // cleanup
+    getFs().delete(dest, true);
+  }
+
+  @Test
+  public void testRenameFile() throws Exception {
+    final String dir = "/dir" + new Random().nextInt(1000);
+    Path dirPath = new Path(getBucketPath() +dir);
+    getFs().mkdirs(dirPath);
+
+    Path file1Source = new Path(getBucketPath() + dir
+        + "/file1_Copy");
+    ContractTestUtils.touch(getFs(), file1Source);
+    Path file1Destin = new Path(getBucketPath() + dir + "/file1");
+    assertTrue("Renamed failed", getFs().rename(file1Source, file1Destin));
+    assertTrue("Renamed failed: /dir/file1", getFs().exists(file1Destin));
+    FileStatus[] fStatus = getFs().listStatus(dirPath);
+    assertEquals("Renamed failed", 1, fStatus.length);
+    getFs().delete(getBucketPath(), true);
+  }
+
+
+
+  /**
+   * Rename file to an existed directory.
+   */
+  @Test
+  public void testRenameFileToDir() throws Exception {
+    final String dir = "/dir" + new Random().nextInt(1000);
+    Path dirPath = new Path(getBucketPath() +dir);
+    getFs().mkdirs(dirPath);
+
+    Path file1Destin = new Path(getBucketPath() + dir  + "/file1");
+    ContractTestUtils.touch(getFs(), file1Destin);
+    Path abcRootPath = new Path(getBucketPath() + "/a/b/c");
+    getFs().mkdirs(abcRootPath);
+    assertTrue("Renamed failed", getFs().rename(file1Destin, abcRootPath));
+    assertTrue("Renamed filed: /a/b/c/file1", getFs().exists(new Path(
+        abcRootPath, "file1")));
+    getFs().delete(getBucketPath(), true);
+  }
+
+  /**
+   * Rename to the source's parent directory, it will succeed.
+   * 1. Rename from /root_dir/dir1/dir2 to /root_dir.
+   * Expected result : /root_dir/dir2
+   * <p>
+   * 2. Rename from /root_dir/dir1/file1 to /root_dir.
+   * Expected result : /root_dir/file1.
+   */
+  @Test
+  public void testRenameToParentDir() throws Exception {
+    final String root = "/root_dir";
+    final String dir1 = root + "/dir1";
+    final String dir2 = dir1 + "/dir2";
+    final Path dir2SourcePath = new Path(getBucketPath() + dir2);
+    getFs().mkdirs(dir2SourcePath);
+    final Path destRootPath = new Path(getBucketPath() + root);
+
+    Path file1Source = new Path(getBucketPath() + dir1 + "/file2");
+    ContractTestUtils.touch(getFs(), file1Source);
+
+    // rename source directory to its parent directory(destination).
+    assertTrue("Rename failed", getFs().rename(dir2SourcePath, destRootPath));
+    final Path expectedPathAfterRename =
+        new Path(getBucketPath() + root + "/dir2");
+    assertTrue("Rename failed",
+        getFs().exists(expectedPathAfterRename));
+
+    // rename source file to its parent directory(destination).
+    assertTrue("Rename failed", getFs().rename(file1Source, destRootPath));
+    final Path expectedFilePathAfterRename =
+        new Path(getBucketPath() + root + "/file2");
+    assertTrue("Rename failed",
+        getFs().exists(expectedFilePathAfterRename));
+    getFs().delete(getBucketPath(), true);
+  }
+
+  /**
+   *  Cannot rename a directory to its own subdirectory.
+   */
+  @Test
+  public void testRenameDirToItsOwnSubDir() throws Exception {
+    final String root = "/root";
+    final String dir1 = root + "/dir1";
+    final Path dir1Path = new Path(getBucketPath() + dir1);
+    // Add a sub-dir1 to the directory to be moved.
+    final Path subDir1 = new Path(dir1Path, "sub_dir1");
+    getFs().mkdirs(subDir1);
+    LOG.info("Created dir1 {}", subDir1);
+
+    final Path sourceRoot = new Path(getBucketPath() + root);
+    LOG.info("Rename op-> source:{} to destin:{}", sourceRoot, subDir1);
+    //  rename should fail and return false
+    try{
+      getFs().rename(sourceRoot, subDir1);
+      fail("Should throw exception : Cannot rename a directory to" +
+          " its own subdirectory");
+    } catch (IllegalArgumentException e){
+      //expected
+    }
+  }
+
+  /**
+   * Cleanup keyTable and directoryTable explicitly as FS delete operation
+   * is not yet supported.
+   * Fails if the (a) parent of dst does not exist or (b) parent is a file.
+   */
+  @Test
+  public void testRenameDestinationParentDoesntExist() throws Exception {
+    final String root = "/root_dir";
+    final String dir1 = root + "/dir1";
+    final String dir2 = dir1 + "/dir2";
+    final Path dir2SourcePath = new Path(getBucketPath() + dir2);
+    getFs().mkdirs(dir2SourcePath);
+    // (a) parent of dst does not exist.  /root_dir/b/c
+    final Path destinPath = new Path(getBucketPath()
+        + root + "/b/c");
+
+    // rename should throw exception
+    try {
+      getFs().rename(dir2SourcePath, destinPath);
+      fail("Should fail as parent of dst does not exist!");
+    } catch (FileNotFoundException fnfe){
+      //expected
+    }
+    // (b) parent of dst is a file. /root_dir/file1/c
+    Path filePath = new Path(getBucketPath() + root + "/file1");
+    ContractTestUtils.touch(getFs(), filePath);
+    Path newDestinPath = new Path(filePath, "c");
+    // rename shouldthrow exception
+    try{
+      getFs().rename(dir2SourcePath, newDestinPath);
+      fail("Should fail as parent of dst is a file!");
+    } catch (IOException e){
+      //expected
+    }
+  }
+
 }
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystemV1.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystemV1.java
new file mode 100644
index 0000000..b0b3152
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystemV1.java
@@ -0,0 +1,117 @@
+/**
+ * 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.fs.ozone;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+
+
+@RunWith(Parameterized.class)
+public class TestRootedOzoneFileSystemV1 extends TestRootedOzoneFileSystem {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRootedOzoneFileSystemV1.class);
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    return Arrays.asList(
+        new Object[]{true, true},
+        new Object[]{true, false},
+        new Object[]{false, true},
+        new Object[]{false, false});
+  }
+
+  public TestRootedOzoneFileSystemV1(boolean setDefaultFs,
+      boolean enableOMRatis) throws Exception {
+    super(setDefaultFs, enableOMRatis);
+  }
+
+  @BeforeClass
+  public static void init() throws Exception {
+    setIsBucketFSOptimized(true);
+    TestRootedOzoneFileSystem.init();
+  }
+
+  /**
+   * OFS: Test recursive listStatus on root and volume.
+   */
+  @Override
+  @Ignore("TODO:HDDS-4360")
+  public void testListStatusRootAndVolumeRecursive() throws IOException {
+  }
+
+  /**
+   * Cleanup keyTable and directoryTable explicitly as FS delete operation
+   * is not yet supported.
+   * Fails if the (a) parent of dst does not exist or (b) parent is a file.
+   */
+  @Override
+  @Test
+  public void testRenameDestinationParentDoesntExist() throws Exception {
+    final String root = "/root_dir";
+    final String dir1 = root + "/dir1";
+    final String dir2 = dir1 + "/dir2";
+    final Path dir2SourcePath = new Path(getBucketPath() + dir2);
+    getFs().mkdirs(dir2SourcePath);
+    // (a) parent of dst does not exist.  /root_dir/b/c
+    final Path destinPath = new Path(getBucketPath()
+        + root + "/b/c");
+
+    // rename should fail and return false
+    Assert.assertFalse(getFs().rename(dir2SourcePath, destinPath));
+    // (b) parent of dst is a file. /root_dir/file1/c
+    Path filePath = new Path(getBucketPath() + root + "/file1");
+    ContractTestUtils.touch(getFs(), filePath);
+    Path newDestinPath = new Path(filePath, "c");
+    // rename should fail and return false
+    Assert.assertFalse(getFs().rename(dir2SourcePath, newDestinPath));
+  }
+
+  /**
+   *  Cannot rename a directory to its own subdirectory.
+   */
+  @Override
+  @Test
+  public void testRenameDirToItsOwnSubDir() throws Exception {
+    final String root = "/root";
+    final String dir1 = root + "/dir1";
+    final Path dir1Path = new Path(getBucketPath() + dir1);
+    // Add a sub-dir1 to the directory to be moved.
+    final Path subDir1 = new Path(dir1Path, "sub_dir1");
+    getFs().mkdirs(subDir1);
+    LOG.info("Created dir1 {}", subDir1);
+
+    final Path sourceRoot = new Path(getBucketPath() + root);
+    LOG.info("Rename op-> source:{} to destin:{}", sourceRoot, subDir1);
+    //  rename should fail and return false
+    Assert.assertFalse(getFs().rename(sourceRoot, subDir1));
+  }
+
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestReadRetries.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestReadRetries.java
index a61c798..446ed5c 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestReadRetries.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestReadRetries.java
@@ -51,9 +51,9 @@ import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.OzoneManager;
-import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
 import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
 
 import org.junit.After;
@@ -68,7 +68,6 @@ import org.junit.rules.Timeout;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
-import static org.junit.Assert.assertEquals;
 
 import org.junit.rules.ExpectedException;
 
@@ -214,10 +213,7 @@ public class TestReadRetries {
     readKey(bucket, keyName, value);
 
     // read intermediate directory
-    verifyIntermediateDir(bucket, "a/b/c/");
     verifyIntermediateDir(bucket, "a/b/c");
-    verifyIntermediateDir(bucket, "/a/b/c/");
-    verifyIntermediateDir(bucket, "/a/b/c");
 
     // shutdown the second datanode
     datanodeDetails = datanodes.get(1);
@@ -241,15 +237,11 @@ public class TestReadRetries {
     factory.releaseClient(clientSpi, false);
   }
 
-  private void verifyIntermediateDir(OzoneBucket bucket,
-      String dir) throws IOException {
-    try {
-      bucket.getKey(dir);
-      fail("Should throw exception for directory listing");
-    } catch (OMException ome) {
-      // expected
-      assertEquals(OMException.ResultCodes.KEY_NOT_FOUND, ome.getResult());
-    }
+  private void verifyIntermediateDir(OzoneBucket bucket, String dir)
+      throws IOException {
+    OzoneFileStatus fileStatus = bucket.getFileStatus(dir);
+    Assert.assertTrue(fileStatus.isDirectory());
+    Assert.assertEquals(dir, fileStatus.getTrimmedName());
   }
 
   private void readKey(OzoneBucket bucket, String keyName, String data)
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
index f6ae506..24afc5f 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
@@ -717,7 +717,13 @@ public class KeyManagerImpl implements KeyManager {
     if (fileStatus == null) {
       return null;
     }
-    return fileStatus.isFile() ? fileStatus.getKeyInfo() : null;
+    // Appended trailing slash to represent directory to the user
+    if (fileStatus.isDirectory()) {
+      String keyPath = OzoneFSUtils.addTrailingSlashIfNeeded(
+          fileStatus.getKeyInfo().getKeyName());
+      fileStatus.getKeyInfo().setKeyName(keyPath);
+    }
+    return fileStatus.getKeyInfo();
   }
 
   private void addBlockToken4Read(OmKeyInfo value) throws IOException {
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java
index 7ce0a16..cce545f 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java
@@ -154,6 +154,8 @@ public class OMMetrics {
   private @Metric MutableCounterLong numTrashFails;
   private @Metric MutableCounterLong numTrashRootsEnqueued;
   private @Metric MutableCounterLong numTrashRootsProcessed;
+  private @Metric MutableCounterLong numTrashAtomicDirRenames;
+  private @Metric MutableCounterLong numTrashAtomicDirDeletes;
 
   private final DBCheckpointMetrics dbCheckpointMetrics;
 
@@ -909,6 +911,13 @@ public class OMMetrics {
     return numTrashFilesDeletes.value();
   }
 
+  public long getNumTrashAtomicDirRenames() {
+    return numTrashAtomicDirRenames.value();
+  }
+
+  public long getNumTrashAtomicDirDeletes() {
+    return numTrashAtomicDirDeletes.value();
+  }
 
   public void incNumTrashActiveCycles() {
     numTrashActiveCycles.incr();
@@ -926,6 +935,14 @@ public class OMMetrics {
     numTrashFails.incr();
   }
 
+  public void incNumTrashAtomicDirRenames() {
+    numTrashAtomicDirRenames.incr();
+  }
+
+  public void incNumTrashAtomicDirDeletes() {
+    numTrashAtomicDirDeletes.incr();
+  }
+
   public void unRegister() {
     MetricsSystem ms = DefaultMetricsSystem.instance();
     ms.unregisterSource(SOURCE_NAME);
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/TrashOzoneFileSystem.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/TrashOzoneFileSystem.java
index a9408a8..9ca673a 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/TrashOzoneFileSystem.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/TrashOzoneFileSystem.java
@@ -170,6 +170,7 @@ public class TrashOzoneFileSystem extends FileSystem {
   }
 
   private boolean renameV1(OFSPath srcPath, OFSPath dstPath) {
+    ozoneManager.getMetrics().incNumTrashAtomicDirRenames();
     OzoneManagerProtocolProtos.OMRequest omRequest =
         getRenameKeyRequest(srcPath, dstPath);
     try {
@@ -199,6 +200,7 @@ public class TrashOzoneFileSystem extends FileSystem {
   }
 
   private boolean deleteV1(OFSPath srcPath) {
+    ozoneManager.getMetrics().incNumTrashAtomicDirDeletes();
     OzoneManagerProtocolProtos.OMRequest omRequest =
         getDeleteKeyRequest(srcPath);
     try {
diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneFileSystem.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneFileSystem.java
index 55edd00..08d2008 100644
--- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneFileSystem.java
+++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneFileSystem.java
@@ -308,6 +308,10 @@ public class BasicRootedOzoneFileSystem extends FileSystem {
     if (!ofsSrc.isInSameBucketAs(ofsDst)) {
       throw new IOException("Cannot rename a key to a different bucket");
     }
+    OzoneBucket bucket = adapterImpl.getBucket(ofsSrc, false);
+    if (OzoneFSUtils.isFSOptimizedBucket(bucket.getMetadata())) {
+      return renameV1(bucket, ofsSrc, ofsDst);
+    }
 
     // Cannot rename a directory to its own subdirectory
     Path dstParent = dst.getParent();
@@ -385,6 +389,29 @@ public class BasicRootedOzoneFileSystem extends FileSystem {
     return result;
   }
 
+  private boolean renameV1(OzoneBucket bucket,
+      OFSPath srcPath, OFSPath dstPath) throws IOException {
+    // construct src and dst key paths
+    String srcKeyPath = srcPath.getNonKeyPathNoPrefixDelim() +
+        OZONE_URI_DELIMITER + srcPath.getKeyName();
+    String dstKeyPath = dstPath.getNonKeyPathNoPrefixDelim() +
+        OZONE_URI_DELIMITER + dstPath.getKeyName();
+    try {
+      adapterImpl.rename(bucket, srcKeyPath, dstKeyPath);
+    } catch (OMException ome) {
+      LOG.error("rename key failed: {}. source:{}, destin:{}",
+          ome.getMessage(), srcKeyPath, dstKeyPath);
+      if (OMException.ResultCodes.KEY_ALREADY_EXISTS == ome.getResult() ||
+          OMException.ResultCodes.KEY_RENAME_ERROR  == ome.getResult() ||
+          OMException.ResultCodes.KEY_NOT_FOUND == ome.getResult()) {
+        return false;
+      } else {
+        throw ome;
+      }
+    }
+    return true;
+  }
+
   /**
    * Intercept rename to trash calls from TrashPolicyDefault.
    */
@@ -501,6 +528,16 @@ public class BasicRootedOzoneFileSystem extends FileSystem {
         return false;
       }
 
+
+      if (!ofsPath.isVolume() && !ofsPath.isBucket()) {
+        OzoneBucket bucket = adapterImpl.getBucket(ofsPath, false);
+        if (OzoneFSUtils.isFSOptimizedBucket(bucket.getMetadata())) {
+          String ofsKeyPath = ofsPath.getNonKeyPathNoPrefixDelim() +
+              OZONE_URI_DELIMITER + ofsPath.getKeyName();
+          return adapterImpl.deleteObject(ofsKeyPath, recursive);
+        }
+      }
+
       // Handle delete volume
       if (ofsPath.isVolume()) {
         String volumeName = ofsPath.getVolumeName();

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