[HUDI-3936] Fix projection for a nested field as pre-combined key (#5379)
This PR fixes the projection logic around a nested field which is used as the pre-combined key field. The fix is to only check and append the root level field for projection, i.e., "a", for a nested field "a.b.c" in the mandatory columns. - Changes the logic to check and append the root level field for a required nested field in the mandatory columns in HoodieBaseRelation.appendMandatoryColumns
This commit is contained in:
@@ -23,6 +23,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.common.util.StringUtils
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.testutils.SparkClientFunctionalTestHarness
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
@@ -32,7 +33,7 @@ import org.apache.spark.sql.functions.{col, lit}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.Tag
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
import org.junit.jupiter.params.provider.CsvSource
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
@@ -57,19 +58,28 @@ class TestMORDataSourceStorage extends SparkClientFunctionalTestHarness {
|
||||
val updatedVerificationVal: String = "driver_update"
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = Array(true, false))
|
||||
def testMergeOnReadStorage(isMetadataEnabled: Boolean) {
|
||||
val dataGen = new HoodieTestDataGenerator()
|
||||
@CsvSource(Array(
|
||||
"true,",
|
||||
"true,fare.currency",
|
||||
"false,",
|
||||
"false,fare.currency"
|
||||
))
|
||||
def testMergeOnReadStorage(isMetadataEnabled: Boolean, preComineField: String) {
|
||||
var options: Map[String, String] = commonOpts +
|
||||
(HoodieMetadataConfig.ENABLE.key -> String.valueOf(isMetadataEnabled))
|
||||
if (!StringUtils.isNullOrEmpty(preComineField)) {
|
||||
options += (DataSourceWriteOptions.PRECOMBINE_FIELD.key() -> preComineField)
|
||||
}
|
||||
val dataGen = new HoodieTestDataGenerator(0xDEEF)
|
||||
val fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration)
|
||||
// Bulk Insert Operation
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).toList
|
||||
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.options(options)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -90,8 +100,7 @@ class TestMORDataSourceStorage extends SparkClientFunctionalTestHarness {
|
||||
val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 100)).toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.options(options)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -110,8 +119,7 @@ class TestMORDataSourceStorage extends SparkClientFunctionalTestHarness {
|
||||
val inputDF3 = hudiSnapshotDF2.filter(col("_row_key") === verificationRowKey).withColumn(verificationCol, lit(updatedVerificationVal))
|
||||
|
||||
inputDF3.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.options(options)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
|
||||
@@ -19,7 +19,7 @@ package org.apache.hudi.functional
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig
|
||||
import org.apache.hudi.common.model.{HoodieRecord, OverwriteNonDefaultsWithLatestAvroPayload, OverwriteWithLatestAvroPayload}
|
||||
import org.apache.hudi.common.model.{HoodieRecord, OverwriteNonDefaultsWithLatestAvroPayload}
|
||||
import org.apache.hudi.common.table.HoodieTableConfig
|
||||
import org.apache.hudi.common.testutils.{HadoopMapRedUtils, HoodieTestDataGenerator}
|
||||
import org.apache.hudi.config.{HoodieStorageConfig, HoodieWriteConfig}
|
||||
@@ -332,7 +332,7 @@ class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with
|
||||
logWarning(s"Not matching bytes read ($bytesRead)")
|
||||
}
|
||||
|
||||
val readColumns = targetColumns ++ relation.mandatoryColumns
|
||||
val readColumns = targetColumns ++ relation.mandatoryFields
|
||||
val (_, projectedStructType, _) = HoodieSparkUtils.getRequiredSchema(tableState.schema, readColumns)
|
||||
|
||||
val row: InternalRow = rows.take(1).head
|
||||
|
||||
Reference in New Issue
Block a user