1
0

Implement Merge on Read Storage (#76)

1. Create HoodieTable abstraction for commits and fileSystemView
2. HoodieMergeOnReadTable created
3. View is now always obtained from the table and the correct view based on the table type is returned
This commit is contained in:
prazanna
2017-02-21 15:24:00 -08:00
committed by Prasanna Rajaperumal
parent 11d2fd3428
commit eb46e7c72b
47 changed files with 1113 additions and 421 deletions

View File

@@ -30,7 +30,6 @@ 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;
import com.uber.hoodie.config.HoodieWriteConfig;
@@ -41,6 +40,7 @@ import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.HoodieCleaner;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -203,7 +203,10 @@ 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 HoodieTableMetaClient(fs, basePath)).collect();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect();
checkTaggedRecords(taggedRecords, "001");
/**
@@ -228,8 +231,11 @@ public class TestHoodieClient implements Serializable {
assertEquals("Expecting two commits.", readClient.listCommitsSince("000").size(), 2);
assertEquals("Latest commit should be 004",readClient.latestCommit(), newCommitTime);
metaClient = new HoodieTableMetaClient(fs, basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig());
// Index should be able to locate all updates in correct locations.
taggedRecords = index.tagLocation(jsc.parallelize(dedupedRecords, 1), new HoodieTableMetaClient(fs, basePath)).collect();
taggedRecords = index.tagLocation(jsc.parallelize(dedupedRecords, 1), table).collect();
checkTaggedRecords(taggedRecords, "004");
// Check the entire dataset has 100 records still
@@ -276,7 +282,9 @@ 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 HoodieTableMetaClient(fs, basePath)).collect();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect();
checkTaggedRecords(taggedRecords, newCommitTime);
// Keep doing some writes and clean inline. Make sure we have expected number of files remaining.
@@ -291,10 +299,10 @@ public class TestHoodieClient implements Serializable {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitTimeline();
table = HoodieTable.getHoodieTable(metadata, getConfig());
HoodieTimeline timeline = table.getCommitTimeline();
TableFileSystemView fsView = new ReadOptimizedTableView(fs, metadata);
TableFileSystemView fsView = table.getFileSystemView();
// Need to ensure the following
for (String partitionPath : dataGen.getPartitionPaths()) {
// compute all the versions of all files, from time 0
@@ -358,7 +366,10 @@ 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 HoodieTableMetaClient(fs, basePath)).collect();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect();
checkTaggedRecords(taggedRecords, newCommitTime);
// Keep doing some writes and clean inline. Make sure we have expected number of files remaining.
@@ -372,7 +383,8 @@ public class TestHoodieClient implements Serializable {
assertNoWriteErrors(statuses);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTimeline activeTimeline = metadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg);
HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline();
Optional<HoodieInstant>
earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1);
Set<HoodieInstant> acceptableCommits =
@@ -384,7 +396,7 @@ public class TestHoodieClient implements Serializable {
acceptableCommits.add(earliestRetainedCommit.get());
}
TableFileSystemView fsView = new ReadOptimizedTableView(fs, metadata);
TableFileSystemView fsView = table1.getFileSystemView();
// Need to ensure the following
for (String partitionPath : dataGen.getPartitionPaths()) {
List<List<HoodieDataFile>> fileVersions = fsView.getEveryVersionInPartition(partitionPath).collect(Collectors.toList());
@@ -637,7 +649,8 @@ public class TestHoodieClient implements Serializable {
assertEquals("2 files needs to be committed.", 2, statuses.size());
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
TableFileSystemView fileSystemView = new ReadOptimizedTableView(fs, metadata);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
TableFileSystemView fileSystemView = table.getFileSystemView();
List<HoodieDataFile> files = fileSystemView.getLatestVersionInPartition(TEST_PARTITION_PATH, commitTime3).collect(
Collectors.toList());
int numTotalInsertsInCommit3 = 0;

View File

@@ -0,0 +1,208 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie;
import com.uber.hoodie.common.HoodieTestDataGenerator;
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.HoodieTableType;
import com.uber.hoodie.common.model.HoodieTestUtils;
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.util.FSUtils;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.compact.CompactionFilter;
import com.uber.hoodie.io.compact.HoodieCompactor;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
import com.uber.hoodie.table.HoodieTable;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SQLContext;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Stream;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
public class TestMergeOnReadTable {
private transient JavaSparkContext jsc = null;
private transient SQLContext sqlContext;
private String basePath = null;
private HoodieCompactor compactor;
private FileSystem fs;
@Before
public void init() throws IOException {
this.fs = FSUtils.getFs();
// Initialize a local spark env
SparkConf sparkConf =
new SparkConf().set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.setAppName("TestHoodieCompactor").setMaster("local[4]");
jsc = new JavaSparkContext(HoodieReadClient.addHoodieSupport(sparkConf));
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ);
compactor = new HoodieRealtimeTableCompactor();
//SQLContext stuff
sqlContext = new SQLContext(jsc);
}
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
@Test
public void testSimpleInsertAndUpdate() throws Exception {
HoodieWriteConfig cfg = getConfig();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
/**
* Write 1 (only inserts)
*/
String newCommitTime = "001";
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
Optional<HoodieInstant> deltaCommit =
metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent());
assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp());
Optional<HoodieInstant> commit =
metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
assertFalse(commit.isPresent());
TableFileSystemView fsView = hoodieTable.getCompactedFileSystemView();
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
Stream<HoodieDataFile> dataFilesToRead = fsView.getLatestVersions(allFiles);
assertTrue(!dataFilesToRead.findAny().isPresent());
fsView = hoodieTable.getFileSystemView();
dataFilesToRead = fsView.getLatestVersions(allFiles);
assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit",
dataFilesToRead.findAny().isPresent());
/**
* Write 2 (updates)
*/
newCommitTime = "004";
records = dataGen.generateUpdates(newCommitTime, 100);
Map<HoodieKey, HoodieRecord> recordsMap = new HashMap<>();
for (HoodieRecord rec : records) {
if (!recordsMap.containsKey(rec.getKey())) {
recordsMap.put(rec.getKey(), rec);
}
}
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant();
assertTrue(deltaCommit.isPresent());
assertEquals("Latest Delta commit should be 004", "004", deltaCommit.get().getTimestamp());
commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
assertFalse(commit.isPresent());
HoodieCompactor compactor = new HoodieRealtimeTableCompactor();
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
compactor.compact(jsc, getConfig(), table, CompactionFilter.allowAll());
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
allFiles = HoodieTestUtils.listAllDataFilesInPath(fs, cfg.getBasePath());
dataFilesToRead = fsView.getLatestVersions(allFiles);
assertTrue(dataFilesToRead.findAny().isPresent());
// verify that there is a commit
HoodieReadClient readClient = new HoodieReadClient(jsc, basePath, sqlContext);
assertEquals("Expecting a single commit.", 1, readClient.listCommitsSince("000").size());
String latestCompactionCommitTime = readClient.latestCommit();
assertTrue(metaClient.getActiveTimeline()
.compareTimestamps("000", latestCompactionCommitTime, HoodieTimeline.LESSER));
assertEquals("Must contain 200 records", 200, readClient.readSince("000").count());
}
private HoodieWriteConfig getConfig() {
return getConfigBuilder().build();
}
private HoodieWriteConfig.Builder getConfigBuilder() {
return HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withCompactionConfig(
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
.forTable("test-trip-table").withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build());
}
private void assertNoWriteErrors(List<WriteStatus> statuses) {
// Verify there are no errors
for (WriteStatus status : statuses) {
assertFalse("Errors found in write of " + status.getFileId(), status.hasErrors());
}
}
}

View File

@@ -57,7 +57,7 @@ public class TestUpdateMapFunction {
// Create a bunch of records with a old version of schema
HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable("100", config, metadata);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
String recordStr1 =
"{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
@@ -78,7 +78,7 @@ public class TestUpdateMapFunction {
records.add(
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()),
rowChange3));
Iterator<List<WriteStatus>> insertResult = table.handleInsert(records.iterator());
Iterator<List<WriteStatus>> insertResult = table.handleInsert("100", records.iterator());
Path commitFile =
new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100"));
FSUtils.getFs().create(commitFile);
@@ -91,7 +91,7 @@ public class TestUpdateMapFunction {
System.out.println(fileId);
table = new HoodieCopyOnWriteTable("101", config, metadata);
table = new HoodieCopyOnWriteTable(config, metadata);
// New content with values for the newly added field
recordStr1 =
"{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";
@@ -104,7 +104,7 @@ public class TestUpdateMapFunction {
records.add(record1);
try {
table.handleUpdate(fileId, records.iterator());
table.handleUpdate("101", fileId, records.iterator());
} catch (ClassCastException e) {
fail(
"UpdateFunction could not read records written with exampleSchema.txt using the exampleEvolvedSchema.txt");

View File

@@ -30,6 +30,7 @@ import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.io.IOUtils;
@@ -127,7 +128,8 @@ public class TestHoodieBloomIndex {
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");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
JavaPairRDD<String, String> rdd = index.loadInvolvedFiles(partitions, metadata);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
JavaPairRDD<String, String> rdd = index.loadInvolvedFiles(partitions, table);
// Still 0, as no valid commit
assertEquals(rdd.count(), 0);
@@ -136,7 +138,7 @@ public class TestHoodieBloomIndex {
new File(basePath + "/.hoodie/20160401010101.commit").createNewFile();
new File(basePath + "/.hoodie/20150312101010.commit").createNewFile();
metadata = new HoodieTableMetaClient(fs, basePath);
rdd = index.loadInvolvedFiles(partitions, metadata);
rdd = index.loadInvolvedFiles(partitions, table);
final List<Tuple2<String, String>> filesList = rdd.collect();
assertEquals(filesList.size(), 4);
@@ -214,12 +216,13 @@ public class TestHoodieBloomIndex {
// Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
// Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
try {
bloomIndex.tagLocation(recordRDD, metadata);
bloomIndex.tagLocation(recordRDD, table);
} catch (IllegalArgumentException e) {
fail("EmptyRDD should not result in IllegalArgumentException: Positive number of slices required");
}
@@ -250,10 +253,11 @@ public class TestHoodieBloomIndex {
// Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
// Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, metadata);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
// Should not find any files
for (HoodieRecord record : taggedRecordRDD.collect()) {
@@ -267,7 +271,9 @@ public class TestHoodieBloomIndex {
// We do the tag again
metadata = new HoodieTableMetaClient(fs, basePath);
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, metadata);
table = HoodieTable.getHoodieTable(metadata, config);
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
// Check results
for (HoodieRecord record : taggedRecordRDD.collect()) {
@@ -311,10 +317,11 @@ public class TestHoodieBloomIndex {
// Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
// Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
JavaPairRDD<HoodieKey, Optional<String>> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, metadata);
JavaPairRDD<HoodieKey, Optional<String>> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table);
// Should not find any files
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
@@ -328,7 +335,8 @@ public class TestHoodieBloomIndex {
// We do the tag again
metadata = new HoodieTableMetaClient(fs, basePath);
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, metadata);
table = HoodieTable.getHoodieTable(metadata, config);
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table);
// Check results
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
@@ -377,8 +385,10 @@ public class TestHoodieBloomIndex {
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, metadata);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
// Check results
for (HoodieRecord record : taggedRecordRDD.collect()) {

View File

@@ -22,6 +22,7 @@ import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.table.HoodieTable;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
@@ -56,7 +57,9 @@ public class TestHoodieCleaner {
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCleaner cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
HoodieCleaner cleaner = new HoodieCleaner(table, config);
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0));
@@ -70,7 +73,9 @@ public class TestHoodieCleaner {
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, partitionPaths[1], "001", file1P1C0); // update
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
table = HoodieTable.getHoodieTable(metadata, config);
cleaner = new HoodieCleaner(table, config);
assertEquals("Must clean 1 file" , 1, cleaner.clean(partitionPaths[0]));
assertEquals("Must clean 1 file" , 1, cleaner.clean(partitionPaths[1]));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1));
@@ -85,7 +90,9 @@ public class TestHoodieCleaner {
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update
String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "002");
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
table = HoodieTable.getHoodieTable(metadata, config);
cleaner = new HoodieCleaner(table, config);
assertEquals("Must clean two files" , 2, cleaner.clean(partitionPaths[0]));
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file1P0C0));
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1));
@@ -113,7 +120,9 @@ public class TestHoodieCleaner {
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCleaner cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
HoodieCleaner cleaner = new HoodieCleaner(table, config);
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0));
@@ -127,7 +136,9 @@ public class TestHoodieCleaner {
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, partitionPaths[1], "001", file1P1C0); // update
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
table = HoodieTable.getHoodieTable(metadata, config);
cleaner = new HoodieCleaner(table, config);
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1));
@@ -142,7 +153,9 @@ public class TestHoodieCleaner {
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update
String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "002");
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
table = HoodieTable.getHoodieTable(metadata, config);
cleaner = new HoodieCleaner(table, config);
assertEquals(
"Must not clean any file. We have to keep 1 version before the latest commit time to keep",
0, cleaner.clean(partitionPaths[0]));
@@ -156,7 +169,9 @@ public class TestHoodieCleaner {
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file2P0C1); // update
String file4P0C3 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "003");
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
table = HoodieTable.getHoodieTable(metadata, config);
cleaner = new HoodieCleaner(table, config);
assertEquals(
"Must not clean one old file", 1, cleaner.clean(partitionPaths[0]));

View File

@@ -28,8 +28,6 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.RealtimeTableView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieIndexConfig;
@@ -41,6 +39,7 @@ import com.uber.hoodie.io.compact.CompactionFilter;
import com.uber.hoodie.io.compact.HoodieCompactionMetadata;
import com.uber.hoodie.io.compact.HoodieCompactor;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
import com.uber.hoodie.table.HoodieTable;
import org.apache.hadoop.fs.FileSystem;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
@@ -112,24 +111,27 @@ public class TestHoodieCompactor {
HoodieTestUtils.initTableType(basePath, HoodieTableType.COPY_ON_WRITE);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
RealtimeTableView fsView = new RealtimeTableView(FSUtils.getFs(), metaClient);
compactor.compact(jsc, getConfig(), metaClient, fsView, CompactionFilter.allowAll());
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
compactor.compact(jsc, getConfig(), table, CompactionFilter.allowAll());
}
@Test
public void testCompactionEmpty() throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
RealtimeTableView fsView = new RealtimeTableView(FSUtils.getFs(), metaClient);
HoodieWriteConfig config = getConfig();
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
String newCommitTime = writeClient.startCommit();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
writeClient.insert(recordsRDD, newCommitTime).collect();
HoodieCompactionMetadata result =
compactor.compact(jsc, getConfig(), metaClient, fsView, CompactionFilter.allowAll());
assertTrue("If there is nothing to compact, result wull be null", result == null);
compactor.compact(jsc, getConfig(), table, CompactionFilter.allowAll());
assertTrue("If there is nothing to compact, result will be empty",
result.getFileIdAndFullPaths().isEmpty());
}
@Test
@@ -145,11 +147,13 @@ public class TestHoodieCompactor {
// Update all the 100 records
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
newCommitTime = "101";
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
HoodieIndex index = new HoodieBloomIndex<>(config, jsc);
updatedRecords = index.tagLocation(updatedRecordsRDD, metaClient).collect();
updatedRecords = index.tagLocation(updatedRecordsRDD, table).collect();
// Write them to corresponding avro logfiles
HoodieTestUtils
@@ -158,10 +162,10 @@ public class TestHoodieCompactor {
// Verify that all data file has one log file
metaClient = new HoodieTableMetaClient(fs, basePath);
RealtimeTableView fsView = new RealtimeTableView(fs, metaClient);
table = HoodieTable.getHoodieTable(metaClient, config);
for (String partitionPath : dataGen.getPartitionPaths()) {
Map<HoodieDataFile, List<HoodieLogFile>> groupedLogFiles =
fsView.groupLatestDataFileWithLogFiles(fs, partitionPath);
table.getFileSystemView().groupLatestDataFileWithLogFiles(partitionPath);
for (List<HoodieLogFile> logFiles : groupedLogFiles.values()) {
assertEquals("There should be 1 log file written for every data file", 1,
logFiles.size());
@@ -170,13 +174,14 @@ public class TestHoodieCompactor {
// Do a compaction
metaClient = new HoodieTableMetaClient(fs, basePath);
fsView = new RealtimeTableView(fs, metaClient);
table = HoodieTable.getHoodieTable(metaClient, config);
HoodieCompactionMetadata result =
compactor.compact(jsc, getConfig(), metaClient, fsView, CompactionFilter.allowAll());
compactor.compact(jsc, getConfig(), table, CompactionFilter.allowAll());
// Verify that recently written compacted data file has no log file
metaClient = new HoodieTableMetaClient(fs, basePath);
fsView = new RealtimeTableView(fs, metaClient);
table = HoodieTable.getHoodieTable(metaClient, config);
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
assertTrue("Compaction commit should be > than last insert", timeline
@@ -185,7 +190,7 @@ public class TestHoodieCompactor {
for (String partitionPath : dataGen.getPartitionPaths()) {
Map<HoodieDataFile, List<HoodieLogFile>> groupedLogFiles =
fsView.groupLatestDataFileWithLogFiles(fs, partitionPath);
table.getFileSystemView().groupLatestDataFileWithLogFiles(partitionPath);
for (List<HoodieLogFile> logFiles : groupedLogFiles.values()) {
assertTrue(
"After compaction there should be no log files visiable on a Realtime view",

View File

@@ -89,8 +89,10 @@ public class TestCopyOnWriteTable {
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieWriteConfig config = makeHoodieClientConfig();
HoodieInsertHandle io = new HoodieInsertHandle(config, commitTime,
new HoodieTableMetaClient(FSUtils.getFs(), basePath), partitionPath);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieInsertHandle io = new HoodieInsertHandle(config, commitTime, table, partitionPath);
Path newPath = io.makeNewPath(record.getPartitionPath(), unitNumber, fileName);
assertTrue(newPath.toString().equals(this.basePath + "/" + partitionPath + "/" + FSUtils
.makeDataFileName(commitTime, unitNumber, fileName)));
@@ -113,8 +115,9 @@ public class TestCopyOnWriteTable {
HoodieWriteConfig config = makeHoodieClientConfig();
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
String partitionPath = "/2016/01/31";
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(firstCommitTime, config, metadata);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Get some records belong to the same partition (2016/01/31)
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
@@ -131,7 +134,7 @@ public class TestCopyOnWriteTable {
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
// Insert new records
HoodieClientTestUtils.collectStatuses(table.handleInsert(records.iterator()));
HoodieClientTestUtils.collectStatuses(table.handleInsert(firstCommitTime, records.iterator()));
// We should have a parquet file generated (TODO: better control # files after we revise AvroParquetIO)
File parquetFile = null;
for (File file : new File(this.basePath + partitionPath).listFiles()) {
@@ -175,8 +178,8 @@ public class TestCopyOnWriteTable {
Thread.sleep(1000);
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());
table = new HoodieCopyOnWriteTable(config, metadata);
Iterator<List<WriteStatus>> iter = table.handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(), updatedRecords.iterator());
// Check the updated file
File updatedParquetFile = null;
@@ -242,7 +245,7 @@ public class TestCopyOnWriteTable {
String commitTime = HoodieTestUtils.makeNewCommitTime();
FileSystem fs = FSUtils.getFs();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(commitTime, config, metadata);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Write a few records, and get atleast one file
// 10 records for partition 1, 1 record for partition 2.
@@ -250,7 +253,7 @@ public class TestCopyOnWriteTable {
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
// Simulate crash after first file
List<WriteStatus> statuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(records.iterator()));
List<WriteStatus> statuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
WriteStatus status = statuses.get(0);
Path partialFile = new Path(String.format("%s/%s/%s",
basePath,
@@ -263,7 +266,7 @@ public class TestCopyOnWriteTable {
records = newHoodieRecords(10, "2016-01-31T03:16:41.415Z");
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
statuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(records.iterator()));
statuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
status = statuses.get(0);
Path retriedFIle = new Path(String.format("%s/%s/%s",
@@ -280,7 +283,7 @@ public class TestCopyOnWriteTable {
HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(commitTime, config, metadata);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Case 1:
// 10 records for partition 1, 1 record for partition 2.
@@ -288,7 +291,7 @@ public class TestCopyOnWriteTable {
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
// Insert new records
List<WriteStatus> returnedStatuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(records.iterator()));
List<WriteStatus> returnedStatuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
// TODO: check the actual files and make sure 11 records, total were written.
@@ -307,7 +310,7 @@ public class TestCopyOnWriteTable {
records.addAll(newHoodieRecords(1, "2016-02-02T03:16:41.415Z"));
// Insert new records
returnedStatuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(records.iterator()));
returnedStatuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
assertEquals(3, returnedStatuses.size());
assertEquals("2016/01/31", returnedStatuses.get(0).getPartitionPath());
@@ -327,7 +330,7 @@ public class TestCopyOnWriteTable {
.parquetPageSize(64 * 1024).build()).build();
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(commitTime, config, metadata);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
List<HoodieRecord> records = new ArrayList<>();
// Approx 1150 records are written for block size of 64KB
@@ -339,7 +342,7 @@ public class TestCopyOnWriteTable {
}
// Insert new records
HoodieClientTestUtils.collectStatuses(table.handleInsert(records.iterator()));
HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
// Check the updated file
int counts = 0;
@@ -371,7 +374,7 @@ public class TestCopyOnWriteTable {
HoodieClientTestUtils.fakeDataFile(basePath, TEST_PARTITION_PATH, "001", "file1", fileSize);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable("001", config, metadata);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[]{TEST_PARTITION_PATH});
List<HoodieRecord> insertRecords = dataGenerator.generateInserts("001", numInserts);