[SPARK-55952][SPARK-55953][SQL] Add ResolveChangelogTable analyzer rule for batch CDC post-processing#55508
Conversation
…CDC post-processing
Post-process a resolved `DataSourceV2Relation(ChangelogTable)` to inject
carry-over removal and/or update detection, fused into a single pass over a
`(rowId, _commit_version)`-partitioned Window. Also reject streaming CDC
reads that would require post-processing, to prevent silent wrong results.
- `ResolveChangelogTable` analyzer rule:
- Batch: carry-over removal is a `Filter` on the Window (drops CoW pairs
where `min(rowVersion) == max(rowVersion)`); update detection is a
`CASE WHEN` over delete/insert counts (relabels pairs as
`update_preimage` / `update_postimage`). Fused into a single Window.
- Streaming: throws `INVALID_CDC_OPTION.STREAMING_POST_PROCESSING_NOT_SUPPORTED`
when the requested options would need post-processing. Streams that
don't need post-processing pass through unchanged.
- Net changes: throws `INVALID_CDC_OPTION.NET_CHANGES_NOT_YET_SUPPORTED`
for both batch and streaming.
- Option validation: throws
`INVALID_CDC_OPTION.UPDATE_DETECTION_REQUIRES_CARRY_OVER_REMOVAL` when
`computeUpdates = true` is combined with a carry-over-surfacing connector
and `deduplicationMode = none`, which would silently misclassify
carry-overs as updates.
- Runtime guard: the generated plan raises
`INVALID_CDC_OPTION.UNEXPECTED_MULTIPLE_CHANGES_PER_ROW_VERSION` when the
connector emits more than one delete or insert for the same
`(rowId, _commit_version)` partition.
- `Analyzer`: register the rule after `ResolveRelations`.
- `InMemoryChangelogCatalog`: `ChangelogProperties` extension so tests can
configure post-processing scenarios.
Actual streaming post-processing implementation and net change computation
are scoped to follow-up PRs.
c2092af to
6e73cc5
Compare
gengliangwang
left a comment
There was a problem hiding this comment.
Summary
Prior state. Before this PR, CHANGES FROM VERSION ... WITH (deduplicationMode = ..., computeUpdates = ...) parsed the options but silently ignored them — the connector's raw change rows leaked through unchanged. On connectors that advertise containsCarryoverRows = true or representsUpdateAsDeleteAndInsert = true, this produces wrong results (carry-overs surface; updates stay as raw delete+insert).
Design approach. A new post-resolution analyzer rule ResolveChangelogTable fires after ResolveRelations on any DataSourceV2Relation(ChangelogTable, ...) / StreamingRelationV2(ChangelogTable, ...). For batch reads, it fuses both post-processing passes into a single (rowId, _commit_version)-partitioned Window (aggregate counts of inserts/deletes + min/max row version), followed by a Filter that drops CoW carry-overs (delCnt=1 ∧ insCnt=1 ∧ minRv=maxRv) and a CaseWhen project that relabels delete/insert pairs as update_preimage/update_postimage (and raises a runtime error on contract violations delCnt>1 ∨ insCnt>1). Streaming reads that would need post-processing are rejected up front via INVALID_CDC_OPTION.STREAMING_POST_PROCESSING_NOT_SUPPORTED to prevent silent wrong results; streams with no required post-processing pass through.
Key decisions made by this PR.
- Fuse the two passes into one Window — avoids a second shuffle; the extra
min_rv/max_rvaggregates make carry-over detection essentially free once update detection is needed. - Raise at runtime for connector contract violations (
>1 delete or insert per (rowId, _commit_version)) viaRaiseError, since the violation is only observable over data. - Reject streaming up front rather than silently fall through when post-processing would be needed.
Implementation sketch. One new rule (ResolveChangelogTable.scala) wired into Analyzer right after ResolveRelations; new sub-conditions under INVALID_CDC_OPTION; a test-harness extension (ChangelogProperties, InMemoryChangelogCatalog) that lets tests flip capability flags without a real connector.
Findings below are all inline — no general comments.
gengliangwang
left a comment
There was a problem hiding this comment.
Follow-up review
All five prior review comments are well addressed:
CHANGELOG_CONTRACT_VIOLATIONis now its own error class withsqlState = XX000— correctly separates the runtime contract violation from the analysis-timeINVALID_CDC_OPTIONsub-conditions.- Helper columns are prefixed
__spark_cdc_*and exposed as a publicResolveChangelogTable.HelperColumnobject — collisions with user columns are now structurally impossible and connectors can reuse the names. resolved: Boolean = falseonChangelogTablereplaces the plan-tag idempotency mechanism cleanly, and the rule now guards on!table.resolved.- The empty-rowId / default-UOE
rowId()/rowVersion()misuses are validated in #55507 — accepted, can verify once that PR lands. - The "no capability flags" test is renamed to match the actual skip condition.
The new NULL-safety addition (_rv_cnt = 2 in the carry-over filter) is a nice defensive touch and is covered by a dedicated regression test.
Two small follow-ups below — one nit, one test-coverage gap.
gengliangwang
left a comment
There was a problem hiding this comment.
Test-suite follow-up
A closer pass over the test cases. Coverage breadth is good, but a handful of tests have weak assertions, slightly misleading names, or rely on schema-violating fixture data. None block merge — most are 1–4-line cleanups.
gengliangwang
left a comment
There was a problem hiding this comment.
One more nit from the close-read of the post-processing logic.
johanl-db
left a comment
There was a problem hiding this comment.
The implementation looks good, we should aim to merge it soon to have time to cover net change computation before the 4.2 cut
This is PR 2 of a split of #55426 (see the split suggestion for the full plan). Its independent of PR 1, so we can merge in any order. For more context see discussion posted to dev@spark.apache.org and linked SPIP.
Introduce the analyzer rule that post-processes a resolved
DataSourceV2Relation(ChangelogTable)to inject carry-over removal and/or update detection, fused into a single pass over a(rowId, _commit_version)-partitioned Window. To prevent silent wrong results, it also includes an explicit rejection path for streaming CDC reads that would require post-processing.Included Changes:
ResolveChangelogTableanalyzer rule:Filteron the Window (drops CoW pairs wheremin(rowVersion) == max(rowVersion)). Update detection is aCASE WHENover delete/insert counts (relabels pairs asupdate_preimage/update_postimage). The two passes are fused into a single Window.INVALID_CDC_OPTION.STREAMING_POST_PROCESSING_NOT_SUPPORTEDwhen the requested options would need post-processing. Streams that don't need post-processing pass through unchanged. Actual streaming support is scoped to a follow-up PR.INVALID_CDC_OPTION.NET_CHANGES_NOT_YET_SUPPORTEDfor both batch and streaming. Actual implementation is scoped to a follow-up PR.INVALID_CDC_OPTION.UPDATE_DETECTION_REQUIRES_CARRY_OVER_REMOVALwhencomputeUpdates = trueis combined with a carry-over-surfacing connector anddeduplicationMode = none, which would silently misclassify carry-overs as updates.INVALID_CDC_OPTION.UNEXPECTED_MULTIPLE_CHANGES_PER_ROW_VERSIONwhen the connector emits morethan one delete or insert for the same
(rowId, _commit_version)partition, violating theChangelogcontract.Analyzer: register the rule afterResolveRelations.InMemoryChangelogCatalog:ChangelogPropertiesextension so tests can configure post-processing scenarios without a real connector.Why are the changes needed?
Currently,
CHANGES FROM VERSION ... WITH (deduplicationMode = ..., computeUpdates = ...)parses the options, but they are silently ignored — connector output is returned raw. This PR wires the options to their actual semantics for batch reads, and prevents silent wrong results for streaming reads.Does this PR introduce any user-facing change?
Yes, for CDC queries against a
Changelogconnector.Before/after example (click to expand)
Given a
Changelogconnector that advertises bothcontainsCarryoverRows = trueandrepresentsUpdateAsDeleteAndInsert = true, with rowIdidand arowVersioncolumn, for versions 1–2:Raw rows emitted by the connector:
Before this PR:
WITH (computeUpdates = 'true')is silently ignored, carry-overs leak through:After this PR:
WITH (computeUpdates = 'true'):How was this patch tested?
ResolveChangelogTablePostProcessingSuiteexercises the batch rule end-to-end via SQL againstInMemoryChangelogCatalog(carry-over removal, update detection, their interaction across the option and connector-flag matrix, data-column handling with mixed types, and plan-shape invariants).ChangelogResolutionSuiteadds streaming-rejection cases for the two capability flags that would require post-processing.Was this patch authored or co-authored using generative AI tooling?
Generated-by: Claude Opus 4.7