Take 2: Refactor hoodie-common and create right abstractions for Hoodie Storage V2.0
- Refactored timelines to be a single timeline for all active events and one for archived events. CommitTimeline and other timelines can be inferred by applying a filter on the activeTimelime - Introduced HoodieInstant to abstract different types of action, commit time and if isInFlight - Implemented other review comments
This commit is contained in:
@@ -16,6 +16,7 @@
|
||||
|
||||
package com.uber.hoodie.cli;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableConfig;
|
||||
import org.springframework.core.Ordered;
|
||||
import org.springframework.core.annotation.Order;
|
||||
import org.springframework.shell.plugin.support.DefaultPromptProvider;
|
||||
@@ -27,17 +28,18 @@ public class HoodiePrompt extends DefaultPromptProvider {
|
||||
|
||||
@Override
|
||||
public String getPrompt() {
|
||||
String tableName = HoodieCLI.tableMetadata.getTableConfig().getTableName();
|
||||
switch (HoodieCLI.state) {
|
||||
case INIT:
|
||||
return "hoodie->";
|
||||
case DATASET:
|
||||
return "hoodie:" + HoodieCLI.tableMetadata.getTableConfig().getTableName() + "->";
|
||||
return "hoodie:" + tableName + "->";
|
||||
case SYNC:
|
||||
return "hoodie:" + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " <==> "
|
||||
return "hoodie:" + tableName + " <==> "
|
||||
+ HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->";
|
||||
}
|
||||
if (HoodieCLI.tableMetadata != null)
|
||||
return "hoodie:" + HoodieCLI.tableMetadata.getTableConfig().getTableName() + "->";
|
||||
return "hoodie:" + tableName + "->";
|
||||
return "hoodie->";
|
||||
}
|
||||
|
||||
|
||||
@@ -24,6 +24,8 @@ import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.NumericUtils;
|
||||
|
||||
import org.apache.spark.launcher.SparkLauncher;
|
||||
@@ -38,6 +40,7 @@ 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
|
||||
@@ -67,15 +70,16 @@ public class CommitsCommand implements CommandMarker {
|
||||
@CliOption(key = {
|
||||
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10")
|
||||
final Integer limit) throws IOException {
|
||||
HoodieTimeline timeline = HoodieCLI.tableMetadata.getActiveCommitTimeline();
|
||||
List<String> commits = timeline.getInstants().collect(Collectors.toList());
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
||||
List<HoodieInstant> commits = timeline.getInstants().collect(Collectors.toList());
|
||||
String[][] rows = new String[commits.size()][];
|
||||
Collections.reverse(commits);
|
||||
for (int i = 0; i < commits.size(); i++) {
|
||||
String commit = commits.get(i);
|
||||
HoodieInstant commit = commits.get(i);
|
||||
HoodieCommitMetadata commitMetadata =
|
||||
HoodieCommitMetadata.fromBytes(timeline.readInstantDetails(commit).get());
|
||||
rows[i] = new String[] {commit,
|
||||
HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get());
|
||||
rows[i] = new String[] {commit.getTimestamp(),
|
||||
NumericUtils.humanReadableByteCount(commitMetadata.fetchTotalBytesWritten()),
|
||||
String.valueOf(commitMetadata.fetchTotalFilesInsert()),
|
||||
String.valueOf(commitMetadata.fetchTotalFilesUpdated()),
|
||||
@@ -104,10 +108,14 @@ public class CommitsCommand implements CommandMarker {
|
||||
final String commitTime,
|
||||
@CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path")
|
||||
final String sparkPropertiesPath) throws Exception {
|
||||
if (!HoodieCLI.tableMetadata.getActiveCommitTimeline().containsInstant(commitTime)) {
|
||||
return "Commit " + commitTime + " not found in Commits " + HoodieCLI.tableMetadata
|
||||
.getActiveCommitTimeline().getInstants().collect(Collectors.toList());
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
|
||||
if (!timeline.containsInstant(commitInstant)) {
|
||||
return "Commit " + commitTime + " not found in Commits " + timeline;
|
||||
}
|
||||
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK.toString(),
|
||||
commitTime,
|
||||
@@ -127,13 +135,15 @@ public class CommitsCommand implements CommandMarker {
|
||||
public String showCommitPartitions(
|
||||
@CliOption(key = {"commit"}, help = "Commit to show")
|
||||
final String commitTime) throws Exception {
|
||||
HoodieTimeline timeline = HoodieCLI.tableMetadata.getActiveCommitTimeline();
|
||||
if (!timeline.containsInstant(commitTime)) {
|
||||
return "Commit " + commitTime + " not found in Commits " + HoodieCLI.tableMetadata
|
||||
.getActiveCommitTimeline().getInstants().collect(Collectors.toList());
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
|
||||
if (!timeline.containsInstant(commitInstant)) {
|
||||
return "Commit " + commitTime + " not found in Commits " + timeline;
|
||||
}
|
||||
HoodieCommitMetadata meta =
|
||||
HoodieCommitMetadata.fromBytes(timeline.readInstantDetails(commitTime).get());
|
||||
HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get());
|
||||
List<String[]> rows = new ArrayList<String[]>();
|
||||
for (Map.Entry<String, List<HoodieWriteStat>> entry : meta.getPartitionToWriteStats()
|
||||
.entrySet()) {
|
||||
@@ -173,13 +183,15 @@ public class CommitsCommand implements CommandMarker {
|
||||
public String showCommitFiles(
|
||||
@CliOption(key = {"commit"}, help = "Commit to show")
|
||||
final String commitTime) throws Exception {
|
||||
HoodieTimeline timeline = HoodieCLI.tableMetadata.getActiveCommitTimeline();
|
||||
if (!timeline.containsInstant(commitTime)) {
|
||||
return "Commit " + commitTime + " not found in Commits " + HoodieCLI.tableMetadata
|
||||
.getActiveCommitTimeline().getInstants().collect(Collectors.toList());
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
|
||||
if (!timeline.containsInstant(commitInstant)) {
|
||||
return "Commit " + commitTime + " not found in Commits " + timeline;
|
||||
}
|
||||
HoodieCommitMetadata meta =
|
||||
HoodieCommitMetadata.fromBytes(timeline.readInstantDetails(commitTime).get());
|
||||
HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get());
|
||||
List<String[]> rows = new ArrayList<String[]>();
|
||||
for (Map.Entry<String, List<HoodieWriteStat>> entry : meta.getPartitionToWriteStats()
|
||||
.entrySet()) {
|
||||
@@ -208,26 +220,26 @@ 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.getActiveCommitTimeline();
|
||||
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitTimeline().filterCompletedInstants();;
|
||||
HoodieTableMetaClient source = HoodieCLI.tableMetadata;
|
||||
HoodieTimeline sourceTimeline = source.getActiveCommitTimeline();
|
||||
HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitTimeline().filterCompletedInstants();;
|
||||
String targetLatestCommit =
|
||||
targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get();
|
||||
targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get().getTimestamp();
|
||||
String sourceLatestCommit =
|
||||
sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get();
|
||||
sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get().getTimestamp();
|
||||
|
||||
if (sourceLatestCommit != null && sourceTimeline
|
||||
.compareInstants(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) {
|
||||
.compareTimestamps(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) {
|
||||
// source is behind the target
|
||||
List<String> commitsToCatchup =
|
||||
targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE)
|
||||
.collect(Collectors.toList());
|
||||
return "Source " + source.getTableConfig().getTableName() + " is behind by " + commitsToCatchup.size()
|
||||
+ " commits. Commits to catch up - " + commitsToCatchup;
|
||||
.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
return "Source " + source.getTableConfig().getTableName() + " is behind by "
|
||||
+ commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup;
|
||||
} else {
|
||||
List<String> commitsToCatchup =
|
||||
sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE)
|
||||
.collect(Collectors.toList());
|
||||
.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
return "Source " + source.getTableConfig().getTableName() + " is ahead by "
|
||||
+ commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup;
|
||||
}
|
||||
|
||||
@@ -21,6 +21,7 @@ import com.uber.hoodie.cli.utils.HiveUtil;
|
||||
import com.uber.hoodie.cli.HoodieCLI;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
@@ -60,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.getActiveCommitTimeline();
|
||||
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitTimeline();
|
||||
HoodieTableMetaClient source = HoodieCLI.tableMetadata;
|
||||
HoodieTimeline sourceTimeline = source.getActiveCommitTimeline();
|
||||
HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitTimeline();
|
||||
long sourceCount = 0;
|
||||
long targetCount = 0;
|
||||
if ("complete".equals(mode)) {
|
||||
@@ -74,36 +75,40 @@ public class HoodieSyncCommand implements CommandMarker {
|
||||
}
|
||||
|
||||
String targetLatestCommit =
|
||||
targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get();
|
||||
targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get().getTimestamp();
|
||||
String sourceLatestCommit =
|
||||
sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get();
|
||||
sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get().getTimestamp();
|
||||
|
||||
if (sourceLatestCommit != null && sourceTimeline
|
||||
.compareInstants(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) {
|
||||
.compareTimestamps(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) {
|
||||
// source is behind the target
|
||||
List<String> commitsToCatchup =
|
||||
targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE)
|
||||
List<HoodieInstant> commitsToCatchup =
|
||||
targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE).getInstants()
|
||||
.collect(Collectors.toList());
|
||||
if (commitsToCatchup.isEmpty()) {
|
||||
return "Count difference now is (count(" + target.getTableConfig().getTableName()
|
||||
+ ") - count(" + source.getTableConfig().getTableName() + ") == " + (targetCount
|
||||
- sourceCount);
|
||||
} else {
|
||||
long newInserts = CommitUtil.countNewRecords(target, commitsToCatchup);
|
||||
long newInserts = CommitUtil.countNewRecords(target,
|
||||
commitsToCatchup.stream().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList()));
|
||||
return "Count difference now is (count(" + target.getTableConfig().getTableName()
|
||||
+ ") - count(" + source.getTableConfig().getTableName() + ") == " + (targetCount
|
||||
- sourceCount) + ". Catch up count is " + newInserts;
|
||||
}
|
||||
} else {
|
||||
List<String> commitsToCatchup =
|
||||
sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE)
|
||||
List<HoodieInstant> commitsToCatchup =
|
||||
sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE).getInstants()
|
||||
.collect(Collectors.toList());
|
||||
if (commitsToCatchup.isEmpty()) {
|
||||
return "Count difference now is (count(" + source.getTableConfig().getTableName()
|
||||
+ ") - count(" + target.getTableConfig().getTableName() + ") == " + (sourceCount
|
||||
- targetCount);
|
||||
} else {
|
||||
long newInserts = CommitUtil.countNewRecords(source, commitsToCatchup);
|
||||
long newInserts = CommitUtil.countNewRecords(source,
|
||||
commitsToCatchup.stream().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList()));
|
||||
return "Count difference now is (count(" + source.getTableConfig().getTableName()
|
||||
+ ") - count(" + target.getTableConfig().getTableName() + ") == " + (sourceCount
|
||||
- targetCount) + ". Catch up count is " + newInserts;
|
||||
|
||||
@@ -24,6 +24,8 @@ import com.uber.hoodie.cli.HoodieCLI;
|
||||
import com.uber.hoodie.cli.HoodiePrintHelper;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.NumericUtils;
|
||||
|
||||
@@ -53,21 +55,22 @@ public class StatsCommand implements CommandMarker {
|
||||
long totalRecordsUpserted = 0;
|
||||
long totalRecordsWritten = 0;
|
||||
|
||||
HoodieTimeline timeline = HoodieCLI.tableMetadata.getActiveCommitTimeline();
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
||||
|
||||
String[][] rows = new String[new Long(timeline.getTotalInstants()).intValue() + 1][];
|
||||
String[][] rows = new String[new Long(timeline.countInstants()).intValue() + 1][];
|
||||
int i = 0;
|
||||
DecimalFormat df = new DecimalFormat("#.00");
|
||||
for (String commitTime : timeline.getInstants().collect(
|
||||
for (HoodieInstant commitTime : timeline.getInstants().collect(
|
||||
Collectors.toList())) {
|
||||
String waf = "0";
|
||||
HoodieCommitMetadata commit = HoodieCommitMetadata.fromBytes(timeline.readInstantDetails(commitTime).get());
|
||||
HoodieCommitMetadata commit = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitTime).get());
|
||||
if (commit.fetchTotalUpdateRecordsWritten() > 0) {
|
||||
waf = df.format(
|
||||
(float) commit.fetchTotalRecordsWritten() / commit
|
||||
.fetchTotalUpdateRecordsWritten());
|
||||
}
|
||||
rows[i++] = new String[] {commitTime,
|
||||
rows[i++] = new String[] {commitTime.getTimestamp(),
|
||||
String.valueOf(commit.fetchTotalUpdateRecordsWritten()),
|
||||
String.valueOf(commit.fetchTotalRecordsWritten()), waf};
|
||||
totalRecordsUpserted += commit.fetchTotalUpdateRecordsWritten();
|
||||
|
||||
@@ -19,6 +19,7 @@ package com.uber.hoodie.cli.utils;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
@@ -27,10 +28,11 @@ public class CommitUtil {
|
||||
public static long countNewRecords(HoodieTableMetaClient target, List<String> commitsToCatchup)
|
||||
throws IOException {
|
||||
long totalNew = 0;
|
||||
HoodieTimeline timeline = target.getActiveCommitTimeline();
|
||||
timeline = timeline.reload();
|
||||
HoodieTimeline timeline = target.getActiveTimeline().reload().getCommitTimeline().filterCompletedInstants();
|
||||
for(String commit:commitsToCatchup) {
|
||||
HoodieCommitMetadata c = HoodieCommitMetadata.fromBytes(timeline.readInstantDetails(commit).get());
|
||||
HoodieCommitMetadata c = HoodieCommitMetadata.fromBytes(timeline
|
||||
.getInstantDetails(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commit))
|
||||
.get());
|
||||
totalNew += c.fetchTotalRecordsWritten() - c.fetchTotalUpdateRecordsWritten();
|
||||
}
|
||||
return totalNew;
|
||||
|
||||
Reference in New Issue
Block a user