Skip to content

Commit 09a90a1

Browse files
committed
HDFS-17632. Resolve to only one file in case of overlapping resolution
1 parent f30dd43 commit 09a90a1

File tree

2 files changed

+30
-10
lines changed

2 files changed

+30
-10
lines changed

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

Lines changed: 25 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -84,6 +84,7 @@
8484
import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
8585
import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
8686
import org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver;
87+
import org.apache.hadoop.hdfs.server.federation.resolver.PathLocation;
8788
import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
8889
import org.apache.hadoop.hdfs.server.federation.resolver.RouterResolveException;
8990
import org.apache.hadoop.hdfs.server.federation.router.async.AsyncErasureCoding;
@@ -2001,8 +2002,9 @@ public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
20012002
minOfMax = Math.min(minOfMax, max);
20022003
}
20032004
// Concatenate all entries into one result, sorted by inodeId
2004-
List<OpenFileEntry> routerEntries = new ArrayList<>();
20052005
boolean hasMore = false;
2006+
Map<String, OpenFileEntry> routerEntries = new HashMap<>();
2007+
Map<String, RemoteLocation> resolvedPaths = new HashMap<>();
20062008
for (Map.Entry<RemoteLocation, BatchedEntries> entry : results.entrySet()) {
20072009
BatchedEntries nsEntries = entry.getValue();
20082010
hasMore |= nsEntries.hasMore();
@@ -2012,16 +2014,33 @@ public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
20122014
hasMore = true;
20132015
break;
20142016
}
2015-
RemoteLocation loc = entry.getKey();
2016-
String routerPath = ofe.getFilePath().replaceFirst(loc.getDest(), loc.getSrc());;
2017+
RemoteLocation remoteLoc = entry.getKey();
2018+
String routerPath = ofe.getFilePath().replaceFirst(remoteLoc.getDest(), remoteLoc.getSrc());;
20172019
OpenFileEntry newEntry =
20182020
new OpenFileEntry(ofe.getId(), routerPath, ofe.getClientName(),
20192021
ofe.getClientMachine());
2020-
routerEntries.add(newEntry);
2022+
// An existing file already resolves to the same path.
2023+
// Resolve according to mount table and keep the best path.
2024+
if (resolvedPaths.containsKey(routerPath)) {
2025+
PathLocation pathLoc = subclusterResolver.getDestinationForPath(routerPath);
2026+
List<String> namespaces = pathLoc.getDestinations().stream().map(
2027+
RemoteLocation::getNameserviceId).collect(
2028+
Collectors.toList());
2029+
int existingIdx = namespaces.indexOf(resolvedPaths.get(routerPath).getNameserviceId());
2030+
int currentIdx = namespaces.indexOf(remoteLoc.getNameserviceId());
2031+
if (currentIdx < existingIdx && currentIdx != -1) {
2032+
routerEntries.put(routerPath, newEntry);
2033+
resolvedPaths.put(routerPath, remoteLoc);
2034+
}
2035+
} else {
2036+
routerEntries.put(routerPath, newEntry);
2037+
resolvedPaths.put(routerPath, remoteLoc);
2038+
}
20212039
}
20222040
}
2023-
routerEntries.sort(Comparator.comparingLong(OpenFileEntry::getId));
2024-
return new BatchedRemoteIterator.BatchedListEntries<>(routerEntries, hasMore);
2041+
List<OpenFileEntry> entryList = new ArrayList<>(routerEntries.values());
2042+
entryList.sort(Comparator.comparingLong(OpenFileEntry::getId));
2043+
return new BatchedRemoteIterator.BatchedListEntries<>(entryList, hasMore);
20252044
}
20262045

20272046
@Override

hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterListOpenFiles.java

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -157,11 +157,12 @@ public void testMultipleDestinations() throws Exception {
157157
result =
158158
routerProtocol.listOpenFiles(0, EnumSet.of(OpenFilesIterator.OpenFilesType.ALL_OPEN_FILES),
159159
testPath);
160-
// Should list both entries
161-
assertEquals(2, result.size());
162-
assertEquals(testPath + "/file2", result.get(0).getFilePath());
163-
assertEquals(testPath + "/file2", result.get(1).getFilePath());
160+
// Should list one file only
161+
assertEquals(1, result.size());
162+
assertEquals(routerClient.getFileInfo(TEST_DESTINATION_PATH + "/file2").getFileId(),
163+
result.get(0).getId());
164164
ite = routerClient.listOpenFiles(testPath);
165+
routerClient.open(testPath + "/file2");
165166
while (ite.hasNext()) {
166167
OpenFileEntry ofe = ite.next();
167168
assertTrue(ofe.getFilePath().equals(testPath + "/file2"));

0 commit comments

Comments
 (0)