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:
@@ -21,12 +21,15 @@ import com.google.common.collect.Iterables;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieCommits;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
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.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.ParquetUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
@@ -59,9 +62,11 @@ 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;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
@@ -87,7 +92,7 @@ public class TestHoodieClient implements Serializable {
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath);
|
||||
HoodieTestUtils.init(basePath);
|
||||
|
||||
dataGen = new HoodieTestDataGenerator();
|
||||
}
|
||||
@@ -200,7 +205,7 @@ public class TestHoodieClient implements Serializable {
|
||||
assertEquals("Latest commit should be 001",readClient.latestCommit(), newCommitTime);
|
||||
assertEquals("Must contain 200 records", readClient.readCommit(newCommitTime).count(), records.size());
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetadata(fs, basePath)).collect();
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetaClient(fs, basePath)).collect();
|
||||
checkTaggedRecords(taggedRecords, "001");
|
||||
|
||||
/**
|
||||
@@ -226,7 +231,7 @@ public class TestHoodieClient implements Serializable {
|
||||
assertEquals("Latest commit should be 004",readClient.latestCommit(), newCommitTime);
|
||||
|
||||
// Index should be able to locate all updates in correct locations.
|
||||
taggedRecords = index.tagLocation(jsc.parallelize(dedupedRecords, 1), new HoodieTableMetadata(fs, basePath)).collect();
|
||||
taggedRecords = index.tagLocation(jsc.parallelize(dedupedRecords, 1), new HoodieTableMetaClient(fs, basePath)).collect();
|
||||
checkTaggedRecords(taggedRecords, "004");
|
||||
|
||||
// Check the entire dataset has 100 records still
|
||||
@@ -273,7 +278,7 @@ public class TestHoodieClient implements Serializable {
|
||||
|
||||
assertEquals("Expecting a single commit.", new HoodieReadClient(jsc, basePath).listCommitsSince("000").size(), 1);
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetadata(fs, basePath)).collect();
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetaClient(fs, basePath)).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
|
||||
// Keep doing some writes and clean inline. Make sure we have expected number of files remaining.
|
||||
@@ -287,34 +292,39 @@ public class TestHoodieClient implements Serializable {
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
SortedMap<String, HoodieCommitMetadata> commitMetadata = metadata.getAllCommitMetadata();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTimeline timeline = metadata.getActiveCommitTimeline();
|
||||
|
||||
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 (Map.Entry<String, HoodieCommitMetadata> entry : commitMetadata.entrySet()) {
|
||||
for (HoodieWriteStat wstat : entry.getValue().getWriteStats(partitionPath)) {
|
||||
for (String entry : timeline.getInstants().collect(Collectors.toList())) {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.readInstantDetails(entry).get());
|
||||
|
||||
for (HoodieWriteStat wstat : commitMetadata.getWriteStats(partitionPath)) {
|
||||
if (!fileIdToVersions.containsKey(wstat.getFileId())) {
|
||||
fileIdToVersions.put(wstat.getFileId(), new TreeSet<String>());
|
||||
fileIdToVersions.put(wstat.getFileId(), new TreeSet<>());
|
||||
}
|
||||
fileIdToVersions.get(wstat.getFileId()).add(entry.getKey());
|
||||
fileIdToVersions.get(wstat.getFileId()).add(FSUtils.getCommitTime(new Path(wstat.getFullPath()).getName()));
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, List<FileStatus>> fileVersions = metadata.getAllVersionsInPartition(fs, partitionPath);
|
||||
for (Map.Entry<String, List<FileStatus>> entry : fileVersions.entrySet()) {
|
||||
List<FileStatus> versions = entry.getValue();
|
||||
|
||||
List<List<HoodieDataFile>> fileVersions = fsView.streamEveryVersionInPartition(partitionPath).collect(Collectors.toList());
|
||||
for (List<HoodieDataFile> entry : fileVersions) {
|
||||
// No file has no more than max versions
|
||||
assertTrue("fileId " + entry.getKey() + " has more than " + maxVersions + " versions",
|
||||
versions.size() <= maxVersions);
|
||||
String fileId = entry.iterator().next().getFileId();
|
||||
|
||||
assertTrue("fileId " + fileId + " has more than " + maxVersions + " versions",
|
||||
entry.size() <= maxVersions);
|
||||
|
||||
// Each file, has the latest N versions (i.e cleaning gets rid of older versions)
|
||||
List<String> commitedVersions = new ArrayList<>(fileIdToVersions.get(entry.getKey()));
|
||||
for (int i = 0; i < versions.size(); i++) {
|
||||
assertEquals("File " + entry.getKey() + " does not have latest versions" + versions + " on commits" + commitedVersions,
|
||||
FSUtils.getCommitTime(Iterables.get(versions, i).getPath().getName()),
|
||||
List<String> commitedVersions = new ArrayList<>(fileIdToVersions.get(fileId));
|
||||
for (int i = 0; i < entry.size(); i++) {
|
||||
assertEquals("File " + fileId + " does not have latest versions on commits" + commitedVersions,
|
||||
Iterables.get(entry, i).getCommitTime(),
|
||||
commitedVersions.get(commitedVersions.size() - 1 - i));
|
||||
}
|
||||
}
|
||||
@@ -349,7 +359,7 @@ public class TestHoodieClient implements Serializable {
|
||||
// verify that there is a commit
|
||||
assertEquals("Expecting a single commit.", new HoodieReadClient(jsc, basePath).listCommitsSince("000").size(), 1);
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetadata(fs, basePath)).collect();
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetaClient(fs, basePath)).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
|
||||
// Keep doing some writes and clean inline. Make sure we have expected number of files remaining.
|
||||
@@ -362,23 +372,29 @@ public class TestHoodieClient implements Serializable {
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
HoodieCommits commits = metadata.getAllCommits();
|
||||
String earliestRetainedCommit = commits.lastCommit(maxCommits - 1);
|
||||
Set<String> acceptableCommits = new HashSet<>(commits.getCommitList());
|
||||
if (earliestRetainedCommit != null) {
|
||||
acceptableCommits.removeAll(commits.findCommitsInRange("000", earliestRetainedCommit));
|
||||
acceptableCommits.add(earliestRetainedCommit);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTimeline activeTimeline = metadata.getActiveCommitTimeline();
|
||||
Optional<String> earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1);
|
||||
Set<String> acceptableCommits =
|
||||
activeTimeline.getInstants().collect(Collectors.toSet());
|
||||
if (earliestRetainedCommit.isPresent()) {
|
||||
acceptableCommits.removeAll(
|
||||
activeTimeline.findInstantsInRange("000", earliestRetainedCommit.get())
|
||||
.collect(Collectors.toSet()));
|
||||
acceptableCommits.add(earliestRetainedCommit.get());
|
||||
}
|
||||
|
||||
TableFileSystemView fsView = new ReadOptimizedTableView(fs, metadata);
|
||||
// Need to ensure the following
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
Map<String, List<FileStatus>> fileVersions = metadata.getAllVersionsInPartition(fs, partitionPath);
|
||||
for (Map.Entry<String, List<FileStatus>> entry : fileVersions.entrySet()) {
|
||||
Set<String> commitTimes = new HashSet<>(entry.getValue().size());
|
||||
for(FileStatus value:entry.getValue()) {
|
||||
commitTimes.add(FSUtils.getCommitTime(value.getPath().getName()));
|
||||
List<List<HoodieDataFile>> fileVersions = fsView.streamEveryVersionInPartition(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);
|
||||
}
|
||||
@@ -620,13 +636,16 @@ public class TestHoodieClient implements Serializable {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
assertEquals("2 files needs to be committed.", 2, statuses.size());
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
FileStatus[] files = metadata.getLatestVersionInPartition(fs, TEST_PARTITION_PATH, commitTime3);
|
||||
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(
|
||||
Collectors.toList());
|
||||
int numTotalInsertsInCommit3 = 0;
|
||||
for (FileStatus file: files) {
|
||||
if (file.getPath().getName().contains(file1)) {
|
||||
assertEquals("Existing file should be expanded", commitTime3, FSUtils.getCommitTime(file.getPath().getName()));
|
||||
records = ParquetUtils.readAvroRecords(file.getPath());
|
||||
for (HoodieDataFile file: files) {
|
||||
if (file.getFileName().contains(file1)) {
|
||||
assertEquals("Existing file should be expanded", commitTime3, file.getCommitTime());
|
||||
records = ParquetUtils.readAvroRecords(new Path(file.getPath()));
|
||||
for (GenericRecord record: records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String recordCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
@@ -641,8 +660,8 @@ public class TestHoodieClient implements Serializable {
|
||||
}
|
||||
assertEquals("All keys added in commit 2 must be updated in commit3 correctly", 0, keys2.size());
|
||||
} else {
|
||||
assertEquals("New file must be written for commit 3", commitTime3, FSUtils.getCommitTime(file.getPath().getName()));
|
||||
records = ParquetUtils.readAvroRecords(file.getPath());
|
||||
assertEquals("New file must be written for commit 3", commitTime3, file.getCommitTime());
|
||||
records = ParquetUtils.readAvroRecords(new Path(file.getPath()));
|
||||
for (GenericRecord record: records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
assertEquals("only expect commit3", commitTime3, record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString());
|
||||
|
||||
Reference in New Issue
Block a user