Split insert() into insert() & bulkInsert() (#69)
- Behaviour change for existing insert() users - Made the current insert() implementation, as something to use for bulkInsert() - Normal inserts now share a lot of code with upsert, which provides benefits like small file handling - Refactored/Cleaned up code in HoodieWriteClient for reuse - Added a unit test, switching few tests to call bulkInsert() and few to call insert()
This commit is contained in:
@@ -137,7 +137,7 @@ public class TestHoodieClient implements Serializable {
|
||||
|
||||
JavaRDD<HoodieRecord> smallRecordsRDD = jsc.parallelize(records.subList(0, 75), 1);
|
||||
// We create three parquet file, each having one record. (two different partitions)
|
||||
List<WriteStatus> statuses = writeClient.insert(smallRecordsRDD, newCommitTime).collect();
|
||||
List<WriteStatus> statuses = writeClient.bulkInsert(smallRecordsRDD, newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
@@ -158,7 +158,7 @@ public class TestHoodieClient implements Serializable {
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
JavaRDD<WriteStatus> result = client.insert(writeRecords, newCommitTime);
|
||||
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, newCommitTime);
|
||||
|
||||
assertFalse("If Autocommit is false, then commit should not be made automatically",
|
||||
HoodieTestUtils.doesCommitExist(basePath, newCommitTime));
|
||||
@@ -169,7 +169,7 @@ public class TestHoodieClient implements Serializable {
|
||||
newCommitTime = "002";
|
||||
records = dataGen.generateUpdates(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> updateRecords = jsc.parallelize(records, 1);
|
||||
result = client.upsert(writeRecords, newCommitTime);
|
||||
result = client.upsert(updateRecords, newCommitTime);
|
||||
assertFalse("If Autocommit is false, then commit should not be made automatically",
|
||||
HoodieTestUtils.doesCommitExist(basePath, newCommitTime));
|
||||
assertTrue("Commit should succeed", client.commit(newCommitTime, result));
|
||||
@@ -542,24 +542,27 @@ public class TestHoodieClient implements Serializable {
|
||||
}
|
||||
|
||||
|
||||
private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize) {
|
||||
HoodieWriteConfig.Builder builder = getConfigBuilder();
|
||||
return builder.withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder()
|
||||
.compactionSmallFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 15)
|
||||
.insertSplitSize(insertSplitSize).build()) // tolerate upto 15 records
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder()
|
||||
.limitFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 20)
|
||||
.build())
|
||||
.build();
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testSmallInsertHandling() throws Exception {
|
||||
public void testSmallInsertHandlingForUpserts() throws Exception {
|
||||
|
||||
HoodieWriteConfig.Builder builder = getConfigBuilder();
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
|
||||
|
||||
final String TEST_PARTITION_PATH = "2016/09/26";
|
||||
final int INSERT_SPLIT_LIMIT = 10;
|
||||
// based on examination of sample file, the schema produces the following per record size
|
||||
final int SIZE_PER_RECORD = 50 * 1024;
|
||||
// setup the small file handling params
|
||||
HoodieWriteConfig config = builder.withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(SIZE_PER_RECORD * 15)
|
||||
.insertSplitSize(INSERT_SPLIT_LIMIT).build()) // tolerate upto 15 records
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(SIZE_PER_RECORD * 20)
|
||||
.build()).build(); // hold upto 20 records max
|
||||
HoodieWriteConfig config = getSmallInsertWriteConfig(INSERT_SPLIT_LIMIT); // hold upto 20 records max
|
||||
dataGen = new HoodieTestDataGenerator(new String[] {TEST_PARTITION_PATH});
|
||||
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, config);
|
||||
@@ -651,6 +654,79 @@ public class TestHoodieClient implements Serializable {
|
||||
assertEquals("Total inserts in commit3 must add up", keys3.size(), numTotalInsertsInCommit3);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSmallInsertHandlingForInserts() throws Exception {
|
||||
|
||||
final String TEST_PARTITION_PATH = "2016/09/26";
|
||||
final int INSERT_SPLIT_LIMIT = 10;
|
||||
// setup the small file handling params
|
||||
HoodieWriteConfig config = getSmallInsertWriteConfig(INSERT_SPLIT_LIMIT); // hold upto 20 records max
|
||||
dataGen = new HoodieTestDataGenerator(new String[] {TEST_PARTITION_PATH});
|
||||
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, config);
|
||||
|
||||
// Inserts => will write file1
|
||||
String commitTime1 = "001";
|
||||
List<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime1, INSERT_SPLIT_LIMIT); // this writes ~500kb
|
||||
Set<String> keys1 = HoodieClientTestUtils.getRecordKeys(inserts1);
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 1);
|
||||
List<WriteStatus> statuses= client.insert(insertRecordsRDD1, commitTime1).collect();
|
||||
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
assertEquals("Just 1 file needs to be added.", 1, statuses.size());
|
||||
String file1 = statuses.get(0).getFileId();
|
||||
assertEquals("file should contain 10 records",
|
||||
ParquetUtils.readRowKeysFromParquet(new Path(basePath, TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(),
|
||||
10);
|
||||
|
||||
// Second, set of Inserts should just expand file1
|
||||
String commitTime2 = "002";
|
||||
List<HoodieRecord> inserts2 = dataGen.generateInserts(commitTime2, 4);
|
||||
Set<String> keys2 = HoodieClientTestUtils.getRecordKeys(inserts2);
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD2 = jsc.parallelize(inserts2, 1);
|
||||
statuses = client.insert(insertRecordsRDD2, commitTime2).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
assertEquals("Just 1 file needs to be updated.", 1, statuses.size());
|
||||
assertEquals("Existing file should be expanded", file1, statuses.get(0).getFileId());
|
||||
assertEquals("Existing file should be expanded", commitTime1, statuses.get(0).getStat().getPrevCommit());
|
||||
Path newFile = new Path(basePath, TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1));
|
||||
assertEquals("file should contain 14 records", ParquetUtils.readRowKeysFromParquet(newFile).size(), 14);
|
||||
|
||||
List<GenericRecord> records = ParquetUtils.readAvroRecords(newFile);
|
||||
for (GenericRecord record: records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String recCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
assertTrue("Record expected to be part of commit 1 or commit2", commitTime1.equals(recCommitTime) || commitTime2.equals(recCommitTime));
|
||||
assertTrue("key expected to be part of commit 1 or commit2", keys2.contains(recordKey) || keys1.contains(recordKey));
|
||||
}
|
||||
|
||||
// Lots of inserts such that file1 is updated and expanded, a new file2 is created.
|
||||
String commitTime3 = "003";
|
||||
List<HoodieRecord> insert3 = dataGen.generateInserts(commitTime3, 20);
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD3 = jsc.parallelize(insert3, 1);
|
||||
statuses = client.insert(insertRecordsRDD3, commitTime3).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
assertEquals("2 files needs to be committed.", 2, statuses.size());
|
||||
|
||||
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
FileStatus[] files = metadata.getLatestVersionInPartition(fs, TEST_PARTITION_PATH, commitTime3);
|
||||
assertEquals("Total of 2 valid data files", 2, files.length);
|
||||
|
||||
|
||||
int totalInserts = 0;
|
||||
for (FileStatus file: files) {
|
||||
assertEquals("All files must be at commit 3", commitTime3, FSUtils.getCommitTime(file.getPath().getName()));
|
||||
records = ParquetUtils.readAvroRecords(file.getPath());
|
||||
totalInserts += records.size();
|
||||
}
|
||||
assertEquals("Total number of records must add up", totalInserts, inserts1.size() + inserts2.size() + insert3.size());
|
||||
}
|
||||
|
||||
|
||||
|
||||
@After
|
||||
public void clean() {
|
||||
|
||||
@@ -62,6 +62,9 @@ public class HoodieTestDataGenerator {
|
||||
+ "{\"name\": \"end_lon\", \"type\": \"double\"},"
|
||||
+ "{\"name\":\"fare\",\"type\": \"double\"}]}";
|
||||
|
||||
// based on examination of sample file, the schema produces the following per record size
|
||||
public static final int SIZE_PER_RECORD = 50 * 1024;
|
||||
|
||||
|
||||
private List<KeyPartition> existingKeysList = new ArrayList<>();
|
||||
private static Schema avroSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA));
|
||||
|
||||
Reference in New Issue
Block a user