1
0

[HUDI-1315] Adding builder for HoodieTableMetaClient initialization (#2534)

This commit is contained in:
Sivabalan Narayanan
2021-02-19 20:54:26 -05:00
committed by GitHub
parent 0d91c451b0
commit c9fcf964b2
64 changed files with 241 additions and 203 deletions

View File

@@ -276,7 +276,7 @@ public class HiveIncrementalPuller {
if (!fs.exists(new Path(targetDataPath)) || !fs.exists(new Path(targetDataPath + "/.hoodie"))) {
return "0";
}
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), targetDataPath);
HoodieTableMetaClient metadata = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(targetDataPath).build();
Option<HoodieInstant> lastCommit =
metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant();
@@ -309,7 +309,7 @@ public class HiveIncrementalPuller {
}
private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) {
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), sourceTableLocation);
HoodieTableMetaClient metadata = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(sourceTableLocation).build();
List<String> commitsToSync = metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()
.findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());

View File

@@ -137,7 +137,7 @@ public class HoodieClusteringJob {
}
private String getSchemaFromLatestInstant() throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.basePath, true);
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(cfg.basePath).setLoadActiveTimelineOnLoad(true).build();
TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient);
if (metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 0) {
throw new HoodieException("Cannot run clustering without any completed commits");

View File

@@ -60,7 +60,7 @@ public class HoodieCompactionAdminTool {
* Executes one of compaction admin operations.
*/
public void run(JavaSparkContext jsc) throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.basePath);
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(cfg.basePath).build();
try (CompactionAdminClient admin = new CompactionAdminClient(new HoodieSparkEngineContext(jsc), cfg.basePath)) {
final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());
if (cfg.outputPath != null && fs.exists(new Path(cfg.outputPath))) {

View File

@@ -83,7 +83,7 @@ public class HoodieSnapshotCopier implements Serializable {
final boolean verifyMetadataFileListing) throws IOException {
FileSystem fs = FSUtils.getFs(baseDir, jsc.hadoopConfiguration());
final SerializableConfiguration serConf = new SerializableConfiguration(jsc.hadoopConfiguration());
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs.getConf(), baseDir);
final HoodieTableMetaClient tableMetadata = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(baseDir).build();
final BaseFileOnlyView fsView = new HoodieTableFileSystemView(tableMetadata,
tableMetadata.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedInstants());
HoodieEngineContext context = new HoodieSparkEngineContext(jsc);

View File

@@ -148,7 +148,7 @@ public class HoodieSnapshotExporter {
}
private Option<String> getLatestCommitTimestamp(FileSystem fs, Config cfg) {
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs.getConf(), cfg.sourceBasePath);
final HoodieTableMetaClient tableMetadata = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(cfg.sourceBasePath).build();
Option<HoodieInstant> latestCommit = tableMetadata.getActiveTimeline().getCommitsAndCompactionTimeline()
.filterCompletedInstants().lastInstant();
return latestCommit.isPresent() ? Option.of(latestCommit.get().getTimestamp()) : Option.empty();
@@ -259,7 +259,7 @@ public class HoodieSnapshotExporter {
private BaseFileOnlyView getBaseFileOnlyView(JavaSparkContext jsc, Config cfg) {
FileSystem fs = FSUtils.getFs(cfg.sourceBasePath, jsc.hadoopConfiguration());
HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs.getConf(), cfg.sourceBasePath);
HoodieTableMetaClient tableMetadata = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(cfg.sourceBasePath).build();
return new HoodieTableFileSystemView(tableMetadata, tableMetadata
.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedInstants());
}

View File

@@ -44,7 +44,7 @@ public class InitialCheckpointFromAnotherHoodieTimelineProvider extends InitialC
@Override
public void init(Configuration config) throws HoodieException {
super.init(config);
this.anotherDsHoodieMetaclient = new HoodieTableMetaClient(config, path.toString());
this.anotherDsHoodieMetaclient = HoodieTableMetaClient.builder().setConf(config).setBasePath(path.toString()).build();
}
@Override

View File

@@ -223,8 +223,7 @@ public class DeltaSync implements Serializable {
*/
public void refreshTimeline() throws IOException {
if (fs.exists(new Path(cfg.targetBasePath))) {
HoodieTableMetaClient meta = new HoodieTableMetaClient(new Configuration(fs.getConf()), cfg.targetBasePath,
cfg.payloadClassName);
HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(new Configuration(fs.getConf())).setBasePath(cfg.targetBasePath).setPayloadClassName(cfg.payloadClassName).build();
switch (meta.getTableType()) {
case COPY_ON_WRITE:
this.commitTimelineOpt = Option.of(meta.getActiveTimeline().getCommitTimeline().filterCompletedInstants());

View File

@@ -528,7 +528,7 @@ public class HoodieDeltaStreamer implements Serializable {
if (fs.exists(new Path(cfg.targetBasePath))) {
HoodieTableMetaClient meta =
new HoodieTableMetaClient(new Configuration(fs.getConf()), cfg.targetBasePath, false);
HoodieTableMetaClient.builder().setConf(new Configuration(fs.getConf())).setBasePath(cfg.targetBasePath).setLoadActiveTimelineOnLoad(false).build();
tableType = meta.getTableType();
// This will guarantee there is no surprise with table type
ValidationUtils.checkArgument(tableType.equals(HoodieTableType.valueOf(cfg.tableType)),
@@ -636,7 +636,7 @@ public class HoodieDeltaStreamer implements Serializable {
asyncCompactService = Option.ofNullable(new SparkAsyncCompactService(new HoodieSparkEngineContext(jssc), writeClient));
// Enqueue existing pending compactions first
HoodieTableMetaClient meta =
new HoodieTableMetaClient(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath, true);
HoodieTableMetaClient.builder().setConf(new Configuration(jssc.hadoopConfiguration())).setBasePath(cfg.targetBasePath).setLoadActiveTimelineOnLoad(true).build();
List<HoodieInstant> pending = CompactionUtils.getPendingCompactionInstantTimes(meta);
pending.forEach(hoodieInstant -> asyncCompactService.get().enqueuePendingCompaction(hoodieInstant));
asyncCompactService.get().start((error) -> {

View File

@@ -100,7 +100,7 @@ public class TimelineServerPerf implements Serializable {
this.hostAddr = cfg.serverHost;
}
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(timelineServer.getConf(), cfg.basePath, true);
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(timelineServer.getConf()).setBasePath(cfg.basePath).setLoadActiveTimelineOnLoad(true).build();
SyncableFileSystemView fsView = new RemoteHoodieTableFileSystemView(this.hostAddr, cfg.serverPort, metaClient);
String reportDir = cfg.reportDir;

View File

@@ -58,7 +58,7 @@ public class IncrSourceHelper {
int numInstantsPerFetch, Option<String> beginInstant, boolean readLatestOnMissingBeginInstant) {
ValidationUtils.checkArgument(numInstantsPerFetch > 0,
"Make sure the config hoodie.deltastreamer.source.hoodieincr.num_instants is set to a positive value");
HoodieTableMetaClient srcMetaClient = new HoodieTableMetaClient(jssc.hadoopConfiguration(), srcBasePath, true);
HoodieTableMetaClient srcMetaClient = HoodieTableMetaClient.builder().setConf(jssc.hadoopConfiguration()).setBasePath(srcBasePath).setLoadActiveTimelineOnLoad(true).build();
final HoodieTimeline activeCommitTimeline =
srcMetaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();

View File

@@ -363,7 +363,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
}
static void assertAtleastNCompactionCommits(int minExpected, String tablePath, FileSystem fs) {
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), tablePath);
HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build();
HoodieTimeline timeline = meta.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList()));
int numCompactionCommits = (int) timeline.getInstants().count();
@@ -371,7 +371,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
}
static void assertAtleastNDeltaCommits(int minExpected, String tablePath, FileSystem fs) {
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), tablePath);
HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build();
HoodieTimeline timeline = meta.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants();
LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList()));
int numDeltaCommits = (int) timeline.getInstants().count();
@@ -380,7 +380,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
static String assertCommitMetadata(String expected, String tablePath, FileSystem fs, int totalCommits)
throws IOException {
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), tablePath);
HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build();
HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
HoodieInstant lastInstant = timeline.lastInstant().get();
HoodieCommitMetadata commitMetadata =
@@ -408,7 +408,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
}
static void assertAtLeastNCommits(int minExpected, String tablePath, FileSystem fs) {
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), tablePath);
HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).build();
HoodieTimeline timeline = meta.getActiveTimeline().filterCompletedInstants();
LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList()));
int numDeltaCommits = (int) timeline.getInstants().count();
@@ -683,13 +683,13 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
cfg.configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP, "2"));
HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc);
deltaStreamerTestRunner(ds, cfg, (r) -> {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(this.dfs.getConf(), tableBasePath, true);
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(this.dfs.getConf()).setBasePath(tableBasePath).setLoadActiveTimelineOnLoad(true).build();
int pendingReplaceSize = metaClient.getActiveTimeline().filterPendingReplaceTimeline().getInstants().toArray().length;
int completeReplaceSize = metaClient.getActiveTimeline().getCompletedReplaceTimeline().getInstants().toArray().length;
LOG.info("PendingReplaceSize=" + pendingReplaceSize + ",completeReplaceSize = " + completeReplaceSize);
return completeReplaceSize > 0;
});
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(this.dfs.getConf(), tableBasePath, true);
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(this.dfs.getConf()).setBasePath(tableBasePath).setLoadActiveTimelineOnLoad(true).build();
assertEquals(1, metaClient.getActiveTimeline().getCompletedReplaceTimeline().getInstants().toArray().length);
}
@@ -739,13 +739,13 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
} else {
LOG.warn("Schedule clustering failed");
}
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(this.dfs.getConf(), tableBasePath, true);
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(this.dfs.getConf()).setBasePath(tableBasePath).setLoadActiveTimelineOnLoad(true).build();
int pendingReplaceSize = metaClient.getActiveTimeline().filterPendingReplaceTimeline().getInstants().toArray().length;
int completeReplaceSize = metaClient.getActiveTimeline().getCompletedReplaceTimeline().getInstants().toArray().length;
System.out.println("PendingReplaceSize=" + pendingReplaceSize + ",completeReplaceSize = " + completeReplaceSize);
return completeReplaceSize > 0;
});
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(this.dfs.getConf(), tableBasePath, true);
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(this.dfs.getConf()).setBasePath(tableBasePath).setLoadActiveTimelineOnLoad(true).build();
assertEquals(1, metaClient.getActiveTimeline().getCompletedReplaceTimeline().getInstants().toArray().length);
}
@@ -921,7 +921,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
assertEquals(1000, counts.get(1).getLong(1));
// Test with empty commits
HoodieTableMetaClient mClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), tableBasePath, true);
HoodieTableMetaClient mClient = HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(tableBasePath).setLoadActiveTimelineOnLoad(true).build();
HoodieInstant lastFinished = mClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get();
HoodieDeltaStreamer.Config cfg2 = TestHelpers.makeDropAllConfig(tableBasePath, WriteOperationType.UPSERT);
cfg2.filterDupes = false;
@@ -930,7 +930,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
cfg2.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP));
HoodieDeltaStreamer ds2 = new HoodieDeltaStreamer(cfg2, jsc);
ds2.sync();
mClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), tableBasePath, true);
mClient = HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(tableBasePath).setLoadActiveTimelineOnLoad(true).build();
HoodieInstant newLastFinished = mClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get();
assertTrue(HoodieTimeline.compareTimestamps(newLastFinished.getTimestamp(), HoodieTimeline.GREATER_THAN, lastFinished.getTimestamp()
));