|
|
|
|
@@ -17,6 +17,7 @@
|
|
|
|
|
|
|
|
|
|
package org.apache.hudi.functional;
|
|
|
|
|
|
|
|
|
|
import org.apache.avro.Schema;
|
|
|
|
|
import org.apache.hudi.AvroConversionUtils;
|
|
|
|
|
import org.apache.hudi.DataSourceWriteOptions;
|
|
|
|
|
import org.apache.hudi.HoodieDatasetBulkInsertHelper;
|
|
|
|
|
@@ -27,10 +28,9 @@ import org.apache.hudi.execution.bulkinsert.NonSortPartitionerWithRows;
|
|
|
|
|
import org.apache.hudi.keygen.ComplexKeyGenerator;
|
|
|
|
|
import org.apache.hudi.keygen.NonpartitionedKeyGenerator;
|
|
|
|
|
import org.apache.hudi.keygen.SimpleKeyGenerator;
|
|
|
|
|
import org.apache.hudi.metadata.HoodieTableMetadata;
|
|
|
|
|
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;
|
|
|
|
|
@@ -46,6 +46,9 @@ 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 scala.Tuple2;
|
|
|
|
|
import scala.collection.JavaConversions;
|
|
|
|
|
import scala.collection.JavaConverters;
|
|
|
|
|
|
|
|
|
|
import java.io.IOException;
|
|
|
|
|
import java.util.ArrayList;
|
|
|
|
|
@@ -56,10 +59,6 @@ import java.util.stream.Collectors;
|
|
|
|
|
import java.util.stream.IntStream;
|
|
|
|
|
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;
|
|
|
|
|
import static org.junit.jupiter.api.Assertions.fail;
|
|
|
|
|
@@ -117,36 +116,42 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|
|
|
|
testBulkInsertHelperFor(keyGenClass, "_row_key");
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private void testBulkInsertHelperFor(String keyGenClass, String recordKey) {
|
|
|
|
|
private void testBulkInsertHelperFor(String keyGenClass, String recordKeyField) {
|
|
|
|
|
Map<String, String> props = null;
|
|
|
|
|
if (keyGenClass.equals(SimpleKeyGenerator.class.getName())) {
|
|
|
|
|
props = getPropsAllSet(recordKey);
|
|
|
|
|
props = getPropsAllSet(recordKeyField);
|
|
|
|
|
} else if (keyGenClass.equals(ComplexKeyGenerator.class.getName())) {
|
|
|
|
|
props = getPropsForComplexKeyGen(recordKey);
|
|
|
|
|
props = getPropsForComplexKeyGen(recordKeyField);
|
|
|
|
|
} else { // NonPartitioned key gen
|
|
|
|
|
props = getPropsForNonPartitionedKeyGen(recordKey);
|
|
|
|
|
props = getPropsForNonPartitionedKeyGen(recordKeyField);
|
|
|
|
|
}
|
|
|
|
|
HoodieWriteConfig config = getConfigBuilder(schemaStr).withProps(props).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(), false, false);
|
|
|
|
|
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(dataset, config,
|
|
|
|
|
new NonSortPartitionerWithRows(), false);
|
|
|
|
|
StructType resultSchema = result.schema();
|
|
|
|
|
|
|
|
|
|
assertEquals(result.count(), 10);
|
|
|
|
|
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());
|
|
|
|
|
assertEquals(entry.getValue(), resultSchema.fieldIndex(entry.getKey()));
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
boolean isNonPartitioned = keyGenClass.equals(NonpartitionedKeyGenerator.class.getName());
|
|
|
|
|
boolean isNonPartitionedKeyGen = keyGenClass.equals(NonpartitionedKeyGenerator.class.getName());
|
|
|
|
|
boolean isComplexKeyGen = keyGenClass.equals(ComplexKeyGenerator.class.getName());
|
|
|
|
|
|
|
|
|
|
result.toJavaRDD().foreach(entry -> {
|
|
|
|
|
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.RECORD_KEY_METADATA_FIELD)).equals(entry.getAs(recordKey).toString()));
|
|
|
|
|
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).equals(isNonPartitioned ? "" : 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(""));
|
|
|
|
|
String recordKey = isComplexKeyGen ? String.format("%s:%s", recordKeyField, entry.getAs(recordKeyField)) : entry.getAs(recordKeyField).toString();
|
|
|
|
|
assertEquals(recordKey, entry.get(resultSchema.fieldIndex(HoodieRecord.RECORD_KEY_METADATA_FIELD)));
|
|
|
|
|
|
|
|
|
|
String partitionPath = isNonPartitionedKeyGen ? HoodieTableMetadata.EMPTY_PARTITION_NAME : entry.getAs("partition").toString();
|
|
|
|
|
assertEquals(partitionPath, entry.get(resultSchema.fieldIndex(HoodieRecord.PARTITION_PATH_METADATA_FIELD)));
|
|
|
|
|
|
|
|
|
|
assertEquals("", entry.get(resultSchema.fieldIndex(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)));
|
|
|
|
|
assertEquals("", entry.get(resultSchema.fieldIndex(HoodieRecord.COMMIT_TIME_METADATA_FIELD)));
|
|
|
|
|
assertEquals("", entry.get(resultSchema.fieldIndex(HoodieRecord.FILENAME_METADATA_FIELD)));
|
|
|
|
|
});
|
|
|
|
|
|
|
|
|
|
Dataset<Row> trimmedOutput = result.drop(HoodieRecord.PARTITION_PATH_METADATA_FIELD).drop(HoodieRecord.RECORD_KEY_METADATA_FIELD)
|
|
|
|
|
@@ -157,8 +162,13 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|
|
|
|
@Test
|
|
|
|
|
public void testBulkInsertHelperNoMetaFields() {
|
|
|
|
|
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
|
|
|
|
|
HoodieWriteConfig config = getConfigBuilder(schemaStr)
|
|
|
|
|
.withProps(getPropsAllSet("_row_key"))
|
|
|
|
|
.withPopulateMetaFields(false)
|
|
|
|
|
.build();
|
|
|
|
|
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
|
|
|
|
|
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsertWithoutMetaFields(dataset);
|
|
|
|
|
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(dataset, config,
|
|
|
|
|
new NonSortPartitionerWithRows(), false);
|
|
|
|
|
StructType resultSchema = result.schema();
|
|
|
|
|
|
|
|
|
|
assertEquals(result.count(), 10);
|
|
|
|
|
@@ -194,8 +204,8 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|
|
|
|
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, false);
|
|
|
|
|
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(dataset, config,
|
|
|
|
|
new NonSortPartitionerWithRows(), false);
|
|
|
|
|
StructType resultSchema = result.schema();
|
|
|
|
|
|
|
|
|
|
assertEquals(result.count(), enablePreCombine ? 10 : 15);
|
|
|
|
|
@@ -211,13 +221,15 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|
|
|
|
int metadataCommitSeqNoIndex = resultSchema.fieldIndex(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD);
|
|
|
|
|
int metadataFilenameIndex = resultSchema.fieldIndex(HoodieRecord.FILENAME_METADATA_FIELD);
|
|
|
|
|
|
|
|
|
|
result.toJavaRDD().foreach(entry -> {
|
|
|
|
|
assertTrue(entry.get(metadataRecordKeyIndex).equals(entry.getAs("_row_key")));
|
|
|
|
|
assertTrue(entry.get(metadataPartitionPathIndex).equals(entry.getAs("partition")));
|
|
|
|
|
assertTrue(entry.get(metadataCommitSeqNoIndex).equals(""));
|
|
|
|
|
assertTrue(entry.get(metadataCommitTimeIndex).equals(""));
|
|
|
|
|
assertTrue(entry.get(metadataFilenameIndex).equals(""));
|
|
|
|
|
});
|
|
|
|
|
result.toJavaRDD()
|
|
|
|
|
.collect()
|
|
|
|
|
.forEach(entry -> {
|
|
|
|
|
assertTrue(entry.get(metadataRecordKeyIndex).equals(entry.getAs("_row_key")));
|
|
|
|
|
assertTrue(entry.get(metadataPartitionPathIndex).equals(entry.getAs("partition")));
|
|
|
|
|
assertTrue(entry.get(metadataCommitSeqNoIndex).equals(""));
|
|
|
|
|
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);
|
|
|
|
|
@@ -226,7 +238,7 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|
|
|
|
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())
|
|
|
|
|
(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");
|
|
|
|
|
@@ -238,9 +250,9 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|
|
|
|
})
|
|
|
|
|
.map((MapFunction<Tuple2<String, Row>, Row>) value -> value._2, encoder);
|
|
|
|
|
|
|
|
|
|
assertTrue(inputSnapshotDf.except(trimmedOutput).count() == 0);
|
|
|
|
|
assertEquals(0, inputSnapshotDf.except(trimmedOutput).count());
|
|
|
|
|
} else {
|
|
|
|
|
assertTrue(dataset.except(trimmedOutput).count() == 0);
|
|
|
|
|
assertEquals(0, dataset.except(trimmedOutput).count());
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@@ -277,7 +289,7 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|
|
|
|
Map<String, String> props = new HashMap<>();
|
|
|
|
|
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), ComplexKeyGenerator.class.getName());
|
|
|
|
|
props.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), recordKey);
|
|
|
|
|
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "simple:partition");
|
|
|
|
|
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition");
|
|
|
|
|
props.put(HoodieWriteConfig.TBL_NAME.key(), recordKey + "_table");
|
|
|
|
|
return props;
|
|
|
|
|
}
|
|
|
|
|
@@ -296,8 +308,9 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|
|
|
|
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
|
|
|
|
|
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
|
|
|
|
|
try {
|
|
|
|
|
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
|
|
|
|
|
"testNamespace", new NonSortPartitionerWithRows(), false, false);
|
|
|
|
|
Dataset<Row> preparedDF = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(dataset, config,
|
|
|
|
|
new NonSortPartitionerWithRows(), false);
|
|
|
|
|
preparedDF.count();
|
|
|
|
|
fail("Should have thrown exception");
|
|
|
|
|
} catch (Exception e) {
|
|
|
|
|
// ignore
|
|
|
|
|
@@ -307,8 +320,9 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|
|
|
|
rows = DataSourceTestUtils.generateRandomRows(10);
|
|
|
|
|
dataset = sqlContext.createDataFrame(rows, structType);
|
|
|
|
|
try {
|
|
|
|
|
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
|
|
|
|
|
"testNamespace", new NonSortPartitionerWithRows(), false, false);
|
|
|
|
|
Dataset<Row> preparedDF = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(dataset, config,
|
|
|
|
|
new NonSortPartitionerWithRows(), false);
|
|
|
|
|
preparedDF.count();
|
|
|
|
|
fail("Should have thrown exception");
|
|
|
|
|
} catch (Exception e) {
|
|
|
|
|
// ignore
|
|
|
|
|
@@ -318,8 +332,9 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|
|
|
|
rows = DataSourceTestUtils.generateRandomRows(10);
|
|
|
|
|
dataset = sqlContext.createDataFrame(rows, structType);
|
|
|
|
|
try {
|
|
|
|
|
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
|
|
|
|
|
"testNamespace", new NonSortPartitionerWithRows(), false, false);
|
|
|
|
|
Dataset<Row> preparedDF = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(dataset, config,
|
|
|
|
|
new NonSortPartitionerWithRows(), false);
|
|
|
|
|
preparedDF.count();
|
|
|
|
|
fail("Should have thrown exception");
|
|
|
|
|
} catch (Exception e) {
|
|
|
|
|
// ignore
|
|
|
|
|
@@ -329,8 +344,9 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
|
|
|
|
rows = DataSourceTestUtils.generateRandomRows(10);
|
|
|
|
|
dataset = sqlContext.createDataFrame(rows, structType);
|
|
|
|
|
try {
|
|
|
|
|
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName",
|
|
|
|
|
"testNamespace", new NonSortPartitionerWithRows(), false, false);
|
|
|
|
|
Dataset<Row> preparedDF = HoodieDatasetBulkInsertHelper.prepareForBulkInsert(dataset, config,
|
|
|
|
|
new NonSortPartitionerWithRows(), false);
|
|
|
|
|
preparedDF.count();
|
|
|
|
|
fail("Should have thrown exception");
|
|
|
|
|
} catch (Exception e) {
|
|
|
|
|
// ignore
|
|
|
|
|
|