You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by ar...@apache.org on 2020/07/27 15:38:18 UTC

[hadoop-ozone] branch master updated: HDDS-4026. Dir rename failed when sets 'ozone.om.enable.filesystem.paths' to true (#1256)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new ddd3211  HDDS-4026. Dir rename failed when sets 'ozone.om.enable.filesystem.paths' to true (#1256)
ddd3211 is described below

commit ddd3211757ad67d1574e0e9e983b182681050a7c
Author: Bharat Viswanadham <bh...@apache.org>
AuthorDate: Mon Jul 27 08:38:07 2020 -0700

    HDDS-4026. Dir rename failed when sets 'ozone.om.enable.filesystem.paths' to true (#1256)
---
 .../fs/ozone/TestOzoneFSWithObjectStoreCreate.java | 99 ++++++++++++++++++++++
 .../hadoop/fs/ozone/TestOzoneFileInterfaces.java   | 11 ++-
 .../hadoop/fs/ozone/TestOzoneFileSystem.java       | 94 +++++++++++++++++++-
 .../ozone/om/request/key/OMKeyRenameRequest.java   | 12 +--
 4 files changed, 204 insertions(+), 12 deletions(-)

diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSWithObjectStoreCreate.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSWithObjectStoreCreate.java
index b872a3d..c4e5435 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSWithObjectStoreCreate.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSWithObjectStoreCreate.java
@@ -35,10 +35,15 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
 
+import java.io.FileNotFoundException;
+import java.io.IOException;
 import java.net.URI;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 
 import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_SCHEME;
+import static org.junit.Assert.fail;
 
 /**
  * Class tests create with object store and getFileStatus.
@@ -120,6 +125,100 @@ public class TestOzoneFSWithObjectStoreCreate {
 
   }
 
+
+  @Test
+  public void testObjectStoreCreateWithO3fs() throws Exception {
+    OzoneVolume ozoneVolume =
+        cluster.getRpcClient().getObjectStore().getVolume(volumeName);
+
+    OzoneBucket ozoneBucket = ozoneVolume.getBucket(bucketName);
+
+
+    // Use ObjectStore API to create keys. This similates how s3 create keys.
+    String parentDir = "/dir1/dir2/dir3/dir4/";
+
+
+    List<String> keys = new ArrayList<>();
+    keys.add("/dir1");
+    keys.add("/dir1/dir2");
+    keys.add("/dir1/dir2/dir3");
+    keys.add("/dir1/dir2/dir3/dir4/");
+    for (int i=1; i <= 3; i++) {
+      int length = 10;
+      String fileName = parentDir.concat("/file" + i + "/");
+      keys.add(fileName);
+      OzoneOutputStream ozoneOutputStream =
+          ozoneBucket.createKey(fileName, length);
+      byte[] b = new byte[10];
+      Arrays.fill(b, (byte)96);
+      ozoneOutputStream.write(b);
+      ozoneOutputStream.close();
+    }
+
+    // check
+    for (int i=1; i <= 3; i++) {
+      String fileName = parentDir.concat("/file" + i + "/");
+      Path p = new Path(fileName);
+      Assert.assertTrue(o3fs.getFileStatus(p).isFile());
+      checkAncestors(p);
+    }
+
+    // Delete keys with object store api delete
+    for (int i = 1; i <= 3; i++) {
+      String fileName = parentDir.concat("/file" + i + "/");
+      ozoneBucket.deleteKey(fileName);
+    }
+
+
+    // Delete parent dir via o3fs.
+    boolean result = o3fs.delete(new Path("/dir1"), true);
+    Assert.assertTrue(result);
+
+    // No Key should exist.
+    for(String key : keys) {
+      checkPath(new Path(key));
+    }
+
+
+    for (int i=1; i <= 3; i++) {
+      int length = 10;
+      String fileName = parentDir.concat("/file" + i + "/");
+      OzoneOutputStream ozoneOutputStream =
+          ozoneBucket.createKey(fileName, length);
+      byte[] b = new byte[10];
+      Arrays.fill(b, (byte)96);
+      ozoneOutputStream.write(b);
+      ozoneOutputStream.close();
+    }
+
+    o3fs.mkdirs(new Path("/dest"));
+    o3fs.rename(new Path("/dir1"), new Path("/dest"));
+
+    // No source Key should exist.
+    for(String key : keys) {
+      checkPath(new Path(key));
+    }
+
+    // check dest path.
+    for (int i=1; i <= 3; i++) {
+      String fileName = "/dest/".concat(parentDir.concat("/file" + i + "/"));
+      Path p = new Path(fileName);
+      Assert.assertTrue(o3fs.getFileStatus(p).isFile());
+      checkAncestors(p);
+    }
+
+  }
+
+  private void checkPath(Path path) {
+    try {
+      o3fs.getFileStatus(path);
+      fail("testObjectStoreCreateWithO3fs failed for Path" + path);
+    } catch (IOException ex) {
+      Assert.assertTrue(ex instanceof FileNotFoundException);
+      Assert.assertTrue(ex.getMessage().contains("No such file or directory"));
+    }
+  }
+
   private void checkAncestors(Path p) throws Exception {
     p = p.getParent();
     while(p.getParent() != null) {
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileInterfaces.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileInterfaces.java
index 15a8fc6..06d1bd3 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileInterfaces.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileInterfaces.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.TestDataUtil;
 import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.OMMetrics;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
@@ -97,7 +98,8 @@ public class TestOzoneFileInterfaces {
    */
   @Parameters
   public static Collection<Object[]> data() {
-    return Arrays.asList(new Object[][] {{false, true}, {true, false}});
+    return Arrays.asList(new Object[][] {{false, true, true},
+        {true, false, false}});
   }
 
   private boolean setDefaultFs;
@@ -118,10 +120,13 @@ public class TestOzoneFileInterfaces {
 
   private OMMetrics omMetrics;
 
+  private boolean enableFileSystemPaths;
+
   public TestOzoneFileInterfaces(boolean setDefaultFs,
-      boolean useAbsolutePath) {
+      boolean useAbsolutePath, boolean enabledFileSystemPaths) {
     this.setDefaultFs = setDefaultFs;
     this.useAbsolutePath = useAbsolutePath;
+    this.enableFileSystemPaths = enabledFileSystemPaths;
     GlobalStorageStatistics.INSTANCE.reset();
   }
 
@@ -131,6 +136,8 @@ public class TestOzoneFileInterfaces {
     bucketName = RandomStringUtils.randomAlphabetic(10).toLowerCase();
 
     OzoneConfiguration conf = new OzoneConfiguration();
+    conf.setBoolean(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS,
+        enableFileSystemPaths);
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(3)
         .build();
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java
index d4a2a46..cdfe0cf 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java
@@ -18,8 +18,10 @@
 
 package org.apache.hadoop.fs.ozone;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Set;
@@ -42,6 +44,7 @@ import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.TestDataUtil;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
 import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -66,14 +69,25 @@ import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
  * Ozone file system tests that are not covered by contract tests.
  */
+@RunWith(Parameterized.class)
 public class TestOzoneFileSystem {
 
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    return Arrays.asList(new Object[]{true}, new Object[]{false});
+  }
+
+  public TestOzoneFileSystem(boolean setDefaultFs) {
+    this.enabledFileSystemPaths = setDefaultFs;
+  }
   /**
    * Set a timeout for each test.
    */
@@ -83,6 +97,8 @@ public class TestOzoneFileSystem {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestOzoneFileSystem.class);
 
+  private boolean enabledFileSystemPaths;
+
   private MiniOzoneCluster cluster;
   private FileSystem fs;
   private OzoneFileSystem o3fs;
@@ -215,6 +231,8 @@ public class TestOzoneFileSystem {
     testRenameDir();
     testSeekOnFileLength();
     testDeleteRoot();
+
+    testRecursiveDelete();
   }
 
   @After
@@ -229,6 +247,8 @@ public class TestOzoneFileSystem {
       throws IOException, TimeoutException, InterruptedException {
     OzoneConfiguration conf = new OzoneConfiguration();
     conf.setInt(FS_TRASH_INTERVAL_KEY, 1);
+    conf.setBoolean(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS,
+        enabledFileSystemPaths);
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(3)
         .build();
@@ -296,7 +316,7 @@ public class TestOzoneFileSystem {
     }
 
     // Delete the child key
-    fs.delete(child, false);
+    fs.delete(child, true);
 
     // Deleting the only child should create the parent dir key if it does
     // not exist
@@ -305,6 +325,78 @@ public class TestOzoneFileSystem {
     assertEquals(parentKey, parentKeyInfo.getName());
   }
 
+
+  private void testRecursiveDelete() throws Exception {
+    Path grandparent = new Path("/gdir1");
+
+    for (int i = 1; i <= 10; i++) {
+      Path parent = new Path(grandparent, "pdir" +i);
+      Path child = new Path(parent, "child");
+      ContractTestUtils.touch(fs, child);
+    }
+
+    // Delete the grandparent, which should delete all keys.
+    fs.delete(grandparent, true);
+
+    checkPath(grandparent);
+
+    for (int i = 1; i <= 10; i++) {
+      Path parent = new Path(grandparent, "dir" +i);
+      Path child = new Path(parent, "child");
+      checkPath(parent);
+      checkPath(child);
+    }
+
+
+    Path level0 = new Path("/level0");
+
+    for (int i = 1; i <= 3; i++) {
+      Path level1 = new Path(level0, "level" +i);
+      Path level2 = new Path(level1, "level" +i);
+      Path level1File = new Path(level1, "file1");
+      Path level2File = new Path(level2, "file1");
+      ContractTestUtils.touch(fs, level1File);
+      ContractTestUtils.touch(fs, level2File);
+    }
+
+    // Delete at sub directory level.
+    for (int i = 1; i <= 3; i++) {
+      Path level1 = new Path(level0, "level" +i);
+      Path level2 = new Path(level1, "level" +i);
+      fs.delete(level2, true);
+      fs.delete(level1, true);
+    }
+
+
+    // Delete level0 finally.
+    fs.delete(grandparent, true);
+
+    // Check if it exists or not.
+    checkPath(grandparent);
+
+    for (int i = 1; i <= 3; i++) {
+      Path level1 = new Path(level0, "level" +i);
+      Path level2 = new Path(level1, "level" +i);
+      Path level1File = new Path(level1, "file1");
+      Path level2File = new Path(level2, "file1");
+      checkPath(level1);
+      checkPath(level2);
+      checkPath(level1File);
+      checkPath(level2File);
+    }
+
+  }
+
+  private void checkPath(Path path) {
+    try {
+      fs.getFileStatus(path);
+      fail("testRecursiveDelete failed");
+    } catch (IOException ex) {
+      Assert.assertTrue(ex instanceof FileNotFoundException);
+      Assert.assertTrue(ex.getMessage().contains("No such file or directory"));
+    }
+  }
+
   private void testFileDelete() throws Exception {
     Path grandparent = new Path("/testBatchDelete");
     Path parent = new Path(grandparent, "parent");
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequest.java
index e6e9839..4e7c05c 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequest.java
@@ -87,19 +87,13 @@ public class OMKeyRenameRequest extends OMKeyRequest {
 
     KeyArgs renameKeyArgs = renameKeyRequest.getKeyArgs();
 
-    // Set modification time and normalize key if needed.
+    // Set modification time.
     KeyArgs.Builder newKeyArgs = renameKeyArgs.toBuilder()
-            .setModificationTime(Time.now())
-        .setKeyName(validateAndNormalizeKey(
-            ozoneManager.getEnableFileSystemPaths(),
-            renameKeyArgs.getKeyName()));
+            .setModificationTime(Time.now());
 
     return getOmRequest().toBuilder()
         .setRenameKeyRequest(renameKeyRequest.toBuilder()
-            .setKeyArgs(newKeyArgs)
-            .setToKeyName(validateAndNormalizeKey(
-                ozoneManager.getEnableFileSystemPaths(),
-                renameKeyRequest.getToKeyName())))
+            .setKeyArgs(newKeyArgs))
         .setUserInfo(getUserInfo()).build();
 
   }


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