Skip to content

Commit c37f01d

Browse files
authored
HDFS-16724. RBF should support get the information about ancestor mount points (#4719)
1 parent a3b1baf commit c37f01d

File tree

4 files changed

+202
-14
lines changed

4 files changed

+202
-14
lines changed
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,33 @@
1+
/**
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.hadoop.hdfs.server.federation.router;
19+
20+
import java.io.IOException;
21+
22+
/**
23+
* This exception is thrown when can not get any mount point for the input path.
24+
* RBF cannot forward any requests for the path.
25+
*/
26+
public class NoLocationException extends IOException {
27+
28+
private static final long serialVersionUID = 1L;
29+
30+
public NoLocationException(String path, Class<?> t) {
31+
super("Cannot find locations for " + path + " in " + t.getSimpleName());
32+
}
33+
}

hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java

Lines changed: 21 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -935,19 +935,22 @@ public BatchedDirectoryListing getBatchedListing(String[] srcs,
935935
public HdfsFileStatus getFileInfo(String src) throws IOException {
936936
rpcServer.checkOperation(NameNode.OperationCategory.READ);
937937

938-
final List<RemoteLocation> locations =
939-
rpcServer.getLocationsForPath(src, false, false);
940-
RemoteMethod method = new RemoteMethod("getFileInfo",
941-
new Class<?>[] {String.class}, new RemoteParam());
942-
943938
HdfsFileStatus ret = null;
944-
// If it's a directory, we check in all locations
945-
if (rpcServer.isPathAll(src)) {
946-
ret = getFileInfoAll(locations, method);
947-
} else {
948-
// Check for file information sequentially
949-
ret = rpcClient.invokeSequential(
950-
locations, method, HdfsFileStatus.class, null);
939+
IOException noLocationException = null;
940+
try {
941+
final List<RemoteLocation> locations = rpcServer.getLocationsForPath(src, false, false);
942+
RemoteMethod method = new RemoteMethod("getFileInfo",
943+
new Class<?>[] {String.class}, new RemoteParam());
944+
945+
// If it's a directory, we check in all locations
946+
if (rpcServer.isPathAll(src)) {
947+
ret = getFileInfoAll(locations, method);
948+
} else {
949+
// Check for file information sequentially
950+
ret = rpcClient.invokeSequential(locations, method, HdfsFileStatus.class, null);
951+
}
952+
} catch (NoLocationException | RouterResolveException e) {
953+
noLocationException = e;
951954
}
952955

953956
// If there is no real path, check mount points
@@ -966,6 +969,12 @@ public HdfsFileStatus getFileInfo(String src) throws IOException {
966969
}
967970
}
968971

972+
// Can't find mount point for path and the path didn't contain any sub monit points,
973+
// throw the NoLocationException to client.
974+
if (ret == null && noLocationException != null) {
975+
throw noLocationException;
976+
}
977+
969978
return ret;
970979
}
971980

hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1731,8 +1731,7 @@ protected List<RemoteLocation> getLocationsForPath(String path,
17311731
final PathLocation location =
17321732
this.subclusterResolver.getDestinationForPath(path);
17331733
if (location == null) {
1734-
throw new IOException("Cannot find locations for " + path + " in " +
1735-
this.subclusterResolver.getClass().getSimpleName());
1734+
throw new NoLocationException(path, this.subclusterResolver.getClass());
17361735
}
17371736

17381737
// We may block some write operations
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,147 @@
1+
/**
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.hadoop.hdfs.server.federation.router;
19+
20+
import org.apache.hadoop.conf.Configuration;
21+
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
22+
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
23+
import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.RouterContext;
24+
import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
25+
import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster;
26+
import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
27+
import org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver;
28+
import org.apache.hadoop.hdfs.server.federation.resolver.RouterResolveException;
29+
import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
30+
import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
31+
import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
32+
import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
33+
import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryRequest;
34+
import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
35+
import org.apache.hadoop.test.LambdaTestUtils;
36+
import org.junit.After;
37+
import org.junit.AfterClass;
38+
import org.junit.BeforeClass;
39+
import org.junit.Test;
40+
41+
import java.io.IOException;
42+
import java.util.Collections;
43+
44+
import static org.junit.Assert.assertEquals;
45+
import static org.junit.Assert.assertNotNull;
46+
import static org.junit.Assert.assertTrue;
47+
48+
/**
49+
* Test a router end-to-end including the MountTable without default nameservice.
50+
*/
51+
public class TestRouterMountTableWithoutDefaultNS {
52+
private static StateStoreDFSCluster cluster;
53+
private static RouterContext routerContext;
54+
private static MountTableResolver mountTable;
55+
private static ClientProtocol routerProtocol;
56+
57+
@BeforeClass
58+
public static void globalSetUp() throws Exception {
59+
// Build and start a federated cluster
60+
cluster = new StateStoreDFSCluster(false, 2);
61+
Configuration conf = new RouterConfigBuilder()
62+
.stateStore()
63+
.admin()
64+
.rpc()
65+
.build();
66+
conf.setInt(RBFConfigKeys.DFS_ROUTER_ADMIN_MAX_COMPONENT_LENGTH_KEY, 20);
67+
conf.setBoolean(RBFConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE_ENABLE, false);
68+
cluster.addRouterOverrides(conf);
69+
cluster.startCluster();
70+
cluster.startRouters();
71+
cluster.waitClusterUp();
72+
73+
// Get the end points
74+
routerContext = cluster.getRandomRouter();
75+
Router router = routerContext.getRouter();
76+
routerProtocol = routerContext.getClient().getNamenode();
77+
mountTable = (MountTableResolver) router.getSubclusterResolver();
78+
}
79+
80+
@AfterClass
81+
public static void tearDown() {
82+
if (cluster != null) {
83+
cluster.stopRouter(routerContext);
84+
cluster.shutdown();
85+
cluster = null;
86+
}
87+
}
88+
89+
@After
90+
public void clearMountTable() throws IOException {
91+
RouterClient client = routerContext.getAdminClient();
92+
MountTableManager mountTableManager = client.getMountTableManager();
93+
GetMountTableEntriesRequest req1 = GetMountTableEntriesRequest.newInstance("/");
94+
GetMountTableEntriesResponse response = mountTableManager.getMountTableEntries(req1);
95+
for (MountTable entry : response.getEntries()) {
96+
RemoveMountTableEntryRequest req2 =
97+
RemoveMountTableEntryRequest.newInstance(entry.getSourcePath());
98+
mountTableManager.removeMountTableEntry(req2);
99+
}
100+
}
101+
102+
/**
103+
* Add a mount table entry to the mount table through the admin API.
104+
* @param entry Mount table entry to add.
105+
* @return If it was succesfully added.
106+
* @throws IOException Problems adding entries.
107+
*/
108+
private boolean addMountTable(final MountTable entry) throws IOException {
109+
RouterClient client = routerContext.getAdminClient();
110+
MountTableManager mountTableManager = client.getMountTableManager();
111+
AddMountTableEntryRequest addRequest = AddMountTableEntryRequest.newInstance(entry);
112+
AddMountTableEntryResponse addResponse = mountTableManager.addMountTableEntry(addRequest);
113+
114+
// Reload the Router cache
115+
mountTable.loadCache(true);
116+
117+
return addResponse.getStatus();
118+
}
119+
120+
/**
121+
* Verify that RBF that disable default nameservice should support
122+
* get information about ancestor mount points.
123+
*/
124+
@Test
125+
public void testGetFileInfoWithSubMountPoint() throws IOException {
126+
MountTable addEntry = MountTable.newInstance("/testdir/1",
127+
Collections.singletonMap("ns0", "/testdir/1"));
128+
assertTrue(addMountTable(addEntry));
129+
HdfsFileStatus finfo = routerProtocol.getFileInfo("/testdir");
130+
assertNotNull(finfo);
131+
assertEquals("supergroup", finfo.getGroup());
132+
assertTrue(finfo.isDirectory());
133+
}
134+
135+
/**
136+
* Verify that RBF doesn't support get the file information
137+
* with no location and sub mount points.
138+
*/
139+
@Test
140+
public void testGetFileInfoWithoutSubMountPoint() throws Exception {
141+
MountTable addEntry = MountTable.newInstance("/testdir/1",
142+
Collections.singletonMap("ns0", "/testdir/1"));
143+
assertTrue(addMountTable(addEntry));
144+
LambdaTestUtils.intercept(RouterResolveException.class,
145+
() -> routerContext.getRouter().getRpcServer().getFileInfo("/testdir2"));
146+
}
147+
}

0 commit comments

Comments
 (0)