Skip to content

[HUDI-6579] Adding support for upsert and deletes with spark datasource for pk less table#9261

Merged
nsivabalan merged 7 commits intoapache:masterfrom
nsivabalan:enableUpsertsDeletesPkless
Aug 6, 2023
Merged

[HUDI-6579] Adding support for upsert and deletes with spark datasource for pk less table#9261
nsivabalan merged 7 commits intoapache:masterfrom
nsivabalan:enableUpsertsDeletesPkless

Conversation

@nsivabalan
Copy link
Contributor

Change Logs

Adding support for upsert and deletes with spark datasource for pk less table.

Impact

This patch opens up possibility to do updates or delete records with spark datasource writes for a primary key less table.

For eg, if for a primary table, user prefers to delete all records for a given employee.

val df = spark.read.format("hudi").load(basePath).filter("emp_id == '100')
df.write.format("hudi").option("hoodie.datasource.write.operation","delete").mode(Append).save(basePath) 

This is feasible for a pk less table.

Risk level (write none, low medium or high below)

low.

Documentation Update

Describe any necessary documentation update if there is any new feature, config, or user-facing change

  • The config description must be updated if new configs are added or the default value of the configs are changed
  • Any new feature or user-facing change requires updating the Hudi website. Please create a Jira ticket, attach the
    ticket number here and follow the instruction to make
    changes to the website.

Contributor's checklist

  • Read through contributor's guide
  • Change Logs and Impact were stated clearly
  • Adequate tests were added if applicable
  • CI passed

@nsivabalan nsivabalan added release-0.14.0 priority:blocker Production down; release blocker labels Jul 21, 2023
@apache apache deleted a comment from hudi-bot Jul 23, 2023
private def canRemoveMetaFields(optParams: Map[String, String]) : Boolean = {
!(optParams.getOrDefault(SPARK_SQL_WRITES_PREPPED_KEY, "false").toBoolean
|| optParams.getOrDefault(SPARK_SQL_MERGE_INTO_PREPPED_KEY, "false").toBoolean
|| !optParams.containsKey(RECORDKEY_FIELD.key()))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For pk-less tables, RECORDKEY_FIELD may not be present and this condition can be true. Do we want to drop the meta fields in such cases? From the comment it seems like for pk-less we don;t want to drop the meta fields.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

not sure I get it. we will drop the meta fields down the line while creating the HoodieRecordPayload.
here we are just trying to gauge if we can go w/ prepped writes or regular non-prepped writes. if incoming df has meta fields and if its pk less table, we go with prepped flow.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My point was for pk-less this condition would be true and canRemoveMetaFields can return true, isn't it? However, we need the meta fields for pk-less.

!optParams.containsKey(RECORDKEY_FIELD.key()

@nsivabalan
Copy link
Contributor Author

@codope : addressed all comments.

Copy link
Member

@codope codope left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you please look into the CI failures?

private def canRemoveMetaFields(optParams: Map[String, String]) : Boolean = {
!(optParams.getOrDefault(SPARK_SQL_WRITES_PREPPED_KEY, "false").toBoolean
|| optParams.getOrDefault(SPARK_SQL_MERGE_INTO_PREPPED_KEY, "false").toBoolean
|| !optParams.containsKey(RECORDKEY_FIELD.key()))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My point was for pk-less this condition would be true and canRemoveMetaFields can return true, isn't it? However, we need the meta fields for pk-less.

!optParams.containsKey(RECORDKEY_FIELD.key()

@nsivabalan
Copy link
Contributor Author

oops, my bad. not sure how I missed. will fix it

@nsivabalan
Copy link
Contributor Author

@codope : Updated

Copy link
Member

@codope codope left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will land once the CI succeeds.

@nsivabalan nsivabalan force-pushed the enableUpsertsDeletesPkless branch 3 times, most recently from 7d85ec6 to 7e7efc7 Compare August 3, 2023 22:28
@codope codope self-assigned this Aug 4, 2023
@nsivabalan nsivabalan force-pushed the enableUpsertsDeletesPkless branch from 7e7efc7 to 3968bf3 Compare August 4, 2023 15:34
@hudi-bot
Copy link
Collaborator

hudi-bot commented Aug 5, 2023

CI report:

Bot commands @hudi-bot supports the following commands:
  • @hudi-bot run azure re-run the last Azure build

@nsivabalan nsivabalan merged commit 7061652 into apache:master Aug 6, 2023
val df = if (preppedWriteOperation || preppedSparkSqlWrites || preppedSparkSqlMergeInto) {
sourceDf
} else {
sourceDf.drop(HoodieRecord.HOODIE_META_COLUMNS: _*)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

dropping meta cols here caused a problem with HoodieStreamingSink: when val sourceDF = spark.readStream.format(hudi).load; sourceDF.writeStream.format(hudi).start(), the source DF is a streaming source and dropping metacols failed the assertion of "Queries with streaming sources must be executed with writeStream.start()" because internally Hudi is writing DF as a batch

we either keep df here the same as sourceDF when using sourceDF.writeStream.format(hudi).start() or use sourceDF.writeStream.foreachBatch{}

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

priority:blocker Production down; release blocker release-0.14.0

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants