1
0

[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:
vinoth chandar
2021-01-19 21:20:28 -08:00
committed by GitHub
parent e23967b9e9
commit 5ca0625b27
55 changed files with 767 additions and 570 deletions

View File

@@ -18,9 +18,6 @@
package org.apache.hudi.sync.common;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieTableType;
@@ -29,12 +26,13 @@ import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.TimelineUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.schema.MessageType;
import java.io.IOException;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
@@ -128,13 +126,9 @@ public abstract class AbstractSyncHoodieClient {
public List<String> getPartitionsWrittenToSince(Option<String> lastCommitTimeSynced) {
if (!lastCommitTimeSynced.isPresent()) {
LOG.info("Last commit time synced is not known, listing all partitions in " + basePath + ",FS :" + fs);
try {
HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf());
return FSUtils.getAllPartitionPaths(engineContext, fs, basePath, useFileListingFromMetadata, verifyMetadataFileListing,
assumeDatePartitioning);
} catch (IOException e) {
throw new HoodieIOException("Failed to list all partitions in " + basePath, e);
}
HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf());
return FSUtils.getAllPartitionPaths(engineContext, basePath, useFileListingFromMetadata, verifyMetadataFileListing,
assumeDatePartitioning);
} else {
LOG.info("Last commit time synced is " + lastCommitTimeSynced.get() + ", Getting commits since then");
return TimelineUtils.getPartitionsWritten(metaClient.getActiveTimeline().getCommitsTimeline()