Introduce getCommitsAndCompactionsTimeline() explicitly & adjust usage across code base
This commit is contained in:
@@ -271,7 +271,7 @@ public class HiveIncrementalPuller {
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, targetDataPath);
|
||||
|
||||
Optional<HoodieInstant>
|
||||
lastCommit = metadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant();
|
||||
lastCommit = metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants().lastInstant();
|
||||
if(lastCommit.isPresent()) {
|
||||
return lastCommit.get().getTimestamp();
|
||||
}
|
||||
@@ -306,12 +306,12 @@ public class HiveIncrementalPuller {
|
||||
|
||||
private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) throws IOException {
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, sourceTableLocation);
|
||||
List<String> commitsToSync = metadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants()
|
||||
List<String> commitsToSync = metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants()
|
||||
.findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
if (commitsToSync.isEmpty()) {
|
||||
log.warn("Nothing to sync. All commits in " + config.sourceTable + " are " + metadata
|
||||
.getActiveTimeline().getCommitTimeline().filterCompletedInstants().getInstants()
|
||||
.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants().getInstants()
|
||||
.collect(Collectors.toList()) + " and from commit time is "
|
||||
+ config.fromCommitTime);
|
||||
return null;
|
||||
|
||||
@@ -30,27 +30,23 @@ import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
import org.apache.spark.api.java.function.VoidFunction;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
@@ -74,10 +70,10 @@ public class HoodieSnapshotCopier implements Serializable {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs, baseDir);
|
||||
final TableFileSystemView fsView = new HoodieTableFileSystemView(tableMetadata,
|
||||
tableMetadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants());
|
||||
tableMetadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants());
|
||||
// Get the latest commit
|
||||
Optional<HoodieInstant> latestCommit = tableMetadata.getActiveTimeline()
|
||||
.getCommitTimeline().filterCompletedInstants().lastInstant();
|
||||
.getCommitsAndCompactionsTimeline().filterCompletedInstants().lastInstant();
|
||||
if(!latestCommit.isPresent()) {
|
||||
logger.warn("No commits present. Nothing to snapshot");
|
||||
return;
|
||||
|
||||
@@ -119,7 +119,7 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
|
||||
if (fs.exists(new Path(cfg.targetBasePath))) {
|
||||
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs, cfg.targetBasePath);
|
||||
this.commitTimelineOpt = Optional.of(meta.getActiveTimeline().getCommitTimeline().filterCompletedInstants());
|
||||
this.commitTimelineOpt = Optional.of(meta.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants());
|
||||
} else {
|
||||
this.commitTimelineOpt = Optional.empty();
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user