[HUDI-2161] Adding support to disable meta columns with bulk insert operation (#3247)
This commit is contained in:
committed by
GitHub
parent
2099bf41db
commit
d5026e9a24
@@ -113,6 +113,33 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
||||
assertTrue(dataset.except(trimmedOutput).count() == 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBulkInsertHelperNoMetaFields() {
|
||||
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
|
||||
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
|
||||
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsertWithoutMetaFields(dataset);
|
||||
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());
|
||||
}
|
||||
|
||||
result.toJavaRDD().foreach(entry -> {
|
||||
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.RECORD_KEY_METADATA_FIELD)).equals(""));
|
||||
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).equals(""));
|
||||
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)).equals(""));
|
||||
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).equals(""));
|
||||
assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.FILENAME_METADATA_FIELD)).equals(""));
|
||||
});
|
||||
|
||||
Dataset<Row> trimmedOutput = result.drop(HoodieRecord.PARTITION_PATH_METADATA_FIELD).drop(HoodieRecord.RECORD_KEY_METADATA_FIELD)
|
||||
.drop(HoodieRecord.FILENAME_METADATA_FIELD).drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD).drop(HoodieRecord.COMMIT_TIME_METADATA_FIELD);
|
||||
assertTrue(dataset.except(trimmedOutput).count() == 0);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("providePreCombineArgs")
|
||||
public void testBulkInsertPreCombine(boolean enablePreCombine) {
|
||||
|
||||
@@ -26,6 +26,7 @@ import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.client.{SparkRDDWriteClient, TestBootstrap}
|
||||
import org.apache.hudi.common.config.HoodieConfig
|
||||
import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord, HoodieRecordPayload}
|
||||
import org.apache.hudi.common.table.HoodieTableConfig
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
@@ -119,61 +120,13 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
}
|
||||
}
|
||||
|
||||
List(BulkInsertSortMode.GLOBAL_SORT.name(), BulkInsertSortMode.NONE.name(), BulkInsertSortMode.PARTITION_SORT.name())
|
||||
List(BulkInsertSortMode.GLOBAL_SORT, BulkInsertSortMode.NONE, BulkInsertSortMode.PARTITION_SORT)
|
||||
.foreach(sortMode => {
|
||||
test("test_bulk_insert_for_" + sortMode) {
|
||||
initSparkContext("test_bulk_insert_datasource")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
try {
|
||||
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL,
|
||||
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY.key -> "true",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
val inserts = DataSourceTestUtils.generateRandomRows(1000)
|
||||
|
||||
// add some updates so that preCombine kicks in
|
||||
val toUpdateDataset = sqlContext.createDataFrame(DataSourceTestUtils.getUniqueRows(inserts, 40), structType)
|
||||
val updates = DataSourceTestUtils.updateRowsWithHigherTs(toUpdateDataset)
|
||||
val records = inserts.union(updates)
|
||||
|
||||
val recordsSeq = convertRowListToSeq(records)
|
||||
val df = spark.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 = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
|
||||
val resultRows = actualDf.collectAsList()
|
||||
|
||||
// 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)
|
||||
testBulkInsertWithSortMode(sortMode, path)
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
@@ -181,6 +134,118 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
}
|
||||
})
|
||||
|
||||
List(true, false)
|
||||
.foreach(populateMetaFields => {
|
||||
test("test_bulk_insert_for_populate_meta_fields_" + populateMetaFields) {
|
||||
initSparkContext("test_bulk_insert_datasource_populate_meta_fields")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path_populate_meta_fields")
|
||||
try {
|
||||
testBulkInsertWithSortMode(BulkInsertSortMode.NONE, path, populateMetaFields)
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
}
|
||||
}
|
||||
})
|
||||
|
||||
def testBulkInsertWithSortMode(sortMode: BulkInsertSortMode, path: java.nio.file.Path, populateMetaFields : Boolean = true) : Unit = {
|
||||
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL,
|
||||
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY.key -> "true",
|
||||
HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key() -> String.valueOf(populateMetaFields),
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
|
||||
HoodieWriteConfig.BULKINSERT_SORT_MODE.key() -> sortMode.name(),
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
val inserts = DataSourceTestUtils.generateRandomRows(1000)
|
||||
|
||||
// add some updates so that preCombine kicks in
|
||||
val toUpdateDataset = sqlContext.createDataFrame(DataSourceTestUtils.getUniqueRows(inserts, 40), structType)
|
||||
val updates = DataSourceTestUtils.updateRowsWithHigherTs(toUpdateDataset)
|
||||
val records = inserts.union(updates)
|
||||
|
||||
val recordsSeq = convertRowListToSeq(records)
|
||||
val df = spark.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 = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
|
||||
|
||||
if (!populateMetaFields) {
|
||||
assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(0)).filter(entry => !(entry.mkString(",").equals(""))).count())
|
||||
assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(1)).filter(entry => !(entry.mkString(",").equals(""))).count())
|
||||
assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(2)).filter(entry => !(entry.mkString(",").equals(""))).count())
|
||||
assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(3)).filter(entry => !(entry.mkString(",").equals(""))).count())
|
||||
assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(4)).filter(entry => !(entry.mkString(",").equals(""))).count())
|
||||
}
|
||||
// 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)
|
||||
}
|
||||
|
||||
test("test disable and enable meta fields") {
|
||||
initSparkContext("test_disable_enable_meta_fields")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
try {
|
||||
testBulkInsertWithSortMode(BulkInsertSortMode.NONE, path, false)
|
||||
|
||||
// enabling meta fields back should throw exception
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL,
|
||||
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY.key -> "true",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
|
||||
HoodieWriteConfig.BULKINSERT_SORT_MODE.key() -> BulkInsertSortMode.NONE.name(),
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
val inserts = DataSourceTestUtils.generateRandomRows(1000)
|
||||
val df = spark.createDataFrame(sc.parallelize(inserts), structType)
|
||||
try {
|
||||
// write to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
|
||||
fail("Should have thrown exception")
|
||||
} catch {
|
||||
case e: HoodieException => assertTrue(e.getMessage.contains("hoodie.populate.meta.fields already disabled for the table. Can't be re-enabled back"))
|
||||
}
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
}
|
||||
}
|
||||
|
||||
test("test drop duplicates row writing for bulk_insert") {
|
||||
initSparkContext("test_append_mode")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
|
||||
Reference in New Issue
Block a user