1
0

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:
Prasanna Rajaperumal
2017-01-18 01:00:36 -08:00
parent 8ee777a9bb
commit ccd8cb2407
46 changed files with 1194 additions and 1106 deletions

View File

@@ -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->";
}

View File

@@ -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;
}

View File

@@ -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;

View File

@@ -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();

View File

@@ -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;

View File

@@ -78,7 +78,7 @@ class DedupeSparkJob (basePath: String,
val fsView = new ReadOptimizedTableView(fs, metadata)
val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"${basePath}/${duplicatedPartitionPath}"))
val latestFiles:java.util.List[HoodieDataFile] = fsView.streamLatestVersions(allFiles).collect(Collectors.toList[HoodieDataFile]())
val latestFiles:java.util.List[HoodieDataFile] = fsView.getLatestVersions(allFiles).collect(Collectors.toList[HoodieDataFile]())
val filteredStatuses = latestFiles.map(f => f.getPath)
LOG.info(s" List of files under partition: ${} => ${filteredStatuses.mkString(" ")}")
@@ -129,7 +129,7 @@ class DedupeSparkJob (basePath: String,
val fsView = new ReadOptimizedTableView(fs, metadata)
val allFiles = fs.listStatus(new Path(s"${basePath}/${duplicatedPartitionPath}"))
val latestFiles:java.util.List[HoodieDataFile] = fsView.streamLatestVersions(allFiles).collect(Collectors.toList[HoodieDataFile]())
val latestFiles:java.util.List[HoodieDataFile] = fsView.getLatestVersions(allFiles).collect(Collectors.toList[HoodieDataFile]())
val fileNameToPathMap = latestFiles.map(f => (f.getFileId, new Path(f.getPath))).toMap
val dupeFixPlan = planDuplicateFix()

View File

@@ -25,6 +25,7 @@ import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
@@ -89,8 +90,10 @@ public class HoodieReadClient implements Serializable {
this.jsc = jsc;
this.fs = FSUtils.getFs();
this.metaClient = new HoodieTableMetaClient(fs, basePath, true);
this.commitTimeline = metaClient.getActiveCommitTimeline();
this.index = new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
this.commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
this.index =
new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
this.sqlContextOpt = Optional.absent();
}
@@ -191,7 +194,7 @@ public class HoodieReadClient implements Serializable {
+ metaClient.getBasePath());
}
List<HoodieDataFile> latestFiles = fileSystemView.streamLatestVersions(fs.globStatus(new Path(path))).collect(
List<HoodieDataFile> latestFiles = fileSystemView.getLatestVersions(fs.globStatus(new Path(path))).collect(
Collectors.toList());
for (HoodieDataFile file : latestFiles) {
filteredPaths.add(file.getPath());
@@ -212,17 +215,17 @@ public class HoodieReadClient implements Serializable {
*/
public Dataset<Row> readSince(String lastCommitTimestamp) {
List<String> commitsToReturn =
List<HoodieInstant> commitsToReturn =
commitTimeline.findInstantsAfter(lastCommitTimestamp, Integer.MAX_VALUE)
.collect(Collectors.toList());
.getInstants().collect(Collectors.toList());
//TODO: we can potentially trim this down to only affected partitions, using CommitMetadata
try {
// Go over the commit metadata, and obtain the new files that need to be read.
HashMap<String, String> fileIdToFullPath = new HashMap<>();
for (String commit: commitsToReturn) {
for (HoodieInstant commit: commitsToReturn) {
HoodieCommitMetadata metadata =
HoodieCommitMetadata.fromBytes(commitTimeline.readInstantDetails(commit).get());
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get());
// get files from each commit, and replace any previous versions
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths());
}
@@ -240,13 +243,15 @@ public class HoodieReadClient implements Serializable {
*/
public Dataset<Row> readCommit(String commitTime) {
assertSqlContext();
if (!commitTimeline.containsInstant(commitTime)) {
HoodieInstant commitInstant =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
if (!commitTimeline.containsInstant(commitInstant)) {
new HoodieException("No commit exists at " + commitTime);
}
try {
HoodieCommitMetadata commitMetdata =
HoodieCommitMetadata.fromBytes(commitTimeline.readInstantDetails(commitTime).get());
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commitInstant).get());
Collection<String> paths = commitMetdata.getFileIdAndFullPaths().values();
return sqlContextOpt.get().read()
.parquet(paths.toArray(new String[paths.size()]))
@@ -298,13 +303,14 @@ public class HoodieReadClient implements Serializable {
* @return
*/
public List<String> listCommitsSince(String commitTimestamp) {
return commitTimeline.findInstantsAfter(commitTimestamp, Integer.MAX_VALUE).collect(Collectors.toList());
return commitTimeline.findInstantsAfter(commitTimestamp, Integer.MAX_VALUE).getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
}
/**
* Returns the last successful commit (a successful write operation) into a Hoodie table.
*/
public String latestCommit() {
return commitTimeline.lastInstant().get();
return commitTimeline.lastInstant().get().getTimestamp();
}
}

View File

@@ -25,6 +25,8 @@ import com.uber.hoodie.common.model.HoodieRecordPayload;
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.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCommitException;
@@ -302,7 +304,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
logger.info("Comitting " + commitTime);
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
List<Tuple2<String, HoodieWriteStat>> stats =
writeStatuses.mapToPair(new PairFunction<WriteStatus, String, HoodieWriteStat>() {
@@ -319,7 +321,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
try {
commitTimeline.saveInstantAsComplete(commitTime,
activeTimeline.saveAsComplete(
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime),
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
// Save was a success
// We cannot have unbounded commit files. Archive commits if we have to archive
@@ -356,17 +359,19 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
final Timer.Context context = metrics.getRollbackCtx();
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
HoodieTimeline inflightTimeline = activeTimeline.getCommitTimeline().filterInflights();
HoodieTimeline commitTimeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
try {
if (commitTimeline.lastInstant().isPresent()
&& commitTimeline.findInstantsAfter(commitTime, Integer.MAX_VALUE).count() > 0) {
&& !commitTimeline.findInstantsAfter(commitTime, Integer.MAX_VALUE).empty()) {
throw new HoodieRollbackException("Found commits after time :" + commitTime +
", please rollback greater commits first");
}
List<String> inflights =
commitTimeline.getInflightInstants().collect(Collectors.toList());
List<String> inflights = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
if (!inflights.isEmpty() && inflights.indexOf(commitTime) != inflights.size() - 1) {
throw new HoodieRollbackException(
"Found in-flight commits after time :" + commitTime +
@@ -374,10 +379,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
if (inflights.contains(commitTime) || (commitTimeline.lastInstant().isPresent()
&& commitTimeline.lastInstant().get().equals(commitTime))) {
&& commitTimeline.lastInstant().get().getTimestamp().equals(commitTime))) {
// 1. Atomically unpublish this commit
if(commitTimeline.containsInstant(commitTime)) {
commitTimeline.revertInstantToInflight(commitTime);
if(!inflights.contains(commitTime)) {
// This is completed commit, first revert it to inflight to unpublish data
activeTimeline.revertToInflight(
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime));
}
// 2. Revert the index changes
logger.info("Clean out index changes at time: " + commitTime);
@@ -415,7 +422,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
});
// 4. Remove commit
logger.info("Clean out metadata files at time: " + commitTime);
commitTimeline.removeInflightFromTimeline(commitTime);
activeTimeline.deleteInflight(
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime));
if (context != null) {
long durationInMs = metrics.getDurationInMs(context.stop());
@@ -446,7 +454,6 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
final Timer.Context context = metrics.getCleanCtx();
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
List<String> partitionsToClean = FSUtils.getAllPartitionPaths(fs, metaClient.getBasePath());
// shuffle to distribute cleaning work across partitions evenly
@@ -497,8 +504,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
logger.info("Generate a new commit time " + commitTime);
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
commitTimeline.saveInstantAsInflight(commitTime);
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
activeTimeline.createInflight(
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime));
}
public static SparkConf registerClasses(SparkConf conf) {
@@ -534,14 +542,17 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
/**
* Cleanup all inflight commits
*
* @throws IOException
*/
private void rollbackInflightCommits() {
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
HoodieTimeline inflightTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterInflights();
List<String> commits = commitTimeline.getInflightInstants().collect(Collectors.toList());
List<String> commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
Collections.reverse(commits);
for (String commit : commits) {
rollback(commit);

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie.index;
import com.google.common.base.Optional;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieKey;
@@ -128,9 +129,12 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
String fileId =
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline()
.filterCompletedInstants();
// if the last commit ts for this row is less than the system commit ts
if (commitTimeline.hasInstants() && commitTimeline.containsInstant(commitTs)) {
if (!commitTimeline.empty() && commitTimeline.containsInstant(
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))) {
rec.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
}
}

View File

@@ -22,6 +22,7 @@ import com.google.common.base.Optional;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
@@ -218,13 +219,13 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
public Iterable<Tuple2<String, String>> call(String partitionPath) {
FileSystem fs = FSUtils.getFs();
TableFileSystemView view = new ReadOptimizedTableView(fs, metaClient);
java.util.Optional<String> latestCommitTime =
metaClient.getActiveCommitTimeline().lastInstant();
java.util.Optional<HoodieInstant> latestCommitTime =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant();
List<Tuple2<String, String>> list = new ArrayList<>();
if (latestCommitTime.isPresent()) {
List<HoodieDataFile> filteredFiles =
view.streamLatestVersionInPartition(partitionPath,
latestCommitTime.get()).collect(Collectors.toList());
view.getLatestVersionInPartition(partitionPath,
latestCommitTime.get().getTimestamp()).collect(Collectors.toList());
for (HoodieDataFile file : filteredFiles) {
list.add(new Tuple2<>(partitionPath, file.getFileName()));
}

View File

@@ -20,6 +20,7 @@ import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.common.util.FSUtils;
@@ -38,55 +39,56 @@ import java.util.stream.Collectors;
/**
* Cleaner is responsible for garbage collecting older files in a given partition path, such that
*
* <p>
* 1) It provides sufficient time for existing queries running on older versions, to finish
*
* <p>
* 2) It bounds the growth of the files in the file system
*
* <p>
* TODO: Should all cleaning be done based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata}
*
*
*/
public class HoodieCleaner {
private static Logger logger = LogManager.getLogger(HoodieCleaner.class);
public enum CleaningPolicy {
KEEP_LATEST_FILE_VERSIONS,
KEEP_LATEST_COMMITS
}
private final TableFileSystemView fileSystemView;
private final HoodieTimeline commitTimeline;
private HoodieTableMetaClient metaClient;
private HoodieWriteConfig config;
private FileSystem fs;
public HoodieCleaner(HoodieTableMetaClient metaClient,
HoodieWriteConfig config,
FileSystem fs) {
public HoodieCleaner(HoodieTableMetaClient metaClient, HoodieWriteConfig config,
FileSystem fs) {
this.metaClient = metaClient;
this.fileSystemView = new ReadOptimizedTableView(fs, metaClient);
this.commitTimeline = metaClient.getActiveCommitTimeline();
this.commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
this.config = config;
this.fs = fs;
}
/**
*
* Selects the older versions of files for cleaning, such that it bounds the number of versions of each file.
* This policy is useful, if you are simply interested in querying the table, and you don't want too many
* versions for a single file (i.e run it with versionsRetained = 1)
*
*
* @param partitionPath
* @return
* @throws IOException
*/
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath) throws IOException {
logger.info("Cleaning "+ partitionPath+", retaining latest "+ config.getCleanerFileVersionsRetained()+" file versions. ");
List<List<HoodieDataFile>> fileVersions = fileSystemView.streamEveryVersionInPartition(partitionPath).collect(
Collectors.toList());
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath)
throws IOException {
logger.info("Cleaning " + partitionPath + ", retaining latest " + config
.getCleanerFileVersionsRetained() + " file versions. ");
List<List<HoodieDataFile>> fileVersions =
fileSystemView.getEveryVersionInPartition(partitionPath)
.collect(Collectors.toList());
List<String> deletePaths = new ArrayList<>();
for (List<HoodieDataFile> versionsForFileId : fileVersions) {
@@ -99,10 +101,8 @@ public class HoodieCleaner {
}
// Delete the remaining files
while (commitItr.hasNext()) {
deletePaths.add(String.format("%s/%s/%s",
config.getBasePath(),
partitionPath,
commitItr.next().getFileName()));
deletePaths.add(String.format("%s/%s/%s", config.getBasePath(), partitionPath,
commitItr.next().getFileName()));
}
}
return deletePaths;
@@ -111,17 +111,17 @@ public class HoodieCleaner {
/**
* Selects the versions for file for cleaning, such that it
*
* - Leaves the latest version of the file untouched
* - For older versions,
* - It leaves all the commits untouched which has occured in last <code>config.getCleanerCommitsRetained()</code> commits
* - It leaves ONE commit before this window. We assume that the max(query execution time) == commit_batch_time * config.getCleanerCommitsRetained(). This is 12 hours by default.
* This is essential to leave the file used by the query thats running for the max time.
*
* This provides the effect of having lookback into all changes that happened in the last X
* commits. (eg: if you retain 24 commits, and commit batch time is 30 mins, then you have 12 hrs of lookback)
*
* This policy is the default.
* <p>
* - Leaves the latest version of the file untouched
* - For older versions,
* - It leaves all the commits untouched which has occured in last <code>config.getCleanerCommitsRetained()</code> commits
* - It leaves ONE commit before this window. We assume that the max(query execution time) == commit_batch_time * config.getCleanerCommitsRetained(). This is 12 hours by default.
* This is essential to leave the file used by the query thats running for the max time.
* <p>
* This provides the effect of having lookback into all changes that happened in the last X
* commits. (eg: if you retain 24 commits, and commit batch time is 30 mins, then you have 12 hrs of lookback)
* <p>
* This policy is the default.
*
* @param partitionPath
* @return
@@ -135,11 +135,12 @@ public class HoodieCleaner {
List<String> deletePaths = new ArrayList<>();
// determine if we have enough commits, to start cleaning.
if (commitTimeline.getTotalInstants() > commitsRetained) {
String earliestCommitToRetain =
commitTimeline.nthInstant(commitTimeline.getTotalInstants() - commitsRetained).get();
if (commitTimeline.countInstants() > commitsRetained) {
HoodieInstant earliestCommitToRetain =
commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained).get();
List<List<HoodieDataFile>> fileVersions =
fileSystemView.streamEveryVersionInPartition(partitionPath).collect(Collectors.toList());
fileSystemView.getEveryVersionInPartition(partitionPath)
.collect(Collectors.toList());
for (List<HoodieDataFile> fileList : fileVersions) {
String lastVersion = FSUtils.getCommitTime(fileList.get(0).getFileName());
String lastVersionBeforeEarliestCommitToRetain =
@@ -160,8 +161,9 @@ public class HoodieCleaner {
}
// Always keep the last commit
if (commitTimeline.compareInstants(earliestCommitToRetain, fileCommitTime,
HoodieTimeline.GREATER)) {
if (commitTimeline
.compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime,
HoodieTimeline.GREATER)) {
// this is a commit, that should be cleaned.
deletePaths.add(String
.format("%s/%s/%s", config.getBasePath(), partitionPath, FSUtils
@@ -177,10 +179,12 @@ public class HoodieCleaner {
/**
* Gets the latest version < commitTime. This version file could still be used by queries.
*/
private String getLatestVersionBeforeCommit(List<HoodieDataFile> fileList, String commitTime) {
private String getLatestVersionBeforeCommit(List<HoodieDataFile> fileList,
HoodieInstant commitTime) {
for (HoodieDataFile file : fileList) {
String fileCommitTime = FSUtils.getCommitTime(file.getFileName());
if (commitTimeline.compareInstants(commitTime, fileCommitTime, HoodieTimeline.GREATER)) {
if (commitTimeline.compareTimestamps(commitTime.getTimestamp(), fileCommitTime,
HoodieTimeline.GREATER)) {
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the one we want
return fileCommitTime;
}

View File

@@ -19,8 +19,8 @@ package com.uber.hoodie.io;
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.HoodieActiveCommitTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedCommitTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.common.file.HoodieAppendLog;
import com.uber.hoodie.exception.HoodieCommitException;
@@ -34,6 +34,7 @@ import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -47,11 +48,10 @@ public class HoodieCommitArchiveLog {
private final FileSystem fs;
private final HoodieWriteConfig config;
public HoodieCommitArchiveLog(HoodieWriteConfig config,
FileSystem fs) {
public HoodieCommitArchiveLog(HoodieWriteConfig config, FileSystem fs) {
this.fs = fs;
this.config = config;
this.archiveFilePath = HoodieArchivedCommitTimeline
this.archiveFilePath = HoodieArchivedTimeline
.getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME);
}
@@ -59,7 +59,7 @@ public class HoodieCommitArchiveLog {
* Check if commits need to be archived. If yes, archive commits.
*/
public boolean archiveIfRequired() {
List<String> commitsToArchive = getCommitsToArchive().collect(Collectors.toList());
List<HoodieInstant> commitsToArchive = getCommitsToArchive().collect(Collectors.toList());
if (commitsToArchive.iterator().hasNext()) {
log.info("Archiving commits " + commitsToArchive);
archive(commitsToArchive);
@@ -70,41 +70,42 @@ public class HoodieCommitArchiveLog {
}
}
private Stream<String> getCommitsToArchive() {
private Stream<HoodieInstant> getCommitsToArchive() {
int maxCommitsToKeep = config.getMaxCommitsToKeep();
int minCommitsToKeep = config.getMinCommitsToKeep();
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
if (commitTimeline.hasInstants() && commitTimeline.getTotalInstants() > maxCommitsToKeep) {
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
// Actually do the commits
return commitTimeline.getInstants()
.limit(commitTimeline.getTotalInstants() - minCommitsToKeep);
.limit(commitTimeline.countInstants() - minCommitsToKeep);
}
return Stream.empty();
}
private boolean deleteCommits(List<String> commitsToArchive) {
private boolean deleteCommits(List<HoodieInstant> commitsToArchive) {
log.info("Deleting commits " + commitsToArchive);
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
boolean success = true;
for(String commitToArchive:commitsToArchive) {
Path commitFile = new Path(metaClient.getMetaPath(),
((HoodieActiveCommitTimeline) commitTimeline)
.getCompletedFileName(commitToArchive));
for (HoodieInstant commitToArchive : commitsToArchive) {
Path commitFile =
new Path(metaClient.getMetaPath(), commitToArchive.getFileName());
try {
if (fs.exists(commitFile)) {
success &= fs.delete(commitFile, false);
log.info("Archived and deleted commit file " + commitFile);
}
} catch (IOException e) {
throw new HoodieIOException(
"Failed to delete archived commit " + commitToArchive, e);
throw new HoodieIOException("Failed to delete archived commit " + commitToArchive,
e);
}
}
return success;
@@ -120,18 +121,19 @@ public class HoodieCommitArchiveLog {
.compression(HoodieAppendLog.CompressionType.RECORD, new BZip2Codec()));
}
private void archive(List<String> commits) throws HoodieCommitException {
private void archive(List<HoodieInstant> commits) throws HoodieCommitException {
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
HoodieAppendLog.Writer writer = null;
try {
writer = openWriter();
for (String commitTime : commits) {
Text k = new Text(commitTime);
for (HoodieInstant commitTime : commits) {
Text k = new Text(commitTime.getTimestamp());
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.readInstantDetails(commitTime).get());
.fromBytes(commitTimeline.getInstantDetails(commitTime).get());
Text v = new Text(commitMetadata.toJsonString());
writer.append(k, v);
log.info("Wrote " + k);

View File

@@ -50,7 +50,8 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
this.config = config;
this.fs = FSUtils.getFs();
this.metaClient = metaClient;
this.hoodieTimeline = metaClient.getActiveCommitTimeline();
this.hoodieTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
this.fileSystemView = new ReadOptimizedTableView(fs, metaClient);
this.schema =
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));

View File

@@ -20,6 +20,7 @@ import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
@@ -291,13 +292,15 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
FileSystem fs = FSUtils.getFs();
List<SmallFile> smallFileLocations = new ArrayList<>();
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
TableFileSystemView fileSystemView = new ReadOptimizedTableView(fs, metaClient);
if (commitTimeline.hasInstants()) { // if we have some commits
String latestCommitTime = commitTimeline.lastInstant().get();
List<HoodieDataFile> allFiles = fileSystemView.streamLatestVersionInPartition(partitionPath, latestCommitTime).collect(
Collectors.toList());
if (!commitTimeline.empty()) { // if we have some commits
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
List<HoodieDataFile> allFiles = fileSystemView
.getLatestVersionInPartition(partitionPath, latestCommitTime.getTimestamp())
.collect(Collectors.toList());
for (HoodieDataFile file : allFiles) {
if (file.getFileSize() < config.getParquetSmallFileLimit()) {
@@ -322,12 +325,13 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
*/
private long averageBytesPerRecord() {
long avgSize = 0L;
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
try {
if (commitTimeline.hasInstants()) {
String latestCommitTime = commitTimeline.lastInstant().get();
if (!commitTimeline.empty()) {
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.readInstantDetails(latestCommitTime).get());
.fromBytes(commitTimeline.getInstantDetails(latestCommitTime).get());
avgSize = (long) Math.ceil(
(1.0 * commitMetadata.fetchTotalBytesWritten()) / commitMetadata
.fetchTotalRecordsWritten());

View File

@@ -29,6 +29,7 @@ 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.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.ParquetUtils;
@@ -41,7 +42,6 @@ import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.HoodieCleaner;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.spark.SparkConf;
@@ -61,10 +61,8 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeSet;
import java.util.stream.Collectors;
@@ -293,15 +291,16 @@ public class TestHoodieClient implements Serializable {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTimeline timeline = metadata.getActiveCommitTimeline();
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitTimeline();
TableFileSystemView fsView = new ReadOptimizedTableView(fs, metadata);
// Need to ensure the following
for (String partitionPath : dataGen.getPartitionPaths()) {
// compute all the versions of all files, from time 0
HashMap<String, TreeSet<String>> fileIdToVersions = new HashMap<>();
for (String entry : timeline.getInstants().collect(Collectors.toList())) {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.readInstantDetails(entry).get());
for (HoodieInstant entry : timeline.getInstants().collect(Collectors.toList())) {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(entry).get());
for (HoodieWriteStat wstat : commitMetadata.getWriteStats(partitionPath)) {
if (!fileIdToVersions.containsKey(wstat.getFileId())) {
@@ -312,7 +311,7 @@ public class TestHoodieClient implements Serializable {
}
List<List<HoodieDataFile>> fileVersions = fsView.streamEveryVersionInPartition(partitionPath).collect(Collectors.toList());
List<List<HoodieDataFile>> fileVersions = fsView.getEveryVersionInPartition(partitionPath).collect(Collectors.toList());
for (List<HoodieDataFile> entry : fileVersions) {
// No file has no more than max versions
String fileId = entry.iterator().next().getFileId();
@@ -373,13 +372,14 @@ public class TestHoodieClient implements Serializable {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTimeline activeTimeline = metadata.getActiveCommitTimeline();
Optional<String> earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1);
Set<String> acceptableCommits =
HoodieTimeline activeTimeline = metadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
Optional<HoodieInstant>
earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1);
Set<HoodieInstant> acceptableCommits =
activeTimeline.getInstants().collect(Collectors.toSet());
if (earliestRetainedCommit.isPresent()) {
acceptableCommits.removeAll(
activeTimeline.findInstantsInRange("000", earliestRetainedCommit.get())
activeTimeline.findInstantsInRange("000", earliestRetainedCommit.get().getTimestamp()).getInstants()
.collect(Collectors.toSet()));
acceptableCommits.add(earliestRetainedCommit.get());
}
@@ -387,16 +387,16 @@ public class TestHoodieClient implements Serializable {
TableFileSystemView fsView = new ReadOptimizedTableView(fs, metadata);
// Need to ensure the following
for (String partitionPath : dataGen.getPartitionPaths()) {
List<List<HoodieDataFile>> fileVersions = fsView.streamEveryVersionInPartition(partitionPath).collect(Collectors.toList());
List<List<HoodieDataFile>> fileVersions = fsView.getEveryVersionInPartition(partitionPath).collect(Collectors.toList());
for (List<HoodieDataFile> entry : fileVersions) {
Set<String> commitTimes = new HashSet<>();
for(HoodieDataFile value:entry) {
System.out.println("Data File - " + value);
commitTimes.add(value.getCommitTime());
}
System.out.println("Existing commits " + activeTimeline.getInstants().collect(Collectors.toList()));
assertEquals("Only contain acceptable versions of file should be present",
acceptableCommits, commitTimes);
acceptableCommits.stream().map(HoodieInstant::getTimestamp)
.collect(Collectors.toSet()), commitTimes);
}
}
}
@@ -637,9 +637,8 @@ public class TestHoodieClient implements Serializable {
assertEquals("2 files needs to be committed.", 2, statuses.size());
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTimeline activeTimeline = metadata.getActiveCommitTimeline();
TableFileSystemView fileSystemView = new ReadOptimizedTableView(fs, metadata);
List<HoodieDataFile> files = fileSystemView.streamLatestVersionInPartition(TEST_PARTITION_PATH, commitTime3).collect(
List<HoodieDataFile> files = fileSystemView.getLatestVersionInPartition(TEST_PARTITION_PATH, commitTime3).collect(
Collectors.toList());
int numTotalInsertsInCommit3 = 0;
for (HoodieDataFile file: files) {

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie.common;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import java.io.File;
@@ -60,11 +61,11 @@ public class HoodieClientTestUtils {
public static void fakeCommitFile(String basePath, String commitTime) throws IOException {
fakeMetaFile(basePath, commitTime, HoodieTableMetaClient.COMMIT_EXTENSION);
fakeMetaFile(basePath, commitTime, HoodieTimeline.COMMIT_EXTENSION);
}
public static void fakeInFlightFile(String basePath, String commitTime) throws IOException {
fakeMetaFile(basePath, commitTime, HoodieTableMetaClient.INFLIGHT_FILE_SUFFIX);
fakeMetaFile(basePath, commitTime, HoodieTimeline.INFLIGHT_EXTENSION);
}
public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId) throws Exception {

View File

@@ -20,6 +20,7 @@ import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
@@ -142,7 +143,7 @@ public class HoodieTestDataGenerator {
public static void createCommitFile(String basePath, String commitTime) throws IOException {
Path commitFile =
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTableMetaClient.makeCommitFileName(commitTime));
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(commitTime));
FileSystem fs = FSUtils.getFs();
FSDataOutputStream os = fs.create(commitFile, true);
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();

View File

@@ -17,6 +17,7 @@
package com.uber.hoodie.func;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.TestRawTripPayload;
@@ -79,7 +80,7 @@ public class TestUpdateMapFunction {
rowChange3));
Iterator<List<WriteStatus>> insertResult = table.handleInsert(records.iterator());
Path commitFile =
new Path(config.getBasePath() + "/.hoodie/" + HoodieTableMetaClient.makeCommitFileName("100"));
new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100"));
FSUtils.getFs().create(commitFile);
// Now try an update with an evolved schema

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie.io;
import com.google.common.collect.Lists;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
@@ -78,15 +79,16 @@ public class TestHoodieCommitArchiveLog {
HoodieTestDataGenerator.createCommitFile(basePath, "102");
HoodieTestDataGenerator.createCommitFile(basePath, "103");
HoodieTimeline timeline = metadata.getActiveCommitTimeline();
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
assertEquals("Loaded 4 commits and the count should match", 4,
timeline.getTotalInstants());
assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
timeline = timeline.reload();
timeline =
metadata.getActiveTimeline().reload().getCommitTimeline().filterCompletedInstants();
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4,
timeline.getTotalInstants());
timeline.countInstants());
}
@Test
@@ -104,19 +106,21 @@ public class TestHoodieCommitArchiveLog {
HoodieTestDataGenerator.createCommitFile(basePath, "104");
HoodieTestDataGenerator.createCommitFile(basePath, "105");
HoodieTimeline timeline = metadata.getActiveCommitTimeline();
List<String> originalCommits = timeline.getInstants().collect(
Collectors.toList());
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
List<HoodieInstant> originalCommits = timeline.getInstants().collect(Collectors.toList());
assertEquals("Loaded 6 commits and the count should match", 6, timeline.getTotalInstants());
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
timeline = timeline.reload();
timeline =
metadata.getActiveTimeline().reload().getCommitTimeline().filterCompletedInstants();
assertEquals(
"Should archive commits when maxCommitsToKeep is 5 and now the commits length should be minCommitsToKeep which is 2",
2, timeline.getTotalInstants());
2, timeline.countInstants());
assertEquals("Archive should not archive the last 2 commits",
Lists.newArrayList("104", "105"), timeline.getInstants().collect(Collectors.toList()));
Lists.newArrayList("104", "105"),
timeline.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()));
// Remove all the commits from the original commits, make it ready to be checked against the read map
timeline.getInstants().forEach(originalCommits::remove);
@@ -134,7 +138,8 @@ public class TestHoodieCommitArchiveLog {
assertEquals(
"Read commits map should match the originalCommits - commitsLoadedAfterArchival",
originalCommits, new ArrayList<>(readCommits.keySet()));
originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()),
new ArrayList<>(readCommits.keySet()));
reader.close();
}
@@ -153,15 +158,21 @@ public class TestHoodieCommitArchiveLog {
HoodieTestDataGenerator.createCommitFile(basePath, "104");
HoodieTestDataGenerator.createCommitFile(basePath, "105");
HoodieTimeline timeline = metadata.getActiveCommitTimeline();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.getTotalInstants());
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
timeline = timeline.reload();
assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("100"));
assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("101"));
assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("102"));
assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("103"));
timeline =
metadata.getActiveTimeline().reload().getCommitTimeline().filterCompletedInstants();
assertTrue("Archived commits should always be safe",
timeline.containsOrBeforeTimelineStarts("100"));
assertTrue("Archived commits should always be safe",
timeline.containsOrBeforeTimelineStarts("101"));
assertTrue("Archived commits should always be safe",
timeline.containsOrBeforeTimelineStarts("102"));
assertTrue("Archived commits should always be safe",
timeline.containsOrBeforeTimelineStarts("103"));
}

View File

@@ -184,8 +184,8 @@ public class TestCopyOnWriteTable {
if (file.getName().endsWith(".parquet")) {
if (FSUtils.getFileId(file.getName())
.equals(FSUtils.getFileId(parquetFile.getName())) && metadata
.getActiveCommitTimeline()
.compareInstants(FSUtils.getCommitTime(file.getName()),
.getActiveTimeline().getCommitTimeline()
.compareTimestamps(FSUtils.getCommitTime(file.getName()),
FSUtils.getCommitTime(parquetFile.getName()), HoodieTimeline.GREATER)) {
updatedParquetFile = file;
break;

View File

@@ -16,12 +16,12 @@
package com.uber.hoodie.common.model;
public enum HoodieStorageType {
public enum HoodieFileFormat {
PARQUET(".parquet");
private final String extension;
HoodieStorageType(String extension) {
HoodieFileFormat(String extension) {
this.extension = extension;
}

View File

@@ -16,9 +16,8 @@
package com.uber.hoodie.common.table;
import com.uber.hoodie.common.model.HoodieStorageType;
import com.uber.hoodie.common.model.HoodieFileFormat;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@@ -46,10 +45,10 @@ public class HoodieTableConfig implements Serializable {
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name";
public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
public static final String HOODIE_RO_STORAGE_FORMAT_PROP_NAME =
"hoodie.table.ro.storage.format";
public static final String HOODIE_RO_FILE_FORMAT_PROP_NAME =
"hoodie.table.ro.file.format";
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
public static final HoodieStorageType DEFAULT_RO_STORAGE_FORMAT = HoodieStorageType.PARQUET;
public static final HoodieFileFormat DEFAULT_RO_FILE_FORMAT = HoodieFileFormat.PARQUET;
private Properties props;
public HoodieTableConfig(FileSystem fs, String metaPath) {
@@ -127,12 +126,12 @@ public class HoodieTableConfig implements Serializable {
/**
* Get the Read Optimized Storage Format
*
* @return HoodieStorageType for the Read Optimized Storage format
* @return HoodieFileFormat for the Read Optimized Storage format
*/
public HoodieStorageType getROStorageFormat() {
if (props.contains(HOODIE_RO_STORAGE_FORMAT_PROP_NAME)) {
return HoodieStorageType.valueOf(props.getProperty(HOODIE_RO_STORAGE_FORMAT_PROP_NAME));
public HoodieFileFormat getROFileFormat() {
if (props.contains(HOODIE_RO_FILE_FORMAT_PROP_NAME)) {
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RO_FILE_FORMAT_PROP_NAME));
}
return DEFAULT_RO_STORAGE_FORMAT;
return DEFAULT_RO_FILE_FORMAT;
}
}

View File

@@ -17,10 +17,8 @@
package com.uber.hoodie.common.table;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.timeline.HoodieActiveCommitTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedCommitTimeline;
import com.uber.hoodie.common.table.timeline.HoodieCleanerTimeline;
import com.uber.hoodie.common.table.timeline.HoodieSavePointTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.DatasetNotFoundException;
import org.apache.hadoop.fs.FileStatus;
@@ -30,8 +28,13 @@ import org.apache.hadoop.fs.PathFilter;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.Externalizable;
import java.io.File;
import java.io.IOException;
import java.io.ObjectInput;
import java.io.ObjectInputStream;
import java.io.ObjectOutput;
import java.io.ObjectOutputStream;
import java.io.Serializable;
import java.util.Objects;
import java.util.Properties;
@@ -50,28 +53,22 @@ import java.util.Properties;
public class HoodieTableMetaClient implements Serializable {
private final transient static Logger log = LogManager.getLogger(HoodieTableMetaClient.class);
public static String METAFOLDER_NAME = ".hoodie";
public static String COMMIT_EXTENSION = ".commit";
public static String CLEAN_EXTENSION = ".clean";
public static String SAVEPOINT_EXTENSION = ".savepoint";
public static String INFLIGHT_FILE_SUFFIX = ".inflight";
private String basePath;
private transient FileSystem fs;
private String metaPath;
private HoodieTableType tableType;
private HoodieTableConfig tableConfig;
private HoodieTimeline activeCommitTimeline;
private HoodieTimeline archivedCommitTimeline;
private HoodieTimeline savePointTimeline;
private HoodieTimeline cleanerTimeline;
private HoodieActiveTimeline activeTimeline;
private HoodieArchivedTimeline archivedTimeline;
public HoodieTableMetaClient(FileSystem fs, String basePath) throws DatasetNotFoundException {
// Do not load any timeline by default
this(fs, basePath, false);
}
public HoodieTableMetaClient(FileSystem fs, String basePath,
boolean loadActiveCommitTimelineOnLoad) throws DatasetNotFoundException {
public HoodieTableMetaClient(FileSystem fs, String basePath, boolean loadActiveTimelineOnLoad)
throws DatasetNotFoundException {
log.info("Loading HoodieTableMetaClient from " + basePath);
this.basePath = basePath;
this.fs = fs;
@@ -82,14 +79,15 @@ public class HoodieTableMetaClient implements Serializable {
this.tableConfig = new HoodieTableConfig(fs, metaPath);
this.tableType = tableConfig.getTableType();
log.info("Finished Loading Table of type " + tableType + " from " + basePath);
if (loadActiveCommitTimelineOnLoad) {
if (loadActiveTimelineOnLoad) {
log.info("Loading Active commit timeline for " + basePath);
getActiveCommitTimeline();
getActiveTimeline();
}
}
/**
* For serailizing and de-serializing
*
* @deprecated
*/
public HoodieTableMetaClient() {
@@ -97,6 +95,7 @@ public class HoodieTableMetaClient implements Serializable {
/**
* This method is only used when this object is deserialized in a spark executor.
*
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
@@ -105,6 +104,11 @@ public class HoodieTableMetaClient implements Serializable {
this.fs = FSUtils.getFs();
}
private void writeObject(java.io.ObjectOutputStream out)
throws IOException {
out.defaultWriteObject();
}
/**
* @return Base path
*/
@@ -134,16 +138,16 @@ public class HoodieTableMetaClient implements Serializable {
}
/**
* Get the active commits as a timeline
* Get the active instants as a timeline
*
* @return Active commit timeline
* @return Active instants timeline
* @throws IOException
*/
public synchronized HoodieTimeline getActiveCommitTimeline() {
if (activeCommitTimeline == null) {
activeCommitTimeline = new HoodieActiveCommitTimeline(fs, metaPath);
public synchronized HoodieActiveTimeline getActiveTimeline() {
if (activeTimeline == null) {
activeTimeline = new HoodieActiveTimeline(fs, metaPath);
}
return activeCommitTimeline;
return activeTimeline;
}
/**
@@ -153,40 +157,13 @@ public class HoodieTableMetaClient implements Serializable {
* @return Active commit timeline
* @throws IOException
*/
public HoodieTimeline getArchivedCommitTimeline() {
if (archivedCommitTimeline == null) {
archivedCommitTimeline = new HoodieArchivedCommitTimeline(fs, metaPath);
public synchronized HoodieArchivedTimeline getArchivedTimeline() {
if (archivedTimeline == null) {
archivedTimeline = new HoodieArchivedTimeline(fs, metaPath);
}
return archivedCommitTimeline;
return archivedTimeline;
}
/**
* Get the save points as a timeline.
*
* @return Savepoint timeline
* @throws IOException
*/
public HoodieTimeline getSavePointsTimeline() {
if (savePointTimeline == null) {
savePointTimeline = new HoodieSavePointTimeline(fs, metaPath);
}
return savePointTimeline;
}
/**
* Get the cleaner activity as a timeline.
*
* @return Cleaner activity
* @throws IOException
*/
public HoodieTimeline getCleanerTimeline() {
if (cleanerTimeline == null) {
cleanerTimeline = new HoodieCleanerTimeline(fs, metaPath);
}
return cleanerTimeline;
}
/**
* Helper method to initialize a given path as a hoodie dataset with configs passed in as as Properties
*
@@ -215,34 +192,6 @@ public class HoodieTableMetaClient implements Serializable {
}
// HELPER METHODS TO CREATE META FILE NAMES
public static String makeCommitFileName(String commitTime) {
return commitTime + COMMIT_EXTENSION;
}
public static String makeInflightCommitFileName(String commitTime) {
return commitTime + INFLIGHT_FILE_SUFFIX;
}
public static String makeCleanerFileName(String instant) {
return instant + CLEAN_EXTENSION;
}
public static String makeInflightCleanerFileName(String instant) {
return instant + CLEAN_EXTENSION + INFLIGHT_FILE_SUFFIX;
}
public static String makeInflightSavePointFileName(String commitTime) {
return commitTime + SAVEPOINT_EXTENSION + INFLIGHT_FILE_SUFFIX;
}
public static String makeSavePointFileName(String commitTime) {
return commitTime + SAVEPOINT_EXTENSION;
}
public static String getCommitFromCommitFile(String commitFileName) {
return commitFileName.split("\\.")[0];
}
public static FileStatus[] scanFiles(FileSystem fs, Path metaPath, PathFilter nameFilter)
throws IOException {
return fs.listStatus(metaPath, nameFilter);

View File

@@ -17,153 +17,125 @@
package com.uber.hoodie.common.table;
import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import java.io.IOException;
import java.io.Serializable;
import java.util.Optional;
import java.util.function.BiPredicate;
import java.util.stream.Stream;
/**
* HoodieTimeline allows representation of meta-data events as a timeline.
* Instants are specific points in time represented as strings.
* in this format YYYYMMDDHHmmSS. e.g. 20170101193218
* Any operation on the timeline starts with the inflight instant and then when complete marks
* the completed instant and removes the inflight instant.
* Completed instants are plainly referred to as just instants
* HoodieTimeline is a view of meta-data instants in the hoodie dataset.
* Instants are specific points in time represented as HoodieInstant.
* <p>
* Timelines as immutable once created. Any operation to change the timeline (like create/delete instants)
* will not be reflected unless explicitly reloaded using the reload()
* Timelines are immutable once created and operations create new instance of
* timelines which filter on the instants and this can be chained.
*
* @see com.uber.hoodie.common.table.HoodieTableMetaClient
* @see HoodieDefaultTimeline
* @see HoodieInstant
* @since 0.3.0
*/
public interface HoodieTimeline extends Serializable {
String COMMIT_ACTION = "commit";
String CLEAN_ACTION = "clean";
String SAVEPOINT_ACTION = "savepoint";
String INFLIGHT_EXTENSION = ".inflight";
String COMMIT_EXTENSION = "." + COMMIT_ACTION;
String CLEAN_EXTENSION = "." + CLEAN_ACTION;
String SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION;
//this is to preserve backwards compatibility on commit in-flight filenames
String INFLIGHT_COMMIT_EXTENSION = INFLIGHT_EXTENSION;
String INFLIGHT_CLEAN_EXTENSION = "." + CLEAN_ACTION + INFLIGHT_EXTENSION;
String INFLIGHT_SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION + INFLIGHT_EXTENSION;
/**
* Find all the completed instants after startTs and before or on endTs
* Filter this timeline to just include the in-flights
*
* @return New instance of HoodieTimeline with just in-flights
*/
HoodieTimeline filterInflights();
/**
* Filter this timeline to just include the completed instants
*
* @return New instance of HoodieTimeline with just completed instants
*/
HoodieTimeline filterCompletedInstants();
/**
* Create a new Timeline with instants after startTs and before or on endTs
*
* @param startTs
* @param endTs
* @return Stream of instants
*/
Stream<String> findInstantsInRange(String startTs, String endTs);
HoodieTimeline findInstantsInRange(String startTs, String endTs);
/**
* Find all the completed instants after startTs
* Create a new Timeline with all the instants after startTs
*
* @param commitTime
* @param numCommits
* @return Stream of instants
*/
Stream<String> findInstantsAfter(String commitTime, int numCommits);
HoodieTimeline findInstantsAfter(String commitTime, int numCommits);
/**
* If the timeline has any completed instants
* If the timeline has any instants
*
* @return true if timeline is not empty
* @return true if timeline is empty
*/
boolean hasInstants();
/**
* If the timeline has any in-complete instants
*
* @return true if timeline has any in-complete instants
*/
boolean hasInflightInstants();
boolean empty();
/**
* @return total number of completed instants
*/
int getTotalInstants();
int countInstants();
/**
* @return first completed instant if available
*/
Optional<String> firstInstant();
Optional<HoodieInstant> firstInstant();
/**
* @param n
* @return nth completed instant from the first completed instant
*/
Optional<String> nthInstant(int n);
Optional<HoodieInstant> nthInstant(int n);
/**
* @return last completed instant if available
*/
Optional<String> lastInstant();
Optional<HoodieInstant> lastInstant();
/**
* @param n
* @return nth completed instant going back from the last completed instant
*/
Optional<String> nthFromLastInstant(int n);
Optional<HoodieInstant> nthFromLastInstant(int n);
/**
* @return true if the passed instant is present as a completed instant on the timeline
*/
boolean containsInstant(String instant);
boolean containsInstant(HoodieInstant instant);
/**
* @return true if the passed instant is present as a completed instant on the timeline or
* if the instant is before the first completed instant in the timeline
*/
boolean containsOrBeforeTimelineStarts(String instant);
boolean containsOrBeforeTimelineStarts(String ts);
/**
* @return Get the stream of completed instants
*/
Stream<String> getInstants();
/**
* @return Get the stream of in-flight instants
*/
Stream<String> getInflightInstants();
Stream<HoodieInstant> getInstants();
/**
* @return true if the passed in instant is before the first completed instant in the timeline
*/
boolean isInstantBeforeTimelineStarts(String instant);
/**
* Register the passed in instant as a in-flight
*
* @param instant
*/
void saveInstantAsInflight(String instant);
/**
* Register the passed in instant as a completed instant.
* It needs to have a corresponding in-flight instant, otherwise it will fail.
* Pass a optional byte[] to save with the instant.
*
* @param instant
* @param data
*/
void saveInstantAsComplete(String instant, Optional<byte[]> data);
/**
* Un-Register a completed instant as in-flight. This is usually atomic way to
* revert the effects of a operation on hoodie datasets
*
* @param instant
*/
void revertInstantToInflight(String instant);
/**
* Remove the in-flight instant from the timeline
*
* @param instant
*/
void removeInflightFromTimeline(String instant);
/**
* Reload the timeline. Timelines are immutable once created.
*
* @return
* @throws IOException
*/
HoodieTimeline reload() throws IOException;
boolean isBeforeTimelineStarts(String ts);
/**
* Read the completed instant details
@@ -171,7 +143,7 @@ public interface HoodieTimeline extends Serializable {
* @param instant
* @return
*/
Optional<byte[]> readInstantDetails(String instant);
Optional<byte[]> getInstantDetails(HoodieInstant instant);
/**
* Helper methods to compare instants
@@ -183,8 +155,55 @@ public interface HoodieTimeline extends Serializable {
(commit1, commit2) -> commit1.compareTo(commit2) <= 0;
BiPredicate<String, String> LESSER = (commit1, commit2) -> commit1.compareTo(commit2) < 0;
default boolean compareInstants(String commit1, String commit2,
default boolean compareTimestamps(String commit1, String commit2,
BiPredicate<String, String> predicateToApply) {
return predicateToApply.test(commit1, commit2);
}
static HoodieInstant getCompletedInstant(final HoodieInstant instant) {
return new HoodieInstant(false, instant.getAction(), instant.getTimestamp());
}
static HoodieInstant getInflightInstant(final HoodieInstant instant) {
return new HoodieInstant(true, instant.getAction(), instant.getTimestamp());
}
static String makeCommitFileName(String commitTime) {
return commitTime + HoodieTimeline.COMMIT_EXTENSION;
}
static String makeInflightCommitFileName(String commitTime) {
return commitTime + HoodieTimeline.INFLIGHT_COMMIT_EXTENSION;
}
static String makeCleanerFileName(String instant) {
return instant + HoodieTimeline.CLEAN_EXTENSION;
}
static String makeInflightCleanerFileName(String instant) {
return instant + HoodieTimeline.INFLIGHT_CLEAN_EXTENSION;
}
static String makeInflightSavePointFileName(String commitTime) {
return commitTime + HoodieTimeline.INFLIGHT_SAVEPOINT_EXTENSION;
}
static String makeSavePointFileName(String commitTime) {
return commitTime + HoodieTimeline.SAVEPOINT_EXTENSION;
}
static String getCommitFromCommitFile(String commitFileName) {
return commitFileName.split("\\.")[0];
}
static String makeFileNameAsComplete(String fileName) {
return fileName.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
}
static String makeFileNameAsInflight(String fileName) {
return fileName + HoodieTimeline.INFLIGHT_EXTENSION;
}
}

View File

@@ -51,7 +51,7 @@ public interface TableFileSystemView {
* @param maxCommitTime
* @return
*/
Stream<HoodieDataFile> streamLatestVersionInPartition(String partitionPathStr,
Stream<HoodieDataFile> getLatestVersionInPartition(String partitionPathStr,
String maxCommitTime);
/**
@@ -60,7 +60,7 @@ public interface TableFileSystemView {
* @param partitionPath
* @return
*/
Stream<List<HoodieDataFile>> streamEveryVersionInPartition(String partitionPath);
Stream<List<HoodieDataFile>> getEveryVersionInPartition(String partitionPath);
/**
* Stream all the versions from the passed in fileStatus[] with commit times containing in commitsToReturn.
@@ -69,7 +69,7 @@ public interface TableFileSystemView {
* @param commitsToReturn
* @return
*/
Stream<HoodieDataFile> streamLatestVersionInRange(FileStatus[] fileStatuses,
Stream<HoodieDataFile> getLatestVersionInRange(FileStatus[] fileStatuses,
List<String> commitsToReturn);
/**
@@ -79,15 +79,15 @@ public interface TableFileSystemView {
* @param maxCommitToReturn
* @return
*/
Stream<HoodieDataFile> streamLatestVersionsBeforeOrOn(FileStatus[] fileStatuses,
Stream<HoodieDataFile> getLatestVersionsBeforeOrOn(FileStatus[] fileStatuses,
String maxCommitToReturn);
/**
* Stream latest versions from the passed in FileStatus[].
* Similar to calling streamLatestVersionsBeforeOrOn(fileStatuses, currentTimeAsCommitTime)
* Similar to calling getLatestVersionsBeforeOrOn(fileStatuses, currentTimeAsCommitTime)
*
* @param fileStatuses
* @return
*/
Stream<HoodieDataFile> streamLatestVersions(FileStatus[] fileStatuses);
Stream<HoodieDataFile> getLatestVersions(FileStatus[] fileStatuses);
}

View File

@@ -1,77 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.timeline;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
import java.util.Arrays;
import java.util.stream.Collectors;
/**
* Active commit timeline. Much optimized for reading than the ArchivedTimeline.
*/
public class HoodieActiveCommitTimeline extends HoodieDefaultTimeline {
public HoodieActiveCommitTimeline(FileSystem fs, String metaPath) {
super(fs, metaPath);
String completedInstantExtension = HoodieTableMetaClient.COMMIT_EXTENSION;
String inflightInstantExtension = INFLIGHT_EXTENSION;
FileStatus[] fileStatuses;
try {
fileStatuses = HoodieTableMetaClient.scanFiles(fs, new Path(metaPath),
path -> path.toString().endsWith(completedInstantExtension) || path.toString()
.endsWith(inflightInstantExtension));
} catch (IOException e) {
throw new HoodieIOException("Failed to scan metadata", e);
}
this.instants = Arrays.stream(fileStatuses)
.filter(status -> status.getPath().getName().endsWith(completedInstantExtension))
.map(fileStatus -> fileStatus.getPath().getName().replaceAll(completedInstantExtension, ""))
.sorted().collect(Collectors.toList());
this.inflights = Arrays.stream(fileStatuses).filter(
status -> status.getPath().getName().endsWith(inflightInstantExtension)).map(
fileStatus -> fileStatus.getPath().getName()
.replaceAll(inflightInstantExtension, "")).sorted()
.collect(Collectors.toList());
}
@Override
public String getInflightFileName(String instant) {
return HoodieTableMetaClient.makeInflightCommitFileName(instant);
}
@Override
public String getCompletedFileName(String instant) {
return HoodieTableMetaClient.makeCommitFileName(instant);
}
@Override
protected String getTimelineName() {
return "commit";
}
@Override
public HoodieTimeline reload() throws IOException {
return new HoodieActiveCommitTimeline(fs, metaPath);
}
}

View File

@@ -0,0 +1,252 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.timeline;
import com.google.common.io.Closeables;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.io.Serializable;
import java.util.Arrays;
import java.util.Comparator;
import java.util.Optional;
import java.util.function.Function;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Represents the Active Timeline for the HoodieDataset. Instants for the last 12 hours (configurable)
* is in the ActiveTimeline and the rest are Archived. ActiveTimeline is a special timeline
* that allows for creation of instants on the timeline.
* <p></p>
* The timeline is not automatically reloaded on any mutation operation, clients have to manually call reload()
* so that they can chain multiple mutations to the timeline and then call reload() once.
* <p></p>
* This class can be serialized and de-serialized and on de-serialization the FileSystem is re-initialized.
*/
public class HoodieActiveTimeline extends HoodieDefaultTimeline {
private final transient static Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
private String metaPath;
private transient FileSystem fs;
protected HoodieActiveTimeline(FileSystem fs, String metaPath, String[] includedExtensions) {
// Filter all the filter in the metapath and include only the extensions passed and
// convert them into HoodieInstant
try {
this.instants =
Arrays.stream(HoodieTableMetaClient.scanFiles(fs, new Path(metaPath), path -> {
// Include only the meta files with extensions that needs to be included
String extension = FSUtils.getFileExtension(path.getName());
return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(extension));
})).sorted(Comparator.comparing(
// Sort the meta-data by the instant time (first part of the file name)
fileStatus -> FSUtils.getInstantTime(fileStatus.getPath().getName())))
// create HoodieInstantMarkers from FileStatus, which extracts properties
.map(HoodieInstant::new).collect(Collectors.toList());
log.info("Loaded instants " + instants);
} catch (IOException e) {
throw new HoodieIOException("Failed to scan metadata", e);
}
this.fs = fs;
this.metaPath = metaPath;
// multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
this.details = (Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
}
public HoodieActiveTimeline(FileSystem fs, String metaPath) {
this(fs, metaPath,
new String[] {COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, SAVEPOINT_EXTENSION,
INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION});
}
/**
* For serialization and de-serialization only.
* @deprecated
*/
public HoodieActiveTimeline() {
}
/**
* This method is only used when this object is deserialized in a spark executor.
*
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
this.fs = FSUtils.getFs();
}
/**
* Get only the commits (inflight and completed) in the active timeline
*
* @return
*/
public HoodieTimeline getCommitTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(COMMIT_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
/**
* Get only the cleaner action (inflight and completed) in the active timeline
*
* @return
*/
public HoodieTimeline getCleanerTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(CLEAN_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
/**
* Get only the save point action (inflight and completed) in the active timeline
*
* @return
*/
public HoodieTimeline getSavePointTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(SAVEPOINT_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
protected Stream<HoodieInstant> filterInstantsByAction(String action) {
return instants.stream().filter(s -> s.getAction().equals(action));
}
public void createInflight(HoodieInstant instant) {
log.info("Creating a new in-flight instant " + instant);
// Create the in-flight file
createFileInMetaPath(instant.getFileName(), Optional.empty());
}
public void saveAsComplete(HoodieInstant instant, Optional<byte[]> data) {
log.info("Marking instant complete " + instant);
moveInflightToComplete(instant, HoodieTimeline.getCompletedInstant(instant), data);
log.info("Completed " + instant);
}
public void revertToInflight(HoodieInstant instant) {
log.info("Reverting instant to inflight " + instant);
moveCompleteToInflight(instant, HoodieTimeline.getInflightInstant(instant));
log.info("Reverted " + instant + " to inflight");
}
public void deleteInflight(HoodieInstant instant) {
log.info("Deleting in-flight " + instant);
Path inFlightCommitFilePath = new Path(metaPath, instant.getFileName());
try {
boolean result = fs.delete(inFlightCommitFilePath, false);
if (result) {
log.info("Removed in-flight " + instant);
} else {
throw new HoodieIOException("Could not delete in-flight instant " + instant);
}
} catch (IOException e) {
throw new HoodieIOException(
"Could not remove inflight commit " + inFlightCommitFilePath, e);
}
}
@Override
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
Path detailPath = new Path(metaPath, instant.getFileName());
return readDataFromPath(detailPath);
}
protected void moveInflightToComplete(HoodieInstant inflight, HoodieInstant completed,
Optional<byte[]> data) {
Path commitFilePath = new Path(metaPath, completed.getFileName());
try {
// open a new file and write the commit metadata in
Path inflightCommitFile = new Path(metaPath, inflight.getFileName());
createFileInMetaPath(inflight.getFileName(), data);
boolean success = fs.rename(inflightCommitFile, commitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + inflightCommitFile + " to " + commitFilePath);
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete " + inflight, e);
}
}
protected void moveCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) {
Path inFlightCommitFilePath = new Path(metaPath, inflight.getFileName());
try {
if (!fs.exists(inFlightCommitFilePath)) {
Path commitFilePath = new Path(metaPath, completed.getFileName());
boolean success = fs.rename(commitFilePath, inFlightCommitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + commitFilePath + " to " + inFlightCommitFilePath);
}
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete revert " + completed, e);
}
}
protected void createFileInMetaPath(String filename, Optional<byte[]> content) {
Path fullPath = new Path(metaPath, filename);
try {
if (!content.isPresent()) {
if (fs.createNewFile(fullPath)) {
log.info("Created a new file in meta path: " + fullPath);
return;
}
} else {
FSDataOutputStream fsout = fs.create(fullPath, true);
fsout.write(content.get());
fsout.close();
return;
}
throw new HoodieIOException("Failed to create file " + fullPath);
} catch (IOException e) {
throw new HoodieIOException("Failed to create file " + fullPath, e);
}
}
protected Optional<byte[]> readDataFromPath(Path detailPath) {
FSDataInputStream is = null;
try {
is = fs.open(detailPath);
return Optional.of(IOUtils.toByteArray(is));
} catch (IOException e) {
throw new HoodieIOException("Could not read commit details from " + detailPath, e);
} finally {
if (is != null) {
Closeables.closeQuietly(is);
}
}
}
public HoodieActiveTimeline reload() {
return new HoodieActiveTimeline(fs, metaPath);
}
}

View File

@@ -18,30 +18,42 @@ package com.uber.hoodie.common.table.timeline;
import com.google.common.io.Closeables;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayList;
import java.io.Serializable;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collectors;
/**
* Archived commit timeline. These commits are usually cleaned up and the meta data is archived for
* future triaging
*
* @since 0.3.0
* Represents the Archived Timeline for the HoodieDataset. Instants for the last 12 hours (configurable)
* is in the ActiveTimeline and the rest are in ArchivedTimeline.
* <p></p>
* Instants are read from the archive file during initialization and never refreshed. To refresh, clients
* need to call reload()
* <p></p>
* This class can be serialized and de-serialized and on de-serialization the FileSystem is re-initialized.
*/
public class HoodieArchivedCommitTimeline extends HoodieDefaultTimeline {
public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits.archived";
private transient Map<String, byte[]> readCommits = new HashMap<>();
private transient FileSystem fs;
private String metaPath;
private Map<String, byte[]> readCommits = new HashMap<>();
public HoodieArchivedCommitTimeline(FileSystem fs, String metaPath) {
private final transient static Logger log = LogManager.getLogger(HoodieArchivedTimeline.class);
public HoodieArchivedTimeline(FileSystem fs, String metaPath) {
// Read back the commits to make sure
Path archiveLogPath = getArchiveLogPath(metaPath);
try {
@@ -55,8 +67,9 @@ public class HoodieArchivedCommitTimeline extends HoodieDefaultTimeline {
// This is okay because only tooling will load the archived commit timeline today
readCommits.put(key.toString(), Arrays.copyOf(val.getBytes(), val.getLength()));
}
this.instants = new ArrayList<>(readCommits.keySet());
this.inflights = new ArrayList<>(0);
this.instants = readCommits.keySet().stream().map(
s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)).collect(
Collectors.toList());
} finally {
Closeables.closeQuietly(reader);
}
@@ -64,58 +77,42 @@ public class HoodieArchivedCommitTimeline extends HoodieDefaultTimeline {
throw new HoodieIOException(
"Could not load archived commit timeline from path " + archiveLogPath, e);
}
// multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
this.details = (Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
this.fs = fs;
this.metaPath = metaPath;
}
@Override
public void saveInstantAsInflight(String instant) {
throw new UnsupportedOperationException(
"Could not save inflight instant in ArchivedTimeline " + instant);
/**
* For serialization and de-serialization only.
* @deprecated
*/
public HoodieArchivedTimeline() {
}
@Override
public void saveInstantAsComplete(String instant, Optional<byte[]> data) {
throw new UnsupportedOperationException(
"Could not save instant as complete in ArchivedTimeline " + instant);
/**
* This method is only used when this object is deserialized in a spark executor.
*
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
this.fs = FSUtils.getFs();
}
@Override
public void revertInstantToInflight(String instant) {
throw new UnsupportedOperationException(
"Could not revert instant in ArchivedTimeline " + instant);
}
@Override
public void removeInflightFromTimeline(String instant) {
throw new UnsupportedOperationException(
"Could not delete inflight instant from ArchivedTimeline " + instant);
}
@Override
public HoodieTimeline reload() throws IOException {
return new HoodieArchivedCommitTimeline(fs, metaPath);
}
@Override
public Optional<byte[]> readInstantDetails(String instant) {
return Optional.ofNullable(readCommits.get(instant));
}
@Override
protected String getInflightFileName(String instant) {
throw new UnsupportedOperationException("No inflight filename for archived commits");
}
@Override
protected String getCompletedFileName(String instant) {
throw new UnsupportedOperationException("No inflight filename for archived commits");
}
@Override
protected String getTimelineName() {
return "archived-commits";
}
public static Path getArchiveLogPath(String metaPath) {
return new Path(metaPath, HOODIE_COMMIT_ARCHIVE_LOG_FILE);
}
@Override
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
return Optional.ofNullable(readCommits.get(instant.getTimestamp()));
}
public HoodieArchivedTimeline reload() {
return new HoodieArchivedTimeline(fs, metaPath);
}
}

View File

@@ -1,57 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.timeline;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
import java.util.Optional;
public class HoodieCleanerTimeline extends HoodieDefaultTimeline {
public HoodieCleanerTimeline(FileSystem fs, String path) {
super(fs, path, HoodieTableMetaClient.CLEAN_EXTENSION);
}
@Override
public HoodieTimeline reload() throws IOException {
return new HoodieCleanerTimeline(fs, metaPath);
}
@Override
public Optional<byte[]> readInstantDetails(String instant) {
// TODO - Nothing about the clean written today - this should change
return Optional.empty();
}
@Override
protected String getInflightFileName(String instant) {
return HoodieTableMetaClient.makeInflightCleanerFileName(instant);
}
@Override
protected String getCompletedFileName(String instant) {
return HoodieTableMetaClient.makeCleanerFileName(instant);
}
@Override
protected String getTimelineName() {
return "cleaner";
}
}

View File

@@ -16,329 +16,131 @@
package com.uber.hoodie.common.table.timeline;
import com.google.common.io.Closeables;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* HoodieTimeline allows representation of meta-data events as a timeline.
* Instants are specific points in time represented as strings.
* in this format YYYYMMDDHHmmSS. e.g. 20170101193218
* Any operation on the timeline starts with the inflight instant and then when complete marks
* the completed instant and removes the inflight instant.
* Completed instants are plainly referred to as just instants
* <p>
* Timelines as immutable once created. Any operation to change the timeline (like create/delete instants)
* will not be reflected unless explicitly reloaded using the reload()
* HoodieDefaultTimeline is a default implementation of the HoodieTimeline.
* It provides methods to inspect a List[HoodieInstant]. Function to get the details of the instant
* is passed in as a lamdba.
*
* @see com.uber.hoodie.common.table.HoodieTableMetaClient
* @see HoodieTimeline
* @since 0.3.0
*/
public abstract class HoodieDefaultTimeline implements HoodieTimeline {
public class HoodieDefaultTimeline implements HoodieTimeline {
private final transient static Logger log = LogManager.getLogger(HoodieDefaultTimeline.class);
public static final String INFLIGHT_EXTENSION = ".inflight";
protected String metaPath;
protected transient FileSystem fs;
protected List<String> inflights;
protected List<String> instants;
protected Function<HoodieInstant, Optional<byte[]>> details;
protected List<HoodieInstant> instants;
public HoodieDefaultTimeline(FileSystem fs, String metaPath, String fileExtension) {
String completedInstantExtension = fileExtension;
String inflightInstantExtension = fileExtension + INFLIGHT_EXTENSION;
FileStatus[] fileStatuses;
try {
fileStatuses = HoodieTableMetaClient.scanFiles(fs, new Path(metaPath),
path -> path.toString().endsWith(completedInstantExtension) || path.toString()
.endsWith(inflightInstantExtension));
} catch (IOException e) {
throw new HoodieIOException("Failed to scan metadata", e);
}
this.instants = Arrays.stream(fileStatuses)
.filter(status -> status.getPath().getName().endsWith(completedInstantExtension))
.map(fileStatus -> fileStatus.getPath().getName().replaceAll(completedInstantExtension, ""))
.sorted().collect(Collectors.toList());
this.inflights = Arrays.stream(fileStatuses).filter(
status -> status.getPath().getName().endsWith(inflightInstantExtension)).map(
fileStatus -> fileStatus.getPath().getName()
.replaceAll(inflightInstantExtension, "")).sorted()
.collect(Collectors.toList());
this.fs = fs;
this.metaPath = metaPath;
}
public HoodieDefaultTimeline(Stream<String> instants, Stream<String> inflights) {
public HoodieDefaultTimeline(Stream<HoodieInstant> instants,
Function<HoodieInstant, Optional<byte[]>> details) {
this.instants = instants.collect(Collectors.toList());
this.inflights = inflights.collect(Collectors.toList());
}
/**
* This constructor only supports backwards compatibility in inflight commits in ActiveCommitTimeline.
* This should never be used.
*
* @param fs
* @param metaPath
* @deprecated
*/
public HoodieDefaultTimeline(FileSystem fs, String metaPath) {
this.fs = fs;
this.metaPath = metaPath;
this.details = details;
}
/**
* For serailizing and de-serializing
*
* @deprecated
*/
public HoodieDefaultTimeline() {
}
public HoodieTimeline filterInflights() {
return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isInflight),
details);
}
/**
* This method is only used when this object is deserialized in a spark executor.
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
this.fs = FSUtils.getFs();
public HoodieTimeline filterCompletedInstants() {
return new HoodieDefaultTimeline(instants.stream().filter(s -> !s.isInflight()), details);
}
@Override
public Stream<String> findInstantsInRange(String startTs, String endTs) {
return instants.stream().filter(
s -> compareInstants(s, startTs, GREATER) && compareInstants(s, endTs,
LESSER_OR_EQUAL));
public HoodieDefaultTimeline findInstantsInRange(String startTs, String endTs) {
return new HoodieDefaultTimeline(instants.stream().filter(
s -> compareTimestamps(s.getTimestamp(), startTs, GREATER) && compareTimestamps(
s.getTimestamp(), endTs, LESSER_OR_EQUAL)), details);
}
@Override
public Stream<String> findInstantsAfter(String commitTime, int numCommits) {
return instants.stream().filter(s -> compareInstants(s, commitTime, GREATER))
.limit(numCommits);
public HoodieDefaultTimeline findInstantsAfter(String commitTime, int numCommits) {
return new HoodieDefaultTimeline(
instants.stream().filter(s -> compareTimestamps(s.getTimestamp(), commitTime, GREATER))
.limit(numCommits), details);
}
@Override
public boolean hasInstants() {
return instants.stream().count() != 0;
public boolean empty() {
return !instants.stream().findFirst().isPresent();
}
@Override
public boolean hasInflightInstants() {
return inflights.stream().count() != 0;
}
@Override
public int getTotalInstants() {
public int countInstants() {
return new Long(instants.stream().count()).intValue();
}
@Override
public Optional<String> firstInstant() {
public Optional<HoodieInstant> firstInstant() {
return instants.stream().findFirst();
}
@Override
public Optional<String> nthInstant(int n) {
if(!hasInstants() || n >= getTotalInstants()) {
public Optional<HoodieInstant> nthInstant(int n) {
if (empty() || n >= countInstants()) {
return Optional.empty();
}
return Optional.of(instants.get(n));
}
@Override
public Optional<String> lastInstant() {
return hasInstants() ? nthInstant(getTotalInstants() - 1) : Optional.empty();
public Optional<HoodieInstant> lastInstant() {
return empty() ? Optional.empty() : nthInstant(countInstants() - 1);
}
@Override
public Optional<String> nthFromLastInstant(int n) {
if(getTotalInstants() < n + 1) {
public Optional<HoodieInstant> nthFromLastInstant(int n) {
if (countInstants() < n + 1) {
return Optional.empty();
}
return nthInstant(getTotalInstants() - 1 - n);
return nthInstant(countInstants() - 1 - n);
}
@Override
public boolean containsInstant(String instant) {
public boolean containsInstant(HoodieInstant instant) {
return instants.stream().anyMatch(s -> s.equals(instant));
}
@Override
public boolean containsOrBeforeTimelineStarts(String instant) {
return containsInstant(instant) || isInstantBeforeTimelineStarts(instant);
return instants.stream().anyMatch(s -> s.getTimestamp().equals(instant))
|| isBeforeTimelineStarts(instant);
}
@Override
public Stream<String> getInstants() {
public Stream<HoodieInstant> getInstants() {
return instants.stream();
}
@Override
public Stream<String> getInflightInstants() {
return inflights.stream();
public boolean isBeforeTimelineStarts(String instant) {
Optional<HoodieInstant> firstCommit = firstInstant();
return firstCommit.isPresent() && compareTimestamps(instant,
firstCommit.get().getTimestamp(), LESSER);
}
@Override
public boolean isInstantBeforeTimelineStarts(String instant) {
Optional<String> firstCommit = firstInstant();
return firstCommit.isPresent() && compareInstants(instant, firstCommit.get(), LESSER);
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
return details.apply(instant);
}
@Override
public void saveInstantAsInflight(String instant) {
log.info("Creating a new in-flight " + getTimelineName() + " " + instant);
// Create the in-flight file
createFileInMetaPath(getInflightFileName(instant), Optional.empty());
}
@Override
public void saveInstantAsComplete(String instant, Optional<byte[]> data) {
log.info("Marking complete " + getTimelineName() + " " + instant);
moveInflightToComplete(instant, data, getCompletedFileName(instant),
HoodieTableMetaClient.makeInflightCommitFileName(instant));
log.info("Completed " + getTimelineName() + " " + instant);
}
@Override
public void revertInstantToInflight(String instant) {
log.info("Reverting instant to inflight " + getTimelineName() + " " + instant);
moveCompleteToInflight(instant, getCompletedFileName(instant),
getInflightFileName(instant));
log.info("Reverted " + getTimelineName() + " " + instant + " to inflight");
}
@Override
public void removeInflightFromTimeline(String instant) {
log.info("Removing in-flight " + getTimelineName() + " " + instant);
String inFlightCommitFileName = getInflightFileName(instant);
Path inFlightCommitFilePath = new Path(metaPath, inFlightCommitFileName);
try {
fs.delete(inFlightCommitFilePath, false);
log.info("Removed in-flight " + getTimelineName() + " " + instant);
} catch (IOException e) {
throw new HoodieIOException(
"Could not remove inflight commit " + inFlightCommitFilePath, e);
}
}
@Override
public Optional<byte[]> readInstantDetails(String instant) {
Path detailPath = new Path(metaPath, getCompletedFileName(instant));
return readDataFromPath(detailPath);
}
/**
* Get the in-flight instant file name
*
* @param instant
* @return
*/
protected abstract String getInflightFileName(String instant);
/**
* Get the completed instant file name
*
* @param instant
* @return
*/
protected abstract String getCompletedFileName(String instant);
/**
* Get the timeline name
*
* @return
*/
protected abstract String getTimelineName();
protected void moveInflightToComplete(String instant, Optional<byte[]> data,
String commitFileName, String inflightFileName) {
Path commitFilePath = new Path(metaPath, commitFileName);
try {
// open a new file and write the commit metadata in
Path inflightCommitFile = new Path(metaPath, inflightFileName);
createFileInMetaPath(inflightFileName, data);
boolean success = fs.rename(inflightCommitFile, commitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + inflightCommitFile + " to " + commitFilePath);
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete commit " + instant, e);
}
}
protected void moveCompleteToInflight(String instant, String commitFileName,
String inflightFileName) {
Path inFlightCommitFilePath = new Path(metaPath, inflightFileName);
try {
if (!fs.exists(inFlightCommitFilePath)) {
Path commitFilePath = new Path(metaPath, commitFileName);
boolean success = fs.rename(commitFilePath, inFlightCommitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + commitFilePath + " to " + inFlightCommitFilePath);
}
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete commit revert " + instant, e);
}
}
protected void createFileInMetaPath(String filename, Optional<byte[]> content) {
Path fullPath = new Path(metaPath, filename);
try {
if (!content.isPresent()) {
if (fs.createNewFile(fullPath)) {
log.info("Created a new file in meta path: " + fullPath);
return;
}
} else {
FSDataOutputStream fsout = fs.create(fullPath, true);
fsout.write(content.get());
fsout.close();
return;
}
throw new HoodieIOException("Failed to create file " + fullPath);
} catch (IOException e) {
throw new HoodieIOException("Failed to create file " + fullPath, e);
}
}
protected Optional<byte[]> readDataFromPath(Path detailPath) {
FSDataInputStream is = null;
try {
is = fs.open(detailPath);
return Optional.of(IOUtils.toByteArray(is));
} catch (IOException e) {
throw new HoodieIOException("Could not read commit details from " + detailPath, e);
} finally {
if (is != null) {
Closeables.closeQuietly(is);
}
}
}
@Override
public String toString() {
return this.getClass().getName() + ": " + instants.stream().map(Object::toString)

View File

@@ -0,0 +1,121 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.timeline;
import com.google.common.io.Files;
import com.uber.hoodie.common.table.HoodieTimeline;
import org.apache.hadoop.fs.FileStatus;
import java.io.Serializable;
import java.util.Objects;
import java.util.Optional;
/**
* A Hoodie Instant represents a action done on a hoodie dataset.
* All actions start with a inflight instant and then create a completed instant after done.
*
* @see HoodieTimeline
*/
public class HoodieInstant implements Serializable {
private boolean isInflight = false;
private String action;
private String timestamp;
/**
* Load the instant from the meta FileStatus
* @param fileStatus
*/
public HoodieInstant(FileStatus fileStatus) {
// First read the instant timestamp. [==>20170101193025<==].commit
String fileName = fileStatus.getPath().getName();
String fileExtension = Files.getFileExtension(fileName);
timestamp = fileName.replace("." + fileExtension, "");
// Next read the action for this marker
action = fileExtension;
if(action.equals("inflight")) {
// This is to support backwards compatibility on how in-flight commit files were written
// General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
action = "commit";
isInflight = true;
} else if (action.contains(HoodieTimeline.INFLIGHT_EXTENSION)) {
isInflight = true;
action = action.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
}
}
public HoodieInstant(boolean isInflight, String action, String timestamp) {
this.isInflight = isInflight;
this.action = action;
this.timestamp = timestamp;
}
public boolean isInflight() {
return isInflight;
}
public String getAction() {
return action;
}
public String getTimestamp() {
return timestamp;
}
/**
* Get the filename for this instant
* @return
*/
public String getFileName() {
if (HoodieTimeline.COMMIT_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightCommitFileName(timestamp) :
HoodieTimeline.makeCommitFileName(timestamp);
} else if (HoodieTimeline.CLEAN_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightCleanerFileName(timestamp) :
HoodieTimeline.makeCleanerFileName(timestamp);
} else if (HoodieTimeline.SAVEPOINT_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightSavePointFileName(timestamp) :
HoodieTimeline.makeSavePointFileName(timestamp);
}
throw new IllegalArgumentException("Cannot get file name for unknown action " + action);
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
HoodieInstant that = (HoodieInstant) o;
return isInflight == that.isInflight &&
Objects.equals(action, that.action) &&
Objects.equals(timestamp, that.timestamp);
}
@Override
public int hashCode() {
return Objects.hash(isInflight, action, timestamp);
}
@Override
public String toString() {
return "[" + ((isInflight) ? "==>" : "") + timestamp + "__" + action + "]";
}
}

View File

@@ -1,50 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.timeline;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
public class HoodieSavePointTimeline extends HoodieDefaultTimeline {
public HoodieSavePointTimeline(FileSystem fs, String metaPath) {
super(fs, metaPath, HoodieTableMetaClient.SAVEPOINT_EXTENSION);
}
@Override
public HoodieTimeline reload() throws IOException {
return new HoodieSavePointTimeline(fs, metaPath);
}
@Override
protected String getInflightFileName(String instant) {
return HoodieTableMetaClient.makeInflightSavePointFileName(instant);
}
@Override
protected String getCompletedFileName(String instant) {
return HoodieTableMetaClient.makeSavePointFileName(instant);
}
@Override
protected String getTimelineName() {
return "savepoint";
}
}

View File

@@ -20,6 +20,7 @@ import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.FileStatus;
@@ -38,7 +39,7 @@ import java.util.stream.Stream;
/**
* Common abstract implementation for multiple TableFileSystemView Implementations.
* 2 possible implementations are ReadOptimizedView and RealtimeView
*
* <p>
* Concrete implementations extending this abstract class, should only implement
* listDataFilesInPartition which includes files to be included in the view
*
@@ -54,24 +55,26 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
public AbstractTableFileSystemView(FileSystem fs, HoodieTableMetaClient metaClient) {
this.metaClient = metaClient;
this.fs = fs;
this.activeCommitTimeline = metaClient.getActiveCommitTimeline();
// Get the active timeline and filter only completed commits
this.activeCommitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
}
public Stream<HoodieDataFile> getLatestDataFilesForFileId(final String partitionPath,
String fileId) {
Optional<String> lastInstant = activeCommitTimeline.lastInstant();
Optional<HoodieInstant> lastInstant = activeCommitTimeline.lastInstant();
if (lastInstant.isPresent()) {
return streamLatestVersionInPartition(partitionPath, lastInstant.get())
return getLatestVersionInPartition(partitionPath, lastInstant.get().getTimestamp())
.filter(hoodieDataFile -> hoodieDataFile.getFileId().equals(fileId));
}
return Stream.empty();
}
@Override
public Stream<HoodieDataFile> streamLatestVersionInPartition(String partitionPathStr,
public Stream<HoodieDataFile> getLatestVersionInPartition(String partitionPathStr,
String maxCommitTime) {
try {
return streamLatestVersionsBeforeOrOn(listDataFilesInPartition(partitionPathStr),
return getLatestVersionsBeforeOrOn(listDataFilesInPartition(partitionPathStr),
maxCommitTime);
} catch (IOException e) {
throw new HoodieIOException(
@@ -81,11 +84,11 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
@Override
public Stream<List<HoodieDataFile>> streamEveryVersionInPartition(String partitionPath) {
public Stream<List<HoodieDataFile>> getEveryVersionInPartition(String partitionPath) {
try {
if(activeCommitTimeline.lastInstant().isPresent()) {
return streamFilesByFileId(listDataFilesInPartition(partitionPath),
activeCommitTimeline.lastInstant().get());
if (activeCommitTimeline.lastInstant().isPresent()) {
return getFilesByFileId(listDataFilesInPartition(partitionPath),
activeCommitTimeline.lastInstant().get().getTimestamp());
}
return Stream.empty();
} catch (IOException e) {
@@ -98,13 +101,14 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
throws IOException;
@Override
public Stream<HoodieDataFile> streamLatestVersionInRange(FileStatus[] fileStatuses,
public Stream<HoodieDataFile> getLatestVersionInRange(FileStatus[] fileStatuses,
List<String> commitsToReturn) {
if (!activeCommitTimeline.hasInstants() || commitsToReturn.isEmpty()) {
if (activeCommitTimeline.empty() || commitsToReturn.isEmpty()) {
return Stream.empty();
}
try {
return streamFilesByFileId(fileStatuses, activeCommitTimeline.lastInstant().get())
return getFilesByFileId(fileStatuses,
activeCommitTimeline.lastInstant().get().getTimestamp())
.map((Function<List<HoodieDataFile>, Optional<HoodieDataFile>>) fss -> {
for (HoodieDataFile fs : fss) {
if (commitsToReturn.contains(fs.getCommitTime())) {
@@ -120,17 +124,18 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
}
@Override
public Stream<HoodieDataFile> streamLatestVersionsBeforeOrOn(FileStatus[] fileStatuses,
public Stream<HoodieDataFile> getLatestVersionsBeforeOrOn(FileStatus[] fileStatuses,
String maxCommitToReturn) {
try {
if (!activeCommitTimeline.hasInstants()) {
if (activeCommitTimeline.empty()) {
return Stream.empty();
}
return streamFilesByFileId(fileStatuses, activeCommitTimeline.lastInstant().get())
return getFilesByFileId(fileStatuses,
activeCommitTimeline.lastInstant().get().getTimestamp())
.map((Function<List<HoodieDataFile>, Optional<HoodieDataFile>>) fss -> {
for (HoodieDataFile fs1 : fss) {
if (activeCommitTimeline
.compareInstants(fs1.getCommitTime(), maxCommitToReturn,
.compareTimestamps(fs1.getCommitTime(), maxCommitToReturn,
HoodieTimeline.LESSER_OR_EQUAL)) {
return Optional.of(fs1);
}
@@ -143,19 +148,20 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
}
@Override
public Stream<HoodieDataFile> streamLatestVersions(FileStatus[] fileStatuses) {
public Stream<HoodieDataFile> getLatestVersions(FileStatus[] fileStatuses) {
try {
if (!activeCommitTimeline.hasInstants()) {
if (activeCommitTimeline.empty()) {
return Stream.empty();
}
return streamFilesByFileId(fileStatuses, activeCommitTimeline.lastInstant().get())
return getFilesByFileId(fileStatuses,
activeCommitTimeline.lastInstant().get().getTimestamp())
.map(statuses -> statuses.get(0));
} catch (IOException e) {
throw new HoodieIOException("Could not filter files for latest version ", e);
}
}
protected Stream<List<HoodieDataFile>> streamFilesByFileId(FileStatus[] files,
protected Stream<List<HoodieDataFile>> getFilesByFileId(FileStatus[] files,
String maxCommitTime) throws IOException {
return groupFilesByFileId(files, maxCommitTime).values().stream();
}
@@ -173,7 +179,7 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
return Arrays.stream(files).flatMap(fileStatus -> {
HoodieDataFile dataFile = new HoodieDataFile(fileStatus);
if (activeCommitTimeline.containsOrBeforeTimelineStarts(dataFile.getCommitTime())
&& activeCommitTimeline.compareInstants(dataFile.getCommitTime(), maxCommitTime,
&& activeCommitTimeline.compareTimestamps(dataFile.getCommitTime(), maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL)) {
return Stream.of(Pair.of(dataFile.getFileId(), dataFile));
}

View File

@@ -36,7 +36,7 @@ public class ReadOptimizedTableView extends AbstractTableFileSystemView {
Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr);
try {
return fs.listStatus(partitionPath, path -> path.getName()
.contains(metaClient.getTableConfig().getROStorageFormat().getFileExtension()));
.contains(metaClient.getTableConfig().getROFileFormat().getFileExtension()));
} catch (IOException e) {
throw new HoodieIOException(
"Failed to list data files in partition " + partitionPathStr, e);

View File

@@ -16,6 +16,10 @@
package com.uber.hoodie.common.util;
import com.google.common.base.Preconditions;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@@ -24,6 +28,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
@@ -91,18 +96,27 @@ public class FSUtils {
/**
* Obtain all the partition paths, that are present in this table.
*/
public static List<String> getAllPartitionPaths(FileSystem fs, String basePath) throws IOException {
public static List<String> getAllPartitionPaths(FileSystem fs, String basePath)
throws IOException {
List<String> partitionsToClean = new ArrayList<>();
// TODO(vc): For now, assume partitions are two levels down from base path.
FileStatus[] folders = fs.globStatus(new Path(basePath + "/*/*/*"));
for (FileStatus status : folders) {
Path path = status.getPath();
partitionsToClean.add(String.format("%s/%s/%s",
path.getParent().getParent().getName(),
path.getParent().getName(),
path.getName()));
partitionsToClean.add(String.format("%s/%s/%s", path.getParent().getParent().getName(),
path.getParent().getName(), path.getName()));
}
return partitionsToClean;
}
public static String getFileExtension(String fullName) {
Preconditions.checkNotNull(fullName);
String fileName = (new File(fullName)).getName();
int dotIndex = fileName.indexOf('.');
return dotIndex == -1 ? "" : fileName.substring(dotIndex);
}
public static String getInstantTime(String name) {
return name.replace(getFileExtension(name), "");
}
}

View File

@@ -19,8 +19,10 @@ package com.uber.hoodie.common.model;
import com.esotericsoftware.kryo.Kryo;
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
import com.esotericsoftware.kryo.serializers.JavaSerializer;
import com.uber.hoodie.common.table.HoodieTableConfig;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.hadoop.fs.FileSystem;
@@ -30,6 +32,8 @@ import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.io.Serializable;
import java.text.SimpleDateFormat;
import java.util.Date;
@@ -66,13 +70,13 @@ public class HoodieTestUtils {
public static final void createCommitFiles(String basePath, String... commitTimes) throws IOException {
for (String commitTime: commitTimes) {
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + HoodieTableMetaClient.makeCommitFileName(commitTime)).createNewFile();
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + HoodieTimeline.makeCommitFileName(commitTime)).createNewFile();
}
}
public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException {
for (String commitTime: commitTimes) {
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + HoodieTableMetaClient.makeInflightCommitFileName(commitTime)).createNewFile();
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + HoodieTimeline.makeInflightCommitFileName(commitTime)).createNewFile();
}
}
@@ -97,15 +101,15 @@ public class HoodieTestUtils {
}
public static final boolean doesCommitExist(String basePath, String commitTime) {
return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + commitTime + HoodieTableMetaClient.COMMIT_EXTENSION).exists();
return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION).exists();
}
public static final boolean doesInflightExist(String basePath, String commitTime) {
return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + commitTime + HoodieTableMetaClient.INFLIGHT_FILE_SUFFIX).exists();
return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + commitTime + HoodieTimeline.INFLIGHT_EXTENSION).exists();
}
public static String makeInflightTestFileName(String instant) {
return instant + TEST_EXTENSION + HoodieTableMetaClient.INFLIGHT_FILE_SUFFIX;
return instant + TEST_EXTENSION + HoodieTimeline.INFLIGHT_EXTENSION;
}
public static String makeTestFileName(String instant) {
@@ -123,9 +127,12 @@ public class HoodieTestUtils {
assert !iter1.hasNext() && !iter2.hasNext();
}
public static <T extends Serializable> T serializeDeserialize(T object, Class<T> clazz) {
public static <T extends Serializable> T serializeDeserialize(T object, Class<T> clazz)
throws IOException, ClassNotFoundException {
// Using Kyro as the default serializer in Spark Jobs
Kryo kryo = new Kryo();
kryo.register(HoodieTableMetaClient.class, new JavaSerializer());
ByteArrayOutputStream baos = new ByteArrayOutputStream();
Output output = new Output(baos);
kryo.writeObject(output, object);

View File

@@ -16,14 +16,12 @@
package com.uber.hoodie.common.table;
import com.esotericsoftware.kryo.Kryo;
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
import com.google.common.collect.Lists;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.timeline.HoodieArchivedCommitTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.ArrayFile;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
@@ -31,13 +29,15 @@ import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.Optional;
import java.util.stream.Collectors;
import static org.junit.Assert.*;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
public class HoodieTableMetaClientTest {
private HoodieTableMetaClient metaClient;
@@ -61,48 +61,53 @@ public class HoodieTableMetaClientTest {
}
@Test
public void checkSerDe() throws IOException {
// check if this object is serialized and se-serialized, we are able to read from the file system
public void checkSerDe() throws IOException, ClassNotFoundException {
// check if this object is serialized and de-serialized, we are able to read from the file system
HoodieTableMetaClient deseralizedMetaClient =
HoodieTestUtils.serializeDeserialize(metaClient, HoodieTableMetaClient.class);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
commitTimeline.saveInstantAsInflight("1");
commitTimeline.saveInstantAsComplete("1", Optional.of("test-detail".getBytes()));
assertNotNull(deseralizedMetaClient);
HoodieActiveTimeline commitTimeline = deseralizedMetaClient.getActiveTimeline();
HoodieInstant instant =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
commitTimeline.createInflight(instant);
commitTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes()));
commitTimeline = commitTimeline.reload();
assertEquals("Commit should be 1", "1", commitTimeline.getInstants().findFirst().get());
HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant);
assertEquals("Commit should be 1 and completed", completedInstant,
commitTimeline.getInstants().findFirst().get());
assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(),
commitTimeline.readInstantDetails("1").get());
commitTimeline.getInstantDetails(completedInstant).get());
}
@Test
public void checkCommitTimeline() throws IOException {
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
assertFalse("Should be empty commit timeline",
commitTimeline.getInstants().findFirst().isPresent());
assertFalse("Should be empty commit timeline",
commitTimeline.getInflightInstants().findFirst().isPresent());
commitTimeline.saveInstantAsInflight("1");
commitTimeline.saveInstantAsComplete("1", Optional.of("test-detail".getBytes()));
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
HoodieTimeline activeCommitTimeline = activeTimeline.getCommitTimeline();
assertTrue("Should be empty commit timeline", activeCommitTimeline.empty());
HoodieInstant instant =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
activeTimeline.createInflight(instant);
activeTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes()));
// Commit timeline should not auto-reload every time getActiveCommitTimeline(), it should be cached
commitTimeline = metaClient.getActiveCommitTimeline();
assertFalse("Should be empty commit timeline",
commitTimeline.getInstants().findFirst().isPresent());
assertFalse("Should be empty commit timeline",
commitTimeline.getInflightInstants().findFirst().isPresent());
activeTimeline = metaClient.getActiveTimeline();
activeCommitTimeline = activeTimeline.getCommitTimeline();
assertTrue("Should be empty commit timeline", activeCommitTimeline.empty());
commitTimeline = commitTimeline.reload();
assertTrue("Should be the 1 commit we made",
commitTimeline.getInstants().findFirst().isPresent());
assertEquals("Commit should be 1", "1", commitTimeline.getInstants().findFirst().get());
HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant);
activeTimeline = activeTimeline.reload();
activeCommitTimeline = activeTimeline.getCommitTimeline();
assertFalse("Should be the 1 commit we made", activeCommitTimeline.empty());
assertEquals("Commit should be 1", completedInstant,
activeCommitTimeline.getInstants().findFirst().get());
assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(),
commitTimeline.readInstantDetails("1").get());
activeCommitTimeline.getInstantDetails(completedInstant).get());
}
@Test
public void checkArchiveCommitTimeline() throws IOException {
Path archiveLogPath =
HoodieArchivedCommitTimeline.getArchiveLogPath(metaClient.getMetaPath());
Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getMetaPath());
SequenceFile.Writer writer = SequenceFile
.createWriter(HoodieTestUtils.fs.getConf(), SequenceFile.Writer.file(archiveLogPath),
SequenceFile.Writer.keyClass(Text.class),
@@ -114,13 +119,24 @@ public class HoodieTableMetaClientTest {
IOUtils.closeStream(writer);
HoodieTimeline archivedTimeline = metaClient.getArchivedCommitTimeline();
assertEquals(Lists.newArrayList("1", "2", "3"),
HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline();
HoodieInstant instant1 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1");
HoodieInstant instant2 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2");
HoodieInstant instant3 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
assertEquals(Lists.newArrayList(instant1, instant2, instant3),
archivedTimeline.getInstants().collect(Collectors.toList()));
System.out.println(new String( archivedTimeline.readInstantDetails("1").get()));
assertArrayEquals(new Text("data1").getBytes(), archivedTimeline.readInstantDetails("1").get());
assertArrayEquals(new Text("data2").getBytes(), archivedTimeline.readInstantDetails("2").get());
assertArrayEquals(new Text("data3").getBytes(), archivedTimeline.readInstantDetails("3").get());
assertArrayEquals(new Text("data1").getBytes(),
archivedTimeline.getInstantDetails(instant1).get());
assertArrayEquals(new Text("data2").getBytes(),
archivedTimeline.getInstantDetails(instant2).get());
assertArrayEquals(new Text("data3").getBytes(),
archivedTimeline.getInstantDetails(instant3).get());
}

View File

@@ -0,0 +1,126 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.string;
import com.uber.hoodie.common.model.HoodieTestUtils;
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 org.apache.hadoop.fs.Path;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.io.IOException;
import java.util.Optional;
import java.util.stream.Stream;
import static org.junit.Assert.*;
public class HoodieActiveTimelineTest {
private HoodieActiveTimeline timeline;
private HoodieTableMetaClient metaClient;
@Rule
public final ExpectedException exception = ExpectedException.none();
@Before
public void setUp() throws Exception {
this.metaClient = HoodieTestUtils.initOnTemp();
}
@After
public void tearDown() throws Exception {
HoodieTestUtils.fs.delete(new Path(this.metaClient.getBasePath()), true);
}
@Test
public void testLoadingInstantsFromFiles() throws IOException {
HoodieInstant instant1 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1");
HoodieInstant instant2 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
HoodieInstant instant3 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "5");
HoodieInstant instant4 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "8");
HoodieInstant instant5 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9");
timeline = new HoodieActiveTimeline(HoodieTestUtils.fs, metaClient.getMetaPath());
timeline.saveAsComplete(instant1, Optional.empty());
timeline.saveAsComplete(instant2, Optional.empty());
timeline.saveAsComplete(instant3, Optional.empty());
timeline.saveAsComplete(instant4, Optional.empty());
timeline.createInflight(instant5);
timeline = timeline.reload();
assertEquals("Total instants should be 5", 5, timeline.countInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream",
Stream.of(instant1, instant2, instant3, instant4, instant5), timeline.getInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream",
Stream.of(instant1, instant2, instant3, instant4, instant5),
timeline.getCommitTimeline().getInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream",
Stream.of(instant1, instant2, instant3, instant4),
timeline.getCommitTimeline().filterCompletedInstants().getInstants());
HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream.of(instant5),
timeline.getCommitTimeline().filterInflights().getInstants());
}
@Test
public void testTimelineOperationsBasic() throws Exception {
timeline = new HoodieActiveTimeline(HoodieTestUtils.fs, metaClient.getMetaPath());
assertTrue(timeline.empty());
assertEquals("", 0, timeline.countInstants());
assertEquals("", Optional.empty(), timeline.firstInstant());
assertEquals("", Optional.empty(), timeline.nthInstant(5));
assertEquals("", Optional.empty(), timeline.nthInstant(-1));
assertEquals("", Optional.empty(), timeline.lastInstant());
assertFalse("", timeline.containsInstant(
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "01")));
}
@Test
public void testTimelineOperations() throws Exception {
timeline = new MockHoodieTimeline(
Stream.of("01", "03", "05", "07", "09", "11", "13", "15", "17", "19"),
Stream.of("21", "23"));
HoodieTestUtils.assertStreamEquals("", Stream.of("05", "07", "09", "11"),
timeline.getCommitTimeline().filterCompletedInstants().findInstantsInRange("04", "11")
.getInstants().map(HoodieInstant::getTimestamp));
HoodieTestUtils.assertStreamEquals("", Stream.of("09", "11"),
timeline.getCommitTimeline().filterCompletedInstants().findInstantsAfter("07", 2)
.getInstants().map(HoodieInstant::getTimestamp));
assertFalse(timeline.empty());
assertFalse(timeline.getCommitTimeline().filterInflights().empty());
assertEquals("", 12, timeline.countInstants());
HoodieTimeline activeCommitTimeline = timeline.getCommitTimeline().filterCompletedInstants();
assertEquals("", 10, activeCommitTimeline.countInstants());
assertEquals("", "01", activeCommitTimeline.firstInstant().get().getTimestamp());
assertEquals("", "11", activeCommitTimeline.nthInstant(5).get().getTimestamp());
assertEquals("", "19", activeCommitTimeline.lastInstant().get().getTimestamp());
assertEquals("", "09", activeCommitTimeline.nthFromLastInstant(5).get().getTimestamp());
assertTrue("", activeCommitTimeline.containsInstant(
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "09")));
assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
}
}

View File

@@ -1,104 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.string;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import org.apache.hadoop.fs.Path;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.io.IOException;
import java.util.Optional;
import java.util.stream.Stream;
import static org.junit.Assert.*;
public class HoodieDefaultTimelineTest {
private HoodieTimeline timeline;
private HoodieTableMetaClient metaClient;
@Rule
public final ExpectedException exception = ExpectedException.none();
@Before
public void setUp() throws Exception {
this.metaClient = HoodieTestUtils.initOnTemp();
}
@After
public void tearDown() throws Exception {
HoodieTestUtils.fs.delete(new Path(this.metaClient.getBasePath()), true);
}
@Test
public void testLoadingInstantsFromFiles() throws IOException {
timeline =
new MockHoodieTimeline(HoodieTestUtils.fs, metaClient.getMetaPath(), ".test");
timeline.saveInstantAsComplete("1", Optional.empty());
timeline.saveInstantAsComplete("3", Optional.empty());
timeline.saveInstantAsComplete("5", Optional.empty());
timeline.saveInstantAsComplete("8", Optional.empty());
timeline.saveInstantAsInflight("9");
timeline = timeline.reload();
assertEquals("Total instants should be 4", 4, timeline.getTotalInstants());
HoodieTestUtils
.assertStreamEquals("Check the instants stream", Stream.of("1", "3", "5", "8"),
timeline.getInstants());
assertTrue("Inflights should be present in the timeline", timeline.hasInflightInstants());
HoodieTestUtils.assertStreamEquals("Check the inflights stream", Stream.of("9"),
timeline.getInflightInstants());
}
@Test
public void testTimelineOperationsBasic() throws Exception {
timeline = new MockHoodieTimeline(Stream.empty(), Stream.empty());
assertFalse(timeline.hasInstants());
assertFalse(timeline.hasInflightInstants());
assertEquals("", 0, timeline.getTotalInstants());
assertEquals("", Optional.empty(), timeline.firstInstant());
assertEquals("", Optional.empty(), timeline.nthInstant(5));
assertEquals("", Optional.empty(), timeline.nthInstant(-1));
assertEquals("", Optional.empty(), timeline.lastInstant());
assertFalse("", timeline.containsInstant("01"));
}
@Test
public void testTimelineOperations() throws Exception {
timeline = new MockHoodieTimeline(
Stream.of("01", "03", "05", "07", "09", "11", "13", "15", "17", "19"),
Stream.of("21", "23"));
HoodieTestUtils.assertStreamEquals("", Stream.of("05", "07", "09", "11"),
timeline.findInstantsInRange("04", "11"));
HoodieTestUtils
.assertStreamEquals("", Stream.of("09", "11"), timeline.findInstantsAfter("07", 2));
assertTrue(timeline.hasInstants());
assertTrue(timeline.hasInflightInstants());
assertEquals("", 10, timeline.getTotalInstants());
assertEquals("", "01", timeline.firstInstant().get());
assertEquals("", "11", timeline.nthInstant(5).get());
assertEquals("", "19", timeline.lastInstant().get());
assertEquals("", "09", timeline.nthFromLastInstant(5).get());
assertTrue("", timeline.containsInstant("09"));
assertFalse("", timeline.isInstantBeforeTimelineStarts("02"));
assertTrue("", timeline.isInstantBeforeTimelineStarts("00"));
}
}

View File

@@ -16,51 +16,29 @@
package com.uber.hoodie.common.table.string;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline;
import org.apache.hadoop.fs.FileSystem;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import java.io.IOException;
import java.util.Optional;
import java.util.Comparator;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
public class MockHoodieTimeline extends HoodieDefaultTimeline {
private String fileExt;
public MockHoodieTimeline(FileSystem fs, String metaPath, String fileExtension)
public class MockHoodieTimeline extends HoodieActiveTimeline {
public MockHoodieTimeline(Stream<String> completed, Stream<String> inflights)
throws IOException {
super(fs, metaPath, fileExtension);
this.fileExt = fileExtension;
}
public MockHoodieTimeline(Stream<String> instants, Stream<String> inflights)
throws IOException {
super(instants, inflights);
}
@Override
public HoodieTimeline reload() throws IOException {
return new MockHoodieTimeline(fs, metaPath, fileExt);
}
@Override
public Optional<byte[]> readInstantDetails(String instant) {
return Optional.empty();
}
@Override
protected String getInflightFileName(String instant) {
return HoodieTestUtils.makeInflightTestFileName(instant);
}
@Override
protected String getCompletedFileName(String instant) {
return HoodieTestUtils.makeTestFileName(instant);
}
@Override
protected String getTimelineName() {
return "mock-test";
super();
this.instants = Stream.concat(completed
.map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)),
inflights.map(
s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s)))
.sorted(Comparator.comparing(new Function<HoodieInstant, String>() {
@Override
public String apply(HoodieInstant hoodieInstant) {
return hoodieInstant.getFileName();
}
})).collect(Collectors.toList());
}
}

View File

@@ -23,6 +23,8 @@ import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
@@ -40,6 +42,7 @@ import java.util.stream.Collectors;
import static org.junit.Assert.*;
@SuppressWarnings("ResultOfMethodCallIgnored")
public class ReadOptimizedTableViewTest {
private HoodieTableMetaClient metaClient;
private String basePath;
@@ -77,8 +80,10 @@ public class ReadOptimizedTableViewTest {
fsView.getLatestDataFilesForFileId(partitionPath, fileId).findFirst().isPresent());
// Make this commit safe
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
commitTimeline.saveInstantAsComplete(commitTime1, Optional.empty());
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
HoodieInstant instant1 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime1);
commitTimeline.saveAsComplete(instant1, Optional.empty());
refreshFsView();
assertEquals("", fileName1,
fsView.getLatestDataFilesForFileId(partitionPath, fileId).findFirst().get()
@@ -94,7 +99,9 @@ public class ReadOptimizedTableViewTest {
.getFileName());
// Make it safe
commitTimeline.saveInstantAsComplete(commitTime2, Optional.empty());
HoodieInstant instant2 =
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime2);
commitTimeline.saveAsComplete(instant2, Optional.empty());
refreshFsView();
assertEquals("", fileName2,
fsView.getLatestDataFilesForFileId(partitionPath, fileId).findFirst().get()
@@ -140,7 +147,7 @@ public class ReadOptimizedTableViewTest {
refreshFsView();
List<HoodieDataFile> statuses1 =
fsView.streamLatestVersionInPartition("2016/05/01", commitTime4)
fsView.getLatestVersionInPartition("2016/05/01", commitTime4)
.collect(Collectors.toList());
assertEquals(statuses1.size(), 3);
Set<String> filenames = Sets.newHashSet();
@@ -153,7 +160,7 @@ public class ReadOptimizedTableViewTest {
// Reset the max commit time
List<HoodieDataFile> statuses2 =
fsView.streamLatestVersionInPartition("2016/05/01", commitTime3)
fsView.getLatestVersionInPartition("2016/05/01", commitTime3)
.collect(Collectors.toList());
assertEquals(statuses2.size(), 3);
filenames = Sets.newHashSet();
@@ -204,7 +211,7 @@ public class ReadOptimizedTableViewTest {
refreshFsView();
List<List<HoodieDataFile>> statuses1 =
fsView.streamEveryVersionInPartition("2016/05/01").collect(Collectors.toList());
fsView.getEveryVersionInPartition("2016/05/01").collect(Collectors.toList());
assertEquals(statuses1.size(), 3);
for (List<HoodieDataFile> status : statuses1) {
@@ -269,9 +276,9 @@ public class ReadOptimizedTableViewTest {
assertEquals(statuses.length, 7);
refreshFsView();
List<HoodieDataFile> statuses1 =
fsView.streamLatestVersionInRange(statuses, Lists.newArrayList(commitTime2, commitTime3))
.collect(Collectors.toList());
List<HoodieDataFile> statuses1 = fsView
.getLatestVersionInRange(statuses, Lists.newArrayList(commitTime2, commitTime3))
.collect(Collectors.toList());
assertEquals(statuses1.size(), 2);
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : statuses1) {
@@ -320,7 +327,7 @@ public class ReadOptimizedTableViewTest {
refreshFsView();
List<HoodieDataFile> statuses1 =
fsView.streamLatestVersionsBeforeOrOn(statuses, commitTime2)
fsView.getLatestVersionsBeforeOrOn(statuses, commitTime2)
.collect(Collectors.toList());
assertEquals(statuses1.size(), 2);
Set<String> filenames = Sets.newHashSet();
@@ -371,8 +378,7 @@ public class ReadOptimizedTableViewTest {
refreshFsView();
List<HoodieDataFile> statuses1 =
fsView.streamLatestVersions(statuses)
.collect(Collectors.toList());
fsView.getLatestVersions(statuses).collect(Collectors.toList());
assertEquals(statuses1.size(), 3);
Set<String> filenames = Sets.newHashSet();
for (HoodieDataFile status : statuses1) {

View File

@@ -79,6 +79,11 @@
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
</dependency>
<dependency>
<groupId>com.esotericsoftware</groupId>
<artifactId>kryo</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>

View File

@@ -21,6 +21,7 @@ import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.InvalidDatasetException;
@@ -95,7 +96,7 @@ public class HoodieInputFormat extends MapredParquetInputFormat
String tableName = metadata.getTableConfig().getTableName();
String mode = HoodieHiveUtil.readMode(Job.getInstance(job), tableName);
TableFileSystemView fsView = new ReadOptimizedTableView(FSUtils.getFs(), metadata);
HoodieTimeline timeline = metadata.getActiveCommitTimeline();
HoodieTimeline timeline = metadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) {
// this is of the form commitTs_partition_sequenceNumber
String lastIncrementalTs = HoodieHiveUtil.readStartCommitTime(Job.getInstance(job), tableName);
@@ -103,10 +104,10 @@ public class HoodieInputFormat extends MapredParquetInputFormat
Integer maxCommits = HoodieHiveUtil.readMaxCommits(Job.getInstance(job), tableName);
LOG.info("Last Incremental timestamp was set as " + lastIncrementalTs);
List<String> commitsToReturn =
timeline.findInstantsAfter(lastIncrementalTs, maxCommits)
.collect(Collectors.toList());
timeline.findInstantsAfter(lastIncrementalTs, maxCommits).getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
List<HoodieDataFile> filteredFiles =
fsView.streamLatestVersionInRange(value, commitsToReturn)
fsView.getLatestVersionInRange(value, commitsToReturn)
.collect(Collectors.toList());
for (HoodieDataFile filteredFile : filteredFiles) {
LOG.info("Processing incremental hoodie file - " + filteredFile.getPath());
@@ -116,7 +117,7 @@ public class HoodieInputFormat extends MapredParquetInputFormat
"Total paths to process after hoodie incremental filter " + filteredFiles.size());
} else {
// filter files on the latest commit found
List<HoodieDataFile> filteredFiles = fsView.streamLatestVersions(value).collect(Collectors.toList());
List<HoodieDataFile> filteredFiles = fsView.getLatestVersions(value).collect(Collectors.toList());
LOG.info("Total paths to process after hoodie filter " + filteredFiles.size());
for (HoodieDataFile filteredFile : filteredFiles) {
LOG.info("Processing latest hoodie file - " + filteredFile.getPath());

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie.utilities;
import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.utilities.exception.HoodieIncrementalPullException;
import com.uber.hoodie.utilities.exception.HoodieIncrementalPullSQLException;
@@ -267,8 +268,12 @@ public class HiveIncrementalPuller {
}
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, targetDataPath);
Optional<String> lastCommit = metadata.getActiveCommitTimeline().lastInstant();
return lastCommit.orElse("0");
Optional<HoodieInstant>
lastCommit = metadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant();
if(lastCommit.isPresent()) {
return lastCommit.get().getTimestamp();
}
return "0";
}
private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime)
@@ -299,13 +304,14 @@ public class HiveIncrementalPuller {
private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) throws IOException {
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, sourceTableLocation);
List<String> commitsToSync = metadata.getActiveCommitTimeline()
.findInstantsAfter(config.fromCommitTime, config.maxCommits)
List<String> commitsToSync = metadata.getActiveTimeline().getCommitTimeline().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
.getActiveCommitTimeline().getInstants().collect(Collectors.toList())
+ " and from commit time is " + config.fromCommitTime);
.getActiveTimeline().getCommitTimeline().filterCompletedInstants().getInstants()
.collect(Collectors.toList()) + " and from commit time is "
+ config.fromCommitTime);
return null;
}
log.info("Syncing commits " + commitsToSync);

View File

@@ -66,11 +66,11 @@ public class HoodieDeltaStreamer implements Serializable {
JavaSparkContext sc = getSparkContext(cfg);
FileSystem fs = FSUtils.getFs();
HoodieTableMetaClient targetHoodieMetadata = new HoodieTableMetaClient(fs, cfg.targetPath);
HoodieTimeline timeline = targetHoodieMetadata.getActiveCommitTimeline();
HoodieTimeline timeline = targetHoodieMetadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
String lastCommitPulled = findLastCommitPulled(fs, cfg.dataPath);
log.info("Last commit pulled on the source dataset is " + lastCommitPulled);
if (!timeline.getInstants().iterator().hasNext() && timeline
.compareInstants(timeline.lastInstant().get(), lastCommitPulled,
.compareTimestamps(timeline.lastInstant().get().getTimestamp(), lastCommitPulled,
HoodieTimeline.GREATER)) {
// this should never be the case
throw new IllegalStateException(

View File

@@ -24,6 +24,7 @@ import com.uber.hoodie.common.table.HoodieTableConfig;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.hadoop.fs.FileStatus;
@@ -66,7 +67,8 @@ public class HoodieSnapshotCopier implements Serializable {
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs, baseDir);
final TableFileSystemView fsView = new ReadOptimizedTableView(fs, tableMetadata);
// Get the latest commit
final Optional<String> latestCommit = tableMetadata.getActiveCommitTimeline().lastInstant();
final Optional<HoodieInstant>
latestCommit = tableMetadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant();
if(!latestCommit.isPresent()) {
logger.warn("No commits present. Nothing to snapshot");
} else {
@@ -91,7 +93,7 @@ public class HoodieSnapshotCopier implements Serializable {
FileSystem fs = FSUtils.getFs();
List<Tuple2<String, String>> filePaths = new ArrayList<>();
for (HoodieDataFile hoodieDataFile : fsView
.streamLatestVersionInPartition(partition, latestCommit.get())
.getLatestVersionInPartition(partition, latestCommit.get().getTimestamp())
.collect(Collectors.toList())) {
filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath()));
}
@@ -124,8 +126,8 @@ public class HoodieSnapshotCopier implements Serializable {
} else {
String commitTime =
FSUtils.getCommitFromCommitFile(commitFilePath.getName());
return tableMetadata.getActiveCommitTimeline()
.compareInstants(commitTime, latestCommit.get(), HoodieTimeline.GREATER);
return tableMetadata.getActiveTimeline().getCommitTimeline()
.compareTimestamps(commitTime, latestCommit.get().getTimestamp(), HoodieTimeline.GREATER);
}
}
});