Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -719,7 +719,7 @@ protected InodeTree(final Configuration config, final String viewName,

if (isMergeSlashConfigured) {
Preconditions.checkNotNull(mergeSlashTarget);
root = new INodeLink<T>(mountTableName, ugi,
root = new INodeLink<T>("/", ugi,
initAndGetTargetFs(), mergeSlashTarget);
mountPoints.add(new MountPoint<T>("/", (INodeLink<T>) root));
rootFallbackLink = null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,9 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileSystemTestHelper;
import org.apache.hadoop.fs.FsConstants;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
Expand All @@ -42,6 +44,7 @@
import org.slf4j.LoggerFactory;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;

Expand Down Expand Up @@ -228,4 +231,136 @@ public void testChildFileSystems() throws Exception {
assertEquals(DistributedFileSystem.class, childFs[0].getClass(),
"Unexpected child filesystem!");
}

@Test
public void testListStatusReturnsCorrectPaths() throws Exception {
String clusterName = "ClusterLMS1";
URI viewFsUri = new URI(FsConstants.VIEWFS_SCHEME, clusterName,
"/", null, null);

Configuration conf = new Configuration();
ConfigUtil.addLinkMergeSlash(conf, clusterName,
fsDefault.getUri());

FileSystem vfs = FileSystem.get(viewFsUri, conf);

// Create test directory structure
Path testDir = new Path("/testListStatus");
Path subDir1 = new Path(testDir, "subdir1");
Path subDir2 = new Path(testDir, "subdir2");
Path file1 = new Path(testDir, "file1.txt");

try {
fsDefault.mkdirs(subDir1);
fsDefault.mkdirs(subDir2);
fsDefault.create(file1).close();

// Test listStatus returns correct ViewFS paths
FileStatus[] statuses = vfs.listStatus(testDir);
assertEquals(3, statuses.length, "Should have 3 items");

for (FileStatus status : statuses) {
Path path = status.getPath();
// Path should be viewfs:// URI, not hdfs://
assertEquals(FsConstants.VIEWFS_SCHEME, path.toUri().getScheme(),
"Scheme should be viewfs");
assertEquals(clusterName, path.toUri().getAuthority(),
"Authority should match cluster name");
// Path should start with /testListStatus, not contain cluster name
assertTrue(path.toString().startsWith("viewfs://" + clusterName + "/testListStatus/"),
"Path should be absolute and correct: " + path);
assertFalse(path.toString().contains(clusterName + "/" + clusterName),
"Path should not contain duplicate cluster name: " + path);
}
} finally {
fsDefault.delete(testDir, true);
vfs.close();
}
}

@Test
public void testListLocatedStatusReturnsCorrectPaths() throws Exception {
String clusterName = "ClusterLMS1";
URI viewFsUri = new URI(FsConstants.VIEWFS_SCHEME, clusterName,
"/", null, null);

Configuration conf = new Configuration();
ConfigUtil.addLinkMergeSlash(conf, clusterName,
fsDefault.getUri());

FileSystem vfs = FileSystem.get(viewFsUri, conf);

// Create test directory structure
Path testDir = new Path("/testListLocatedStatus");
Path file1 = new Path(testDir, "file1.txt");
Path file2 = new Path(testDir, "file2.txt");

try {
fsDefault.mkdirs(testDir);
fsDefault.create(file1).close();
fsDefault.create(file2).close();

// Test listLocatedStatus returns correct ViewFS paths
RemoteIterator<LocatedFileStatus> iter = vfs.listLocatedStatus(testDir);
int count = 0;
while (iter.hasNext()) {
LocatedFileStatus status = iter.next();
Path path = status.getPath();
count++;

// Path should be viewfs:// URI
assertEquals(FsConstants.VIEWFS_SCHEME, path.toUri().getScheme(),
"Scheme should be viewfs");
assertEquals(clusterName, path.toUri().getAuthority(),
"Authority should match cluster name");
// Path should be absolute and not contain relative path issues
assertTrue(path.toString().startsWith("viewfs://" + clusterName + "/testListLocatedStatus/"),
"Path should start correctly: " + path);
assertFalse(path.toString().contains(clusterName + "/" + clusterName),
"Path should not contain duplicate cluster name: " + path);
}
assertEquals(2, count, "Should have 2 files");
} finally {
fsDefault.delete(testDir, true);
vfs.close();
}
}

@Test
public void testResolvedPathIsAbsolute() throws Exception {
String clusterName = "ClusterLMS1";
URI viewFsUri = new URI(FsConstants.VIEWFS_SCHEME, clusterName,
"/", null, null);

Configuration conf = new Configuration();
ConfigUtil.addLinkMergeSlash(conf, clusterName,
fsDefault.getUri());

FileSystem vfs = FileSystem.get(viewFsUri, conf);

// Create nested directory structure
Path baseDir = new Path("/user/history/done");
Path yearDir = new Path(baseDir, "2021");

try {
fsDefault.mkdirs(yearDir);

// This is the exact scenario that caused the bug
FileStatus[] statuses = vfs.listStatus(baseDir);
assertEquals(1, statuses.length, "Should have 1 directory");

Path resultPath = statuses[0].getPath();
String expectedPath = "viewfs://" + clusterName + "/user/history/done/2021";
assertEquals(expectedPath, resultPath.toString(),
"Path should be correctly formed");

// Verify path is absolute (starts with /)
assertTrue(resultPath.toUri().getPath().startsWith("/"),
"Path should be absolute");
} finally {
fsDefault.delete(new Path("/user"), true);
vfs.close();
}
}
}

Loading