1
0

[HUDI-3731] Fixing Column Stats Index record Merging sequence missing columnName (#5159)

* Added `DataSkippingFailureMode` to control how DS handles failures in the flow (either "strict", when exception would be thrown, or "fallback" when it will just fallback to the full-scan)

* Make sure tests execute in `DataSkippingFailureMode.Strict`

* Fixed Column Stats Index record merging sequence missing `columnName`
This commit is contained in:
Alexey Kudinkin
2022-03-29 08:39:56 -07:00
committed by GitHub
parent 1b2fb71afc
commit fcb003ec76
5 changed files with 58 additions and 21 deletions

View File

@@ -20,6 +20,7 @@ package org.apache.hudi
import org.apache.hadoop.conf.Configuration
import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_SNAPSHOT_OPT_VAL}
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.HoodieFileIndex.DataSkippingFailureMode
import org.apache.hudi.client.HoodieJavaWriteClient
import org.apache.hudi.client.common.HoodieJavaEngineContext
import org.apache.hudi.common.config.HoodieMetadataConfig
@@ -354,6 +355,9 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
HoodieTableConfig.POPULATE_META_FIELDS.key -> "true"
)
// If there are any failures in the Data Skipping flow, test should fail
spark.sqlContext.setConf(DataSkippingFailureMode.configName, DataSkippingFailureMode.Strict.value);
inputDF.repartition(4)
.write
.format("hudi")

View File

@@ -18,13 +18,14 @@
package org.apache.hudi.functional
import org.apache.hudi.HoodieFileIndex.DataSkippingFailureMode
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig}
import org.apache.hudi.testutils.HoodieClientTestBase
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions}
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieFileIndex}
import org.apache.spark.sql._
import org.apache.spark.sql.types._
import org.junit.jupiter.api.Assertions.assertEquals
@@ -92,6 +93,9 @@ class TestLayoutOptimization extends HoodieClientTestBase {
val records = recordsToStrings(dataGen.generateInserts("001", targetRecordsCount)).toList
val writeDf: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records, 2))
// If there are any failures in the Data Skipping flow, test should fail
spark.sqlContext.setConf(DataSkippingFailureMode.configName, DataSkippingFailureMode.Strict.value);
writeDf.write.format("org.apache.hudi")
.options(commonOpts)
.option("hoodie.compact.inline", "false")