1
0

[HUDI-1479] Use HoodieEngineContext to parallelize fetching of partiton paths (#2417)

* [HUDI-1479] Use HoodieEngineContext to parallelize fetching of partition paths

* Adding testClass for FileSystemBackedTableMetadata

Co-authored-by: Nishith Agarwal <nagarwal@uber.com>
This commit is contained in:
Udit Mehrotra
2021-01-10 21:19:52 -08:00
committed by GitHub
parent 23e93d05c0
commit 7ce3ac778e
38 changed files with 509 additions and 100 deletions

View File

@@ -101,7 +101,7 @@ public class TestClientRollback extends HoodieClientTestBase {
assertNoWriteErrors(statuses);
HoodieWriteConfig config = getConfig();
List<String> partitionPaths =
FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), config.useFileListingMetadata(),
FSUtils.getAllPartitionPaths(context, fs, cfg.getBasePath(), config.useFileListingMetadata(),
config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning());
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieSparkTable table = HoodieSparkTable.create(getConfig(), context, metaClient);

View File

@@ -23,6 +23,7 @@ import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.metrics.Registry;
import org.apache.hudi.common.model.FileSlice;
@@ -454,7 +455,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
records = dataGen.generateUniqueUpdates(newCommitTime, 5);
writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
assertNoWriteErrors(writeStatuses);
assertFalse(metadata(client).isInSync());
assertTrue(metadata(client).isInSync());
// updates and inserts
newCommitTime = HoodieActiveTimeline.createNewInstantTime();
@@ -462,21 +463,21 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
records = dataGen.generateUpdates(newCommitTime, 10);
writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
assertNoWriteErrors(writeStatuses);
assertFalse(metadata(client).isInSync());
assertTrue(metadata(client).isInSync());
// Compaction
if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) {
newCommitTime = HoodieActiveTimeline.createNewInstantTime();
client.scheduleCompactionAtInstant(newCommitTime, Option.empty());
client.compact(newCommitTime);
assertFalse(metadata(client).isInSync());
assertTrue(metadata(client).isInSync());
}
// Savepoint
restoreToInstant = newCommitTime;
if (metaClient.getTableType() == HoodieTableType.COPY_ON_WRITE) {
client.savepoint("hoodie", "metadata test");
assertFalse(metadata(client).isInSync());
assertTrue(metadata(client).isInSync());
}
// Deletes
@@ -485,12 +486,12 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
JavaRDD<HoodieKey> deleteKeys = jsc.parallelize(records, 1).map(r -> r.getKey());
client.startCommitWithTime(newCommitTime);
client.delete(deleteKeys, newCommitTime);
assertFalse(metadata(client).isInSync());
assertTrue(metadata(client).isInSync());
// Clean
newCommitTime = HoodieActiveTimeline.createNewInstantTime();
client.clean(newCommitTime);
assertFalse(metadata(client).isInSync());
assertTrue(metadata(client).isInSync());
// updates
newCommitTime = HoodieActiveTimeline.createNewInstantTime();
@@ -498,8 +499,8 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
records = dataGen.generateUniqueUpdates(newCommitTime, 10);
writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
assertNoWriteErrors(writeStatuses);
assertFalse(metadata(client).isInSync());
assertTrue(metadata(client).isInSync());
// insert overwrite to test replacecommit
newCommitTime = HoodieActiveTimeline.createNewInstantTime();
client.startCommitWithTime(newCommitTime, HoodieTimeline.REPLACE_COMMIT_ACTION);
@@ -507,7 +508,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
HoodieWriteResult replaceResult = client.insertOverwrite(jsc.parallelize(records, 1), newCommitTime);
writeStatuses = replaceResult.getWriteStatuses().collect();
assertNoWriteErrors(writeStatuses);
assertFalse(metadata(client).isInSync());
assertTrue(metadata(client).isInSync());
}
// Enable metadata table and ensure it is synced
@@ -757,7 +758,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
private void validateMetadata(SparkRDDWriteClient client) throws IOException {
HoodieWriteConfig config = client.getConfig();
HoodieBackedTableMetadata tableMetadata = metadata(client);
HoodieTableMetadata tableMetadata = metadata(client);
assertNotNull(tableMetadata, "MetadataReader should have been initialized");
if (!config.useFileListingMetadata()) {
return;
@@ -767,7 +768,9 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
// Partitions should match
List<String> fsPartitions = FSUtils.getAllFoldersWithPartitionMetaFile(fs, basePath);
FileSystemBackedTableMetadata fsBackedTableMetadata = new FileSystemBackedTableMetadata(engineContext,
new SerializableConfiguration(hadoopConf), config.getBasePath(), config.shouldAssumeDatePartitioning());
List<String> fsPartitions = fsBackedTableMetadata.getAllPartitionPaths();
List<String> metadataPartitions = tableMetadata.getAllPartitionPaths();
Collections.sort(fsPartitions);
@@ -849,7 +852,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
// Metadata table has a fixed number of partitions
// Cannot use FSUtils.getAllFoldersWithPartitionMetaFile for this as that function filters all directory
// in the .hoodie folder.
List<String> metadataTablePartitions = FSUtils.getAllPartitionPaths(fs, HoodieTableMetadata.getMetadataTableBasePath(basePath),
List<String> metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, fs, HoodieTableMetadata.getMetadataTableBasePath(basePath),
false, false, false);
assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size());
@@ -873,10 +876,11 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
.create(hadoopConf, client.getConfig(), new HoodieSparkEngineContext(jsc));
}
private HoodieBackedTableMetadata metadata(SparkRDDWriteClient client) {
private HoodieTableMetadata metadata(SparkRDDWriteClient client) {
HoodieWriteConfig clientConfig = client.getConfig();
return (HoodieBackedTableMetadata) HoodieTableMetadata.create(hadoopConf, clientConfig.getBasePath(), clientConfig.getSpillableMapBasePath(),
clientConfig.useFileListingMetadata(), clientConfig.getFileListingMetadataVerify(), false, clientConfig.shouldAssumeDatePartitioning());
return HoodieTableMetadata.create(client.getEngineContext(), clientConfig.getBasePath(),
clientConfig.getSpillableMapBasePath(), clientConfig.useFileListingMetadata(),
clientConfig.getFileListingMetadataVerify(), false, clientConfig.shouldAssumeDatePartitioning());
}
// TODO: this can be moved to TestHarness after merge from master
@@ -912,4 +916,4 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
protected HoodieTableType getTableType() {
return tableType;
}
}
}