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:
@@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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()));
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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()));
|
||||
|
||||
@@ -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());
|
||||
|
||||
Reference in New Issue
Block a user