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

@@ -20,7 +20,9 @@ package org.apache.hudi.utilities.perf;
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.engine.HoodieEngineContext;
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -37,6 +39,7 @@ import com.beust.jcommander.Parameter;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.Snapshot;
import com.codahale.metrics.UniformReservoir;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -71,7 +74,12 @@ public class TimelineServerPerf implements Serializable {
public TimelineServerPerf(Config cfg) throws IOException {
this.cfg = cfg;
useExternalTimelineServer = (cfg.serverHost != null);
this.timelineServer = new TimelineService(cfg.getTimelinServerConfig());
TimelineService.Config timelineServiceConf = cfg.getTimelinServerConfig();
this.timelineServer = new TimelineService(
new HoodieLocalEngineContext(FSUtils.prepareHadoopConf(new Configuration())),
new Configuration(), timelineServiceConf, FileSystem.get(new Configuration()),
TimelineService.buildFileSystemViewManager(timelineServiceConf,
new SerializableConfiguration(FSUtils.prepareHadoopConf(new Configuration()))));
}
private void setHostAddrFromSparkConf(SparkConf sparkConf) {