Skip to content

[Improve][Flink]supports multiple parallelisms and remove flink-specific logic from API#10107

Merged
Carl-Zhou-CN merged 3 commits intoapache:devfrom
CloverDew:feature/supports-multiple-parallelisms-on-flink
Feb 10, 2026
Merged

[Improve][Flink]supports multiple parallelisms and remove flink-specific logic from API#10107
Carl-Zhou-CN merged 3 commits intoapache:devfrom
CloverDew:feature/supports-multiple-parallelisms-on-flink

Conversation

@CloverDew
Copy link
Contributor

Purpose of this pull request: Fixes #9980

Problem

After implementing CDC schema evolution support in Flink engine, several issues were identified:

  1. The coordinator's state resides in the TaskManager's JVM; if a job fails and restarts, this state is easily lost.
  2. Flink's proprietary implementation logic should not be placed in the public API SupportSchemaEvolutionSinkWriter. Furthermore, flush requires manual API implementation.
  3. Multi-parallelism configuration is not supported.

Solution

This PR made some minor adjustments to the architecture.:

  • BroadcastSchemaSinkOperator operator is introduced to solidify the response state.
  • Coordinator is only responsible for communication: pure messenger, stateless.
  • Flushing is guaranteed by checkpoints, not manual implementation

Key Changes:

1. Enhanced LocalSchemaCoordinator

  • JobId isolation: Prevents multi-job interference using Map<String, WeakReference<LocalSchemaCoordinator>>
  • Pure messenger: No persistent state, only temporary communication futures

3. Streamlined BroadcastSchemaSinkOperator

  • Idempotency: tracks lastProcessedEpoch using Flink state

4. API Compliance (Addresses #9980)

  • API separation: SupportSchemaEvolutionSinkWriter contains only generic applySchemaChange() method
  • Flink-specific logic moved: All coordination logic moved from API to Flink translation layer

Testing

  • Verified MySQL CDC → MySQL sink schema evolution scenarios
  • Confirmed no data loss during schema changes
  • Tested multi-job concurrent execution

Breaking Changes

None. This is a refactoring that maintains API compatibility while improving internal implementation.

Related Issues

Check list

@CloverDew CloverDew force-pushed the feature/supports-multiple-parallelisms-on-flink branch 3 times, most recently from 1e42e4d to 460e584 Compare November 24, 2025 14:00
@Carl-Zhou-CN Carl-Zhou-CN changed the title [Fix][Flink]supports multiple parallelisms and remove flink-specific logic from API [Impove][Flink]supports multiple parallelisms and remove flink-specific logic from API Nov 25, 2025
@CloverDew CloverDew changed the title [Impove][Flink]supports multiple parallelisms and remove flink-specific logic from API [Improve][Flink]supports multiple parallelisms and remove flink-specific logic from API Nov 25, 2025
@CloverDew CloverDew force-pushed the feature/supports-multiple-parallelisms-on-flink branch from 460e584 to ae2c6b3 Compare November 27, 2025 09:38
@CloverDew CloverDew force-pushed the feature/supports-multiple-parallelisms-on-flink branch from 4886556 to 35fa5a4 Compare December 5, 2025 08:43
}
}

private void sendSchemaChangeEventToDownstream(SchemaChangeEvent schemaChangeEvent) {
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
private void sendSchemaChangeEventToDownstream(SchemaChangeEvent schemaChangeEvent) {
private void sendSchemaChangeEventToDownStream(SchemaChangeEvent schemaChangeEvent) {

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Is there a problem here?

Copy link
Member

Choose a reason for hiding this comment

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

You are correct.

private final Config pluginConfig;
private volatile Long lastProcessedEventTime;
private transient LocalSchemaCoordinator coordinator;
private transient Map<String, List<BufferedDataRow>> bufferedDataRows;
Copy link
Member

Choose a reason for hiding this comment

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

Doesn't it need to be stored in the checkpoint?

Copy link
Contributor Author

@CloverDew CloverDew Dec 16, 2025

Choose a reason for hiding this comment

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

It's also possible to store the data, but I've already stored it at the sink end. The operators closer to the source end don't necessarily need to be materialized either, especially if the source can provide a replay strategy.
The coordinator does not need to be persisted because it is only a communication component;The BroadcastSchemaSinkOperator is the component that actually holds the coordinator's response state. The coordinator can be rebuilt after a crash, and the job manager will read the state from the sink operator, buffered data can be persisted, but the latest processing time does not need to be persisted because it has already been persisted by the sink.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'm going to finalize the following variables; the buffered data rows can be rebuilt through reprocessing:
localSchemaState - Stores the schema state of each table; updated when the schema changes.
lastProcessedEventTime - The time of the last processed event, used to ensure event order.
schemaChangePending - Indicates whether a schema change is currently in progress.

Copy link
Member

Choose a reason for hiding this comment

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

After the downstream schema change is completed and a checkpoint is performed, failures occur during the process of re-consuming the cache. Is there a possibility that this batch of data will be lost? Because seemingly, this segment of data does not affect the checkpoint process of source and sink data, and it cannot be recovered from the state upon restart.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

You're right, this situation can indeed happen. After the checkpoint, the source side submits the offset, assuming that this portion of the data has been processed and won't be replayed, so the data is lost. Caching is indeed necessary.

}
}

private void performPeriodicCleanup() {
Copy link
Member

Choose a reason for hiding this comment

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

Under what circumstances will an expired task occur?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

For example, the following scenario could lead to a timeout:

  1. The SchemaOperator initiates an alter table add column n request.
  2. The coordinator waits for acknowledgments from 3 sink subtasks (parallelism=3).
  3. Subtask-0 and Subtask-1 successfully apply the schema change and send acknowledgments.
  4. Subtask-2 crashes or is restarted during the application process.
  5. Subtask-2 does not recover and send an acknowledgment within 5 minutes.
  6. The request times out and is cleaned up.

long eventTime = schemaChangeEvent.getCreatedTime();

try {
if (lastProcessedEventTime != null && eventTime <= lastProcessedEventTime) {
Copy link
Member

Choose a reason for hiding this comment

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

Could you give an example of the scene that occurred?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Each time a change event is processed, lastProcessedEventTime is updated. Any event with a timestamp less than or equal to the current watermark is rejected; otherwise, it would lead to inconsistencies in the evolution process.

Copy link
Contributor Author

@CloverDew CloverDew Dec 16, 2025

Choose a reason for hiding this comment

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

For example, consider executing the following SQL statements:

Original table structure: orders(id, user_id, amount)
-- T1=1000: Add discount field
ALTER TABLE orders ADD COLUMN discount DECIMAL(10,2);
-- T2=2000: Add status field
ALTER TABLE orders ADD COLUMN status VARCHAR(20);
-- T3=3000: Drop discount field
ALTER TABLE orders DROP COLUMN discount;

The expected final correct schema should be: orders(id, user_id, amount, status)

  1. The correct epoch processing order should be: 1000 → 3000 → Completion
  2. However, if T2 (2000) is processed late, the following will happen:
  3. After processing T3 (3000), the Sink considers the current schema to be: orders(id, user_id, amount). If this late data is not rejected, and T2 (2000) is suddenly processed, the schema becomes: orders(id, user_id, amount, status). At this point, if data is written:
    SeaTunnelRow row = new SeaTunnelRow(new Object[]{
    1001L, // id
    501L, // user_id
    new BigDecimal("99.99"), // amount
    "PAID" // status - but the Sink might interpret this as the discount field
    });

This would result in a mapping mismatch, so I made a judgment here.

Copy link
Member

Choose a reason for hiding this comment

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

ok, thank you.

jobId,
eventTime);

String key = tableId.toString() + "#" + eventTime;
Copy link
Member

Choose a reason for hiding this comment

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

One method can be used. I see that it has been applied in several places

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok

@CloverDew CloverDew force-pushed the feature/supports-multiple-parallelisms-on-flink branch 2 times, most recently from 95fc353 to c9bd451 Compare December 17, 2025 13:55
@github-actions github-actions bot added the CI&CD label Dec 18, 2025
@CloverDew CloverDew force-pushed the feature/supports-multiple-parallelisms-on-flink branch from a39c03d to 83897d0 Compare January 9, 2026 13:22
@github-actions github-actions bot removed the CI&CD label Jan 9, 2026
Copy link
Member

@TyrantLucifer TyrantLucifer left a comment

Choose a reason for hiding this comment

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

LGTM, thank you for your contribution.

Copy link
Collaborator

@LiJie20190102 LiJie20190102 left a comment

Choose a reason for hiding this comment

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

LGTM, thank you for your contribution.

@corgy-w corgy-w requested a review from Carl-Zhou-CN February 2, 2026 13:37
Copy link
Member

@Carl-Zhou-CN Carl-Zhou-CN left a comment

Choose a reason for hiding this comment

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

+1

@Carl-Zhou-CN
Copy link
Member

@CloverDew Thank you for your contribution

@Carl-Zhou-CN Carl-Zhou-CN merged commit 89b3e1c into apache:dev Feb 10, 2026
5 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[Bug][Flink] Move Flink-specific flush coordination from API to translation layer

4 participants