1
0

[HUDI-1013] Adding Bulk Insert V2 implementation (#1834)

- Adding ability to use native spark row writing for bulk_insert
 - Controlled by `ENABLE_ROW_WRITER_OPT_KEY` datasource write option
 - Introduced KeyGeneratorInterface in hudi-client, moved KeyGenerator back to hudi-spark
 - Simplified the new API additions to just two new methods : getRecordKey(row), getPartitionPath(row)
 - Fixed all built-in key generators with new APIs
 - Made the field position map lazily created upon the first call to row based apis
 - Implemented native row based key generators for CustomKeyGenerator
 - Fixed all the tests, with these new APIs

Co-authored-by: Balaji Varadarajan <varadarb@uber.com>
Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
Sivabalan Narayanan
2020-08-13 03:33:39 -04:00
committed by GitHub
parent 8d04268264
commit 379cf0786f
62 changed files with 4682 additions and 485 deletions

View File

@@ -151,7 +151,7 @@ public class HoodieJavaApp {
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName())
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_KEY(), "false")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "false")
// This will remove any existing data at path below, and create a
.mode(SaveMode.Overwrite);
@@ -178,7 +178,7 @@ public class HoodieJavaApp {
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_KEY(), "false")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "false")
.option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append);
updateHiveSyncConfig(writer);
@@ -204,7 +204,7 @@ public class HoodieJavaApp {
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_KEY(), "false")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "false")
.option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append);
updateHiveSyncConfig(writer);

View File

@@ -358,7 +358,7 @@ public class HoodieJavaStreamingApp {
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_KEY(), "true")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "true")
.option(HoodieWriteConfig.TABLE_NAME, tableName).option("checkpointLocation", checkpointLocation)
.outputMode(OutputMode.Append());

View File

@@ -0,0 +1,156 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.testutils.DataSourceTestUtils;
import org.apache.hudi.testutils.HoodieClientTestBase;
import org.apache.avro.Schema;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.types.StructType;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
/**
* Tests {@link HoodieDatasetBulkInsertHelper}.
*/
public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
private String schemaStr;
private Schema schema;
private StructType structType;
public TestHoodieDatasetBulkInsertHelper() throws IOException {
init();
}
private void init() throws IOException {
schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
schema = DataSourceTestUtils.getStructTypeExampleSchema();
structType = AvroConversionUtils.convertAvroSchemaToStructType(schema);
}
@Test
public void testBulkInsertHelper() throws IOException {
HoodieWriteConfig config = getConfigBuilder(schemaStr).withProps(getPropsAllSet()).build();
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
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());
}
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);
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(metadataParitionPathIndex).equals(entry.getAs("partition")));
assertTrue(entry.get(metadataCommitSeqNoIndex).equals(""));
assertTrue(entry.get(metadataCommitTimeIndex).equals(""));
assertTrue(entry.get(metadataFilenameIndex).equals(""));
});
}
private Map<String, String> getPropsAllSet() {
return getProps(true, true, true, true);
}
private Map<String, String> getProps(boolean setAll, boolean setKeyGen, boolean setRecordKey, boolean setPartitionPath) {
Map<String, String> props = new HashMap<>();
if (setAll) {
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), "org.apache.hudi.keygen.SimpleKeyGenerator");
props.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition");
} else {
if (setKeyGen) {
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), "org.apache.hudi.keygen.SimpleKeyGenerator");
}
if (setRecordKey) {
props.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
}
if (setPartitionPath) {
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition");
}
}
return props;
}
@Test
public void testNoPropsSet() {
HoodieWriteConfig config = getConfigBuilder(schemaStr).build();
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
try {
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
fail("Should have thrown exception");
} catch (Exception e) {
// ignore
}
config = getConfigBuilder(schemaStr).withProps(getProps(false, false, true, true)).build();
rows = DataSourceTestUtils.generateRandomRows(10);
dataset = sqlContext.createDataFrame(rows, structType);
try {
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
fail("Should have thrown exception");
} catch (Exception e) {
// ignore
}
config = getConfigBuilder(schemaStr).withProps(getProps(false, true, false, true)).build();
rows = DataSourceTestUtils.generateRandomRows(10);
dataset = sqlContext.createDataFrame(rows, structType);
try {
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
fail("Should have thrown exception");
} catch (Exception e) {
// ignore
}
config = getConfigBuilder(schemaStr).withProps(getProps(false, true, true, false)).build();
rows = DataSourceTestUtils.generateRandomRows(10);
dataset = sqlContext.createDataFrame(rows, structType);
try {
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
fail("Should have thrown exception");
} catch (Exception e) {
// ignore
}
}
}

View File

@@ -0,0 +1,213 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
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;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
/**
* Unit tests {@link HoodieBulkInsertDataInternalWriter}.
*/
public class TestHoodieBulkInsertDataInternalWriter extends HoodieClientTestHarness {
private static final Random RANDOM = new Random();
@BeforeEach
public void setUp() throws Exception {
initSparkContexts("TestHoodieBulkInsertDataInternalWriter");
initPath();
initFileSystem();
initTestDataGenerator();
initMetaClient();
}
@AfterEach
public void tearDown() throws Exception {
cleanupResources();
}
@Test
public void testDataInternalWriter() throws IOException {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
// 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(), RANDOM.nextLong(), STRUCT_TYPE);
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
int batches = 5;
Dataset<Row> totalInputRows = null;
for (int j = 0; j < batches; j++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
writeRows(inputRows, writer);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
List<String> fileAbsPaths = new ArrayList<>();
List<String> fileNames = new ArrayList<>();
// verify write statuses
assertWriteStatuses(commitMetadata.getWriteStatuses(), batches, size, fileAbsPaths, fileNames);
// verify rows
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0]));
assertOutput(totalInputRows, result, instantTime, fileNames);
}
}
/**
* Issue some corrupted or wrong schematized InternalRow after few valid InternalRows so that global error is thrown. write batch 1 of valid records write batch2 of invalid records which is expected
* to throw Global Error. Verify global error is set appropriately and only first batch of records are written to disk.
*/
@Test
public void testGlobalFailure() throws IOException {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
String instantTime = "001";
HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE);
int size = 10 + RANDOM.nextInt(100);
int totalFailures = 5;
// Generate first batch of valid rows
Dataset<Row> inputRows = getRandomRows(sqlContext, size / 2, partitionPath, false);
List<InternalRow> internalRows = toInternalRows(inputRows, ENCODER);
// generate some failures rows
for (int i = 0; i < totalFailures; i++) {
internalRows.add(getInternalRowWithError(partitionPath));
}
// generate 2nd batch of valid rows
Dataset<Row> inputRows2 = getRandomRows(sqlContext, size / 2, partitionPath, false);
internalRows.addAll(toInternalRows(inputRows2, ENCODER));
// issue writes
try {
for (InternalRow internalRow : internalRows) {
writer.write(internalRow);
}
fail("Should have failed");
} catch (Throwable e) {
// expected
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
List<String> fileAbsPaths = new ArrayList<>();
List<String> fileNames = new ArrayList<>();
// verify write statuses
assertWriteStatuses(commitMetadata.getWriteStatuses(), 1, size / 2, fileAbsPaths, fileNames);
// verify rows
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0]));
assertOutput(inputRows, result, instantTime, fileNames);
}
private void writeRows(Dataset<Row> inputRows, HoodieBulkInsertDataInternalWriter writer) throws IOException {
List<InternalRow> internalRows = toInternalRows(inputRows, ENCODER);
// issue writes
for (InternalRow internalRow : internalRows) {
writer.write(internalRow);
}
}
private void assertWriteStatuses(List<HoodieInternalWriteStatus> writeStatuses, int batches, int size, List<String> fileAbsPaths, List<String> fileNames) {
assertEquals(batches, writeStatuses.size());
int counter = 0;
for (HoodieInternalWriteStatus writeStatus : writeStatuses) {
// verify write status
assertEquals(writeStatus.getTotalRecords(), size);
assertNull(writeStatus.getGlobalError());
assertEquals(writeStatus.getFailedRowsSize(), 0);
assertNotNull(writeStatus.getFileId());
String fileId = writeStatus.getFileId();
assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3], writeStatus.getPartitionPath());
fileAbsPaths.add(basePath + "/" + writeStatus.getStat().getPath());
fileNames.add(writeStatus.getStat().getPath().substring(writeStatus.getStat().getPath().lastIndexOf('/') + 1));
HoodieWriteStat writeStat = writeStatus.getStat();
assertEquals(size, writeStat.getNumInserts());
assertEquals(size, writeStat.getNumWrites());
assertEquals(fileId, writeStat.getFileId());
assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter++ % 3], writeStat.getPartitionPath());
assertEquals(0, writeStat.getNumDeletes());
assertEquals(0, writeStat.getNumUpdateWrites());
assertEquals(0, writeStat.getTotalWriteErrors());
}
}
private void assertOutput(Dataset<Row> expectedRows, Dataset<Row> actualRows, String instantTime, List<String> fileNames) {
// verify 3 meta fields that are filled in within create handle
actualRows.collectAsList().forEach(entry -> {
assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime);
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD)));
assertTrue(fileNames.contains(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD))));
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)));
});
// after trimming 2 of the meta fields, rest of the fields should match
Dataset<Row> trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
Dataset<Row> trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
assertEquals(0, trimmedActual.except(trimmedExpected).count());
}
}

View File

@@ -0,0 +1,321 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.internal;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.HoodieClientTestUtils;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.sources.v2.writer.DataWriter;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Random;
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;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
/**
* Unit tests {@link HoodieDataSourceInternalWriter}.
*/
public class TestHoodieDataSourceInternalWriter extends HoodieClientTestHarness {
private static final Random RANDOM = new Random();
@BeforeEach
public void setUp() throws Exception {
initSparkContexts("TestHoodieDataSourceInternalWriter");
initPath();
initFileSystem();
initTestDataGenerator();
initMetaClient();
}
@AfterEach
public void tearDown() throws Exception {
cleanupResources();
}
@Test
public void testDataSourceWriter() throws IOException {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
String instantTime = "001";
// init writer
HoodieDataSourceInternalWriter dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong());
List<String> partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS);
List<String> partitionPathsAbs = new ArrayList<>();
for (String partitionPath : partitionPaths) {
partitionPathsAbs.add(basePath + "/" + partitionPath + "/*");
}
int size = 10 + RANDOM.nextInt(1000);
int batches = 5;
Dataset<Row> totalInputRows = null;
for (int j = 0; j < batches; j++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
writeRows(inputRows, writer);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
commitMessages.add(commitMetadata);
dataSourceInternalWriter.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
// verify output
assertOutput(totalInputRows, result, instantTime);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size);
}
@Test
public void testMultipleDataSourceWrites() throws IOException {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
int partitionCounter = 0;
// execute N rounds
for (int i = 0; i < 5; i++) {
String instantTime = "00" + i;
// init writer
HoodieDataSourceInternalWriter dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
Dataset<Row> totalInputRows = null;
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(partitionCounter++, RANDOM.nextLong(), RANDOM.nextLong());
int size = 10 + RANDOM.nextInt(1000);
int batches = 5; // one batch per partition
for (int j = 0; j < batches; j++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
writeRows(inputRows, writer);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
commitMessages.add(commitMetadata);
dataSourceInternalWriter.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime);
// verify output
assertOutput(totalInputRows, result, instantTime);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size);
}
}
@Test
public void testLargeWrites() throws IOException {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
int partitionCounter = 0;
// execute N rounds
for (int i = 0; i < 3; i++) {
String instantTime = "00" + i;
// init writer
HoodieDataSourceInternalWriter dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
Dataset<Row> totalInputRows = null;
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(partitionCounter++, RANDOM.nextLong(), RANDOM.nextLong());
int size = 10000 + RANDOM.nextInt(10000);
int batches = 3; // one batch per partition
for (int j = 0; j < batches; j++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
writeRows(inputRows, writer);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
commitMessages.add(commitMetadata);
dataSourceInternalWriter.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime);
// verify output
assertOutput(totalInputRows, result, instantTime);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size);
}
}
/**
* Tests that DataSourceWriter.abort() will abort the written records of interest write and commit batch1 write and abort batch2 Read of entire dataset should show only records from batch1.
* commit batch1
* abort batch2
* verify only records from batch1 is available to read
*/
@Test
public void testAbort() throws IOException {
// init config and table
HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
String instantTime0 = "00" + 0;
// init writer
HoodieDataSourceInternalWriter dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
DataWriter<InternalRow> writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong());
List<String> partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS);
List<String> partitionPathsAbs = new ArrayList<>();
for (String partitionPath : partitionPaths) {
partitionPathsAbs.add(basePath + "/" + partitionPath + "/*");
}
int size = 10 + RANDOM.nextInt(100);
int batches = 1;
Dataset<Row> totalInputRows = null;
for (int j = 0; j < batches; j++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
writeRows(inputRows, writer);
if (totalInputRows == null) {
totalInputRows = inputRows;
} else {
totalInputRows = totalInputRows.union(inputRows);
}
}
HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit();
List<HoodieWriterCommitMessage> commitMessages = new ArrayList<>();
commitMessages.add(commitMetadata);
// commit 1st batch
dataSourceInternalWriter.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
Dataset<Row> result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0]));
// verify rows
assertOutput(totalInputRows, result, instantTime0);
assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size);
// 2nd batch. abort in the end
String instantTime1 = "00" + 1;
dataSourceInternalWriter =
new HoodieDataSourceInternalWriter(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf);
writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(1, RANDOM.nextLong(), RANDOM.nextLong());
for (int j = 0; j < batches; j++) {
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3];
Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
writeRows(inputRows, writer);
}
commitMetadata = (HoodieWriterCommitMessage) writer.commit();
commitMessages = new ArrayList<>();
commitMessages.add(commitMetadata);
// commit 1st batch
dataSourceInternalWriter.abort(commitMessages.toArray(new HoodieWriterCommitMessage[0]));
metaClient.reloadActiveTimeline();
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);
}
private void writeRows(Dataset<Row> inputRows, DataWriter<InternalRow> writer) throws IOException {
List<InternalRow> internalRows = toInternalRows(inputRows, ENCODER);
// issue writes
for (InternalRow internalRow : internalRows) {
writer.write(internalRow);
}
}
private void assertWriteStatuses(List<HoodieInternalWriteStatus> writeStatuses, int batches, int size) {
assertEquals(batches, writeStatuses.size());
int counter = 0;
for (HoodieInternalWriteStatus writeStatus : writeStatuses) {
assertEquals(writeStatus.getPartitionPath(), HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3]);
assertEquals(writeStatus.getTotalRecords(), size);
assertEquals(writeStatus.getFailedRowsSize(), 0);
assertEquals(writeStatus.getTotalErrorRecords(), 0);
assertFalse(writeStatus.hasErrors());
assertNull(writeStatus.getGlobalError());
assertNotNull(writeStatus.getFileId());
String fileId = writeStatus.getFileId();
HoodieWriteStat writeStat = writeStatus.getStat();
assertEquals(size, writeStat.getNumInserts());
assertEquals(size, writeStat.getNumWrites());
assertEquals(fileId, writeStat.getFileId());
assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter++ % 3], writeStat.getPartitionPath());
assertEquals(0, writeStat.getNumDeletes());
assertEquals(0, writeStat.getNumUpdateWrites());
assertEquals(0, writeStat.getTotalWriteErrors());
}
}
private void assertOutput(Dataset<Row> expectedRows, Dataset<Row> actualRows, String instantTime) {
// verify 3 meta fields that are filled in within create handle
actualRows.collectAsList().forEach(entry -> {
assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime);
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD)));
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)));
});
// after trimming 2 of the meta fields, rest of the fields should match
Dataset<Row> trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
Dataset<Row> trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
assertEquals(0, trimmedActual.except(trimmedExpected).count());
}
}

View File

@@ -23,10 +23,13 @@ import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
import org.apache.spark.sql.Row;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
public class TestComplexKeyGenerator extends TestKeyGeneratorUtilities {
public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps(boolean getComplexRecordKey) {
TypedProperties properties = new TypedProperties();
@@ -76,13 +79,18 @@ public class TestComplexKeyGenerator extends TestKeyGeneratorUtilities {
public void testWrongRecordKeyField() {
ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(getWrongRecordKeyFieldProps());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testHappyFlow() {
ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(getProps());
HoodieKey key = keyGenerator.getKey(getRecord());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686/ts_ms=2020-03-21");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686/ts_ms=2020-03-21");
}
}

View File

@@ -22,10 +22,13 @@ import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
import org.apache.spark.sql.Row;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
public class TestCustomKeyGenerator extends TestKeyGeneratorUtilities {
public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps(boolean getComplexRecordKey) {
TypedProperties properties = new TypedProperties();
@@ -97,25 +100,37 @@ public class TestCustomKeyGenerator extends TestKeyGeneratorUtilities {
@Test
public void testSimpleKeyGenerator() {
KeyGenerator keyGenerator = new CustomKeyGenerator(getPropertiesForSimpleKeyGen());
HoodieKey key = keyGenerator.getKey(getRecord());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "key1");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686");
}
@Test
public void testTimestampBasedKeyGenerator() {
KeyGenerator keyGenerator = new CustomKeyGenerator(getPropertiesForTimestampBasedKeyGen());
HoodieKey key = keyGenerator.getKey(getRecord());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "key1");
Assertions.assertEquals(key.getPartitionPath(), "ts_ms=20200321");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "ts_ms=20200321");
}
@Test
public void testNonPartitionedKeyGenerator() {
KeyGenerator keyGenerator = new CustomKeyGenerator(getPropertiesForNonPartitionedKeyGen());
HoodieKey key = keyGenerator.getKey(getRecord());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "key1");
Assertions.assertTrue(key.getPartitionPath().isEmpty());
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertTrue(keyGenerator.getPartitionPath(row).isEmpty());
}
@Test
@@ -127,6 +142,16 @@ public class TestCustomKeyGenerator extends TestKeyGeneratorUtilities {
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("No enum constant org.apache.hudi.keygen.CustomKeyGenerator.PartitionKeyType.DUMMY"));
}
try {
KeyGenerator keyGenerator = new CustomKeyGenerator(getInvalidPartitionKeyTypeProps());
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
keyGenerator.getPartitionPath(row);
Assertions.fail("should fail when invalid PartitionKeyType is provided!");
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("No enum constant org.apache.hudi.keygen.CustomKeyGenerator.PartitionKeyType.DUMMY"));
}
}
@Test
@@ -138,6 +163,16 @@ public class TestCustomKeyGenerator extends TestKeyGeneratorUtilities {
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("Property hoodie.datasource.write.recordkey.field not found"));
}
try {
KeyGenerator keyGenerator = new CustomKeyGenerator(getPropsWithoutRecordKeyFieldProps());
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
keyGenerator.getRecordKey(row);
Assertions.fail("should fail when record key field is not provided!");
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("Property hoodie.datasource.write.recordkey.field not found"));
}
}
@Test
@@ -149,21 +184,41 @@ public class TestCustomKeyGenerator extends TestKeyGeneratorUtilities {
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("Unable to find field names for partition path in proper format"));
}
try {
KeyGenerator keyGenerator = new CustomKeyGenerator(getImproperPartitionFieldFormatProp());
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
keyGenerator.getPartitionPath(row);
Assertions.fail("should fail when partition key field is provided in improper format!");
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("Unable to find field names for partition path in proper format"));
}
}
@Test
public void testComplexRecordKeyWithSimplePartitionPath() {
KeyGenerator keyGenerator = new CustomKeyGenerator(getComplexRecordKeyWithSimplePartitionProps());
HoodieKey key = keyGenerator.getKey(getRecord());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686");
}
@Test
public void testComplexRecordKeysWithComplexPartitionPath() {
KeyGenerator keyGenerator = new CustomKeyGenerator(getComplexRecordKeyAndPartitionPathProps());
HoodieKey key = keyGenerator.getKey(getRecord());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686/ts_ms=20200321");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686/ts_ms=20200321");
}
}

View File

@@ -23,10 +23,13 @@ import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
import org.apache.spark.sql.Row;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
public class TestGlobalDeleteKeyGenerator extends TestKeyGeneratorUtilities {
public class TestGlobalDeleteKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps(boolean getComplexRecordKey) {
TypedProperties properties = new TypedProperties();
@@ -66,13 +69,19 @@ public class TestGlobalDeleteKeyGenerator extends TestKeyGeneratorUtilities {
public void testWrongRecordKeyField() {
GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getWrongRecordKeyFieldProps());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testHappyFlow() {
GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getProps());
HoodieKey key = keyGenerator.getKey(getRecord());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(key.getPartitionPath(), "");
keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType);
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "");
}
}

View File

@@ -23,10 +23,13 @@ import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
import org.apache.spark.sql.Row;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
public class TestSimpleKeyGenerator extends TestKeyGeneratorUtilities {
public class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps() {
TypedProperties properties = new TypedProperties();
@@ -52,6 +55,13 @@ public class TestSimpleKeyGenerator extends TestKeyGeneratorUtilities {
return properties;
}
private TypedProperties getWrongPartitionPathFieldProps() {
TypedProperties properties = new TypedProperties();
properties.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "_wrong_partition_path");
properties.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
return properties;
}
private TypedProperties getComplexRecordKeyProp() {
TypedProperties properties = new TypedProperties();
properties.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "timestamp");
@@ -79,19 +89,36 @@ public class TestSimpleKeyGenerator extends TestKeyGeneratorUtilities {
public void testWrongRecordKeyField() {
SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongRecordKeyFieldProps());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testWrongPartitionPathField() {
SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongPartitionPathFieldProps());
GenericRecord record = getRecord();
Assertions.assertEquals(keyGenerator.getPartitionPath(record), KeyGenUtils.DEFAULT_PARTITION_PATH);
Assertions.assertEquals(keyGenerator.getPartitionPath(KeyGeneratorTestUtilities.getRow(record)),
KeyGenUtils.DEFAULT_PARTITION_PATH);
}
@Test
public void testComplexRecordKeyField() {
SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getComplexRecordKeyProp());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testHappyFlow() {
SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getProps());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(getRecord());
Assertions.assertEquals(key.getRecordKey(), "key1");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686");
}
}

View File

@@ -18,31 +18,45 @@
package org.apache.hudi.keygen;
import org.apache.hudi.AvroConversionHelper;
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.testutils.SchemaTestUtil;
import org.apache.hudi.exception.HoodieDeltaStreamerException;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.exception.HoodieDeltaStreamerException;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
import org.apache.spark.sql.types.StructType;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import scala.Function1;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class TestTimestampBasedKeyGenerator {
private GenericRecord baseRecord;
private TypedProperties properties = new TypedProperties();
private Schema schema;
private StructType structType;
private Row baseRow;
@BeforeEach
public void initialize() throws IOException {
Schema schema = SchemaTestUtil.getTimestampEvolvedSchema();
schema = SchemaTestUtil.getTimestampEvolvedSchema();
structType = AvroConversionUtils.convertAvroSchemaToStructType(schema);
baseRecord = SchemaTestUtil
.generateAvroRecordFromJson(schema, 1, "001", "f1");
.generateAvroRecordFromJson(schema, 1, "001", "f1");
baseRow = genericRecordToRow(baseRecord);
properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "field1");
properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "createTime");
@@ -61,6 +75,17 @@ public class TestTimestampBasedKeyGenerator {
return properties;
}
private Row genericRecordToRow(GenericRecord baseRecord) {
Function1<Object, Object> convertor = AvroConversionHelper.createConverterToRow(schema, structType);
Row row = (Row) convertor.apply(baseRecord);
int fieldCount = structType.fieldNames().length;
Object[] values = new Object[fieldCount];
for (int i = 0; i < fieldCount; i++) {
values[i] = row.get(i);
}
return new GenericRowWithSchema(values, structType);
}
private TypedProperties getBaseKeyConfig(String timestampType, String inputFormatList, String inputFormatDelimiterRegex, String inputTimezone, String outputFormat, String outputTimezone) {
if (timestampType != null) {
properties.setProperty(TimestampBasedKeyGenerator.Config.TIMESTAMP_TYPE_FIELD_PROP, timestampType);
@@ -88,25 +113,43 @@ public class TestTimestampBasedKeyGenerator {
// timezone is GMT+8:00
baseRecord.put("createTime", 1578283932000L);
properties = getBaseKeyConfig("EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null);
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 12", hk1.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
// timezone is GMT
properties = getBaseKeyConfig("EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT", null);
HoodieKey hk2 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk2 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 04", hk2.getPartitionPath());
// test w/ Row
assertEquals("2020-01-06 04", keyGen.getPartitionPath(baseRow));
// timestamp is DATE_STRING, timezone is GMT+8:00
baseRecord.put("createTime", "2020-01-06 12:12:12");
properties = getBaseKeyConfig("DATE_STRING", "yyyy-MM-dd hh", "GMT+8:00", null);
properties.setProperty("hoodie.deltastreamer.keygen.timebased.input.dateformat", "yyyy-MM-dd hh:mm:ss");
HoodieKey hk3 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk3 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 12", hk3.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
// timezone is GMT
properties = getBaseKeyConfig("DATE_STRING", "yyyy-MM-dd hh", "GMT", null);
HoodieKey hk4 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk4 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 12", hk4.getPartitionPath());
// test w/ Row
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
}
@Test
@@ -116,140 +159,173 @@ public class TestTimestampBasedKeyGenerator {
// timezone is GMT
properties = getBaseKeyConfig("SCALAR", "yyyy-MM-dd hh", "GMT", "days");
HoodieKey hk5 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk5 = keyGen.getKey(baseRecord);
assertEquals(hk5.getPartitionPath(), "2024-10-04 12");
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("2024-10-04 12", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_SingleInputFormat_ISO8601WithMsZ_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.428Z");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"GMT");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"GMT");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_SingleInputFormats_ISO8601WithMsZ_OutputTimezoneAsInputDateTimeZone() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.428Z");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.428Z");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsZ_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33Z");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsWithOffset_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33-05:00");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040118", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040118", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsWithOffset_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.123-05:00");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040118", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040118", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsEST() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.123Z");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"EST");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"EST");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040109", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040109", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_Throws_MultipleInputFormats_InputDateNotMatchingFormats() {
public void test_Throws_MultipleInputFormats_InputDateNotMatchingFormats() throws IOException {
baseRecord.put("createTime", "2020-04-01 13:01:33.123-05:00");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
Assertions.assertThrows(HoodieDeltaStreamerException.class, () -> new TimestampBasedKeyGenerator(properties).getKey(baseRecord));
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
Assertions.assertThrows(HoodieDeltaStreamerException.class, () -> keyGen.getKey(baseRecord));
baseRow = genericRecordToRow(baseRecord);
Assertions.assertThrows(HoodieDeltaStreamerException.class, () -> keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ShortDate_OutputCustomDate() throws IOException {
baseRecord.put("createTime", "20200401");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ,yyyyMMdd",
"",
"UTC",
"MM/dd/yyyy",
"UTC");
HoodieKey hk1 = new TimestampBasedKeyGenerator(properties).getKey(baseRecord);
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ,yyyyMMdd",
"",
"UTC",
"MM/dd/yyyy",
"UTC");
KeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("04/01/2020", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("04/01/2020", keyGen.getPartitionPath(baseRow));
}
}

View File

@@ -0,0 +1,106 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.testutils;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.testutils.RawTripTestPayload;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.avro.Schema;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.RowFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Date;
import java.util.List;
import java.util.Random;
import java.util.UUID;
import java.util.stream.Collectors;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH;
/**
* Test utils for data source tests.
*/
public class DataSourceTestUtils {
public static Option<String> convertToString(HoodieRecord record) {
try {
String str = ((RawTripTestPayload) record.getData()).getJsonData();
str = "{" + str.substring(str.indexOf("\"timestamp\":"));
// Remove the last } bracket
str = str.substring(0, str.length() - 1);
return Option.of(str + ", \"partition\": \"" + record.getPartitionPath() + "\"}");
} catch (IOException e) {
return Option.empty();
}
}
public static List<String> convertToStringList(List<HoodieRecord> records) {
return records.stream().map(DataSourceTestUtils::convertToString).filter(Option::isPresent).map(Option::get)
.collect(Collectors.toList());
}
public static List<String> convertKeysToStringList(List<HoodieKey> keys) {
return keys.stream()
.map(hr -> "{\"_row_key\":\"" + hr.getRecordKey() + "\",\"partition\":\"" + hr.getPartitionPath() + "\"}")
.collect(Collectors.toList());
}
public static class NoOpBulkInsertPartitioner<T extends HoodieRecordPayload>
implements BulkInsertPartitioner<T> {
@Override
public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions) {
return records;
}
@Override
public boolean arePartitionRecordsSorted() {
return false;
}
}
public static Schema getStructTypeExampleSchema() throws IOException {
return new Schema.Parser().parse(FileIOUtils.readAsUTFString(DataSourceTestUtils.class.getResourceAsStream("/exampleSchema.txt")));
}
public static List<Row> generateRandomRows(int count) {
Random random = new Random();
List<Row> toReturn = new ArrayList<>();
List<String> partitions = Arrays.asList(new String[] {DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH, DEFAULT_THIRD_PARTITION_PATH});
for (int i = 0; i < count; i++) {
Object[] values = new Object[3];
values[0] = UUID.randomUUID().toString();
values[1] = partitions.get(random.nextInt(3));
values[2] = new Date().getTime();
toReturn.add(RowFactory.create(values));
}
return toReturn;
}
}

View File

@@ -16,19 +16,32 @@
* limitations under the License.
*/
package org.apache.hudi.keygen;
package org.apache.hudi.testutils;
import org.apache.hudi.AvroConversionHelper;
import org.apache.hudi.AvroConversionUtils;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
import org.apache.spark.sql.types.StructType;
public class TestKeyGeneratorUtilities {
import scala.Function1;
public String exampleSchema = "{\"type\": \"record\",\"name\": \"testrec\",\"fields\": [ "
public class KeyGeneratorTestUtilities {
public static String exampleSchema = "{\"type\": \"record\",\"name\": \"testrec\",\"fields\": [ "
+ "{\"name\": \"timestamp\",\"type\": \"long\"},{\"name\": \"_row_key\", \"type\": \"string\"},"
+ "{\"name\": \"ts_ms\", \"type\": \"string\"},"
+ "{\"name\": \"pii_col\", \"type\": \"string\"}]}";
public static final String TEST_STRUCTNAME = "test_struct_name";
public static final String TEST_RECORD_NAMESPACE = "test_record_namespace";
public static Schema schema = new Schema.Parser().parse(exampleSchema);
public static StructType structType = AvroConversionUtils.convertAvroSchemaToStructType(schema);
public GenericRecord getRecord() {
GenericRecord record = new GenericData.Record(new Schema.Parser().parse(exampleSchema));
record.put("timestamp", 4357686);
@@ -37,4 +50,19 @@ public class TestKeyGeneratorUtilities {
record.put("pii_col", "pi");
return record;
}
public static Row getRow(GenericRecord record) {
return getRow(record, schema, structType);
}
public static Row getRow(GenericRecord record, Schema schema, StructType structType) {
Function1<Object, Object> converterFn = AvroConversionHelper.createConverterToRow(schema, structType);
Row row = (Row) converterFn.apply(record);
int fieldCount = structType.fieldNames().length;
Object[] values = new Object[fieldCount];
for (int i = 0; i < fieldCount; i++) {
values[i] = row.get(i);
}
return new GenericRowWithSchema(values, structType);
}
}

View File

@@ -0,0 +1,36 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
{
"namespace": "example.schema",
"type": "record",
"name": "trip",
"fields": [
{
"name": "_row_key",
"type": "string"
},
{
"name": "partition",
"type": "string"
},
{
"name": "ts",
"type": ["long", "null"]
}
]
}

View File

@@ -17,13 +17,18 @@
package org.apache.hudi
import java.util
import org.apache.avro.generic.GenericRecord
import org.apache.hudi.avro.HoodieAvroUtils
import org.apache.hudi.common.config.TypedProperties
import org.apache.hudi.common.model.{EmptyHoodieRecordPayload, OverwriteWithLatestAvroPayload}
import org.apache.hudi.common.model.{EmptyHoodieRecordPayload, HoodieKey, OverwriteWithLatestAvroPayload}
import org.apache.hudi.common.testutils.SchemaTestUtil
import org.apache.hudi.common.util.Option
import org.apache.hudi.exception.{HoodieException, HoodieKeyException}
import org.apache.hudi.keygen.{ComplexKeyGenerator, GlobalDeleteKeyGenerator, SimpleKeyGenerator}
import org.apache.hudi.keygen._
import org.apache.hudi.testutils.KeyGeneratorTestUtilities
import org.apache.spark.sql.Row
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{BeforeEach, Test}
import org.scalatest.Assertions.fail
@@ -34,14 +39,18 @@ import org.scalatest.Assertions.fail
class TestDataSourceDefaults {
val schema = SchemaTestUtil.getComplexEvolvedSchema
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
var baseRecord: GenericRecord = _
var baseRow: Row = _
val testStructName = "testStructName"
val testNamespace = "testNamespace"
@BeforeEach def initialize(): Unit = {
baseRecord = SchemaTestUtil
.generateAvroRecordFromJson(schema, 1, "001", "f1")
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
}
private def getKeyConfig(recordKeyFieldName: String, partitionPathField: String, hiveStylePartitioning: String): TypedProperties = {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, recordKeyFieldName)
@@ -51,11 +60,16 @@ class TestDataSourceDefaults {
}
@Test def testSimpleKeyGenerator() = {
// top level, valid fields
val hk1 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false")).getKey(baseRecord)
var keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
val hk1 = keyGen.getKey(baseRecord)
assertEquals("field1", hk1.getRecordKey)
assertEquals("name1", hk1.getPartitionPath)
assertEquals("field1", keyGen.getRecordKey(baseRow))
assertEquals("name1", keyGen.getPartitionPath(baseRow))
// partition path field not specified
try {
val props = new TypedProperties()
@@ -64,7 +78,19 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
// do nothing
}
// partition path field not specified using Row
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
val keyGen = new SimpleKeyGenerator(props)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
}
// recordkey field not specified
@@ -75,7 +101,19 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
// do nothing
}
// recordkey field not specified using Row
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
val keyGen = new SimpleKeyGenerator(props)
keyGen.getPartitionPath(baseRow)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
}
// nested field as record key and partition path
@@ -91,7 +129,7 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: HoodieException =>
// do nothing
// do nothing
}
// if partition path can't be found, return default partition path
@@ -99,22 +137,44 @@ class TestDataSourceDefaults {
.getKey(baseRecord)
assertEquals("default", hk3.getPartitionPath)
// if partition path can't be found, return default partition path using row
keyGen = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
val hk3_row = keyGen.getPartitionPath(baseRow)
assertEquals("default", hk3_row)
// if enable hive style partitioning
val hk4 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "true")).getKey(baseRecord)
assertEquals("name=name1", hk4.getPartitionPath)
// if enable hive style partitioning using row
keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "true"))
val hk4_row = keyGen.getPartitionPath(baseRow)
assertEquals("name=name1", hk4_row)
// if partition is null, return default partition path
baseRecord.put("name", "")
val hk5 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
.getKey(baseRecord)
assertEquals("default", hk5.getPartitionPath)
// if partition is null, return default partition path using Row
keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
val hk5_row = keyGen.getPartitionPath(baseRow)
assertEquals("default", hk5_row)
// if partition is empty, return default partition path
baseRecord.put("name", null)
val hk6 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
.getKey(baseRecord)
assertEquals("default", hk6.getPartitionPath)
// if partition is empty, return default partition path using Row
keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
val hk6_row = keyGen.getPartitionPath(baseRow)
assertEquals("default", hk6_row)
// if record key is empty, throw error
try {
baseRecord.put("field1", "")
@@ -125,7 +185,21 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
// do nothing
}
// if record key is empty, throw error. Using Row
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
keyGen = new SimpleKeyGenerator(props)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
}
// if record key is null, throw error
@@ -138,16 +212,51 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
// do nothing
}
// if record key is null, throw error. Using Row
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
keyGen = new SimpleKeyGenerator(props)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
}
}
@Test def testUserDefinedKeyGeneratorWorksWithRows(): Unit = {
val keyGen = new UserDefinedKeyGenerator(getKeyConfig("field1", "name", "false"))
assertEquals("field1", keyGen.getRecordKey(baseRow))
assertEquals("name1", keyGen.getPartitionPath(baseRow))
}
class UserDefinedKeyGenerator(props: TypedProperties) extends KeyGenerator(props) {
val recordKeyProp: String = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY)
val partitionPathProp: String = props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY)
override def getKey(record: GenericRecord): HoodieKey = {
new HoodieKey(HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyProp, true),
HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathProp, true))
}
}
@Test def testComplexKeyGenerator() = {
// top level, valid fields
val hk1 = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")).getKey(baseRecord)
var keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
val hk1 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:name1", hk1.getRecordKey)
assertEquals("field1/name1", hk1.getPartitionPath)
// top level, valid fields with Row
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
assertEquals("field1/name1", keyGen.getPartitionPath(baseRow))
// partition path field not specified
try {
val props = new TypedProperties()
@@ -156,7 +265,19 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
// do nothing
}
// partition path field not specified using Row
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
val keyGen = new ComplexKeyGenerator(props)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
}
// recordkey field not specified
@@ -167,15 +288,31 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
// do nothing
}
// recordkey field not specified
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
val keyGen = new ComplexKeyGenerator(props)
keyGen.getPartitionPath(baseRow)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
}
// nested field as record key and partition path
val hk2 = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId,testNestedRecord.isAdmin", "testNestedRecord.userId,testNestedRecord.isAdmin", "false"))
.getKey(baseRecord)
keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId,testNestedRecord.isAdmin", "testNestedRecord.userId,testNestedRecord.isAdmin", "false"))
val hk2 = keyGen.getKey(baseRecord)
assertEquals("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", hk2.getRecordKey)
assertEquals("UserId1@001/false", hk2.getPartitionPath)
// nested field as record key and partition path
assertEquals("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", keyGen.getRecordKey(baseRow))
assertEquals("UserId1@001/false", keyGen.getPartitionPath(baseRow))
// Nested record key not found
try {
new ComplexKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
@@ -183,31 +320,57 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: HoodieException =>
// do nothing
// do nothing
}
// Nested record key not found
try {
val keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: HoodieException =>
// do nothing
}
// if partition path can't be found, return default partition path
val hk3 = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
.getKey(baseRecord)
keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
val hk3 = keyGen.getKey(baseRecord)
assertEquals("default", hk3.getPartitionPath)
assertEquals("default", keyGen.getPartitionPath(baseRow))
// if enable hive style partitioning
val hk4 = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "true")).getKey(baseRecord)
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "true"))
val hk4 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:name1", hk4.getRecordKey)
assertEquals("field1=field1/name=name1", hk4.getPartitionPath)
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
assertEquals("field1=field1/name=name1", keyGen.getPartitionPath(baseRow))
// if one part of the record key is empty, replace with "__empty__"
baseRecord.put("name", "")
val hk5 = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")).getKey(baseRecord)
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
val hk5 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:__empty__", hk5.getRecordKey)
assertEquals("field1/default", hk5.getPartitionPath)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
assertEquals("field1:field1,name:__empty__", keyGen.getRecordKey(baseRow))
assertEquals("field1/default", keyGen.getPartitionPath(baseRow))
// if one part of the record key is null, replace with "__null__"
baseRecord.put("name", null)
val hk6 = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")).getKey(baseRecord)
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
val hk6 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:__null__", hk6.getRecordKey)
assertEquals("field1/default", hk6.getPartitionPath)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow))
assertEquals("field1/default", keyGen.getPartitionPath(baseRow))
// if all parts of the composite record key are null/empty, throw error
try {
baseRecord.put("name", "")
@@ -219,46 +382,89 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
// do nothing
}
// if all parts of the composite record key are null/empty, throw error
try {
baseRecord.put("name", "")
baseRecord.put("field1", null)
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "field1,name")
keyGen = new ComplexKeyGenerator(props)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
}
// reset name and field1 values.
baseRecord.put("name", "name1")
baseRecord.put("field1", "field1")
val hk7 = new ComplexKeyGenerator(getKeyConfig("field1, name", "field1, name", "false")).getKey(baseRecord)
keyGen = new ComplexKeyGenerator(getKeyConfig("field1, name", "field1, name", "false"))
val hk7 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:name1", hk7.getRecordKey)
assertEquals("field1/name1", hk7.getPartitionPath)
val hk8 = new ComplexKeyGenerator(getKeyConfig("field1,", "field1,", "false")).getKey(baseRecord)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
assertEquals("field1/name1", keyGen.getPartitionPath(baseRow))
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,", "field1,", "false"))
val hk8 = keyGen.getKey(baseRecord)
assertEquals("field1:field1", hk8.getRecordKey)
assertEquals("field1", hk8.getPartitionPath)
assertEquals("field1:field1", keyGen.getRecordKey(baseRow))
assertEquals("field1", keyGen.getPartitionPath(baseRow))
}
@Test def testGlobalDeleteKeyGenerator() = {
// top level, partition value included but not actually used
val hk1 = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")).getKey(baseRecord)
var keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
val hk1 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:name1", hk1.getRecordKey)
assertEquals("", hk1.getPartitionPath)
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
assertEquals("", keyGen.getPartitionPath(baseRow))
// top level, partition value not included
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
val hk2 = new GlobalDeleteKeyGenerator(props).getKey(baseRecord)
keyGen = new GlobalDeleteKeyGenerator(props)
val hk2 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:name1", hk2.getRecordKey)
assertEquals("", hk2.getPartitionPath)
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
assertEquals("", keyGen.getPartitionPath(baseRow))
// if one part of the record key is empty, replace with "__empty__"
baseRecord.put("name", "")
val hk3 = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")).getKey(baseRecord)
keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
val hk3 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:__empty__", hk3.getRecordKey)
assertEquals("", hk3.getPartitionPath)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
assertEquals("field1:field1,name:__empty__", keyGen.getRecordKey(baseRow))
assertEquals("", keyGen.getPartitionPath(baseRow))
// if one part of the record key is null, replace with "__null__"
baseRecord.put("name", null)
val hk4 = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false")).getKey(baseRecord)
keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
val hk4 = keyGen.getKey(baseRecord)
assertEquals("field1:field1,name:__null__", hk4.getRecordKey)
assertEquals("", hk4.getPartitionPath)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow))
assertEquals("", keyGen.getPartitionPath(baseRow))
// recordkey field not specified
try {
val props = new TypedProperties()
@@ -267,7 +473,19 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
// do nothing
}
// recordkey field not specified
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
val keyGen = new GlobalDeleteKeyGenerator(props)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
}
// Nested record key not found
@@ -277,7 +495,17 @@ class TestDataSourceDefaults {
fail("Should have errored out")
} catch {
case e: HoodieException =>
// do nothing
// do nothing
}
// Nested record key not found
try {
val keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: HoodieException =>
// do nothing
}
// if all parts of the composite record key are null/empty, throw error
@@ -292,6 +520,21 @@ class TestDataSourceDefaults {
case e: HoodieKeyException =>
// do nothing
}
// if all parts of the composite record key are null/empty, throw error
try {
baseRecord.put("name", "")
baseRecord.put("field1", null)
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
val keyGen = new GlobalDeleteKeyGenerator(props)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: HoodieKeyException =>
// do nothing
}
}
@Test def testOverwriteWithLatestAvroPayload() = {

View File

@@ -17,24 +17,28 @@
package org.apache.hudi.functional
import java.util
import java.util.{Date, UUID}
import org.apache.commons.io.FileUtils
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.HoodieSparkSqlWriter
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.exception.HoodieException
import org.apache.spark.sql.{SaveMode, SparkSession}
import org.apache.hudi.testutils.DataSourceTestUtils
import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWriterUtils}
import org.apache.spark.sql.{Row, SaveMode, SparkSession}
import org.scalatest.{FunSuite, Matchers}
class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
test("Parameters With Write Defaults") {
val originals = HoodieSparkSqlWriter.parametersWithWriteDefaults(Map.empty)
val originals = HoodieWriterUtils.parametersWithWriteDefaults(Map.empty)
val rhsKey = "hoodie.right.hand.side.key"
val rhsVal = "hoodie.right.hand.side.val"
val modifier = Map(OPERATION_OPT_KEY -> INSERT_OPERATION_OPT_VAL, TABLE_TYPE_OPT_KEY -> MOR_TABLE_TYPE_OPT_VAL, rhsKey -> rhsVal)
val modified = HoodieSparkSqlWriter.parametersWithWriteDefaults(modifier)
val modified = HoodieWriterUtils.parametersWithWriteDefaults(modifier)
val matcher = (k: String, v: String) => modified(k) should be(v)
originals foreach {
@@ -77,7 +81,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
"hoodie.insert.shuffle.parallelism" -> "4",
"hoodie.upsert.shuffle.parallelism" -> "4")
val fooTableParams = HoodieSparkSqlWriter.parametersWithWriteDefaults(fooTableModifier)
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
val dataFrame = session.createDataFrame(Seq(Test(UUID.randomUUID().toString, new Date().getTime)))
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, dataFrame)
@@ -86,7 +90,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
HoodieWriteConfig.TABLE_NAME -> "hoodie_bar_tbl",
"hoodie.insert.shuffle.parallelism" -> "4",
"hoodie.upsert.shuffle.parallelism" -> "4")
val barTableParams = HoodieSparkSqlWriter.parametersWithWriteDefaults(barTableModifier)
val barTableParams = HoodieWriterUtils.parametersWithWriteDefaults(barTableModifier)
val dataFrame2 = session.createDataFrame(Seq(Test(UUID.randomUUID().toString, new Date().getTime)))
val tableAlreadyExistException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, barTableParams, dataFrame2))
assert(tableAlreadyExistException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist"))
@@ -101,6 +105,129 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
}
}
test("test bulk insert dataset with datasource impl") {
val session = SparkSession.builder()
.appName("test_bulk_insert_datasource")
.master("local[2]")
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.getOrCreate()
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
try {
val sqlContext = session.sqlContext
val sc = session.sparkContext
val hoodieFooTableName = "hoodie_foo_tbl"
//create a new table
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY -> "true",
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> "org.apache.hudi.keygen.SimpleKeyGenerator")
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
// generate the inserts
val schema = DataSourceTestUtils.getStructTypeExampleSchema
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
val records = DataSourceTestUtils.generateRandomRows(100)
val recordsSeq = convertRowListToSeq(records)
val df = session.createDataFrame(sc.parallelize(recordsSeq), structType)
// write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
// collect all parition paths to issue read of parquet files
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
// Check the entire dataset has all records still
val fullPartitionPaths = new Array[String](3)
for (i <- 0 until fullPartitionPaths.length) {
fullPartitionPaths(i) = String.format("%s/%s/*", path.toAbsolutePath.toString, partitions(i))
}
// fetch all records from parquet files generated from write to hudi
val actualDf = session.sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
// remove metadata columns so that expected and actual DFs can be compared as is
val trimmedDf = actualDf.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(2)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(3))
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(4))
assert(df.except(trimmedDf).count() == 0)
} finally {
session.stop()
FileUtils.deleteDirectory(path.toFile)
}
}
test("test bulk insert dataset with datasource impl multiple rounds") {
val session = SparkSession.builder()
.appName("test_bulk_insert_datasource")
.master("local[2]")
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.getOrCreate()
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
try {
val sqlContext = session.sqlContext
val sc = session.sparkContext
val hoodieFooTableName = "hoodie_foo_tbl"
//create a new table
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> "org.apache.hudi.keygen.SimpleKeyGenerator")
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
val fullPartitionPaths = new Array[String](3)
for (i <- 0 to 2) {
fullPartitionPaths(i) = String.format("%s/%s/*", path.toAbsolutePath.toString, partitions(i))
}
val schema = DataSourceTestUtils.getStructTypeExampleSchema
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
var totalExpectedDf = session.createDataFrame(sc.emptyRDD[Row], structType)
for (_ <- 0 to 2) {
// generate the inserts
val records = DataSourceTestUtils.generateRandomRows(200)
val recordsSeq = convertRowListToSeq(records)
val df = session.createDataFrame(sc.parallelize(recordsSeq), structType)
// write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
// Fetch records from entire dataset
val actualDf = session.sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
// remove metadata columns so that expected and actual DFs can be compared as is
val trimmedDf = actualDf.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(2)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(3))
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(4))
// find total df (union from multiple rounds)
totalExpectedDf = totalExpectedDf.union(df)
// find mismatch between actual and expected df
assert(totalExpectedDf.except(trimmedDf).count() == 0)
}
} finally {
session.stop()
FileUtils.deleteDirectory(path.toFile)
}
}
case class Test(uuid: String, ts: Long)
import scala.collection.JavaConverters
def convertRowListToSeq(inputList: util.List[Row]): Seq[Row] =
JavaConverters.asScalaIteratorConverter(inputList.iterator).asScala.toSeq
}