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