-
Notifications
You must be signed in to change notification settings - Fork 2.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[HUDI-8758] Enforce insert deduplicate policy for spark SQL #12588
base: master
Are you sure you want to change the base?
Changes from all commits
400caf6
9c14654
4a99cb7
d928b49
bf6e175
092c792
46dc4d7
845d6c3
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -38,6 +38,7 @@ | |
import org.apache.hudi.config.HoodieCompactionConfig; | ||
import org.apache.hudi.config.HoodiePayloadConfig; | ||
import org.apache.hudi.config.HoodieWriteConfig; | ||
import org.apache.hudi.exception.HoodieDuplicateKeyException; | ||
import org.apache.hudi.exception.HoodieException; | ||
import org.apache.hudi.exception.TableNotFoundException; | ||
import org.apache.hudi.storage.HoodieStorage; | ||
|
@@ -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, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. again, lets rename this There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can we add java docs please |
||
JavaRDD<HoodieRecord> incomingHoodieRecords, | ||
Map<String, String> parameters, | ||
boolean failOnDuplicates) { | ||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() | ||
.withPath(parameters.get("path")) | ||
.withProps(parameters).build(); | ||
return doDropDuplicates( | ||
new HoodieSparkEngineContext(jssc), incomingHoodieRecords, writeConfig, failOnDuplicates); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -29,6 +29,7 @@ import org.apache.hudi.DataSourceOptionsHelper.fetchMissingWriteConfigsFromTable | |
import org.apache.hudi.DataSourceWriteOptions._ | ||
import org.apache.hudi.HoodieConversionUtils.{toProperties, toScalaOption} | ||
import org.apache.hudi.HoodieSparkSqlWriter.StreamingWriteParams | ||
import org.apache.hudi.HoodieSparkSqlWriterInternal.{handleInsertDuplicates, shouldDropDuplicatesForInserts, shouldFailWhenDuplicatesFound} | ||
import org.apache.hudi.HoodieSparkUtils.sparkAdapter | ||
import org.apache.hudi.HoodieWriterUtils._ | ||
import org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema | ||
|
@@ -45,7 +46,7 @@ import org.apache.hudi.common.model._ | |
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType | ||
import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator | ||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} | ||
import org.apache.hudi.common.util.ConfigUtils.getAllConfigKeys | ||
import org.apache.hudi.common.util.ConfigUtils.{getAllConfigKeys, getStringWithAltKeys} | ||
import org.apache.hudi.common.util.{CommitUtils, StringUtils, Option => HOption} | ||
import org.apache.hudi.config.HoodieBootstrapConfig.{BASE_PATH, INDEX_CLASS_NAME} | ||
import org.apache.hudi.config.HoodieWriteConfig.{SPARK_SQL_MERGE_INTO_PREPPED_KEY, WRITE_TABLE_VERSION} | ||
|
@@ -66,8 +67,7 @@ import org.apache.hudi.sync.common.HoodieSyncConfig | |
import org.apache.hudi.sync.common.util.SyncUtilHelpers | ||
import org.apache.hudi.sync.common.util.SyncUtilHelpers.getHoodieMetaSyncException | ||
import org.apache.hudi.util.SparkKeyGenUtils | ||
|
||
import org.apache.spark.api.java.JavaSparkContext | ||
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} | ||
import org.apache.spark.sql.HoodieDataTypeUtils.tryOverrideParquetWriteLegacyFormatProperty | ||
import org.apache.spark.sql._ | ||
import org.apache.spark.sql.catalyst.TableIdentifier | ||
|
@@ -79,7 +79,6 @@ import org.apache.spark.{SPARK_VERSION, SparkContext} | |
import org.slf4j.LoggerFactory | ||
|
||
import java.util.function.BiConsumer | ||
|
||
import scala.collection.JavaConverters._ | ||
import scala.collection.mutable | ||
import scala.util.{Failure, Success, Try} | ||
|
@@ -506,12 +505,8 @@ class HoodieSparkSqlWriterInternal { | |
case Failure(e) => throw new HoodieRecordCreationException("Failed to create Hoodie Spark Record", e) | ||
} | ||
|
||
val dedupedHoodieRecords = | ||
if (hoodieConfig.getBoolean(INSERT_DROP_DUPS) && operation != WriteOperationType.INSERT_OVERWRITE_TABLE && operation != WriteOperationType.INSERT_OVERWRITE) { | ||
DataSourceUtils.dropDuplicates(jsc, hoodieRecords, parameters.asJava) | ||
} else { | ||
hoodieRecords | ||
} | ||
// Remove duplicates from incoming records based on existing keys from storage. | ||
val dedupedHoodieRecords = handleInsertDuplicates(hoodieRecords, hoodieConfig, operation, jsc, parameters) | ||
client.startCommitWithTime(instantTime, commitActionType) | ||
try { | ||
val writeResult = DataSourceUtils.doWriteOperation(client, dedupedHoodieRecords, instantTime, operation, | ||
|
@@ -554,13 +549,16 @@ class HoodieSparkSqlWriterInternal { | |
var operation = WriteOperationType.fromValue(hoodieConfig.getString(OPERATION)) | ||
// TODO clean up | ||
// It does not make sense to allow upsert() operation if INSERT_DROP_DUPS is true | ||
// or INSERT_DUP_POLICY is `drop` or `fail`. | ||
// Auto-correct the operation to "insert" if OPERATION is set to "upsert" wrongly | ||
// or not set (in which case it will be set as "upsert" by parametersWithWriteDefaults()) . | ||
if (hoodieConfig.getBoolean(INSERT_DROP_DUPS) && | ||
if ((hoodieConfig.getBoolean(INSERT_DROP_DUPS) || | ||
shouldFailWhenDuplicatesFound(hoodieConfig) || | ||
shouldDropDuplicatesForInserts(hoodieConfig)) && | ||
operation == WriteOperationType.UPSERT) { | ||
|
||
log.warn(s"$UPSERT_OPERATION_OPT_VAL is not applicable " + | ||
s"when $INSERT_DROP_DUPS is set to be true, " + | ||
s"when $INSERT_DROP_DUPS is set to be true, or $INSERT_DUP_POLICY is set to fail or drop, " + | ||
s"overriding the $OPERATION to be $INSERT_OPERATION_OPT_VAL") | ||
|
||
operation = WriteOperationType.INSERT | ||
|
@@ -1165,3 +1163,45 @@ 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 = { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. from where did you pull this from? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I take it from the master, LINE 510. I modified it to narrow down the impact to only insert. Previously it could impact upsert also. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. gotcha. |
||
operation == WriteOperationType.INSERT || | ||
operation == WriteOperationType.INSERT_PREPPED | ||
} | ||
|
||
def handleInsertDuplicates(incomingRecords: JavaRDD[HoodieRecord[_]], | ||
hoodieConfig: HoodieConfig, | ||
operation: WriteOperationType, | ||
jsc: JavaSparkContext, | ||
parameters: Map[String, String]): JavaRDD[HoodieRecord[_]] = { | ||
// If no deduplication is needed, return the incoming records as is | ||
if (!isDeduplicationRequired(hoodieConfig) || !isDeduplicationNeeded(operation)) { | ||
incomingRecords | ||
} else { | ||
// Perform deduplication | ||
DataSourceUtils.dropDuplicates( | ||
jsc, incomingRecords, parameters.asJava, shouldFailWhenDuplicatesFound(hoodieConfig)) | ||
} | ||
} | ||
} |
This file was deleted.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we should rename this to handleDuplicates