1
0

[HUDI-1138] Add timeline-server-based marker file strategy for improving marker-related latency (#3233)

- Can be enabled for cloud stores like S3. Not supported for hdfs yet, due to partial write failures.
This commit is contained in:
Y Ethan Guo
2021-08-11 08:48:13 -07:00
committed by GitHub
parent 29332498af
commit 4783176554
52 changed files with 2144 additions and 353 deletions

View File

@@ -30,6 +30,8 @@ import org.apache.hudi.common.table.view.SyncableFileSystemView;
import org.apache.hudi.common.table.view.TestHoodieTableFileSystemView;
import org.apache.hudi.timeline.service.TimelineService;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -51,7 +53,8 @@ public class TestRemoteHoodieTableFileSystemView extends TestHoodieTableFileSyst
HoodieLocalEngineContext localEngineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf());
try {
server = new TimelineService(0,
server = new TimelineService(localEngineContext, new Configuration(),
TimelineService.Config.builder().serverPort(0).build(), FileSystem.get(new Configuration()),
FileSystemViewManager.createViewManager(localEngineContext, metadataConfig, sConf, commonConfig));
server.startService();
} catch (Exception ex) {