Hi Anton,

   - Yes, it is our intention to compile it down to a MERGE.
   - The detailed design will follow once the work is started; the purpose
   of this SPIP is to form alignment on this being a favorable improvement to
   Spark.
   - To support AutoCDC, changelogs are not required (producing a change
   log is the topic of an earlier SPIP). Keep in mind that the change feed
   originates from a different data source than the target of the Auto CDC.
   - What we will require for the target is only to support MERGE.

Cheers -Andreas

On Mon, Mar 30, 2026 at 8:49 AM Anton Okolnychyi <[email protected]>
wrote:

> Will auto CDC compile into or use MERGE under the hood? If yes, can we
> include a sketch of how that rewrite will look like in the SPIP? What
> exactly do connectors need to support to benefit from auto CDC? Just
> support changelogs and MERGE?
>
> - Anton
>
> пн, 30 бер. 2026 р. о 07:44 Andreas Neumann <[email protected]> пише:
>
>> Hi Vaquar,
>>
>> I responded to most of your comments on the document itself.
>> Addtional comments inline
>>
>> Cheers -Andreas
>>
>> On Sat, Mar 28, 2026 at 4:42 PM vaquar khan <[email protected]>
>> wrote:
>>
>>> HI .
>>> Thanks for the SPIP. I fully support the goal-abstracting CDC merge
>>> logic is a huge win for the community. However, looking at the current
>>> Spark versions, there are significant architectural gaps between Databricks
>>> Lakeflow's proprietary implementation and OSS Spark.
>>>
>>> A few technical blockers need clarification before we move forward:
>>>
>>> - OSS Compatibility: Databricks documentation explicitly states that the
>>> AUTO CDC APIs are not supported by Apache Spark Declarative Pipelines
>>> <https://docs.databricks.com/gcp/en/ldp/cdc>.
>>>
>> That will change with the implementation of this SPIP.
>>
>>
>>> - Streaming MERGE: The proposed flow requires continuous upsert/delete
>>> semantics, but Dataset.mergeInto() currently does not support streaming
>>> queries. Does this SPIP introduce an entirely new execution path to bypass
>>> this restriction
>>>
>> This works with foreachBatch.
>>
>>
>>> - Tombstone Garbage Collection: Handling stream deletes safely requires
>>> state store tombstone retention (e.g., configuring
>>> pipelines.cdc.tombstoneGCThresholdInSeconds) to prevent late-arriving data
>>> from resurrecting deleted keys. How will this be implemented natively in
>>> OSS Spark state stores?
>>>
>> That's an interesting question. Tombstones could be modeled in the state
>> store, but we are thinking that they will be modeled as an explicit output
>> of the flow, either as records in the output table with a "deleted at"
>> marker, possibly with a view on top to project away these rows; or in a
>> separate output that contains only the tombstones. The exact design is not
>> finalized, that is part of the first phase of the project.
>>
>> - Sequencing Constraints: SEQUENCE BY enforces strict ordering where NULL
>>> sequencing values are explicitly not supported. How will the engine handle
>>> malformed or non-monotonic upstream sequences compared to our existing
>>> time-based watermarks?
>>>
>> I think malformed change events should, at least in the first iteration,
>> fail the stream. Otherwise there is a risk of writing incorrect data.
>>
>>
>>>
>>> - Given the massive surface area (new SQL DDL, streaming MERGE paths,
>>> SCD Type 1/2 state logic, tombstone GC), a phased delivery plan would be
>>> very helpful. It would also clarify exactly which Lakeflow components are
>>> being contributed to open-source versus what needs to be rebuilt from
>>> scratch.
>>>
>>>
>>> Best regards,
>>> Viquar Khan
>>>
>>> On Sat, 28 Mar 2026 at 08:35, 陈 小健 <[email protected]> wrote:
>>>
>>>> unsubscribe
>>>>
>>>> 获取Outlook for Android <https://aka.ms/AAb9ysg>
>>>> ------------------------------
>>>> *From:* Andreas Neumann <[email protected]>
>>>> *Sent:* Saturday, March 28, 2026 2:43:54 AM
>>>> *To:* [email protected] <[email protected]>
>>>> *Subject:* Re: SPIP: Auto CDC support for Apache Spark
>>>>
>>>> Hi Vaibhav,
>>>>
>>>> The goal of this proposal is not to replace MERGE but to provide a
>>>> simple abstraction for the common use case of CDC.
>>>> MERGE itself is a very powerful operator and there will always be use
>>>> cases outside of CDC that will require MERGE.
>>>>
>>>> And thanks for spotting the typo in the SPIP. It is fixed now!
>>>>
>>>> Cheers -Andreas
>>>>
>>>>
>>>> On Fri, Mar 27, 2026 at 10:53 AM Vaibhav Kumar <[email protected]>
>>>> wrote:
>>>>
>>>> Hi Andrew,
>>>>
>>>> Thanks for sharing the SPIP, Does that mean the MERGE statement would
>>>> be deprecated? Also I think there was a small typo I have suggested in the
>>>> doc.
>>>>
>>>> Regards,
>>>> Vaibhav
>>>>
>>>> On Fri, Mar 27, 2026 at 10:15 AM DB Tsai <[email protected]> wrote:
>>>>
>>>> +1
>>>>
>>>> DB Tsai  |  https://www.dbtsai.com/  |  PGP 42E5B25A8F7A82C1
>>>>
>>>> On Mar 26, 2026, at 6:08 PM, Andreas Neumann <[email protected]> wrote:
>>>>
>>>> Hi all,
>>>>
>>>> I’d like to start a discussion on a new SPIP to introduce Auto CDC
>>>> support to Apache Spark.
>>>>
>>>>    - SPIP Document:
>>>>    
>>>> https://docs.google.com/document/d/1Hp5BGEYJRHbk6J7XUph3bAPZKRQXKOuV1PEaqZMMRoQ/
>>>>    -
>>>>
>>>>    JIRA: <https://issues.apache.org/jira/browse/SPARK-55668>
>>>>    https://issues.apache.org/jira/browse/SPARK-5566
>>>>
>>>> Motivation
>>>>
>>>> With the upcoming introduction of standardized CDC support
>>>> <https://issues.apache.org/jira/browse/SPARK-55668>, Spark will soon
>>>> have a unified way to produce change data feeds. However, consuming
>>>> these feeds and applying them to a target table remains a significant
>>>> challenge.
>>>>
>>>> Common patterns like SCD Type 1 (maintaining a 1:1 replica) and SCD
>>>> Type 2 (tracking full change history) often require hand-crafted,
>>>> complex MERGE logic. In distributed systems, these implementations are
>>>> frequently error-prone when handling deletions or out-of-order data.
>>>> Proposal
>>>>
>>>> This SPIP proposes a new "Auto CDC" flow type for Spark. It
>>>> encapsulates the complex logic for SCD types and out-of-order data,
>>>> allowing data engineers to configure a declarative flow instead of writing
>>>> manual MERGE statements. This feature will be available in both Python
>>>> and SQL.
>>>> Example SQL:
>>>> -- Produce a change feed
>>>> CREATE STREAMING TABLE cdc.users AS
>>>> SELECT * FROM STREAM my_table CHANGES FROM VERSION 10;
>>>>
>>>> -- Consume the change feed
>>>> CREATE FLOW flow
>>>> AS AUTO CDC INTO
>>>>   target
>>>> FROM stream(cdc_data.users)
>>>>   KEYS (userId)
>>>>   APPLY AS DELETE WHEN operation = "DELETE"
>>>>   SEQUENCE BY sequenceNum
>>>>   COLUMNS * EXCEPT (operation, sequenceNum)
>>>>   STORED AS SCD TYPE 2
>>>>   TRACK HISTORY ON * EXCEPT (city);
>>>>
>>>>
>>>> Please review the full SPIP for the technical details. Looking forward
>>>> to your feedback and discussion!
>>>>
>>>> Best regards,
>>>>
>>>> Andreas
>>>>
>>>>
>>>>

Reply via email to