You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ha...@apache.org on 2018/10/18 00:52:16 UTC

hadoop git commit: HDDS-670. Fix OzoneFS directory rename.

Repository: hadoop
Updated Branches:
  refs/heads/trunk 4c4364ddd -> 9146d33e1


HDDS-670. Fix OzoneFS directory rename.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9146d33e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9146d33e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9146d33e

Branch: refs/heads/trunk
Commit: 9146d33e1843524885938f60c77b47d4f52e80fb
Parents: 4c4364d
Author: Hanisha Koneru <ha...@apache.org>
Authored: Wed Oct 17 17:51:29 2018 -0700
Committer: Hanisha Koneru <ha...@apache.org>
Committed: Wed Oct 17 17:51:29 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/fs/ozone/OzoneFileSystem.java |   2 +-
 .../hadoop/fs/ozone/TestOzoneFsRenameDir.java   | 121 +++++++++++++++++++
 2 files changed, 122 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9146d33e/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
index d52f3b7..50a63b5 100644
--- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
+++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
@@ -349,7 +349,7 @@ public class OzoneFileSystem extends FileSystem {
     }
 
     if (srcStatus.isDirectory()) {
-      if (dst.toString().startsWith(src.toString())) {
+      if (dst.toString().startsWith(src.toString() + OZONE_URI_DELIMITER)) {
         LOG.trace("Cannot rename a directory to a subdirectory of self");
         return false;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9146d33e/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsRenameDir.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsRenameDir.java b/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsRenameDir.java
new file mode 100644
index 0000000..0c639e7
--- /dev/null
+++ b/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsRenameDir.java
@@ -0,0 +1,121 @@
+/**
+ * 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.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.web.handlers.BucketArgs;
+import org.apache.hadoop.ozone.web.handlers.UserArgs;
+import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
+import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
+import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Unit Test for verifying directory rename operation through OzoneFS.
+ */
+public class TestOzoneFsRenameDir {
+  public static final Logger LOG = LoggerFactory.getLogger(
+      TestOzoneFsRenameDir.class);
+
+  private MiniOzoneCluster cluster = null;
+  private OzoneConfiguration conf = null;
+  private static StorageHandler storageHandler;
+  private static FileSystem fs;
+
+  @Before
+  public void init() throws Exception {
+    conf = new OzoneConfiguration();
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(1)
+        .build();
+    cluster.waitForClusterToBeReady();
+    storageHandler =
+        new ObjectStoreHandler(conf).getStorageHandler();
+
+    // create a volume and a bucket to be used by OzoneFileSystem
+    String userName = "user" + RandomStringUtils.randomNumeric(5);
+    String adminName = "admin" + RandomStringUtils.randomNumeric(5);
+    String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
+    String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
+    UserArgs userArgs = new UserArgs(null, OzoneUtils.getRequestID(),
+        null, null, null, null);
+    VolumeArgs volumeArgs = new VolumeArgs(volumeName, userArgs);
+    volumeArgs.setUserName(userName);
+    volumeArgs.setAdminName(adminName);
+    storageHandler.createVolume(volumeArgs);
+    BucketArgs bucketArgs = new BucketArgs(volumeName, bucketName, userArgs);
+    storageHandler.createBucket(bucketArgs);
+
+    // Fetch the host and port for File System init
+    DatanodeDetails datanodeDetails = cluster.getHddsDatanodes().get(0)
+        .getDatanodeDetails();
+
+    // Set the fs.defaultFS and start the filesystem
+    String uri = String.format("%s://%s.%s/",
+        OzoneConsts.OZONE_URI_SCHEME, bucketName, volumeName);
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, uri);
+    fs =  FileSystem.get(conf);
+    LOG.info("fs.defaultFS=" + fs.getUri());
+  }
+
+  @After
+  public void teardown() {
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }
+  }
+
+  /**
+   * Tests directory rename opertion through OzoneFS.
+   */
+  @Test(timeout=300_000)
+  public void testRenameDir() throws IOException {
+    final String dir = "/root_dir/dir1";
+    final Path source = new Path(fs.getUri().toString() + 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");
+    fs.mkdirs(subdir);
+    LOG.info("Created dir {}", subdir);
+    LOG.info("Will move {} to {}", source, dest);
+    fs.rename(source, dest);
+    assertTrue("Directory rename failed", fs.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 direcotry not renamed",
+        fs.exists(new Path(dest, "sub_dir1")));
+  }
+}


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