Browse Source

[Fix-16331] Fix the resource file cannot be displayed hierarchically (#16799)

dev
xiangzihao 1 week ago committed by GitHub
parent
commit
854098bb44
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 12
      dolphinscheduler-storage-plugin/dolphinscheduler-storage-hdfs/src/main/java/org/apache/dolphinscheduler/plugin/storage/hdfs/HdfsStorageOperator.java
  2. 12
      dolphinscheduler-storage-plugin/dolphinscheduler-storage-hdfs/src/test/java/org/apache/dolphinscheduler/plugin/storage/hdfs/LocalStorageOperatorTest.java

12
dolphinscheduler-storage-plugin/dolphinscheduler-storage-hdfs/src/main/java/org/apache/dolphinscheduler/plugin/storage/hdfs/HdfsStorageOperator.java

@ -32,9 +32,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.security.UserGroupInformation;
@ -235,10 +233,12 @@ public class HdfsStorageOperator extends AbstractStorageOperator implements Clos
if (!fs.exists(path)) {
continue;
}
RemoteIterator<LocatedFileStatus> remoteIterator = fs.listFiles(path, true);
while (remoteIterator.hasNext()) {
LocatedFileStatus locatedFileStatus = remoteIterator.next();
result.add(transformFileStatusToResourceMetadata(locatedFileStatus));
FileStatus[] fileStatuses = fs.listStatus(path);
for (FileStatus fileStatus : fileStatuses) {
if (fileStatus.isDirectory()) {
foldersToFetch.addLast(fileStatus.getPath().toString());
}
result.add(transformFileStatusToResourceMetadata(fileStatus));
}
}
return result;

12
dolphinscheduler-storage-plugin/dolphinscheduler-storage-hdfs/src/test/java/org/apache/dolphinscheduler/plugin/storage/hdfs/LocalStorageOperatorTest.java

@ -296,10 +296,14 @@ class LocalStorageOperatorTest {
String resourceFileAbsolutePath = "file:" + baseDir;
List<StorageEntity> storageEntities =
storageOperator.listFileStorageEntityRecursively(resourceFileAbsolutePath);
assertThat(storageEntities.size()).isEqualTo(1);
StorageEntity storageEntity2 = storageEntities.get(0);
assertThat(storageEntity2.getFullName()).isEqualTo("file:" + Paths.get(baseDir, "sqlDirectory", "demo.sql"));
assertThat(storageEntities.size()).isEqualTo(3);
StorageEntity storageEntity2 = storageEntities.stream()
.filter(storageEntity -> storageEntity.getFileName().equals("demo.sql"))
.findFirst()
.get();
assertThat(storageEntity2.getFullName())
.isEqualTo("file:" + Paths.get(baseDir, "sqlDirectory", "demo.sql"));
assertThat(storageEntity2.getFileName()).isEqualTo("demo.sql");
assertThat(storageEntity2.getPfullName()).isEqualTo("file:" + Paths.get(baseDir, "sqlDirectory"));
assertThat(storageEntity2.isDirectory()).isFalse();

Loading…
Cancel
Save