1
0

[HUDI-1371] [HUDI-1893] Support metadata based listing for Spark DataSource and Spark SQL (#2893)

This commit is contained in:
Udit Mehrotra
2021-08-03 14:47:40 -07:00
committed by GitHub
parent 245e1fd17d
commit 1ff2d3459a
13 changed files with 383 additions and 125 deletions

View File

@@ -18,8 +18,6 @@
package org.apache.hudi.metadata;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.client.HoodieWriteResult;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
@@ -55,6 +53,9 @@ import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
@@ -71,6 +72,7 @@ import java.nio.file.Paths;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
@@ -227,9 +229,14 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
"Must not contain the filtered directory " + filteredDirectoryThree);
FileStatus[] statuses = metadata(client).getAllFilesInPartition(new Path(basePath, "p1"));
assertTrue(statuses.length == 2);
assertEquals(2, statuses.length);
statuses = metadata(client).getAllFilesInPartition(new Path(basePath, "p2"));
assertTrue(statuses.length == 5);
assertEquals(5, statuses.length);
Map<String, FileStatus[]> partitionsToFilesMap = metadata(client).getAllFilesInPartitions(
Arrays.asList(basePath + "/p1", basePath + "/p2"));
assertEquals(2, partitionsToFilesMap.size());
assertEquals(2, partitionsToFilesMap.get(basePath + "/p1").length);
assertEquals(5, partitionsToFilesMap.get(basePath + "/p2").length);
}
}
@@ -881,6 +888,10 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTable table = HoodieSparkTable.create(config, engineContext);
TableFileSystemView tableView = table.getHoodieView();
List<String> fullPartitionPaths = fsPartitions.stream().map(partition -> basePath + "/" + partition).collect(Collectors.toList());
Map<String, FileStatus[]> partitionToFilesMap = tableMetadata.getAllFilesInPartitions(fullPartitionPaths);
assertEquals(fsPartitions.size(), partitionToFilesMap.size());
fsPartitions.forEach(partition -> {
try {
Path partitionPath;
@@ -899,6 +910,8 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
Collections.sort(fsFileNames);
Collections.sort(metadataFilenames);
assertEquals(fsStatuses.length, partitionToFilesMap.get(basePath + "/" + partition).length);
// File sizes should be valid
Arrays.stream(metaStatuses).forEach(s -> assertTrue(s.getLen() > 0));