[HUDI 1308] Harden RFC-15 Implementation based on production testing (#2441)
Addresses leaks, perf degradation observed during testing. These were regressions from the original rfc-15 PoC implementation. * Pass a single instance of HoodieTableMetadata everywhere * Fix tests and add config for enabling metrics - Removed special casing of assumeDatePartitioning inside FSUtils#getAllPartitionPaths() - Consequently, IOException is never thrown and many files had to be adjusted - More diligent handling of open file handles in metadata table - Added config for controlling reuse of connections - Added config for turning off fallback to listing, so we can see tests fail - Changed all ipf listing code to cache/amortize the open/close for better performance - Timelineserver also reuses connections, for better performance - Without timelineserver, when metadata table is opened from executors, reuse is not allowed - HoodieMetadataConfig passed into HoodieTableMetadata#create as argument. - Fix TestHoodieBackedTableMetadata#testSync
This commit is contained in:
@@ -41,8 +41,10 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieClusteringException;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieMetadataException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.SparkHoodieIndex;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
import org.apache.hudi.metrics.DistributedRegistry;
|
||||
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
@@ -395,7 +397,11 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
|
||||
@Override
|
||||
public void syncTableMetadata() {
|
||||
// Open up the metadata table again, for syncing
|
||||
SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context);
|
||||
try (HoodieTableMetadataWriter writer = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context)) {
|
||||
LOG.info("Successfully synced to metadata table");
|
||||
} catch (Exception e) {
|
||||
throw new HoodieMetadataException("Error syncing to metadata table.", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -28,7 +28,6 @@ import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.index.HoodieIndexUtils;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
@@ -36,7 +35,6 @@ import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.Optional;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
@@ -62,13 +60,8 @@ public class SparkHoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends
|
||||
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final HoodieEngineContext context,
|
||||
final HoodieTable hoodieTable) {
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
try {
|
||||
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(context, metaClient.getFs(), metaClient.getBasePath(),
|
||||
config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning());
|
||||
return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to load all partitions", e);
|
||||
}
|
||||
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath());
|
||||
return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -31,14 +31,12 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.index.HoodieIndexUtils;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
@@ -103,14 +101,9 @@ public class SparkHoodieGlobalSimpleIndex<T extends HoodieRecordPayload> extends
|
||||
protected List<Pair<String, HoodieBaseFile>> getAllBaseFilesInTable(final HoodieEngineContext context,
|
||||
final HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
try {
|
||||
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(context, metaClient.getFs(), metaClient.getBasePath(),
|
||||
config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning());
|
||||
// Obtain the latest data files from all the partitions.
|
||||
return getLatestBaseFilesForAllPartitions(allPartitionPaths, context, hoodieTable);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to load all partitions", e);
|
||||
}
|
||||
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath());
|
||||
// Obtain the latest data files from all the partitions.
|
||||
return getLatestBaseFilesForAllPartitions(allPartitionPaths, context, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -99,8 +99,6 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
|
||||
@Override
|
||||
protected void commit(List<HoodieRecord> records, String partitionName, String instantTime) {
|
||||
ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled");
|
||||
metadata.closeReaders();
|
||||
|
||||
JavaRDD<HoodieRecord> recordRDD = prepRecords(records, partitionName);
|
||||
|
||||
try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) {
|
||||
|
||||
@@ -26,13 +26,11 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@@ -49,20 +47,13 @@ public class SparkInsertOverwriteTableCommitActionExecutor<T extends HoodieRecor
|
||||
@Override
|
||||
protected Map<String, List<String>> getPartitionToReplacedFileIds(JavaRDD<WriteStatus> writeStatuses) {
|
||||
Map<String, List<String>> partitionToExistingFileIds = new HashMap<>();
|
||||
try {
|
||||
List<String> partitionPaths = FSUtils.getAllPartitionPaths(context, table.getMetaClient().getFs(),
|
||||
table.getMetaClient().getBasePath(), config.useFileListingMetadata(), config.getFileListingMetadataVerify(),
|
||||
false);
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
if (partitionPaths != null && partitionPaths.size() > 0) {
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions");
|
||||
JavaRDD<String> partitionPathRdd = jsc.parallelize(partitionPaths, partitionPaths.size());
|
||||
partitionToExistingFileIds = partitionPathRdd.mapToPair(
|
||||
partitionPath -> new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCommitException("In InsertOverwriteTable action failed to get existing fileIds of all partition "
|
||||
+ config.getBasePath() + " at time " + instantTime, e);
|
||||
List<String> partitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath());
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
if (partitionPaths != null && partitionPaths.size() > 0) {
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions");
|
||||
JavaRDD<String> partitionPathRdd = jsc.parallelize(partitionPaths, partitionPaths.size());
|
||||
partitionToExistingFileIds = partitionPathRdd.mapToPair(
|
||||
partitionPath -> new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap();
|
||||
}
|
||||
return partitionToExistingFileIds;
|
||||
}
|
||||
|
||||
@@ -195,8 +195,7 @@ public class HoodieSparkMergeOnReadTableCompactor<T extends HoodieRecordPayload>
|
||||
// TODO - rollback any compactions in flight
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
LOG.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime);
|
||||
List<String> partitionPaths = FSUtils.getAllPartitionPaths(context, metaClient.getFs(), metaClient.getBasePath(),
|
||||
config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning());
|
||||
List<String> partitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath());
|
||||
|
||||
// filter the partition paths if needed to reduce list status
|
||||
partitionPaths = config.getCompactionStrategy().filterPartitionPaths(config, partitionPaths);
|
||||
|
||||
@@ -67,7 +67,7 @@ public class SparkCopyOnWriteRollbackActionExecutor<T extends HoodieRecordPayloa
|
||||
@Override
|
||||
protected List<HoodieRollbackStat> executeRollbackUsingFileListing(HoodieInstant instantToRollback) {
|
||||
List<ListingBasedRollbackRequest> rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context,
|
||||
table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), config);
|
||||
table.getMetaClient().getBasePath(), config);
|
||||
return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, instantToRollback, rollbackRequests);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -92,8 +92,7 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler {
|
||||
// generate rollback stats
|
||||
List<ListingBasedRollbackRequest> rollbackRequests;
|
||||
if (table.getMetaClient().getTableType() == HoodieTableType.COPY_ON_WRITE) {
|
||||
rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context, table.getMetaClient().getFs(),
|
||||
table.getMetaClient().getBasePath(), table.getConfig());
|
||||
rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context, table.getMetaClient().getBasePath(), table.getConfig());
|
||||
} else {
|
||||
rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(commitInstantOpt.get(), table, context);
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user