1
0

Refactor hoodie-common and create right abstractions for Hoodie Storage V2.0

The following is the gist of changes done

- All low-level operation of creating a commit code was in HoodieClient which made it hard to share code if there was a compaction commit.
- HoodieTableMetadata contained a mix of metadata and filtering files. (Also few operations required FileSystem to be passed in because those were called from TaskExecutors and others had FileSystem as a global variable). Since merge-on-read requires a lot of that code, but will have to change slightly on how it operates on the metadata and how it filters the files. The two set of operation are split into HoodieTableMetaClient and TableFileSystemView.
- Everything (active commits, archived commits, cleaner log, save point log and in future delta and compaction commits) in HoodieTableMetaClient is a HoodieTimeline. Timeline is a series of instants, which has an in-built concept of inflight and completed commit markers.
- A timeline can be queries for ranges, contains and also use to create new datapoint (create a new commit etc). Commit (and all the above metadata) creation/deletion is streamlined in a timeline
- Multiple timelines can be merged into a single timeline, giving us an audit timeline to whatever happened in a hoodie dataset. This also helps with #55.
- Move to java 8 and introduce java 8 succinct syntax in refactored code
This commit is contained in:
Prasanna Rajaperumal
2017-01-04 14:40:26 -08:00
parent 283269e57f
commit 8ee777a9bb
76 changed files with 3480 additions and 1932 deletions

View File

@@ -17,10 +17,11 @@
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.config.HoodieWriteConfig;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieCompactionConfig;
@@ -32,8 +33,11 @@ import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.stream.Collectors;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
@@ -47,7 +51,7 @@ public class TestHoodieCommitArchiveLog {
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.initializeHoodieDirectory(basePath);
HoodieTestUtils.init(basePath);
fs = FSUtils.getFs();
}
@@ -56,7 +60,7 @@ public class TestHoodieCommitArchiveLog {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").build();
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
}
@@ -67,20 +71,22 @@ public class TestHoodieCommitArchiveLog {
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101");
HoodieTestDataGenerator.createCommitFile(basePath, "102");
HoodieTestDataGenerator.createCommitFile(basePath, "103");
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
HoodieTimeline timeline = metadata.getActiveCommitTimeline();
assertEquals("Loaded 4 commits and the count should match", 4,
metadata.getAllCommits().getCommitList().size());
timeline.getTotalInstants());
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
metadata = new HoodieTableMetadata(fs, basePath);
timeline = timeline.reload();
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4,
metadata.getAllCommits().getCommitList().size());
timeline.getTotalInstants());
}
@Test
@@ -89,7 +95,8 @@ public class TestHoodieCommitArchiveLog {
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101");
HoodieTestDataGenerator.createCommitFile(basePath, "102");
@@ -97,24 +104,22 @@ public class TestHoodieCommitArchiveLog {
HoodieTestDataGenerator.createCommitFile(basePath, "104");
HoodieTestDataGenerator.createCommitFile(basePath, "105");
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
SortedMap<String, HoodieCommitMetadata> originalCommits = new TreeMap<>(metadata.getAllCommitMetadata());
HoodieTimeline timeline = metadata.getActiveCommitTimeline();
List<String> originalCommits = timeline.getInstants().collect(
Collectors.toList());
assertEquals("Loaded 6 commits and the count should match", 6,
metadata.getAllCommits().getCommitList().size());
assertEquals("Loaded 6 commits and the count should match", 6, timeline.getTotalInstants());
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
metadata = new HoodieTableMetadata(fs, basePath);
timeline = timeline.reload();
assertEquals(
"Should archive commits when maxCommitsToKeep is 5 and now the commits length should be minCommitsToKeep which is 2",
2, metadata.getAllCommits().getCommitList().size());
2, timeline.getTotalInstants());
assertEquals("Archive should not archive the last 2 commits",
Lists.newArrayList("104", "105"), metadata.getAllCommits().getCommitList());
Lists.newArrayList("104", "105"), timeline.getInstants().collect(Collectors.toList()));
// Remove all the commits from the original commits, make it ready to be checked against the read map
for(String key:metadata.getAllCommitMetadata().keySet()) {
originalCommits.remove(key);
}
timeline.getInstants().forEach(originalCommits::remove);
// Read back the commits to make sure
SequenceFile.Reader reader = new SequenceFile.Reader(fs.getConf(),
@@ -129,7 +134,7 @@ public class TestHoodieCommitArchiveLog {
assertEquals(
"Read commits map should match the originalCommits - commitsLoadedAfterArchival",
originalCommits, readCommits);
originalCommits, new ArrayList<>(readCommits.keySet()));
reader.close();
}
@@ -139,7 +144,8 @@ public class TestHoodieCommitArchiveLog {
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101");
HoodieTestDataGenerator.createCommitFile(basePath, "102");
@@ -147,16 +153,15 @@ public class TestHoodieCommitArchiveLog {
HoodieTestDataGenerator.createCommitFile(basePath, "104");
HoodieTestDataGenerator.createCommitFile(basePath, "105");
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
assertEquals("Loaded 6 commits and the count should match", 6,
metadata.getAllCommits().getCommitList().size());
HoodieTimeline timeline = metadata.getActiveCommitTimeline();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.getTotalInstants());
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
metadata = new HoodieTableMetadata(fs, basePath);
assertTrue("Archived commits should always be safe", metadata.isCommitTsSafe("100"));
assertTrue("Archived commits should always be safe", metadata.isCommitTsSafe("101"));
assertTrue("Archived commits should always be safe", metadata.isCommitTsSafe("102"));
assertTrue("Archived commits should always be safe", metadata.isCommitTsSafe("103"));
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"));
}