From 1ff7a65b9ffdea9652d1b5b6ddc249548f827cea Mon Sep 17 00:00:00 2001 From: Neil Date: Thu, 20 Apr 2023 17:49:18 +0800 Subject: [PATCH] HDFS-16954. RBF: The operation of renaming a multi-subcluster directory to a single-cluster directory should throw ioexception. (#5483). Contributed by Max Xie. Reviewed-by: Inigo Goiri Signed-off-by: Ayush Saxena --- .../router/RouterClientProtocol.java | 10 +++++ ...MultipleDestinationMountTableResolver.java | 41 +++++++++++++++++++ 2 files changed, 51 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java index ee8ae5885a..34e3666a94 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java @@ -614,6 +614,11 @@ public boolean rename(final String src, final String dst) new Class[] {String.class, String.class}, new RemoteParam(), dstParam); if (isMultiDestDirectory(src)) { + if (locs.size() != srcLocations.size()) { + throw new IOException("Rename of " + src + " to " + dst + " is not" + + " allowed. The number of remote locations for both source and" + + " target should be same."); + } return rpcClient.invokeAll(locs, method); } else { return rpcClient.invokeSequential(locs, method, Boolean.class, @@ -641,6 +646,11 @@ public void rename2(final String src, final String dst, new Class[] {String.class, String.class, options.getClass()}, new RemoteParam(), dstParam, options); if (isMultiDestDirectory(src)) { + if (locs.size() != srcLocations.size()) { + throw new IOException("Rename of " + src + " to " + dst + " is not" + + " allowed. The number of remote locations for both source and" + + " target should be same."); + } rpcClient.invokeConcurrent(locs, method); } else { rpcClient.invokeSequential(locs, method, null, null); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCMultipleDestinationMountTableResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCMultipleDestinationMountTableResolver.java index b05337443f..cbc11b27b2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCMultipleDestinationMountTableResolver.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCMultipleDestinationMountTableResolver.java @@ -720,6 +720,47 @@ public void testWriteWithUnavailableSubCluster() throws IOException { } } + /** + * Test rename a dir from src dir (mapped to both ns0 and ns1) to ns0. + */ + @Test + public void testRenameWithMultiDestinations() throws Exception { + //create a mount point with multiple destinations + String srcDir = "/mount-source-dir"; + Path path = new Path(srcDir); + Map destMap = new HashMap<>(); + destMap.put("ns0", srcDir); + destMap.put("ns1", srcDir); + nnFs0.mkdirs(path); + nnFs1.mkdirs(path); + MountTable addEntry = + MountTable.newInstance(srcDir, destMap); + addEntry.setDestOrder(DestinationOrder.RANDOM); + assertTrue(addMountTable(addEntry)); + + //create a mount point with a single destinations ns0 + String targetDir = "/ns0_test"; + nnFs0.mkdirs(new Path(targetDir)); + MountTable addDstEntry = MountTable.newInstance(targetDir, + Collections.singletonMap("ns0", targetDir)); + assertTrue(addMountTable(addDstEntry)); + + //mkdir sub dirs in srcDir mapping ns0 & ns1 + routerFs.mkdirs(new Path(srcDir + "/dir1")); + routerFs.mkdirs(new Path(srcDir + "/dir1/dir_1")); + routerFs.mkdirs(new Path(srcDir + "/dir1/dir_2")); + routerFs.mkdirs(new Path(targetDir)); + + //try to rename sub dir in srcDir (mapping to ns0 & ns1) to targetDir + // (mapping ns0) + LambdaTestUtils.intercept(IOException.class, "The number of" + + " remote locations for both source and target should be same.", + () -> { + routerFs.rename(new Path(srcDir + "/dir1/dir_1"), + new Path(targetDir)); + }); + } + /** * Test to verify rename operation on directories in case of multiple * destinations.