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

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