[HUDI-1234] Insert new records to data files without merging for "Insert" operation. (#2111)
* Added HoodieConcatHandle to skip merging for "insert" operation when the corresponding config is set Co-authored-by: Sivabalan Narayanan <sivabala@uber.com>
This commit is contained in:
@@ -79,7 +79,9 @@ import org.apache.spark.sql.Row;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.EnumSource;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.io.IOException;
|
||||
@@ -128,6 +130,10 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
}
|
||||
};
|
||||
|
||||
private static Stream<Arguments> configParams() {
|
||||
return Arrays.stream(new Boolean[][] {{true},{false}}).map(Arguments::of);
|
||||
}
|
||||
|
||||
private HoodieTestTable testTable;
|
||||
|
||||
@BeforeEach
|
||||
@@ -451,7 +457,62 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
}
|
||||
|
||||
/**
|
||||
* Tesst deletion of records.
|
||||
* Test Insert API for HoodieConcatHandle.
|
||||
*/
|
||||
@Test
|
||||
public void testInsertsWithHoodieConcatHandle() throws Exception {
|
||||
testHoodieConcatHandle(getConfig(), false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test InsertPrepped API for HoodieConcatHandle.
|
||||
*/
|
||||
@Test
|
||||
public void testInsertsPreppedWithHoodieConcatHandle() throws Exception {
|
||||
testHoodieConcatHandle(getConfig(), true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test one of HoodieConcatHandle w/ {@link AbstractHoodieWriteClient#insert(Object, String)} API.
|
||||
*
|
||||
* @param config Write Config
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
private void testHoodieConcatHandle(HoodieWriteConfig config, boolean isPrepped)
|
||||
throws Exception {
|
||||
// Force using older timeline layout
|
||||
HoodieWriteConfig hoodieWriteConfig = getConfigBuilder()
|
||||
.withProps(config.getProps()).withMergeAllowDuplicateOnInserts(true).withTimelineLayoutVersion(
|
||||
VERSION_0).build();
|
||||
|
||||
HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(), metaClient.getTableType(),
|
||||
metaClient.getTableConfig().getTableName(), metaClient.getArchivePath(),
|
||||
metaClient.getTableConfig().getPayloadClass(), VERSION_0);
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||
|
||||
// Write 1 (only inserts)
|
||||
String newCommitTime = "001";
|
||||
String initCommitTime = "000";
|
||||
int numRecords = 200;
|
||||
insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime, numRecords, SparkRDDWriteClient::insert,
|
||||
isPrepped, true, numRecords);
|
||||
|
||||
// Write 2 (updates)
|
||||
String prevCommitTime = newCommitTime;
|
||||
newCommitTime = "004";
|
||||
numRecords = 100;
|
||||
String commitTimeBetweenPrevAndNew = "002";
|
||||
|
||||
final Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
|
||||
generateWrapRecordsFn(isPrepped, hoodieWriteConfig, dataGen::generateUniqueUpdates);
|
||||
|
||||
writeBatch(client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), initCommitTime,
|
||||
numRecords, recordGenFunction, SparkRDDWriteClient::insert, true, numRecords, 300,
|
||||
2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests deletion of records.
|
||||
*/
|
||||
@Test
|
||||
public void testDeletes() throws Exception {
|
||||
@@ -877,13 +938,13 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
/**
|
||||
* Test scenario of new file-group getting added during insert().
|
||||
*/
|
||||
@Test
|
||||
public void testSmallInsertHandlingForInserts() throws Exception {
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("configParams")
|
||||
public void testSmallInsertHandlingForInserts(boolean mergeAllowDuplicateInserts) throws Exception {
|
||||
final String testPartitionPath = "2016/09/26";
|
||||
final int insertSplitLimit = 100;
|
||||
// setup the small file handling params
|
||||
HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max
|
||||
HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit, false, mergeAllowDuplicateInserts); // hold upto 200 records max
|
||||
dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath});
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(config, false);
|
||||
|
||||
@@ -894,10 +955,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
Set<String> keys1 = recordsToRecordKeySet(inserts1);
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 1);
|
||||
List<WriteStatus> statuses = client.insert(insertRecordsRDD1, commitTime1).collect();
|
||||
|
||||
assertNoWriteErrors(statuses);
|
||||
assertPartitionMetadata(new String[] {testPartitionPath}, fs);
|
||||
|
||||
assertEquals(1, statuses.size(), "Just 1 file needs to be added.");
|
||||
String file1 = statuses.get(0).getFileId();
|
||||
assertEquals(100,
|
||||
@@ -912,14 +971,13 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD2 = jsc.parallelize(inserts2, 1);
|
||||
statuses = client.insert(insertRecordsRDD2, commitTime2).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
assertEquals(1, statuses.size(), "Just 1 file needs to be updated.");
|
||||
assertEquals(file1, statuses.get(0).getFileId(), "Existing file should be expanded");
|
||||
assertEquals(commitTime1, statuses.get(0).getStat().getPrevCommit(), "Existing file should be expanded");
|
||||
|
||||
Path newFile = new Path(basePath, statuses.get(0).getStat().getPath());
|
||||
assertEquals(140, readRowKeysFromParquet(hadoopConf, newFile).size(),
|
||||
"file should contain 140 records");
|
||||
|
||||
List<GenericRecord> records = ParquetUtils.readAvroRecords(hadoopConf, newFile);
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
@@ -933,11 +991,15 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
// Lots of inserts such that file1 is updated and expanded, a new file2 is created.
|
||||
String commitTime3 = "003";
|
||||
client.startCommitWithTime(commitTime3);
|
||||
List<HoodieRecord> insert3 = dataGen.generateInserts(commitTime3, 200);
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD3 = jsc.parallelize(insert3, 1);
|
||||
List<HoodieRecord> inserts3 = dataGen.generateInserts(commitTime3, 200);
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD3 = jsc.parallelize(inserts3, 1);
|
||||
statuses = client.insert(insertRecordsRDD3, commitTime3).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
assertEquals(2, statuses.size(), "2 files needs to be committed.");
|
||||
assertEquals(340,
|
||||
readRowKeysFromParquet(hadoopConf, new Path(basePath, statuses.get(0).getStat().getPath())).size()
|
||||
+ readRowKeysFromParquet(hadoopConf, new Path(basePath, statuses.get(1).getStat().getPath())).size(),
|
||||
"file should contain 340 records");
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, basePath);
|
||||
HoodieTable table = getHoodieTable(metaClient, config);
|
||||
@@ -948,11 +1010,9 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
int totalInserts = 0;
|
||||
for (HoodieBaseFile file : files) {
|
||||
assertEquals(commitTime3, file.getCommitTime(), "All files must be at commit 3");
|
||||
records = ParquetUtils.readAvroRecords(hadoopConf, new Path(file.getPath()));
|
||||
totalInserts += records.size();
|
||||
totalInserts += ParquetUtils.readAvroRecords(hadoopConf, new Path(file.getPath())).size();
|
||||
}
|
||||
assertEquals(totalInserts, inserts1.size() + inserts2.size() + insert3.size(),
|
||||
"Total number of records must add up");
|
||||
assertEquals(totalInserts, inserts1.size() + inserts2.size() + inserts3.size(), "Total number of records must add up");
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -1040,7 +1100,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
|
||||
testClustering(clusteringConfig);
|
||||
}
|
||||
|
||||
|
||||
private void testClustering(HoodieClusteringConfig clusteringConfig) throws Exception {
|
||||
// create config to not update small files.
|
||||
HoodieWriteConfig config = getSmallInsertWriteConfig(2000, false, 10);
|
||||
@@ -1642,22 +1702,45 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
* Build Hoodie Write Config for small data file sizes.
|
||||
*/
|
||||
private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, boolean useNullSchema) {
|
||||
return getSmallInsertWriteConfig(insertSplitSize, useNullSchema, dataGen.getEstimatedFileSizeInBytes(150));
|
||||
return getSmallInsertWriteConfig(insertSplitSize, useNullSchema, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Build Hoodie Write Config for small data file sizes.
|
||||
*/
|
||||
private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, boolean useNullSchema, boolean mergeAllowDuplicateInserts) {
|
||||
return getSmallInsertWriteConfig(insertSplitSize, useNullSchema, dataGen.getEstimatedFileSizeInBytes(150), mergeAllowDuplicateInserts);
|
||||
}
|
||||
|
||||
/**
|
||||
* Build Hoodie Write Config for specified small file sizes.
|
||||
*/
|
||||
private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, boolean useNullSchema, long smallFileSize) {
|
||||
return getSmallInsertWriteConfig(insertSplitSize, useNullSchema, smallFileSize, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Build Hoodie Write Config for specified small file sizes.
|
||||
*/
|
||||
private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, boolean useNullSchema, long smallFileSize, boolean mergeAllowDuplicateInserts) {
|
||||
String schemaStr = useNullSchema ? NULL_SCHEMA : TRIP_EXAMPLE_SCHEMA;
|
||||
return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize);
|
||||
return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize, mergeAllowDuplicateInserts);
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize) {
|
||||
return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize, new Properties());
|
||||
return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize, false);
|
||||
}
|
||||
|
||||
|
||||
private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize, boolean mergeAllowDuplicateInserts) {
|
||||
return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize, mergeAllowDuplicateInserts, new Properties());
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize, Properties props) {
|
||||
return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize, false, props);
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize, boolean mergeAllowDuplicateInserts,
|
||||
Properties props) {
|
||||
HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr);
|
||||
return builder
|
||||
.withCompactionConfig(
|
||||
@@ -1668,6 +1751,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
HoodieStorageConfig.newBuilder()
|
||||
.hfileMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200))
|
||||
.parquetMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)).build())
|
||||
.withMergeAllowDuplicateOnInserts(mergeAllowDuplicateInserts)
|
||||
.withProps(props)
|
||||
.build();
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user