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

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

View File

@@ -18,7 +18,7 @@ package com.uber.hoodie.common;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.FSUtils;
import java.io.File;
@@ -53,18 +53,18 @@ public class HoodieClientTestUtils {
}
private static void fakeMetaFile(String basePath, String commitTime, String suffix) throws IOException {
String parentPath = basePath + "/"+ HoodieTableMetadata.METAFOLDER_NAME;
String parentPath = basePath + "/"+ HoodieTableMetaClient.METAFOLDER_NAME;
new File(parentPath).mkdirs();
new File(parentPath + "/" + commitTime + suffix).createNewFile();
}
public static void fakeCommitFile(String basePath, String commitTime) throws IOException {
fakeMetaFile(basePath, commitTime, HoodieTableMetadata.COMMIT_FILE_SUFFIX);
fakeMetaFile(basePath, commitTime, HoodieTableMetaClient.COMMIT_EXTENSION);
}
public static void fakeInFlightFile(String basePath, String commitTime) throws IOException {
fakeMetaFile(basePath, commitTime, HoodieTableMetadata.INFLIGHT_FILE_SUFFIX);
fakeMetaFile(basePath, commitTime, HoodieTableMetaClient.INFLIGHT_FILE_SUFFIX);
}
public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId) throws Exception {

View File

@@ -19,7 +19,7 @@ package com.uber.hoodie.common;
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.model.HoodieTableMetadata;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
@@ -29,8 +29,6 @@ import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
@@ -144,7 +142,7 @@ public class HoodieTestDataGenerator {
public static void createCommitFile(String basePath, String commitTime) throws IOException {
Path commitFile =
new Path(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME + "/" + FSUtils.makeCommitFileName(commitTime));
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTableMetaClient.makeCommitFileName(commitTime));
FileSystem fs = FSUtils.getFs();
FSDataOutputStream os = fs.create(commitFile, true);
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();

View File

@@ -16,13 +16,13 @@
package com.uber.hoodie.func;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.TestRawTripPayload;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
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.table.HoodieCopyOnWriteTable;
@@ -48,14 +48,14 @@ public class TestUpdateMapFunction {
TemporaryFolder folder = new TemporaryFolder();
folder.create();
this.basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.initializeHoodieDirectory(basePath);
HoodieTestUtils.init(basePath);
}
@Test
public void testSchemaEvolutionOnUpdate() throws Exception {
// Create a bunch of records with a old version of schema
HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable("100", config, metadata);
String recordStr1 =
@@ -79,13 +79,13 @@ public class TestUpdateMapFunction {
rowChange3));
Iterator<List<WriteStatus>> insertResult = table.handleInsert(records.iterator());
Path commitFile =
new Path(config.getBasePath() + "/.hoodie/" + FSUtils.makeCommitFileName("100"));
new Path(config.getBasePath() + "/.hoodie/" + HoodieTableMetaClient.makeCommitFileName("100"));
FSUtils.getFs().create(commitFile);
// Now try an update with an evolved schema
// Evolved schema does not have guarantee on preserving the original field ordering
config = makeHoodieClientConfig("/exampleEvolvedSchema.txt");
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath);
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
String fileId = insertResult.next().get(0).getFileId();
System.out.println(fileId);

View File

@@ -19,13 +19,13 @@ package com.uber.hoodie.index;
import com.google.common.base.Optional;
import com.google.common.collect.Lists;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.TestRawTripPayload;
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.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
@@ -74,7 +74,7 @@ public class TestHoodieBloomIndex {
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.initializeHoodieDirectory(basePath);
HoodieTestUtils.init(basePath);
}
@Test
@@ -126,7 +126,7 @@ public class TestHoodieBloomIndex {
new File(basePath + "/2015/03/12/3_0_20150312101010.parquet").createNewFile();
new File(basePath + "/2015/03/12/4_0_20150312101010.parquet").createNewFile();
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
JavaPairRDD<String, String> rdd = index.loadInvolvedFiles(partitions, metadata);
// Still 0, as no valid commit
assertEquals(rdd.count(), 0);
@@ -135,7 +135,7 @@ public class TestHoodieBloomIndex {
new File(basePath + "/.hoodie").mkdirs();
new File(basePath + "/.hoodie/20160401010101.commit").createNewFile();
new File(basePath + "/.hoodie/20150312101010.commit").createNewFile();
metadata = new HoodieTableMetadata(fs, basePath, "testTable");
metadata = new HoodieTableMetaClient(fs, basePath);
rdd = index.loadInvolvedFiles(partitions, metadata);
final List<Tuple2<String, String>> filesList = rdd.collect();
assertEquals(filesList.size(), 4);
@@ -212,7 +212,7 @@ public class TestHoodieBloomIndex {
// We have some records to be tagged (two different partitions)
JavaRDD<HoodieRecord> recordRDD = jsc.emptyRDD();
// Also create the metadata and config
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
// Let's tag
@@ -248,7 +248,7 @@ public class TestHoodieBloomIndex {
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
// Also create the metadata and config
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
// Let's tag
@@ -266,7 +266,7 @@ public class TestHoodieBloomIndex {
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
// We do the tag again
metadata = new HoodieTableMetadata(fs, basePath, "testTable");
metadata = new HoodieTableMetaClient(fs, basePath);
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, metadata);
// Check results
@@ -309,7 +309,7 @@ public class TestHoodieBloomIndex {
JavaRDD<HoodieKey> keysRDD = jsc.parallelize(Arrays.asList(key1, key2, key3, key4));
// Also create the metadata and config
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
// Let's tag
@@ -327,7 +327,7 @@ public class TestHoodieBloomIndex {
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
// We do the tag again
metadata = new HoodieTableMetadata(fs, basePath, "testTable");
metadata = new HoodieTableMetaClient(fs, basePath);
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, metadata);
// Check results
@@ -375,7 +375,7 @@ public class TestHoodieBloomIndex {
// We do the tag
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, metadata);
@@ -421,8 +421,8 @@ public class TestHoodieBloomIndex {
if (createCommitTime) {
// Also make sure the commit is valid
new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME).mkdirs();
new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME + "/" + commitTime + ".commit").createNewFile();
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME).mkdirs();
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + ".commit").createNewFile();
}
return filename;
}

View File

@@ -16,8 +16,8 @@
package com.uber.hoodie.io;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.util.FSUtils;
@@ -34,10 +34,12 @@ public class TestHoodieCleaner {
private String basePath = null;
private String[] partitionPaths = {"2016/01/01", "2016/02/02"};
private HoodieTableMetaClient metaClient;
@Before
public void init() throws Exception {
this.basePath = HoodieTestUtils.initializeTempHoodieBasePath();
this.metaClient = HoodieTestUtils.initOnTemp();
this.basePath = metaClient.getBasePath();
}
@Test
@@ -53,7 +55,7 @@ public class TestHoodieCleaner {
String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000");
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000");
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCleaner cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]));
@@ -67,7 +69,7 @@ public class TestHoodieCleaner {
String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "001"); // insert
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, partitionPaths[1], "001", file1P1C0); // update
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
assertEquals("Must clean 1 file" , 1, cleaner.clean(partitionPaths[0]));
assertEquals("Must clean 1 file" , 1, cleaner.clean(partitionPaths[1]));
@@ -82,7 +84,7 @@ public class TestHoodieCleaner {
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update
String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "002");
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
assertEquals("Must clean two files" , 2, cleaner.clean(partitionPaths[0]));
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file1P0C0));
@@ -110,7 +112,7 @@ public class TestHoodieCleaner {
String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000");
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000");
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCleaner cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]));
@@ -124,7 +126,7 @@ public class TestHoodieCleaner {
String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "001"); // insert
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, partitionPaths[1], "001", file1P1C0); // update
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]));
@@ -139,7 +141,7 @@ public class TestHoodieCleaner {
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update
String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "002");
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
assertEquals(
"Must not clean any file. We have to keep 1 version before the latest commit time to keep",
@@ -153,7 +155,7 @@ public class TestHoodieCleaner {
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file2P0C1); // update
String file4P0C3 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "003");
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
assertEquals(
"Must not clean one old file", 1, cleaner.clean(partitionPaths[0]));

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"));
}

View File

@@ -16,17 +16,17 @@
package com.uber.hoodie.table;
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.BloomFilter;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.TestRawTripPayload;
import com.uber.hoodie.common.model.HoodieCommits;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
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.common.util.ParquetUtils;
@@ -76,7 +76,7 @@ public class TestCopyOnWriteTable {
TemporaryFolder folder = new TemporaryFolder();
folder.create();
this.basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.initializeHoodieDirectory(basePath);
HoodieTestUtils.init(basePath);
}
@Test
@@ -87,9 +87,10 @@ public class TestCopyOnWriteTable {
HoodieRecord record = mock(HoodieRecord.class);
when(record.getPartitionPath()).thenReturn(partitionPath);
String commitTime = HoodieTestUtils.getNewCommitTime();
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieWriteConfig config = makeHoodieClientConfig();
HoodieInsertHandle io = new HoodieInsertHandle(config, commitTime, null, partitionPath);
HoodieInsertHandle io = new HoodieInsertHandle(config, commitTime,
new HoodieTableMetaClient(FSUtils.getFs(), basePath), partitionPath);
Path newPath = io.makeNewPath(record.getPartitionPath(), unitNumber, fileName);
assertTrue(newPath.toString().equals(this.basePath + "/" + partitionPath + "/" + FSUtils
.makeDataFileName(commitTime, unitNumber, fileName)));
@@ -110,8 +111,8 @@ public class TestCopyOnWriteTable {
public void testUpdateRecords() throws Exception {
// Prepare the AvroParquetIO
HoodieWriteConfig config = makeHoodieClientConfig();
String firstCommitTime = HoodieTestUtils.getNewCommitTime();
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
String partitionPath = "/2016/01/31";
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(firstCommitTime, config, metadata);
@@ -148,7 +149,7 @@ public class TestCopyOnWriteTable {
assertTrue(filter.mightContain(record.getRecordKey()));
}
// Create a commit file
new File(this.basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME + "/"
new File(this.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
+ FSUtils.getCommitTime(parquetFile.getName()) + ".commit").createNewFile();
// Read the parquet file, check the record content
@@ -172,8 +173,8 @@ public class TestCopyOnWriteTable {
List<HoodieRecord> updatedRecords = Arrays.asList(updatedRecord1, insertedRecord1);
Thread.sleep(1000);
String newCommitTime = HoodieTestUtils.getNewCommitTime();
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
String newCommitTime = HoodieTestUtils.makeNewCommitTime();
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
table = new HoodieCopyOnWriteTable(newCommitTime, config, metadata);
Iterator<List<WriteStatus>> iter = table.handleUpdate(updatedRecord1.getCurrentLocation().getFileId(), updatedRecords.iterator());
@@ -181,9 +182,11 @@ public class TestCopyOnWriteTable {
File updatedParquetFile = null;
for (File file : new File(basePath + "/2016/01/31").listFiles()) {
if (file.getName().endsWith(".parquet")) {
if (FSUtils.getFileId(file.getName()).equals(FSUtils.getFileId(parquetFile.getName()))
&& HoodieCommits
.isCommit1After(FSUtils.getCommitTime(file.getName()), FSUtils.getCommitTime(parquetFile.getName()))) {
if (FSUtils.getFileId(file.getName())
.equals(FSUtils.getFileId(parquetFile.getName())) && metadata
.getActiveCommitTimeline()
.compareInstants(FSUtils.getCommitTime(file.getName()),
FSUtils.getCommitTime(parquetFile.getName()), HoodieTimeline.GREATER)) {
updatedParquetFile = file;
break;
}
@@ -236,9 +239,9 @@ public class TestCopyOnWriteTable {
@Test public void testInsertWithPartialFailures() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.getNewCommitTime();
String commitTime = HoodieTestUtils.makeNewCommitTime();
FileSystem fs = FSUtils.getFs();
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(commitTime, config, metadata);
// Write a few records, and get atleast one file
@@ -275,8 +278,8 @@ public class TestCopyOnWriteTable {
@Test public void testInsertRecords() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.getNewCommitTime();
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(commitTime, config, metadata);
// Case 1:
@@ -322,8 +325,8 @@ public class TestCopyOnWriteTable {
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withStorageConfig(
HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024)
.parquetPageSize(64 * 1024).build()).build();
String commitTime = HoodieTestUtils.getNewCommitTime();
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(commitTime, config, metadata);
List<HoodieRecord> records = new ArrayList<>();
@@ -367,7 +370,7 @@ public class TestCopyOnWriteTable {
HoodieClientTestUtils.fakeCommitFile(basePath, "001");
HoodieClientTestUtils.fakeDataFile(basePath, TEST_PARTITION_PATH, "001", "file1", fileSize);
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable("001", config, metadata);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[]{TEST_PARTITION_PATH});