nsivabalan commented on code in PR #12588:
URL: https://github.com/apache/hudi/pull/12588#discussion_r1911034131
##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java:
##########
@@ -284,24 +285,47 @@ public static HoodieRecord
createHoodieRecord(GenericRecord gr, HoodieKey hKey,
* @param writeConfig HoodieWriteConfig
*/
@SuppressWarnings("unchecked")
- public static JavaRDD<HoodieRecord> dropDuplicates(HoodieSparkEngineContext
engineContext, JavaRDD<HoodieRecord> incomingHoodieRecords,
- HoodieWriteConfig writeConfig) {
+ public static JavaRDD<HoodieRecord>
doDropDuplicates(HoodieSparkEngineContext engineContext,
+ JavaRDD<HoodieRecord>
incomingHoodieRecords,
+ HoodieWriteConfig
writeConfig,
+ boolean
failOnDuplicates) {
try {
SparkRDDReadClient client = new SparkRDDReadClient<>(engineContext,
writeConfig);
return client.tagLocation(incomingHoodieRecords)
- .filter(r -> !((HoodieRecord<HoodieRecordPayload>)
r).isCurrentLocationKnown());
+ .filter(r -> shouldIncludeRecord((HoodieRecord<HoodieRecordPayload>)
r, failOnDuplicates));
} catch (TableNotFoundException e) {
- // this will be executed when there is no hoodie table yet
- // so no dups to drop
+ // No table exists yet, so no duplicates to drop
return incomingHoodieRecords;
}
}
+ /**
+ * Determines if a record should be included in the result after
deduplication.
+ *
+ * @param record The Hoodie record to evaluate.
+ * @param failOnDuplicates Whether to fail on detecting duplicates.
+ * @return true if the record should be included; false otherwise.
+ */
+ private static boolean shouldIncludeRecord(HoodieRecord<?> record, boolean
failOnDuplicates) {
+ if (!record.isCurrentLocationKnown()) {
+ return true;
+ }
+ if (failOnDuplicates) {
+ // Fail if duplicates are found and the flag is set
+ throw new HoodieDuplicateKeyException(record.getRecordKey());
+ }
+ return false;
+ }
+
@SuppressWarnings("unchecked")
- public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc,
JavaRDD<HoodieRecord> incomingHoodieRecords,
- Map<String, String> parameters) {
- HoodieWriteConfig writeConfig =
-
HoodieWriteConfig.newBuilder().withPath(parameters.get("path")).withProps(parameters).build();
- return dropDuplicates(new HoodieSparkEngineContext(jssc),
incomingHoodieRecords, writeConfig);
+ public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc,
Review Comment:
again, lets rename this
##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java:
##########
@@ -284,24 +285,47 @@ public static HoodieRecord
createHoodieRecord(GenericRecord gr, HoodieKey hKey,
* @param writeConfig HoodieWriteConfig
*/
@SuppressWarnings("unchecked")
- public static JavaRDD<HoodieRecord> dropDuplicates(HoodieSparkEngineContext
engineContext, JavaRDD<HoodieRecord> incomingHoodieRecords,
- HoodieWriteConfig writeConfig) {
+ public static JavaRDD<HoodieRecord>
doDropDuplicates(HoodieSparkEngineContext engineContext,
Review Comment:
we should rename this to handleDuplicates
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/TestInsertTable.scala:
##########
@@ -3087,4 +3089,69 @@ class TestInsertTable extends HoodieSparkSqlTestBase {
})
}
}
+
+ test("Test table with insert dup policy") {
+ withTempDir { tmp =>
+ Seq(
+ NONE_INSERT_DUP_POLICY,
+ FAIL_INSERT_DUP_POLICY,
+ DROP_INSERT_DUP_POLICY).foreach(policy => {
+ val targetTable = generateTableName
+ val tablePath = s"${tmp.getCanonicalPath}/$targetTable"
+
+ spark.sql(s"set hoodie.datasource.insert.dup.policy=$policy")
+ spark.sql(
+ s"""
+ |create table ${targetTable} (
+ | `id` string,
+ | `name` string,
+ | `dt` bigint,
+ | `day` STRING,
+ | `hour` INT
+ |) using hudi
+ |tblproperties (
+ | 'primaryKey' = 'id',
+ | 'type' = 'MOR',
+ | 'preCombineField'= 'dt',
+ | 'hoodie.index.type' = 'BUCKET',
+ | 'hoodie.bucket.index.hash.field' = 'id',
+ | 'hoodie.bucket.index.num.buckets'= 512
+ | )
+ partitioned by (`day`,`hour`)
+ location '${tablePath}'
+ """.stripMargin)
+
+ spark.sql("set spark.sql.shuffle.partitions = 11")
+ spark.sql(
+ s"""
+ |insert into ${targetTable}
+ |select '1' as id, 'aa' as name, 123 as dt, '2024-02-19' as
`day`, 10 as `hour`
+ |""".stripMargin)
+ if (policy.equals(FAIL_INSERT_DUP_POLICY)) {
+ checkExceptionContain(
+ () => spark.sql(
+ s"""
+ |insert into ${targetTable}
+ |select '1' as id, 'aa' as name, 1234 as dt, '2024-02-19' as
`day`, 10 as `hour`
+ |""".stripMargin))(s"Duplicate key found for insert statement,
key is: 1")
+ } else {
+ spark.sql(
+ s"""
+ |insert into ${targetTable}
+ |select '1' as id, 'aa' as name, 1234 as dt, '2024-02-19' as
`day`, 10 as `hour`
+ |""".stripMargin)
+ if (policy.equals(NONE_INSERT_DUP_POLICY)) {
+ checkAnswer(s"select id, name, dt, day, hour from $targetTable
limit 10")(
+ Seq("1", "aa", 1234, "2024-02-19", 10)
Review Comment:
same comment as above. we should see duplicates.
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestInsertDedupPolicy.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.DataSourceWriteOptions._
+import org.apache.hudi.common.config.{HoodieReaderConfig, HoodieStorageConfig}
+import org.apache.hudi.config.{HoodieCompactionConfig, HoodieWriteConfig}
+import org.apache.hudi.testutils.SparkClientFunctionalTestHarness
+import org.apache.spark.SparkException
+import org.apache.spark.sql.{Dataset, Row, SaveMode}
+import org.junit.jupiter.api.Assertions
+import org.junit.jupiter.api.Assertions.assertTrue
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.{Arguments, MethodSource}
+
+class TestInsertDedupPolicy extends SparkClientFunctionalTestHarness {
+ val firstInsertData: Seq[(Int, String, String, String, Double, Int)] = Seq(
+ (10, "1", "rider-A", "driver-A", 19.10, 7),
+ (10, "2", "rider-B", "driver-B", 27.70, 1),
+ (10, "3", "rider-C", "driver-C", 33.90, 10))
+ val secondInsertData: Seq[(Int, String, String, String, Double, Int)] = Seq(
+ (11, "1", "rider-A", "driver-A", 1.1, 1),
+ (11, "2", "rider-B", "driver-B", 2.2, 2),
+ (11, "5", "rider-C", "driver-C", 3.3, 3))
+ val expectedForDrop: Seq[(Int, String, String, String, Double, Int)] = Seq(
+ (10, "1", "rider-A", "driver-A", 19.10, 7),
+ (10, "2", "rider-B", "driver-B", 27.70, 1),
+ (10, "3", "rider-C", "driver-C", 33.90, 10),
+ (11, "5", "rider-C", "driver-C", 3.3, 3))
+ val expectedForNone: Seq[(Int, String, String, String, Double, Int)] = Seq(
+ (11, "1", "rider-A", "driver-A", 1.1, 1),
+ (11, "2", "rider-B", "driver-B", 2.2, 2),
+ (10, "3", "rider-C", "driver-C", 33.90, 10),
+ (11, "5", "rider-C", "driver-C", 3.3, 3))
+
+ @ParameterizedTest
+ @MethodSource(Array("provideParams"))
+ def testInsertLogic(tableType: String,
+ recordType: String,
+ dedupPolicy: String): Unit = {
+ val sparkOpts: Map[String, String] = Map(
+ HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> "parquet",
+ HoodieWriteConfig.RECORD_MERGE_IMPL_CLASSES.key ->
classOf[DefaultSparkRecordMerger].getName)
+ val fgReaderOpts: Map[String, String] = Map(
+ HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key -> "true",
+ HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS.key -> "true",
+ HoodieWriteConfig.RECORD_MERGE_MODE.key -> "EVENT_TIME_ORDERING")
+ val insertDedupPolicy: Map[String, String] = Map(INSERT_DUP_POLICY.key ->
dedupPolicy)
+ val opts = if (recordType.equals("SPARK")) {
+ sparkOpts ++ fgReaderOpts ++ insertDedupPolicy
+ } else {
+ fgReaderOpts ++ insertDedupPolicy
+ }
+ val columns = Seq("ts", "key", "rider", "driver", "fare", "number")
+
+ // Write the first batch of data.
+ val inserts = spark.createDataFrame(firstInsertData).toDF(columns: _*)
+ inserts.write.format("hudi").
+ option(RECORDKEY_FIELD.key, "key").
+ option(PRECOMBINE_FIELD.key, "ts").
+ option(TABLE_TYPE.key, tableType).
+ option(DataSourceWriteOptions.TABLE_NAME.key, "test_table").
+ option(HoodieCompactionConfig.INLINE_COMPACT.key, "false").
+ options(opts).
+ mode(SaveMode.Overwrite).
+ save(basePath)
+
+ val insertsWithDup = spark.createDataFrame(secondInsertData).toDF(columns:
_*)
+ if (dedupPolicy.equals(FAIL_INSERT_DUP_POLICY)) {
+ // Write and check throws.
+ Assertions.assertThrows(
+ classOf[SparkException],
+ () =>
insertsWithDup.write.format("hudi").options(opts).mode(SaveMode.Append).save(basePath)
+ )
+ } else {
+ // Write data.
+
insertsWithDup.write.format("hudi").options(opts).mode(SaveMode.Append).save(basePath)
+ // Validate the data.
+ val df = spark.read.format("hudi").options(opts).load(basePath)
+ val finalDf = df.select("ts", "key", "rider", "driver", "fare",
"number").sort("key")
+ val expected = if (dedupPolicy.equals(DROP_INSERT_DUP_POLICY))
expectedForDrop else expectedForNone
+ val expectedDf = spark.createDataFrame(expected).toDF(columns:
_*).sort("key")
+ TestInsertDedupPolicy.validate(expectedDf, finalDf)
+ }
+ }
+}
+
+object TestInsertDedupPolicy {
+ def provideParams(): java.util.List[Arguments] = {
+ java.util.Arrays.asList(
+ Arguments.of("MERGE_ON_READ", "AVRO", NONE_INSERT_DUP_POLICY),
+ Arguments.of("MERGE_ON_READ", "SPARK", NONE_INSERT_DUP_POLICY),
+ Arguments.of("MERGE_ON_READ", "AVRO", DROP_INSERT_DUP_POLICY),
Review Comment:
we can avoid multiple record types. just table type and dup policy combos
would do.
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestInsertDedupPolicy.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.DataSourceWriteOptions._
+import org.apache.hudi.common.config.{HoodieReaderConfig, HoodieStorageConfig}
+import org.apache.hudi.config.{HoodieCompactionConfig, HoodieWriteConfig}
+import org.apache.hudi.testutils.SparkClientFunctionalTestHarness
+import org.apache.spark.SparkException
+import org.apache.spark.sql.{Dataset, Row, SaveMode}
+import org.junit.jupiter.api.Assertions
+import org.junit.jupiter.api.Assertions.assertTrue
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.{Arguments, MethodSource}
+
+class TestInsertDedupPolicy extends SparkClientFunctionalTestHarness {
+ val firstInsertData: Seq[(Int, String, String, String, Double, Int)] = Seq(
+ (10, "1", "rider-A", "driver-A", 19.10, 7),
+ (10, "2", "rider-B", "driver-B", 27.70, 1),
+ (10, "3", "rider-C", "driver-C", 33.90, 10))
+ val secondInsertData: Seq[(Int, String, String, String, Double, Int)] = Seq(
+ (11, "1", "rider-A", "driver-A", 1.1, 1),
+ (11, "2", "rider-B", "driver-B", 2.2, 2),
+ (11, "5", "rider-C", "driver-C", 3.3, 3))
+ val expectedForDrop: Seq[(Int, String, String, String, Double, Int)] = Seq(
+ (10, "1", "rider-A", "driver-A", 19.10, 7),
+ (10, "2", "rider-B", "driver-B", 27.70, 1),
+ (10, "3", "rider-C", "driver-C", 33.90, 10),
+ (11, "5", "rider-C", "driver-C", 3.3, 3))
+ val expectedForNone: Seq[(Int, String, String, String, Double, Int)] = Seq(
+ (11, "1", "rider-A", "driver-A", 1.1, 1),
Review Comment:
shouldn't we expect to see duplicate records with NONE as policy value ?
##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java:
##########
@@ -284,24 +285,47 @@ public static HoodieRecord
createHoodieRecord(GenericRecord gr, HoodieKey hKey,
* @param writeConfig HoodieWriteConfig
*/
@SuppressWarnings("unchecked")
- public static JavaRDD<HoodieRecord> dropDuplicates(HoodieSparkEngineContext
engineContext, JavaRDD<HoodieRecord> incomingHoodieRecords,
- HoodieWriteConfig writeConfig) {
+ public static JavaRDD<HoodieRecord>
doDropDuplicates(HoodieSparkEngineContext engineContext,
+ JavaRDD<HoodieRecord>
incomingHoodieRecords,
+ HoodieWriteConfig
writeConfig,
+ boolean
failOnDuplicates) {
try {
SparkRDDReadClient client = new SparkRDDReadClient<>(engineContext,
writeConfig);
return client.tagLocation(incomingHoodieRecords)
- .filter(r -> !((HoodieRecord<HoodieRecordPayload>)
r).isCurrentLocationKnown());
+ .filter(r -> shouldIncludeRecord((HoodieRecord<HoodieRecordPayload>)
r, failOnDuplicates));
} catch (TableNotFoundException e) {
- // this will be executed when there is no hoodie table yet
- // so no dups to drop
+ // No table exists yet, so no duplicates to drop
return incomingHoodieRecords;
}
}
+ /**
+ * Determines if a record should be included in the result after
deduplication.
+ *
+ * @param record The Hoodie record to evaluate.
+ * @param failOnDuplicates Whether to fail on detecting duplicates.
+ * @return true if the record should be included; false otherwise.
+ */
+ private static boolean shouldIncludeRecord(HoodieRecord<?> record, boolean
failOnDuplicates) {
+ if (!record.isCurrentLocationKnown()) {
+ return true;
+ }
+ if (failOnDuplicates) {
+ // Fail if duplicates are found and the flag is set
+ throw new HoodieDuplicateKeyException(record.getRecordKey());
+ }
+ return false;
+ }
+
@SuppressWarnings("unchecked")
- public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc,
JavaRDD<HoodieRecord> incomingHoodieRecords,
- Map<String, String> parameters) {
- HoodieWriteConfig writeConfig =
-
HoodieWriteConfig.newBuilder().withPath(parameters.get("path")).withProps(parameters).build();
- return dropDuplicates(new HoodieSparkEngineContext(jssc),
incomingHoodieRecords, writeConfig);
+ public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc,
Review Comment:
can we add java docs please
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestInsertDedupPolicy.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.DataSourceWriteOptions._
+import org.apache.hudi.common.config.{HoodieReaderConfig, HoodieStorageConfig}
+import org.apache.hudi.config.{HoodieCompactionConfig, HoodieWriteConfig}
+import org.apache.hudi.testutils.SparkClientFunctionalTestHarness
+import org.apache.spark.SparkException
+import org.apache.spark.sql.{Dataset, Row, SaveMode}
+import org.junit.jupiter.api.Assertions
+import org.junit.jupiter.api.Assertions.assertTrue
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.{Arguments, MethodSource}
+
+class TestInsertDedupPolicy extends SparkClientFunctionalTestHarness {
+ val firstInsertData: Seq[(Int, String, String, String, Double, Int)] = Seq(
+ (10, "1", "rider-A", "driver-A", 19.10, 7),
+ (10, "2", "rider-B", "driver-B", 27.70, 1),
+ (10, "3", "rider-C", "driver-C", 33.90, 10))
+ val secondInsertData: Seq[(Int, String, String, String, Double, Int)] = Seq(
+ (11, "1", "rider-A", "driver-A", 1.1, 1),
+ (11, "2", "rider-B", "driver-B", 2.2, 2),
+ (11, "5", "rider-C", "driver-C", 3.3, 3))
+ val expectedForDrop: Seq[(Int, String, String, String, Double, Int)] = Seq(
+ (10, "1", "rider-A", "driver-A", 19.10, 7),
+ (10, "2", "rider-B", "driver-B", 27.70, 1),
+ (10, "3", "rider-C", "driver-C", 33.90, 10),
+ (11, "5", "rider-C", "driver-C", 3.3, 3))
+ val expectedForNone: Seq[(Int, String, String, String, Double, Int)] = Seq(
+ (11, "1", "rider-A", "driver-A", 1.1, 1),
Review Comment:
concat handle should be enabled by default right
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1165,3 +1163,54 @@ class HoodieSparkSqlWriterInternal {
.map(_.toLong).filter(SQLExecution.getQueryExecution(_) eq
newQueryExecution)
}
}
+
+object HoodieSparkSqlWriterInternal {
+ // Check if duplicates should be dropped.
+ def shouldDropDuplicatesForInserts(hoodieConfig: HoodieConfig): Boolean = {
+ hoodieConfig.contains(INSERT_DUP_POLICY) &&
+
hoodieConfig.getString(INSERT_DUP_POLICY).equalsIgnoreCase(DROP_INSERT_DUP_POLICY)
+ }
+
+ // Check if we should fail if duplicates are found.
+ def shouldFailWhenDuplicatesFound(hoodieConfig: HoodieConfig): Boolean = {
+ hoodieConfig.contains(INSERT_DUP_POLICY) &&
+
hoodieConfig.getString(INSERT_DUP_POLICY).equalsIgnoreCase(FAIL_INSERT_DUP_POLICY)
+ }
+
+ // Check if deduplication is required.
+ def isDeduplicationRequired(hoodieConfig: HoodieConfig): Boolean = {
+ hoodieConfig.getBoolean(INSERT_DROP_DUPS) ||
+ shouldFailWhenDuplicatesFound(hoodieConfig) ||
+ shouldDropDuplicatesForInserts(hoodieConfig)
+ }
+
+ // Check if deduplication is needed.
+ def isDeduplicationNeeded(operation: WriteOperationType): Boolean = {
Review Comment:
gotcha.
btw, this could apply only for INSERT and not INSERT_PREPPED
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]