[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:
@@ -18,7 +18,8 @@
|
||||
|
||||
package org.apache.hudi.timeline.service.functional;
|
||||
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
@@ -44,9 +45,12 @@ public class TestRemoteHoodieTableFileSystemView extends TestHoodieTableFileSyst
|
||||
protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline) {
|
||||
FileSystemViewStorageConfig sConf =
|
||||
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.SPILLABLE_DISK).build();
|
||||
HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().build();
|
||||
HoodieLocalEngineContext localEngineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf());
|
||||
|
||||
try {
|
||||
server = new TimelineService(0,
|
||||
FileSystemViewManager.createViewManager(new SerializableConfiguration(metaClient.getHadoopConf()), sConf));
|
||||
FileSystemViewManager.createViewManager(localEngineContext, metadataConfig, sConf));
|
||||
server.startService();
|
||||
} catch (Exception ex) {
|
||||
throw new RuntimeException(ex);
|
||||
|
||||
Reference in New Issue
Block a user