[HUDI-1315] Adding builder for HoodieTableMetaClient initialization (#2534)
This commit is contained in:
committed by
GitHub
parent
0d91c451b0
commit
c9fcf964b2
@@ -45,8 +45,9 @@ public class CompactNode extends DagNode<JavaRDD<WriteStatus>> {
|
||||
*/
|
||||
@Override
|
||||
public void execute(ExecutionContext executionContext, int curItrCount) throws Exception {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(executionContext.getHoodieTestSuiteWriter().getConfiguration(),
|
||||
executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath);
|
||||
HoodieTableMetaClient metaClient =
|
||||
HoodieTableMetaClient.builder().setConf(executionContext.getHoodieTestSuiteWriter().getConfiguration()).setBasePath(executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath)
|
||||
.build();
|
||||
Option<HoodieInstant> lastInstant = metaClient.getActiveTimeline()
|
||||
.getCommitsAndCompactionTimeline().filterPendingCompactionTimeline().lastInstant();
|
||||
if (lastInstant.isPresent()) {
|
||||
|
||||
@@ -49,8 +49,9 @@ public class RollbackNode extends DagNode<Option<HoodieInstant>> {
|
||||
log.info("Executing rollback node {}", this.getName());
|
||||
// Can only be done with an instantiation of a new WriteClient hence cannot be done during DeltaStreamer
|
||||
// testing for now
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(executionContext.getHoodieTestSuiteWriter().getConfiguration(),
|
||||
executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath);
|
||||
HoodieTableMetaClient metaClient =
|
||||
HoodieTableMetaClient.builder().setConf(executionContext.getHoodieTestSuiteWriter().getConfiguration()).setBasePath(executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath)
|
||||
.build();
|
||||
Option<HoodieInstant> lastInstant = metaClient.getActiveTimeline().getCommitsTimeline().lastInstant();
|
||||
if (lastInstant.isPresent()) {
|
||||
log.info("Rolling back last instant {}", lastInstant.get());
|
||||
|
||||
@@ -41,8 +41,9 @@ public class ScheduleCompactNode extends DagNode<Option<String>> {
|
||||
// testing for now
|
||||
// Find the last commit and extra the extra metadata to be passed to the schedule compaction. This is
|
||||
// done to ensure the CHECKPOINT is correctly passed from commit to commit
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(executionContext.getHoodieTestSuiteWriter().getConfiguration(),
|
||||
executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath);
|
||||
HoodieTableMetaClient metaClient =
|
||||
HoodieTableMetaClient.builder().setConf(executionContext.getHoodieTestSuiteWriter().getConfiguration()).setBasePath(executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath)
|
||||
.build();
|
||||
Option<HoodieInstant> lastInstant = metaClient.getActiveTimeline().getCommitsTimeline().lastInstant();
|
||||
if (lastInstant.isPresent()) {
|
||||
HoodieCommitMetadata metadata = org.apache.hudi.common.model.HoodieCommitMetadata.fromBytes(metaClient
|
||||
|
||||
@@ -37,7 +37,6 @@ import org.apache.hudi.config.HoodieMemoryConfig;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.avro.AvroParquetReader;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
@@ -80,7 +79,7 @@ public class DFSHoodieDatasetInputReader extends DFSDeltaInputReader {
|
||||
public DFSHoodieDatasetInputReader(JavaSparkContext jsc, String basePath, String schemaStr) {
|
||||
this.jsc = jsc;
|
||||
this.schemaStr = schemaStr;
|
||||
this.metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
this.metaClient = HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build();
|
||||
}
|
||||
|
||||
protected List<String> getPartitions(Option<Integer> partitionsLimit) throws IOException {
|
||||
|
||||
Reference in New Issue
Block a user