[HUDI-2161] Adding support to disable meta columns with bulk insert operation (#3247)
This commit is contained in:
committed by
GitHub
parent
2099bf41db
commit
d5026e9a24
@@ -39,7 +39,6 @@ import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder;
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getInternalRowWithError;
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows;
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows;
|
||||
@@ -52,6 +51,16 @@ public class TestHoodieBulkInsertDataInternalWriter extends
|
||||
HoodieBulkInsertInternalWriterTestBase {
|
||||
|
||||
private static Stream<Arguments> configParams() {
|
||||
Object[][] data = new Object[][] {
|
||||
{true, true},
|
||||
{true, false},
|
||||
{false, true},
|
||||
{false, false}
|
||||
};
|
||||
return Stream.of(data).map(Arguments::of);
|
||||
}
|
||||
|
||||
private static Stream<Arguments> bulkInsertTypeParams() {
|
||||
Object[][] data = new Object[][] {
|
||||
{true},
|
||||
{false}
|
||||
@@ -61,16 +70,16 @@ public class TestHoodieBulkInsertDataInternalWriter extends
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("configParams")
|
||||
public void testDataInternalWriter(boolean sorted) throws Exception {
|
||||
public void testDataInternalWriter(boolean sorted, boolean populateMetaFields) throws Exception {
|
||||
// init config and table
|
||||
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
|
||||
HoodieWriteConfig cfg = getWriteConfig(populateMetaFields);
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
// execute N rounds
|
||||
for (int i = 0; i < 5; i++) {
|
||||
String instantTime = "00" + i;
|
||||
// init writer
|
||||
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), STRUCT_TYPE,
|
||||
sorted);
|
||||
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000),
|
||||
RANDOM.nextLong(), STRUCT_TYPE, populateMetaFields, sorted);
|
||||
|
||||
int size = 10 + RANDOM.nextInt(1000);
|
||||
// write N rows to partition1, N rows to partition2 and N rows to partition3 ... Each batch should create a new RowCreateHandle and a new file
|
||||
@@ -97,7 +106,7 @@ public class TestHoodieBulkInsertDataInternalWriter extends
|
||||
|
||||
// verify rows
|
||||
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0]));
|
||||
assertOutput(totalInputRows, result, instantTime, fileNames);
|
||||
assertOutput(totalInputRows, result, instantTime, fileNames, populateMetaFields);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -109,12 +118,13 @@ public class TestHoodieBulkInsertDataInternalWriter extends
|
||||
@Test
|
||||
public void testGlobalFailure() throws Exception {
|
||||
// init config and table
|
||||
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
|
||||
HoodieWriteConfig cfg = getWriteConfig(true);
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
|
||||
|
||||
String instantTime = "001";
|
||||
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), STRUCT_TYPE, false);
|
||||
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000),
|
||||
RANDOM.nextLong(), STRUCT_TYPE, true, false);
|
||||
|
||||
int size = 10 + RANDOM.nextInt(100);
|
||||
int totalFailures = 5;
|
||||
@@ -150,7 +160,7 @@ public class TestHoodieBulkInsertDataInternalWriter extends
|
||||
|
||||
// verify rows
|
||||
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0]));
|
||||
assertOutput(inputRows, result, instantTime, fileNames);
|
||||
assertOutput(inputRows, result, instantTime, fileNames, true);
|
||||
}
|
||||
|
||||
private void writeRows(Dataset<Row> inputRows, HoodieBulkInsertDataInternalWriter writer)
|
||||
|
||||
@@ -33,6 +33,9 @@ import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.connector.write.DataWriter;
|
||||
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.MethodSource;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
@@ -40,10 +43,10 @@ import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder;
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows;
|
||||
import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
@@ -55,19 +58,28 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
public class TestHoodieDataSourceInternalBatchWrite extends
|
||||
HoodieBulkInsertInternalWriterTestBase {
|
||||
|
||||
@Test
|
||||
public void testDataSourceWriter() throws Exception {
|
||||
testDataSourceWriterInternal(Collections.EMPTY_MAP, Collections.EMPTY_MAP);
|
||||
private static Stream<Arguments> bulkInsertTypeParams() {
|
||||
Object[][] data = new Object[][] {
|
||||
{true},
|
||||
{false}
|
||||
};
|
||||
return Stream.of(data).map(Arguments::of);
|
||||
}
|
||||
|
||||
private void testDataSourceWriterInternal(Map<String, String> extraMetadata, Map<String, String> expectedExtraMetadata) throws Exception {
|
||||
@ParameterizedTest
|
||||
@MethodSource("bulkInsertTypeParams")
|
||||
public void testDataSourceWriter(boolean populateMetaFields) throws Exception {
|
||||
testDataSourceWriterInternal(Collections.EMPTY_MAP, Collections.EMPTY_MAP, populateMetaFields);
|
||||
}
|
||||
|
||||
private void testDataSourceWriterInternal(Map<String, String> extraMetadata, Map<String, String> expectedExtraMetadata, boolean populateMetaFields) throws Exception {
|
||||
// init config and table
|
||||
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
|
||||
HoodieWriteConfig cfg = getWriteConfig(populateMetaFields);
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
String instantTime = "001";
|
||||
// init writer
|
||||
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
|
||||
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, extraMetadata, false);
|
||||
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, extraMetadata, populateMetaFields, false);
|
||||
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(0, RANDOM.nextLong());
|
||||
|
||||
String[] partitionPaths = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS;
|
||||
@@ -99,7 +111,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
|
||||
metaClient.reloadActiveTimeline();
|
||||
Dataset<Row> result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
|
||||
// verify output
|
||||
assertOutput(totalInputRows, result, instantTime, Option.empty());
|
||||
assertOutput(totalInputRows, result, instantTime, Option.empty(), populateMetaFields);
|
||||
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty());
|
||||
|
||||
// verify extra metadata
|
||||
@@ -125,7 +137,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
|
||||
expectedMetadata.remove(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY().key());
|
||||
expectedMetadata.remove("commit_extra_c");
|
||||
|
||||
testDataSourceWriterInternal(extraMeta, expectedMetadata);
|
||||
testDataSourceWriterInternal(extraMeta, expectedMetadata, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -137,13 +149,14 @@ public class TestHoodieDataSourceInternalBatchWrite extends
|
||||
extraMeta.put("keyB", "valB");
|
||||
extraMeta.put("commit_extra_c", "valC");
|
||||
// none of the keys has commit metadata key prefix.
|
||||
testDataSourceWriterInternal(extraMeta, Collections.EMPTY_MAP);
|
||||
testDataSourceWriterInternal(extraMeta, Collections.EMPTY_MAP, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultipleDataSourceWrites() throws Exception {
|
||||
@ParameterizedTest
|
||||
@MethodSource("bulkInsertTypeParams")
|
||||
public void testMultipleDataSourceWrites(boolean populateMetaFields) throws Exception {
|
||||
// init config and table
|
||||
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
|
||||
HoodieWriteConfig cfg = getWriteConfig(populateMetaFields);
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
int partitionCounter = 0;
|
||||
|
||||
@@ -152,7 +165,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
|
||||
String instantTime = "00" + i;
|
||||
// init writer
|
||||
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
|
||||
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, false);
|
||||
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, populateMetaFields, false);
|
||||
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
|
||||
Dataset<Row> totalInputRows = null;
|
||||
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(partitionCounter++, RANDOM.nextLong());
|
||||
@@ -176,18 +189,19 @@ public class TestHoodieDataSourceInternalBatchWrite extends
|
||||
dataSourceInternalBatchWrite.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
|
||||
metaClient.reloadActiveTimeline();
|
||||
|
||||
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime);
|
||||
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime, populateMetaFields);
|
||||
|
||||
// verify output
|
||||
assertOutput(totalInputRows, result, instantTime, Option.empty());
|
||||
assertOutput(totalInputRows, result, instantTime, Option.empty(), populateMetaFields);
|
||||
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLargeWrites() throws Exception {
|
||||
@ParameterizedTest
|
||||
@MethodSource("bulkInsertTypeParams")
|
||||
public void testLargeWrites(boolean populateMetaFields) throws Exception {
|
||||
// init config and table
|
||||
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
|
||||
HoodieWriteConfig cfg = getWriteConfig(populateMetaFields);
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
int partitionCounter = 0;
|
||||
|
||||
@@ -196,7 +210,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
|
||||
String instantTime = "00" + i;
|
||||
// init writer
|
||||
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
|
||||
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, false);
|
||||
new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, populateMetaFields, false);
|
||||
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
|
||||
Dataset<Row> totalInputRows = null;
|
||||
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(partitionCounter++, RANDOM.nextLong());
|
||||
@@ -220,10 +234,11 @@ public class TestHoodieDataSourceInternalBatchWrite extends
|
||||
dataSourceInternalBatchWrite.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
|
||||
metaClient.reloadActiveTimeline();
|
||||
|
||||
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime);
|
||||
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime,
|
||||
populateMetaFields);
|
||||
|
||||
// verify output
|
||||
assertOutput(totalInputRows, result, instantTime, Option.empty());
|
||||
assertOutput(totalInputRows, result, instantTime, Option.empty(), populateMetaFields);
|
||||
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty());
|
||||
}
|
||||
}
|
||||
@@ -234,15 +249,16 @@ public class TestHoodieDataSourceInternalBatchWrite extends
|
||||
* abort batch2
|
||||
* verify only records from batch1 is available to read
|
||||
*/
|
||||
@Test
|
||||
public void testAbort() throws Exception {
|
||||
@ParameterizedTest
|
||||
@MethodSource("bulkInsertTypeParams")
|
||||
public void testAbort(boolean populateMetaFields) throws Exception {
|
||||
// init config and table
|
||||
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
|
||||
HoodieWriteConfig cfg = getWriteConfig(populateMetaFields);
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
String instantTime0 = "00" + 0;
|
||||
// init writer
|
||||
HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite =
|
||||
new HoodieDataSourceInternalBatchWrite(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, false);
|
||||
new HoodieDataSourceInternalBatchWrite(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, populateMetaFields, false);
|
||||
DataWriter<InternalRow> writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(0, RANDOM.nextLong());
|
||||
|
||||
List<String> partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS);
|
||||
@@ -274,13 +290,13 @@ public class TestHoodieDataSourceInternalBatchWrite extends
|
||||
metaClient.reloadActiveTimeline();
|
||||
Dataset<Row> result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
|
||||
// verify rows
|
||||
assertOutput(totalInputRows, result, instantTime0, Option.empty());
|
||||
assertOutput(totalInputRows, result, instantTime0, Option.empty(), populateMetaFields);
|
||||
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty());
|
||||
|
||||
// 2nd batch. abort in the end
|
||||
String instantTime1 = "00" + 1;
|
||||
dataSourceInternalBatchWrite =
|
||||
new HoodieDataSourceInternalBatchWrite(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, false);
|
||||
new HoodieDataSourceInternalBatchWrite(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf, Collections.EMPTY_MAP, populateMetaFields, false);
|
||||
writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(1, RANDOM.nextLong());
|
||||
|
||||
for (int j = 0; j < batches; j++) {
|
||||
@@ -298,7 +314,7 @@ public class TestHoodieDataSourceInternalBatchWrite extends
|
||||
result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
|
||||
// verify rows
|
||||
// only rows from first batch should be present
|
||||
assertOutput(totalInputRows, result, instantTime0, Option.empty());
|
||||
assertOutput(totalInputRows, result, instantTime0, Option.empty(), populateMetaFields);
|
||||
}
|
||||
|
||||
private void writeRows(Dataset<Row> inputRows, DataWriter<InternalRow> writer) throws Exception {
|
||||
|
||||
Reference in New Issue
Block a user