1
0

[HUDI-1105] Adding dedup support for Bulk Insert w/ Rows (#2206)

This commit is contained in:
Sivabalan Narayanan
2021-07-07 17:38:26 -04:00
committed by GitHub
parent 8f7ad8b178
commit 16e90d30ea
7 changed files with 265 additions and 23 deletions

View File

@@ -25,15 +25,32 @@ import org.apache.hudi.testutils.DataSourceTestUtils;
import org.apache.hudi.testutils.HoodieClientTestBase;
import org.apache.avro.Schema;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.ReduceFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
import org.apache.spark.sql.catalyst.encoders.RowEncoder;
import org.apache.spark.sql.catalyst.expressions.Attribute;
import org.apache.spark.sql.types.StructType;
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.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import scala.Tuple2;
import scala.collection.JavaConversions;
import scala.collection.JavaConverters;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -45,13 +62,22 @@ import static org.junit.jupiter.api.Assertions.fail;
public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
private String schemaStr;
private Schema schema;
private transient Schema schema;
private StructType structType;
public TestHoodieDatasetBulkInsertHelper() throws IOException {
init();
}
/**
* args for schema evolution test.
*/
private static Stream<Arguments> providePreCombineArgs() {
return Stream.of(
Arguments.of(false),
Arguments.of(true));
}
private void init() throws IOException {
schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
schema = DataSourceTestUtils.getStructTypeExampleSchema();
@@ -59,12 +85,12 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
}
@Test
public void testBulkInsertHelper() throws IOException {
HoodieWriteConfig config = getConfigBuilder(schemaStr).withProps(getPropsAllSet()).build();
public void testBulkInsertHelper() {
HoodieWriteConfig config = getConfigBuilder(schemaStr).withProps(getPropsAllSet()).combineInput(false, false).build();
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace",
new NonSortPartitionerWithRows());
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
"testNamespace", new NonSortPartitionerWithRows(), false);
StructType resultSchema = result.schema();
assertEquals(result.count(), 10);
@@ -74,6 +100,42 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
assertTrue(resultSchema.fieldIndex(entry.getKey()) == entry.getValue());
}
result.toJavaRDD().foreach(entry -> {
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.RECORD_KEY_METADATA_FIELD)).equals(entry.getAs("_row_key")));
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).equals(entry.getAs("partition")));
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)).equals(""));
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).equals(""));
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.FILENAME_METADATA_FIELD)).equals(""));
});
Dataset<Row> trimmedOutput = result.drop(HoodieRecord.PARTITION_PATH_METADATA_FIELD).drop(HoodieRecord.RECORD_KEY_METADATA_FIELD)
.drop(HoodieRecord.FILENAME_METADATA_FIELD).drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD).drop(HoodieRecord.COMMIT_TIME_METADATA_FIELD);
assertTrue(dataset.except(trimmedOutput).count() == 0);
}
@ParameterizedTest
@MethodSource("providePreCombineArgs")
public void testBulkInsertPreCombine(boolean enablePreCombine) {
HoodieWriteConfig config = getConfigBuilder(schemaStr).withProps(getPropsAllSet()).combineInput(enablePreCombine, enablePreCombine)
.withPreCombineField("ts").build();
List<Row> inserts = DataSourceTestUtils.generateRandomRows(10);
Dataset<Row> toUpdateDataset = sqlContext.createDataFrame(inserts.subList(0, 5), structType);
List<Row> updates = DataSourceTestUtils.updateRowsWithHigherTs(toUpdateDataset);
List<Row> rows = new ArrayList<>();
rows.addAll(inserts);
rows.addAll(updates);
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
"testNamespace", new NonSortPartitionerWithRows(), false);
StructType resultSchema = result.schema();
assertEquals(result.count(), enablePreCombine ? 10 : 15);
assertEquals(resultSchema.fieldNames().length, structType.fieldNames().length + HoodieRecord.HOODIE_META_COLUMNS.size());
for (Map.Entry<String, Integer> entry : HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.entrySet()) {
assertTrue(resultSchema.fieldIndex(entry.getKey()) == entry.getValue());
}
int metadataRecordKeyIndex = resultSchema.fieldIndex(HoodieRecord.RECORD_KEY_METADATA_FIELD);
int metadataParitionPathIndex = resultSchema.fieldIndex(HoodieRecord.PARTITION_PATH_METADATA_FIELD);
int metadataCommitTimeIndex = resultSchema.fieldIndex(HoodieRecord.COMMIT_TIME_METADATA_FIELD);
@@ -87,6 +149,30 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
assertTrue(entry.get(metadataCommitTimeIndex).equals(""));
assertTrue(entry.get(metadataFilenameIndex).equals(""));
});
Dataset<Row> trimmedOutput = result.drop(HoodieRecord.PARTITION_PATH_METADATA_FIELD).drop(HoodieRecord.RECORD_KEY_METADATA_FIELD)
.drop(HoodieRecord.FILENAME_METADATA_FIELD).drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD).drop(HoodieRecord.COMMIT_TIME_METADATA_FIELD);
// find resolved input snapshot
ExpressionEncoder encoder = getEncoder(dataset.schema());
if (enablePreCombine) {
Dataset<Row> inputSnapshotDf = dataset.groupByKey(
(MapFunction<Row, String>) value -> value.getAs("partition") + "+" + value.getAs("_row_key"), Encoders.STRING())
.reduceGroups((ReduceFunction<Row>) (v1, v2) -> {
long ts1 = v1.getAs("ts");
long ts2 = v2.getAs("ts");
if (ts1 >= ts2) {
return v1;
} else {
return v2;
}
})
.map((MapFunction<Tuple2<String, Row>, Row>) value -> value._2, encoder);
assertTrue(inputSnapshotDf.except(trimmedOutput).count() == 0);
} else {
assertTrue(dataset.except(trimmedOutput).count() == 0);
}
}
private Map<String, String> getPropsAllSet() {
@@ -120,7 +206,7 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
try {
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
"testNamespace", new NonSortPartitionerWithRows());
"testNamespace", new NonSortPartitionerWithRows(), false);
fail("Should have thrown exception");
} catch (Exception e) {
// ignore
@@ -131,7 +217,7 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
dataset = sqlContext.createDataFrame(rows, structType);
try {
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
"testNamespace", new NonSortPartitionerWithRows());
"testNamespace", new NonSortPartitionerWithRows(), false);
fail("Should have thrown exception");
} catch (Exception e) {
// ignore
@@ -142,7 +228,7 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
dataset = sqlContext.createDataFrame(rows, structType);
try {
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
"testNamespace", new NonSortPartitionerWithRows());
"testNamespace", new NonSortPartitionerWithRows(), false);
fail("Should have thrown exception");
} catch (Exception e) {
// ignore
@@ -153,10 +239,18 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
dataset = sqlContext.createDataFrame(rows, structType);
try {
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
"testNamespace", new NonSortPartitionerWithRows());
"testNamespace", new NonSortPartitionerWithRows(), false);
fail("Should have thrown exception");
} catch (Exception e) {
// ignore
}
}
private ExpressionEncoder getEncoder(StructType schema) {
List<Attribute> attributes = JavaConversions.asJavaCollection(schema.toAttributes()).stream()
.map(Attribute::toAttribute).collect(Collectors.toList());
return RowEncoder.apply(schema)
.resolveAndBind(JavaConverters.asScalaBufferConverter(attributes).asScala().toSeq(),
SimpleAnalyzer$.MODULE$);
}
}

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.testutils;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.avro.Schema;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.RowFactory;
@@ -75,6 +76,20 @@ public class DataSourceTestUtils {
return toReturn;
}
public static List<Row> getUniqueRows(List<Row> inserts, int count) {
List<Row> toReturn = new ArrayList<>();
int soFar = 0;
int curIndex = 0;
while (soFar < count) {
if (!toReturn.contains(inserts.get(curIndex))) {
toReturn.add(inserts.get(curIndex));
soFar++;
}
curIndex++;
}
return toReturn;
}
public static List<Row> generateRandomRowsEvolvedSchema(int count) {
Random random = new Random();
List<Row> toReturn = new ArrayList<>();
@@ -89,4 +104,18 @@ public class DataSourceTestUtils {
}
return toReturn;
}
public static List<Row> updateRowsWithHigherTs(Dataset<Row> inputDf) {
Random random = new Random();
List<Row> input = inputDf.collectAsList();
List<Row> rows = new ArrayList<>();
for (Row row : input) {
Object[] values = new Object[3];
values[0] = row.getAs("_row_key");
values[1] = row.getAs("partition");
values[2] = ((Long) row.getAs("ts")) + random.nextInt(1000);
rows.add(RowFactory.create(values));
}
return rows;
}
}