Skip to content

feat: streaming snapshot flush with persistent writer and adaptive file splitting#693

Closed
MrIvv wants to merge 1 commit into
memiiso:masterfrom
MrIvv:debezium-evolution-mt
Closed

feat: streaming snapshot flush with persistent writer and adaptive file splitting#693
MrIvv wants to merge 1 commit into
memiiso:masterfrom
MrIvv:debezium-evolution-mt

Conversation

@MrIvv

@MrIvv MrIvv commented Apr 16, 2026

Copy link
Copy Markdown
Contributor

Summary

Adds a streaming flush mode for the Iceberg sink that keeps a single persistent writer per data collection across an entire incremental snapshot, replacing the writer-per-chunk pattern that produced one small Parquet file per chunk and inflated catalog metadata.

The persistent writer is opened on the first chunk of a table and stays open across subsequent chunks, accumulating events in memory or spilling to a rolling temporary file when adaptive thresholds are crossed (rows-per-file, bytes-per-file, time-since-first-row). The writer is committed and closed on the corresponding TABLE_SCAN_COMPLETED notification emitted by Debezium core.

Per-table completion via existing notification channels

Per-table completion is consumed via the standard Debezium SinkNotificationChannelno producer-side SPI is added. The consumer subscribes at startup, filters for aggregateType="Incremental Snapshot" + type=TABLE_SCAN_COMPLETED, and finalizes the per-table writer using the scanned_collection field. JSON-serialized notifications (the default with debezium.format.value=json) are unwrapped from the {schema, payload} envelope before reading the fields.

This follows the architectural guidance in debezium/debezium#7362 — keep producer-side concerns on Debezium core and consume completion signals through the existing notification machinery on the sink side.

Status REST endpoint

A read-only endpoint /v1/snapshot-status/incremental exposed on Quarkus' management interface (port 9000) reports per-table progress:

  • completed[] — tables whose TABLE_SCAN_COMPLETED arrived, with rows / files / bytes committed
  • inProgress[] — tables currently being scanned (tracked from STARTED notifications, refreshed from in-flight writes)
  • summary — aggregate counts and last-completion / last-data-received timestamps

The status snapshot covers both the streaming-writer path and the direct-commit path (via per-table counters), so the endpoint surfaces a correct view regardless of which write path is in use. External orchestrators can observe completion without parsing the binary offset file.

Dependency on debezium-core changes

This PR depends on debezium/debezium#7362 which introduces parallel incremental snapshot in Debezium core, with per-table TABLE_SCAN_COMPLETED notification semantics this sink consumes. Until that PR is merged and a Debezium snapshot is published, this PR cannot build against an upstream artifact and must be built with a locally-installed debezium-core snapshot (mvn install of the debezium/debezium fork at the matching branch, then mvn install of this sink).

Production validation

The same code is running in production on a 65-table workload with snapshot.max.threads=2, incremental.snapshot.chunk.size=20000. Throughput observed ~42K rows/min, ~2.2× the single-threaded baseline measured on the same workload. Status endpoint reports correct per-table completion, no data loss across multiple pod restarts.

Scope and overlap with separate sub-PRs

The single commit in this PR contains the full streaming flush feature. Three small companion fixes that ship in their own atomic PRs (and that the streaming flush code path imports indirectly) are included here as a build-time dependency:

These overlap with files touched by the streaming flush feature, so removing them would break compilation. They will collapse to no-op once the companion PRs are merged into master.

The data-loss re-throw fix (processTablesInParallel) was already merged via #699 and is no longer in the diff.

Backward compatibility

  • Streaming flush is opt-in: writers fall back to the existing direct-commit path when not configured
  • TABLE_SCAN_COMPLETED notifications are already emitted by Debezium core today; this PR only adds a consumer
  • REST status endpoint is read-only and lives on Quarkus' management port (separate from the main sink port)
  • No existing public API is removed

@MrIvv MrIvv changed the title fix: Prevent silent data loss when parallel table processing fails feat: Streaming snapshot flush with persistent writer, adaptive file splitting, and data loss fix Apr 16, 2026
@MrIvv

MrIvv commented Apr 17, 2026

Copy link
Copy Markdown
Contributor Author

Note on CI failure: The build fails because this PR depends on debezium/debezium#7362 which introduces new SPI classes (SnapshotTableCompletionHandler, SnapshotTableMetadata, etc.) and the debezium-bom:3.6.0-SNAPSHOT artifact.

The CI cannot resolve this dependency until PR-A is merged and the snapshot is published. Once the core changes are available, this PR will build successfully.

The code has been tested extensively in production with a locally-built Debezium core (PostgreSQL 16, 116 tables, ~128M rows, zero data loss).

@MrIvv MrIvv force-pushed the debezium-evolution-mt branch from 935a168 to 540cbd8 Compare April 17, 2026 08:28
@MrIvv MrIvv force-pushed the debezium-evolution-mt branch from f8de242 to 88b64e6 Compare April 29, 2026 15:13

@ismailsimsek ismailsimsek left a comment

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

@MrIvv thank you for working on the improvements. the PR is large and contains multiple fixes and improvements. could we create small PRs to merge them. i added inline comments for the ones can be merged seperately, these fixes/features can be merged independently. spliting will also simplify the main PR for review

hasFailures = true;
failureCount++;
// The original exception from the Callable is wrapped in ExecutionException
LOGGER.error("A task failed with an exception: {}", e.getCause().getMessage(), e.getCause());

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

This is important data loss fix, could you create separate PR to merge this.

This change is completely independent it will also hep the review

if (namespace == null || namespace.isEmpty()) {
return Namespace.of("default");
}
// Support both dot separator and 0x1F (Unit Separator) used by Iceberg REST catalog

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

could we add this also with seperate PR?

is there iceberg code we can use to read string as namespace, something does the speeration for us? that would be ideal instead of doing split manually here.

.unionByNameWith(newSchema)
.setIdentifierFields(newSchema.identifierFieldNames());
// Pre-scan: detect conditions that require allowIncompatibleChanges()
boolean needsIncompatibleChanges = false;

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

could we also split this change and create standalone PR, for all the schema change fixes?

if (!icebergTable.schema().sameSchema(newSchemaCombined)) {
LOGGER.warn("Extending schema of {}", icebergTable.name());
us.commit();
icebergTable.refresh();

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

+1

if (isNewKey && !keepDeletes && rowOperation != Operation.DELETE) {
// Optimization: Pure INSERT (new key) - direct write,
// skipping the costly pre-delete since we know no previous version exists.
if ((isNewKey || rowOperation == Operation.READ) && !keepDeletes && rowOperation != Operation.DELETE) {

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

could we also add thgis fix as separate PR? thank you this is also good fix.

Comment on lines +95 to +97
public StructEventConverter(EmbeddedEngineChangeEvent e, GlobalConfig config,
StructSchemaConverter sharedSchemaConverter) {
super(config);

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

this can also be seperate PR, we can review and merge independently

@MrIvv

MrIvv commented May 4, 2026

Copy link
Copy Markdown
Contributor Author

@ismailsimsek ok, I will try to split this work into different PRs (maybe 3 or 4, no more)

MrIvv added a commit to MrIvv/debezium that referenced this pull request May 4, 2026
…trics, RetryExecutor

Addresses code review feedback from @Naros on PR debezium#7362:

- Remove dead code: RecordTransformer (deleted), DebeziumEventFactory
  (deleted entirely — its single wrapSourceRecord() was redundant with
  ConverterBuilder.toFormat() configured with Connect format, which
  already does the same raw SourceRecord -> ChangeEvent wrap)
- Move IncrementalSnapshotRetryPolicy to debezium-util as RetryExecutor,
  generalized (no SQLException-specific handling) and reusable across
  Debezium components
- Extract config defaults to public static final constants with
  Field::isPositiveInteger / Field::isPositiveLong validation on all
  numeric properties
- Lower DEFAULT_INCREMENTAL_SNAPSHOT_BATCH_FLUSH_SIZE from 20_000 to
  5_000 for safety on wide-row tables (VARCHAR/TEXT/JSONB rows can
  saturate worker memory at higher batch sizes)
- Wire PostProcessorRegistry and SnapshotProgressListener into
  TableSnapshotWorker so post-processors and metrics now run on snapshot
  events the same as on CDC events
- Refactor SnapshotRecordBuilder to use SnapshotChangeRecordEmitter +
  EventDispatcher.emitReadRecord() (same path as the chunked initial
  snapshot) instead of a custom flat builder; the builder shrinks from
  126 to 66 lines and is now a thin envelope wrapper
- Revert unrelated debezium-api/pom.xml change (restore
  "Used for unit testing with Kafka" comment block)

The remaining SnapshotTableCompletionHandler SPI exists solely for the
onTableSnapshotFinished(tableName) callback — a per-table boundary signal
that EventDispatcher does not currently expose. It enables sinks with
batched-write semantics to switch from writer-per-chunk to
writer-per-table-with-periodic-split. Concrete consumer implementation:
memiiso/debezium-server-iceberg#693, which now uses
ConverterBuilder.toFormat() to wrap raw SourceRecords into change events.

Signed-off-by: ivan.senyk <ivan.senyk94@gmail.com>
@MrIvv MrIvv force-pushed the debezium-evolution-mt branch from 88b64e6 to 42bff6e Compare May 5, 2026 07:53
@MrIvv MrIvv changed the title feat: Streaming snapshot flush with persistent writer, adaptive file splitting, and data loss fix feat: streaming snapshot flush with persistent writer and adaptive file splitting May 5, 2026
…le splitting

Adds a streaming flush mode for the Iceberg sink that keeps a single
persistent writer per data collection across an entire incremental
snapshot, replacing the writer-per-chunk pattern that produced one
small Parquet file per chunk and inflated catalog metadata.

A persistent writer is opened on the first chunk of a table and stays
open across subsequent chunks, accumulating events in memory or
spilling to a rolling temporary file when adaptive thresholds are
crossed (rows-per-file, bytes-per-file, time-since-first-row). The
writer is committed and closed on the corresponding TABLE_SCAN_COMPLETED
notification emitted by Debezium core.

Per-table completion is consumed via the standard Debezium notification
channels (`SinkNotificationChannel`) — no producer-side SPI is added.
The consumer subscribes at startup, filters for
`aggregateType="Incremental Snapshot"` and
`type=TABLE_SCAN_COMPLETED`, and finalizes the per-table writer using
the `scanned_collection` field. JSON-serialized notifications (the
default with `debezium.format.value=json`) are unwrapped from the
`{schema, payload}` envelope before reading the fields.

A read-only REST endpoint `/v1/snapshot-status/incremental` exposed
on Quarkus' management interface (port 9000) reports per-table progress
(rows, files, bytes committed; current/in-progress tables) so external
orchestrators can observe completion without parsing the offset file.
The status snapshot tracks both the streaming-writer path and the
direct-commit path (via per-table counters) so the endpoint surfaces a
correct view regardless of which write path is in use.

This PR depends on the parallel incremental snapshot work in
debezium/debezium#7362 — that PR introduces the parallel scheduler
and the per-table TABLE_SCAN_COMPLETED notification semantics this
sink consumes. Until that PR is merged and a Debezium snapshot is
published, this PR cannot build against an upstream artifact and must
be built with a locally-installed `debezium-core` snapshot.

Signed-off-by: ivan.senyk <ivan.senyk94@gmail.com>
@MrIvv MrIvv force-pushed the debezium-evolution-mt branch from 42bff6e to d6b363e Compare May 10, 2026 15:28
@github-actions

Copy link
Copy Markdown
Contributor

This pull request has been marked as stale due to 30 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time. Thank you for your contributions.

@github-actions github-actions Bot added the stale label Jun 10, 2026
@github-actions

Copy link
Copy Markdown
Contributor

This pull request has been closed due to lack of activity. This is not a judgement on the merit of the PR in any way. It is just a way of keeping the PR queue manageable. If you think that is incorrect, or the pull request requires review, you can revive the PR at any time.

@github-actions github-actions Bot closed this Jun 17, 2026
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants