1
0

Introduce getCommitsAndCompactionsTimeline() explicitly & adjust usage across code base

This commit is contained in:
Vinoth Chandar
2017-04-26 13:36:49 -07:00
committed by prazanna
parent bae0528013
commit da17c5c607
15 changed files with 42 additions and 48 deletions

View File

@@ -40,7 +40,6 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
@Component
@@ -71,7 +70,7 @@ public class CommitsCommand implements CommandMarker {
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10")
final Integer limit) throws IOException {
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline().filterCompletedInstants();
List<HoodieInstant> commits = timeline.getInstants().collect(Collectors.toList());
String[][] rows = new String[commits.size()][];
Collections.reverse(commits);
@@ -109,7 +108,7 @@ public class CommitsCommand implements CommandMarker {
@CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path")
final String sparkPropertiesPath) throws Exception {
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline().filterCompletedInstants();
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
if (!timeline.containsInstant(commitInstant)) {
@@ -136,7 +135,7 @@ public class CommitsCommand implements CommandMarker {
@CliOption(key = {"commit"}, help = "Commit to show")
final String commitTime) throws Exception {
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline().filterCompletedInstants();
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
if (!timeline.containsInstant(commitInstant)) {
@@ -184,7 +183,7 @@ public class CommitsCommand implements CommandMarker {
@CliOption(key = {"commit"}, help = "Commit to show")
final String commitTime) throws Exception {
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline().filterCompletedInstants();
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
if (!timeline.containsInstant(commitInstant)) {
@@ -220,9 +219,9 @@ public class CommitsCommand implements CommandMarker {
@CliOption(key = {"path"}, help = "Path of the dataset to compare to")
final String path) throws Exception {
HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.fs, path);
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitTimeline().filterCompletedInstants();;
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();;
HoodieTableMetaClient source = HoodieCLI.tableMetadata;
HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitTimeline().filterCompletedInstants();;
HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();;
String targetLatestCommit =
targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get().getTimestamp();
String sourceLatestCommit =

View File

@@ -61,9 +61,9 @@ public class HoodieSyncCommand implements CommandMarker {
"hivePass"}, mandatory = true, unspecifiedDefaultValue = "", help = "hive password to connect to")
final String hivePass) throws Exception {
HoodieTableMetaClient target = HoodieCLI.syncTableMetadata;
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitTimeline();
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsAndCompactionsTimeline();
HoodieTableMetaClient source = HoodieCLI.tableMetadata;
HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitTimeline();
HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsAndCompactionsTimeline();
long sourceCount = 0;
long targetCount = 0;
if ("complete".equals(mode)) {

View File

@@ -16,7 +16,6 @@
package com.uber.hoodie.cli.commands;
import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.utils.InputStreamConsumer;
@@ -38,7 +37,6 @@ import org.springframework.stereotype.Component;
import java.io.IOException;
import java.util.Collections;
import java.util.Date;
import java.util.List;
import java.util.stream.Collectors;