From 910753d87688cd18e2d912393829868f987e0385 Mon Sep 17 00:00:00 2001 From: CloverDew Date: Mon, 25 May 2026 22:02:12 +0800 Subject: [PATCH 1/7] fix flink1.13 --- .../flink/schema/SchemaOperator.java | 181 ++++++++++++++++-- .../translation/flink/sink/FlinkSink.java | 16 +- .../flink/sink/FlinkSinkWriter.java | 127 +++++++++++- .../flink/schema/SchemaOperatorTest.java | 37 ++++ .../flink/sink/FlinkSinkWriterTest.java | 50 ++++- 5 files changed, 392 insertions(+), 19 deletions(-) diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java index 0f993dcda0ff..badc5d5ee2a2 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java @@ -44,12 +44,19 @@ import lombok.extern.slf4j.Slf4j; import java.io.Serializable; +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Queue; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; /** * Operator placed after the source to handle schema evolution. @@ -64,6 +71,11 @@ * *

Per checkpoint cycle, at most ONE schema change is applied. If multiple DDLs arrive between * two checkpoints, they are processed across successive checkpoint cycles. + * + *

Flink 1.13 cannot continue checkpointing after some source subtasks have finished. When high + * parallelism CDC jobs hit that condition, pending schema changes would otherwise stay blocked + * forever. A lightweight fallback timer detects the checkpoint stall and re-enters the task thread + * through Flink's processing-time service so the deferred DDL can still be applied safely. */ @Slf4j public class SchemaOperator extends AbstractStreamOperator @@ -72,6 +84,7 @@ public class SchemaOperator extends AbstractStreamOperator private static final int MAX_BUFFERED_RECORDS = 100000; private static final long SCHEMA_CHANGE_TIMEOUT_MS = 300_000L; private static final int CHECKPOINT_WAIT_ROUNDS = 1; + private static final long CHECKPOINT_STALL_TIMEOUT_MS = 15_000L; private final Map localSchemaState; private String jobId; @@ -82,6 +95,10 @@ public class SchemaOperator extends AbstractStreamOperator private transient Queue pendingQueue; private volatile boolean schemaChangePending = false; private long firstSeenCheckpointId = -1L; + private transient ScheduledExecutorService fallbackScheduler; + private transient volatile ScheduledFuture pendingFallbackFuture; + private volatile long lastCheckpointCompletedMs = -1L; + private volatile boolean fallbackTimerFired = false; private transient ListState localSchemaStateStore; private transient ListState lastProcessedEventTimeState; @@ -107,6 +124,13 @@ public void open() throws Exception { this.pendingQueue = new LinkedList<>(); } this.coordinator = LocalSchemaCoordinator.getInstance(this.jobId); + this.fallbackScheduler = + Executors.newSingleThreadScheduledExecutor( + runnable -> { + Thread thread = new Thread(runnable, "schema-fallback-timer-" + jobId); + thread.setDaemon(true); + return thread; + }); log.info( "SchemaOperator opened for job: {}, schemaChangePending: {}, pendingQueue size: {}", @@ -116,7 +140,7 @@ public void open() throws Exception { } @Override - public void processElement(StreamRecord streamRecord) { + public void processElement(StreamRecord streamRecord) throws InterruptedException { SeaTunnelRow element = streamRecord.getValue(); if (!isSchemaEvolutionEnabled(pluginConfig)) { @@ -124,6 +148,10 @@ public void processElement(StreamRecord streamRecord) { return; } + if (fallbackTimerFired) { + handleFallbackTimerOnTaskThread(); + } + // detect schema change events if ("__SCHEMA_CHANGE_EVENT__".equals(element.getTableId()) && element.getOptions() != null) { @@ -166,6 +194,7 @@ private void handleSchemaChangeDetected(SchemaChangeEvent event, long timestamp) pendingQueue.add(BufferedRecord.schemaChange(event)); schemaChangePending = true; + scheduleFallbackTimer(); } private void enqueueDataRecord(SeaTunnelRow row, long timestamp) { @@ -210,20 +239,14 @@ private TableIdentifier getPendingSchemaTableIdentifier() { @Override public void notifyCheckpointComplete(long checkpointId) throws Exception { super.notifyCheckpointComplete(checkpointId); + lastCheckpointCompletedMs = System.currentTimeMillis(); if (!schemaChangePending || pendingQueue.isEmpty()) { return; } - BufferedRecord head = pendingQueue.peek(); - while (head != null && !head.isSchemaChange) { - output.collect(new StreamRecord<>(head.row, head.timestamp)); - pendingQueue.poll(); - head = pendingQueue.peek(); - } + BufferedRecord head = advancePastDataRecords(); if (head == null) { - schemaChangePending = false; - firstSeenCheckpointId = -1L; return; } @@ -266,12 +289,133 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { tableId, eventTime); + applyNextPendingSchemaChange(); + } + + private void handleFallbackTimerOnTaskThread() throws InterruptedException { + fallbackTimerFired = false; + + if (!schemaChangePending || pendingQueue.isEmpty()) { + return; + } + + if (lastCheckpointCompletedMs > 0 + && System.currentTimeMillis() - lastCheckpointCompletedMs + < CHECKPOINT_STALL_TIMEOUT_MS) { + scheduleFallbackTimer(); + return; + } + + log.warn( + "No checkpoint completed within {}ms while schema change is pending. " + + "Applying deferred DDL via fallback timer.", + CHECKPOINT_STALL_TIMEOUT_MS); + + BufferedRecord head = advancePastDataRecords(); + if (head == null) { + return; + } + + applyNextPendingSchemaChange(); + } + + private void scheduleFallbackTimer() { + if (fallbackScheduler == null || fallbackScheduler.isShutdown()) { + return; + } + ScheduledFuture existing = pendingFallbackFuture; + if (existing != null && !existing.isDone()) { + return; + } + pendingFallbackFuture = + fallbackScheduler.schedule( + () -> { + fallbackTimerFired = true; + registerProcessingTimeCallback(); + }, + CHECKPOINT_STALL_TIMEOUT_MS, + TimeUnit.MILLISECONDS); + } + + private void registerProcessingTimeCallback() { + try { + Object processingTimeService = getProcessingTimeService(); + Method getCurrentTimeMethod = null; + Method registerMethod = null; + Class callbackClass = null; + + for (Method method : processingTimeService.getClass().getMethods()) { + if ("getCurrentProcessingTime".equals(method.getName()) + && method.getParameterTypes().length == 0) { + getCurrentTimeMethod = method; + } else if ("registerTimer".equals(method.getName()) + && method.getParameterTypes().length == 2 + && method.getParameterTypes()[0] == long.class) { + registerMethod = method; + callbackClass = method.getParameterTypes()[1]; + } + } + + if (getCurrentTimeMethod == null || registerMethod == null || callbackClass == null) { + log.warn( + "Could not find required ProcessingTimeService methods, " + + "falling back to flag-based handling."); + return; + } + + getCurrentTimeMethod.setAccessible(true); + registerMethod.setAccessible(true); + long now = (long) getCurrentTimeMethod.invoke(processingTimeService); + + InvocationHandler handler = + (proxy, method, args) -> { + if ("onProcessingTime".equals(method.getName())) { + handleFallbackTimerOnTaskThread(); + } + return null; + }; + Object callbackProxy = + Proxy.newProxyInstance( + callbackClass.getClassLoader(), + new Class[] {callbackClass}, + handler); + + registerMethod.invoke(processingTimeService, now + 1, callbackProxy); + } catch (Exception e) { + log.warn("Failed to register processing time callback for fallback timer", e); + } + } + + private BufferedRecord advancePastDataRecords() { + BufferedRecord head = pendingQueue.peek(); + while (head != null && !head.isSchemaChange) { + output.collect(new StreamRecord<>(head.row, head.timestamp)); + pendingQueue.poll(); + head = pendingQueue.peek(); + } + if (head == null) { + schemaChangePending = false; + firstSeenCheckpointId = -1L; + } + return head; + } + + private void applyNextPendingSchemaChange() throws InterruptedException { + BufferedRecord head = pendingQueue.poll(); + if (head == null || !head.isSchemaChange) { + return; + } + + firstSeenCheckpointId = -1L; + SchemaChangeEvent event = head.schemaEvent; + TableIdentifier tableId = event.tableIdentifier(); + long eventTime = event.getCreatedTime(); + if (lastProcessedEventTime != null && eventTime <= lastProcessedEventTime) { log.warn( "Skipping outdated schema change event (epoch {} <= last processed {})", eventTime, lastProcessedEventTime); - pendingQueue.poll(); firstSeenCheckpointId = -1L; drainDataUntilNextSchemaChange(); return; @@ -295,9 +439,6 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { tableId, eventTime); - pendingQueue.poll(); - firstSeenCheckpointId = -1L; - CatalogTable newSchema = event.getChangeAfter(); if (newSchema != null) { localSchemaState.put(tableId, newSchema); @@ -323,6 +464,7 @@ private void drainDataUntilNextSchemaChange() { "Released {} buffered data records. Another schema change pending, " + "waiting for next checkpoint.", released); + scheduleFallbackTimer(); return; } pendingQueue.poll(); @@ -333,6 +475,16 @@ private void drainDataUntilNextSchemaChange() { // queue is empty schemaChangePending = false; log.info("Released {} buffered data records. Normal data flow resumed.", released); + sendFlushSignalToDownstream(); + } + + private void sendFlushSignalToDownstream() { + SeaTunnelRow flushRow = new SeaTunnelRow(0); + Map options = new HashMap<>(); + options.put("flush_signal", true); + flushRow.setOptions(options); + output.collect(new StreamRecord<>(flushRow)); + log.info("Sent flush signal to downstream sink after schema change completion."); } @Override @@ -476,6 +628,9 @@ private void sendSchemaChangeEventToDownstream(SchemaChangeEvent schemaChangeEve @Override public void close() throws Exception { + if (fallbackScheduler != null && !fallbackScheduler.isShutdown()) { + fallbackScheduler.shutdownNow(); + } super.close(); } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java index f961b67b9266..e76345eed1c8 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java @@ -18,6 +18,8 @@ package org.apache.seatunnel.translation.flink.sink; import org.apache.seatunnel.api.sink.SeaTunnelSink; +import org.apache.seatunnel.api.sink.SinkAggregatedCommitter; +import org.apache.seatunnel.api.sink.SinkCommitter; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.translation.flink.serialization.CommitWrapperSerializer; @@ -80,15 +82,25 @@ public SinkWriter, FlinkWriterState> throws IOException { org.apache.seatunnel.api.sink.SinkWriter.Context stContext = new FlinkSinkWriterContext(context, parallelism); + SinkCommitter sinkCommitter = sink.createCommitter().orElse(null); + SinkAggregatedCommitter sinkAggregatedCommitter = + sinkCommitter == null ? sink.createAggregatedCommitter().orElse(null) : null; if (states == null || states.isEmpty()) { - return new FlinkSinkWriter<>(sink.createWriter(stContext), 1, stContext); + return new FlinkSinkWriter<>( + sink.createWriter(stContext), + 1, + stContext, + sinkCommitter, + sinkAggregatedCommitter); } else { List restoredState = states.stream().map(FlinkWriterState::getState).collect(Collectors.toList()); return new FlinkSinkWriter<>( sink.restoreWriter(stContext, restoredState), states.get(0).getCheckpointId() + 1, - stContext); + stContext, + sinkCommitter, + sinkAggregatedCommitter); } } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java index f776520acded..30da6298d102 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java @@ -22,6 +22,8 @@ import org.apache.seatunnel.api.common.metrics.MetricNames; import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.sink.MultiTableResourceManager; +import org.apache.seatunnel.api.sink.SinkAggregatedCommitter; +import org.apache.seatunnel.api.sink.SinkCommitter; import org.apache.seatunnel.api.sink.SupportResourceShare; import org.apache.seatunnel.api.sink.SupportSchemaEvolutionSinkWriter; import org.apache.seatunnel.api.sink.event.WriterCloseEvent; @@ -68,8 +70,16 @@ public class FlinkSinkWriter private long checkpointId; + private volatile boolean checkpointStalled; + + private final SinkCommitter sinkCommitter; + + private final SinkAggregatedCommitter sinkAggregatedCommitter; + private MultiTableResourceManager resourceManager; + private MultiTableResourceManager aggregatedCommitterResourceManager; + /** * Cached writer states produced together with {@link #prepareCommit(boolean)}. * @@ -86,10 +96,17 @@ public class FlinkSinkWriter FlinkSinkWriter( org.apache.seatunnel.api.sink.SinkWriter sinkWriter, long checkpointId, - org.apache.seatunnel.api.sink.SinkWriter.Context context) { + org.apache.seatunnel.api.sink.SinkWriter.Context context, + SinkCommitter sinkCommitter, + SinkAggregatedCommitter sinkAggregatedCommitter) { this.context = context; this.sinkWriter = sinkWriter; this.checkpointId = checkpointId; + this.sinkCommitter = sinkCommitter; + this.sinkAggregatedCommitter = + sinkAggregatedCommitter == null + ? null + : (SinkAggregatedCommitter) sinkAggregatedCommitter; MetricsContext metricsContext = context.getMetricsContext(); this.sinkWriteCount = metricsContext.counter(MetricNames.SINK_WRITE_COUNT); this.sinkWriteBytes = metricsContext.counter(MetricNames.SINK_WRITE_BYTES); @@ -99,6 +116,16 @@ public class FlinkSinkWriter ((SupportResourceShare) sinkWriter).initMultiTableResourceManager(1, 1); ((SupportResourceShare) sinkWriter).setMultiTableResourceManager(resourceManager, 0); } + if (this.sinkAggregatedCommitter != null) { + this.sinkAggregatedCommitter.init(); + if (this.sinkAggregatedCommitter instanceof SupportResourceShare) { + aggregatedCommitterResourceManager = + ((SupportResourceShare) this.sinkAggregatedCommitter) + .initMultiTableResourceManager(1, 1); + ((SupportResourceShare) this.sinkAggregatedCommitter) + .setMultiTableResourceManager(aggregatedCommitterResourceManager, 0); + } + } } @Override @@ -118,6 +145,15 @@ public void write(InputT element, SinkWriter.Context context) throws IOException sinkWriteCount.inc(); sinkWriteBytes.inc(seaTunnelRow.getBytesSize()); sinkWriterQPS.markEvent(); + + if (checkpointStalled) { + try { + Optional commitInfo = sinkWriter.prepareCommit(checkpointId); + immediateCommitIfStalled(commitInfo); + } catch (Exception e) { + log.warn("Auto-flush after write failed (checkpoint stalled mode)", e); + } + } } private boolean handleControlMessage(Map options) throws IOException { @@ -132,9 +168,38 @@ private boolean handleControlMessage(Map options) throws IOExcep return true; } + if (options.containsKey("flush_signal")) { + flushBufferedData(); + return true; + } + return false; } + private void flushBufferedData() { + checkpointStalled = true; + try { + Optional commitInfo = sinkWriter.prepareCommit(checkpointId); + immediateCommitIfStalled(commitInfo); + log.info( + "Flushed buffered data in response to flush signal. " + + "Checkpoint-stalled mode enabled: subsequent writes will auto-flush."); + } catch (Exception e) { + log.warn("Failed to flush buffered data on flush signal", e); + } + } + + private void immediateCommitIfStalled(Optional commitInfo) { + if (!commitInfo.isPresent()) { + return; + } + try { + commitPreparedData(commitInfo); + } catch (Exception e) { + log.warn("Failed to immediately commit prepared data (checkpoint-stalled mode)", e); + } + } + private void handleSchemaChangeEvent( SchemaChangeEvent schemaChangeEvent, Map options) throws IOException { log.info( @@ -155,6 +220,7 @@ private void handleSchemaChangeEvent( boolean success = false; try { + flushBeforeSchemaChange(schemaChangeEvent); ((SupportSchemaEvolutionSinkWriter) sinkWriter).applySchemaChange(schemaChangeEvent); log.info( "FlinkSinkWriter successfully applied SchemaChangeEvent for table: {}", @@ -179,6 +245,47 @@ private void handleSchemaChangeEvent( } } + private void flushBeforeSchemaChange(SchemaChangeEvent schemaChangeEvent) throws IOException { + if (sinkCommitter == null && sinkAggregatedCommitter == null) { + return; + } + + try { + Optional commitInfo = sinkWriter.prepareCommit(checkpointId); + commitPreparedData(commitInfo); + log.info( + "Committed pending sink data before schema change for table: {}", + schemaChangeEvent.tableIdentifier()); + } catch (Exception e) { + throw new IOException( + String.format( + "Failed to commit pending sink data before schema change for table %s", + schemaChangeEvent.tableIdentifier()), + e); + } + } + + private void commitPreparedData(Optional commitInfo) throws Exception { + if (!commitInfo.isPresent()) { + return; + } + if (sinkCommitter != null) { + sinkCommitter.commit(Collections.singletonList(commitInfo.get())); + return; + } + if (sinkAggregatedCommitter != null) { + Object aggregatedCommitInfo = + sinkAggregatedCommitter.combine(Collections.singletonList(commitInfo.get())); + List retryCommitInfos = + sinkAggregatedCommitter.commit(Collections.singletonList(aggregatedCommitInfo)); + if (!retryCommitInfos.isEmpty()) { + log.warn( + "Aggregated committer returned {} retry commit(s).", + retryCommitInfos.size()); + } + } + } + private void sendSchemaChangeAck( SchemaChangeEvent schemaChangeEvent, long epoch, int subtaskId, boolean success) { if (subtaskId < 0) { @@ -259,6 +366,14 @@ public List> snapshotState() throws IOException { @Override public void close() throws Exception { + try { + if (checkpointStalled) { + Optional commitInfo = sinkWriter.prepareCommit(checkpointId); + immediateCommitIfStalled(commitInfo); + } + } catch (Exception e) { + log.warn("Failed to perform final stalled-mode flush before close", e); + } sinkWriter.close(); context.getEventListener().onEvent(new WriterCloseEvent()); try { @@ -268,5 +383,15 @@ public void close() throws Exception { } catch (Throwable e) { log.error("close resourceManager error", e); } + try { + if (sinkAggregatedCommitter != null) { + sinkAggregatedCommitter.close(); + } + if (aggregatedCommitterResourceManager != null) { + aggregatedCommitterResourceManager.close(); + } + } catch (Throwable e) { + log.error("close aggregated committer resource error", e); + } } } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java index fd3b7d212616..5065e6be5fba 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java @@ -176,6 +176,37 @@ void testCoordinationFailureKeepsBufferedRecordsBlocked() throws Exception { assertTrue(pendingQueue.peek().isSchemaChange); } + @Test + void testFallbackTimerAppliesSchemaChangeAndFlushesBufferedData() throws Exception { + LocalSchemaCoordinator coordinator = Mockito.mock(LocalSchemaCoordinator.class); + Mockito.when( + coordinator.requestSchemaChange( + Mockito.any(), Mockito.anyLong(), Mockito.anyLong())) + .thenReturn(true); + + OperatorTestContext context = createOperator(false); + setField(context.operator, "coordinator", coordinator); + + AlterTableAddColumnEvent event = createSchemaChangeEvent(); + SeaTunnelRow row = createDataRow("row-released-after-fallback"); + + context.operator.processElement(new StreamRecord<>(createSchemaRow(event), 400L)); + context.operator.processElement(new StreamRecord<>(row, 401L)); + + invokeNoArgMethod(context.operator, "handleFallbackTimerOnTaskThread"); + + assertEquals(3, context.output.records.size()); + assertSchemaBroadcast(context.output.records.get(0), event); + assertEquals(row, context.output.records.get(1).getValue()); + assertTrue( + Boolean.TRUE.equals( + context.output.records.get(2).getValue().getOptions().get("flush_signal"))); + assertFalse(getBooleanField(context.operator, "schemaChangePending")); + assertTrue(getPendingQueue(context.operator).isEmpty()); + Mockito.verify(coordinator) + .requestSchemaChange(event.tableIdentifier(), event.getCreatedTime(), 300_000L); + } + private static OperatorTestContext createOperator(boolean restored) throws Exception { return createOperator(new OperatorStateStoreStub(), restored); } @@ -285,6 +316,12 @@ private static void setField(Object target, Class owner, String fieldName, Ob field.set(target, value); } + private static Object invokeNoArgMethod(Object target, String methodName) throws Exception { + java.lang.reflect.Method method = target.getClass().getDeclaredMethod(methodName); + method.setAccessible(true); + return method.invoke(target); + } + private static Field findField(Class type, String fieldName) throws NoSuchFieldException { Class current = type; while (current != null) { diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java index 50be1a39d279..f9ce1cb29062 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.event.EventListener; +import org.apache.seatunnel.api.sink.SinkCommitter; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -27,8 +28,11 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import java.util.Optional; class FlinkSinkWriterTest { @@ -39,7 +43,7 @@ void testPrepareCommitSnapshotsStateAndAdvancesCheckpoint() throws Exception { RecordingContext context = new RecordingContext(); FlinkSinkWriter flinkSinkWriter = - new FlinkSinkWriter<>(delegate, 1L, context); + new FlinkSinkWriter<>(delegate, 1L, context, null, null); // first checkpoint List> commits = flinkSinkWriter.prepareCommit(false); @@ -72,7 +76,7 @@ void testSnapshotStateWithoutPrepareCommitFallsBack() throws Exception { RecordingContext context = new RecordingContext(); FlinkSinkWriter flinkSinkWriter = - new FlinkSinkWriter<>(delegate, 3L, context); + new FlinkSinkWriter<>(delegate, 3L, context, null, null); // Direct snapshotState should call delegate.snapshotState with checkpointId 3 List> states = flinkSinkWriter.snapshotState(); @@ -83,13 +87,40 @@ void testSnapshotStateWithoutPrepareCommitFallsBack() throws Exception { Assertions.assertEquals("state-3", states.get(0).getState()); } + @Test + void testFlushSignalEnablesImmediateCommitForStalledCheckpoint() throws Exception { + RecordingSinkWriter delegate = new RecordingSinkWriter(); + RecordingCommitter committer = new RecordingCommitter(); + RecordingContext context = new RecordingContext(); + + FlinkSinkWriter flinkSinkWriter = + new FlinkSinkWriter<>(delegate, 7L, context, committer, null); + + SeaTunnelRow flushSignal = new SeaTunnelRow(0); + Map options = new LinkedHashMap<>(); + options.put("flush_signal", true); + flushSignal.setOptions(options); + flinkSinkWriter.write(flushSignal, null); + + SeaTunnelRow row = new SeaTunnelRow(1); + row.setField(0, "value"); + flinkSinkWriter.write(row, null); + + Assertions.assertEquals(1, delegate.writtenRows.size()); + Assertions.assertEquals(Arrays.asList(7L, 7L), delegate.prepareCommitCalls); + Assertions.assertEquals(Arrays.asList("commit-7", "commit-7"), committer.committed); + } + private static class RecordingSinkWriter implements SinkWriter { private final List prepareCommitCalls = new ArrayList<>(); private final List snapshotCalls = new ArrayList<>(); + private final List writtenRows = new ArrayList<>(); @Override - public void write(SeaTunnelRow element) throws IOException {} + public void write(SeaTunnelRow element) throws IOException { + writtenRows.add(element); + } @Override public Optional prepareCommit() { @@ -116,6 +147,19 @@ public void abortPrepare() {} public void close() throws IOException {} } + private static class RecordingCommitter implements SinkCommitter { + private final List committed = new ArrayList<>(); + + @Override + public List commit(List commitInfos) { + committed.addAll(commitInfos); + return Collections.emptyList(); + } + + @Override + public void abort(List commitInfos) {} + } + private static class RecordingContext implements SinkWriter.Context { @Override From e428ced938ce2f301b91ae5436d5774eaa3839c6 Mon Sep 17 00:00:00 2001 From: CloverDew Date: Tue, 26 May 2026 13:18:19 +0800 Subject: [PATCH 2/7] fix flink1.13 --- ...cdc_to_mysql_with_flink_schema_change.conf | 2 +- ...with_flink_schema_change_exactly_once.conf | 2 +- .../flink/schema/SchemaOperator.java | 26 ++++++++++++------- 3 files changed, 18 insertions(+), 12 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change.conf index 9dd595f54785..7ffbf1c1c614 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change.conf @@ -20,7 +20,7 @@ env { # You can set engine configuration here - parallelism = 1 + parallelism = 5 job.mode = "STREAMING" checkpoint.interval = 5000 read_limit.bytes_per_second=7000000 diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once.conf index 09c88f312068..69816a41b9b6 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once.conf @@ -20,7 +20,7 @@ env { # You can set engine configuration here - parallelism = 1 + parallelism = 5 job.mode = "STREAMING" checkpoint.interval = 5000 read_limit.bytes_per_second=7000000 diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java index badc5d5ee2a2..760a0d565438 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java @@ -140,7 +140,8 @@ public void open() throws Exception { } @Override - public void processElement(StreamRecord streamRecord) throws InterruptedException { + public void processElement(StreamRecord streamRecord) + throws InterruptedException { SeaTunnelRow element = streamRecord.getValue(); if (!isSchemaEvolutionEnabled(pluginConfig)) { @@ -289,7 +290,7 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { tableId, eventTime); - applyNextPendingSchemaChange(); + applyNextPendingSchemaChange(false); } private void handleFallbackTimerOnTaskThread() throws InterruptedException { @@ -316,7 +317,7 @@ private void handleFallbackTimerOnTaskThread() throws InterruptedException { return; } - applyNextPendingSchemaChange(); + applyNextPendingSchemaChange(true); } private void scheduleFallbackTimer() { @@ -400,13 +401,13 @@ private BufferedRecord advancePastDataRecords() { return head; } - private void applyNextPendingSchemaChange() throws InterruptedException { - BufferedRecord head = pendingQueue.poll(); + private void applyNextPendingSchemaChange(boolean emitFlushSignalAfterDrain) + throws InterruptedException { + BufferedRecord head = pendingQueue.peek(); if (head == null || !head.isSchemaChange) { return; } - firstSeenCheckpointId = -1L; SchemaChangeEvent event = head.schemaEvent; TableIdentifier tableId = event.tableIdentifier(); long eventTime = event.getCreatedTime(); @@ -416,8 +417,9 @@ private void applyNextPendingSchemaChange() throws InterruptedException { "Skipping outdated schema change event (epoch {} <= last processed {})", eventTime, lastProcessedEventTime); + pendingQueue.poll(); firstSeenCheckpointId = -1L; - drainDataUntilNextSchemaChange(); + drainDataUntilNextSchemaChange(emitFlushSignalAfterDrain); return; } @@ -438,6 +440,8 @@ private void applyNextPendingSchemaChange() throws InterruptedException { "Schema change for table {} (epoch {}) confirmed by all sink subtasks.", tableId, eventTime); + pendingQueue.poll(); + firstSeenCheckpointId = -1L; CatalogTable newSchema = event.getChangeAfter(); if (newSchema != null) { @@ -445,7 +449,7 @@ private void applyNextPendingSchemaChange() throws InterruptedException { } lastProcessedEventTime = eventTime; - drainDataUntilNextSchemaChange(); + drainDataUntilNextSchemaChange(emitFlushSignalAfterDrain); log.info( "Schema change for table {} (epoch {}) processing complete. pendingQueue remaining: {}", @@ -454,7 +458,7 @@ private void applyNextPendingSchemaChange() throws InterruptedException { pendingQueue.size()); } - private void drainDataUntilNextSchemaChange() { + private void drainDataUntilNextSchemaChange(boolean emitFlushSignalAfterDrain) { int released = 0; while (!pendingQueue.isEmpty()) { BufferedRecord record = pendingQueue.peek(); @@ -475,7 +479,9 @@ private void drainDataUntilNextSchemaChange() { // queue is empty schemaChangePending = false; log.info("Released {} buffered data records. Normal data flow resumed.", released); - sendFlushSignalToDownstream(); + if (emitFlushSignalAfterDrain) { + sendFlushSignalToDownstream(); + } } private void sendFlushSignalToDownstream() { From 96be8e0b221deec49ec915011360b581698be977 Mon Sep 17 00:00:00 2001 From: CloverDew Date: Wed, 27 May 2026 20:46:02 +0800 Subject: [PATCH 3/7] try another plan --- .../execution/SourceExecuteProcessor.java | 41 ++++- .../MysqlCDCWithFlinkSchemaChangeIT.java | 43 +++++ ...cdc_to_mysql_with_flink_schema_change.conf | 2 +- ...with_flink_schema_change_exactly_once.conf | 2 +- ...ma_change_exactly_once_multi_parallel.conf | 55 +++++++ ...th_flink_schema_change_multi_parallel.conf | 53 ++++++ .../flink/schema/SchemaOperator.java | 155 +----------------- .../flink/sink/FlinkSinkWriter.java | 90 ---------- .../flink/source/FlinkSourceReader.java | 21 ++- .../source/FlinkSourceReaderContext.java | 4 + .../flink/schema/SchemaOperatorTest.java | 36 ---- .../flink/sink/FlinkSinkWriterTest.java | 48 +++++- 12 files changed, 260 insertions(+), 290 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once_multi_parallel.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_multi_parallel.conf diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java index 94393b0ec45c..ef7ea9ac632a 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.shade.com.google.common.collect.Lists; import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigValueFactory; import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.common.PluginIdentifier; @@ -63,6 +64,8 @@ @SuppressWarnings("unchecked,rawtypes") public class SourceExecuteProcessor extends FlinkAbstractPluginExecuteProcessor { + private static final String SOURCE_KEEP_ALIVE_CONFIG = "schema-changes.source-keep-alive"; + public SourceExecuteProcessor( List jarPaths, Config envConfig, @@ -80,11 +83,13 @@ public List execute(List upstreamDataS SourceTableInfo sourceTableInfo = plugins.get(i); SeaTunnelSource internalSource = sourceTableInfo.getSource(); Config pluginConfig = pluginConfigs.get(i); - FlinkSource flinkSource = new FlinkSource<>(internalSource, envConfig); DataStreamSource sourceStream = executionEnvironment.fromSource( - flinkSource, + new FlinkSource<>( + internalSource, + enableSourceKeepAliveIfNeeded( + internalSource, pluginConfig, envConfig)), WatermarkStrategy.noWatermarks(), String.format("%s-Source", internalSource.getPluginName())); @@ -139,6 +144,38 @@ public List execute(List upstreamDataS return sources; } + private Config enableSourceKeepAliveIfNeeded( + SeaTunnelSource source, Config pluginConfig, Config currentEnvConfig) { + boolean isStreaming = + currentEnvConfig.hasPath("job.mode") + && STREAMING + .toString() + .equalsIgnoreCase(currentEnvConfig.getString("job.mode")); + boolean enableSchemaChange = + pluginConfig.hasPath("schema-changes.enabled") + && pluginConfig.getBoolean("schema-changes.enabled"); + boolean shouldEnableKeepAlive = + isStreaming + && enableSchemaChange + && source instanceof SupportSchemaEvolution + && !supportsSinkFunctionFinish(); + if (!shouldEnableKeepAlive) { + return currentEnvConfig; + } + return currentEnvConfig.withValue( + SOURCE_KEEP_ALIVE_CONFIG, ConfigValueFactory.fromAnyRef(true)); + } + + private boolean supportsSinkFunctionFinish() { + for (java.lang.reflect.Method method : + org.apache.flink.streaming.api.functions.sink.SinkFunction.class.getMethods()) { + if ("finish".equals(method.getName()) && method.getParameterCount() == 0) { + return true; + } + } + return false; + } + @Override protected List initializePlugins( List jarPaths, List pluginConfigs) { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCWithFlinkSchemaChangeIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCWithFlinkSchemaChangeIT.java index 37fea77fe210..ec7dc4e67fc2 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCWithFlinkSchemaChangeIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCWithFlinkSchemaChangeIT.java @@ -167,6 +167,49 @@ public void testMysqlCdcWithSchemaEvolutionCaseExactlyOnce(TestContainer contain assertSchemaEvolution(MYSQL_DATABASE, SOURCE_TABLE, SINK_TABLE2); } + @Order(3) + @TestTemplate + public void testMysqlCdcWithSchemaEvolutionCaseMultiParallel(TestContainer container) { + resetDatabaseToInitialState(); + + CompletableFuture.runAsync( + () -> { + try { + container.executeJob( + "/mysqlcdc_to_mysql_with_flink_schema_change_multi_parallel.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + }); + + assertSchemaEvolutionForAddColumns(MYSQL_DATABASE, SOURCE_TABLE, SINK_TABLE); + shopDatabase.setTemplateName("drop_columns").createAndInitialize(); + assertTableStructureAndData(MYSQL_DATABASE, SOURCE_TABLE, SINK_TABLE); + shopDatabase.setTemplateName("change_columns").createAndInitialize(); + shopDatabase.setTemplateName("modify_columns").createAndInitialize(); + assertTableStructureAndData(MYSQL_DATABASE, SOURCE_TABLE, SINK_TABLE); + } + + @Order(4) + @TestTemplate + public void testMysqlCdcWithSchemaEvolutionCaseExactlyOnceMultiParallel( + TestContainer container) { + shopDatabase.setTemplateName("shop").createAndInitialize(); + CompletableFuture.runAsync( + () -> { + try { + container.executeJob( + "/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once_multi_parallel.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + }); + + assertSchemaEvolution(MYSQL_DATABASE, SOURCE_TABLE, SINK_TABLE2); + } + private void assertSchemaEvolution(String database, String sourceTable, String sinkTable) { await().atMost(180000, TimeUnit.MILLISECONDS) .untilAsserted( diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change.conf index 7ffbf1c1c614..9dd595f54785 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change.conf @@ -20,7 +20,7 @@ env { # You can set engine configuration here - parallelism = 5 + parallelism = 1 job.mode = "STREAMING" checkpoint.interval = 5000 read_limit.bytes_per_second=7000000 diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once.conf index 69816a41b9b6..09c88f312068 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once.conf @@ -20,7 +20,7 @@ env { # You can set engine configuration here - parallelism = 5 + parallelism = 1 job.mode = "STREAMING" checkpoint.interval = 5000 read_limit.bytes_per_second=7000000 diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once_multi_parallel.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once_multi_parallel.conf new file mode 100644 index 000000000000..69816a41b9b6 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once_multi_parallel.conf @@ -0,0 +1,55 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + parallelism = 5 + job.mode = "STREAMING" + checkpoint.interval = 5000 + read_limit.bytes_per_second=7000000 + read_limit.rows_per_second=400 +} + +source { + MySQL-CDC { + server-id = 5652-5657 + username = "st_user_source" + password = "mysqlpw" + table-names = ["shop.products"] + url = "jdbc:mysql://mysql_cdc_e2e:3306/shop" + + schema-changes.enabled = true + } +} + +sink { + jdbc { + url = "jdbc:mysql://mysql_cdc_e2e:3306/shop" + driver = "com.mysql.cj.jdbc.Driver" + user = "st_user_sink" + password = "mysqlpw" + generate_sink_sql = true + database = shop + table = mysql_cdc_e2e_sink_table_with_schema_change_exactly_once + primary_keys = ["id"] + is_exactly_once = true + xa_data_source_class_name = "com.mysql.cj.jdbc.MysqlXADataSource" + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_multi_parallel.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_multi_parallel.conf new file mode 100644 index 000000000000..7ffbf1c1c614 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_multi_parallel.conf @@ -0,0 +1,53 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + parallelism = 5 + job.mode = "STREAMING" + checkpoint.interval = 5000 + read_limit.bytes_per_second=7000000 + read_limit.rows_per_second=400 +} + +source { + MySQL-CDC { + server-id = 5652-5657 + username = "st_user_source" + password = "mysqlpw" + table-names = ["shop.products"] + url = "jdbc:mysql://mysql_cdc_e2e:3306/shop" + + schema-changes.enabled = true + } +} + +sink { + jdbc { + url = "jdbc:mysql://mysql_cdc_e2e:3306/shop" + driver = "com.mysql.cj.jdbc.Driver" + user = "st_user_sink" + password = "mysqlpw" + generate_sink_sql = true + database = shop + table = mysql_cdc_e2e_sink_table_with_schema_change + primary_keys = ["id"] + } +} diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java index 760a0d565438..d3499800d30a 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java @@ -44,19 +44,12 @@ import lombok.extern.slf4j.Slf4j; import java.io.Serializable; -import java.lang.reflect.InvocationHandler; -import java.lang.reflect.Method; -import java.lang.reflect.Proxy; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Queue; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; /** * Operator placed after the source to handle schema evolution. @@ -71,11 +64,6 @@ * *

Per checkpoint cycle, at most ONE schema change is applied. If multiple DDLs arrive between * two checkpoints, they are processed across successive checkpoint cycles. - * - *

Flink 1.13 cannot continue checkpointing after some source subtasks have finished. When high - * parallelism CDC jobs hit that condition, pending schema changes would otherwise stay blocked - * forever. A lightweight fallback timer detects the checkpoint stall and re-enters the task thread - * through Flink's processing-time service so the deferred DDL can still be applied safely. */ @Slf4j public class SchemaOperator extends AbstractStreamOperator @@ -84,7 +72,6 @@ public class SchemaOperator extends AbstractStreamOperator private static final int MAX_BUFFERED_RECORDS = 100000; private static final long SCHEMA_CHANGE_TIMEOUT_MS = 300_000L; private static final int CHECKPOINT_WAIT_ROUNDS = 1; - private static final long CHECKPOINT_STALL_TIMEOUT_MS = 15_000L; private final Map localSchemaState; private String jobId; @@ -95,10 +82,6 @@ public class SchemaOperator extends AbstractStreamOperator private transient Queue pendingQueue; private volatile boolean schemaChangePending = false; private long firstSeenCheckpointId = -1L; - private transient ScheduledExecutorService fallbackScheduler; - private transient volatile ScheduledFuture pendingFallbackFuture; - private volatile long lastCheckpointCompletedMs = -1L; - private volatile boolean fallbackTimerFired = false; private transient ListState localSchemaStateStore; private transient ListState lastProcessedEventTimeState; @@ -124,13 +107,6 @@ public void open() throws Exception { this.pendingQueue = new LinkedList<>(); } this.coordinator = LocalSchemaCoordinator.getInstance(this.jobId); - this.fallbackScheduler = - Executors.newSingleThreadScheduledExecutor( - runnable -> { - Thread thread = new Thread(runnable, "schema-fallback-timer-" + jobId); - thread.setDaemon(true); - return thread; - }); log.info( "SchemaOperator opened for job: {}, schemaChangePending: {}, pendingQueue size: {}", @@ -149,16 +125,12 @@ public void processElement(StreamRecord streamRecord) return; } - if (fallbackTimerFired) { - handleFallbackTimerOnTaskThread(); - } - // detect schema change events if ("__SCHEMA_CHANGE_EVENT__".equals(element.getTableId()) && element.getOptions() != null) { Object object = element.getOptions().get("schema_change_event"); if (object instanceof SchemaChangeEvent) { - handleSchemaChangeDetected((SchemaChangeEvent) object, streamRecord.getTimestamp()); + handleSchemaChangeDetected((SchemaChangeEvent) object); return; } } @@ -172,7 +144,7 @@ public void processElement(StreamRecord streamRecord) output.collect(streamRecord); } - private void handleSchemaChangeDetected(SchemaChangeEvent event, long timestamp) { + private void handleSchemaChangeDetected(SchemaChangeEvent event) { List supportedTypes = source.supports(); if (supportedTypes == null || supportedTypes.isEmpty()) { log.info("Source does not support any schema change types, skipping"); @@ -195,7 +167,6 @@ private void handleSchemaChangeDetected(SchemaChangeEvent event, long timestamp) pendingQueue.add(BufferedRecord.schemaChange(event)); schemaChangePending = true; - scheduleFallbackTimer(); } private void enqueueDataRecord(SeaTunnelRow row, long timestamp) { @@ -240,7 +211,6 @@ private TableIdentifier getPendingSchemaTableIdentifier() { @Override public void notifyCheckpointComplete(long checkpointId) throws Exception { super.notifyCheckpointComplete(checkpointId); - lastCheckpointCompletedMs = System.currentTimeMillis(); if (!schemaChangePending || pendingQueue.isEmpty()) { return; @@ -290,101 +260,7 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { tableId, eventTime); - applyNextPendingSchemaChange(false); - } - - private void handleFallbackTimerOnTaskThread() throws InterruptedException { - fallbackTimerFired = false; - - if (!schemaChangePending || pendingQueue.isEmpty()) { - return; - } - - if (lastCheckpointCompletedMs > 0 - && System.currentTimeMillis() - lastCheckpointCompletedMs - < CHECKPOINT_STALL_TIMEOUT_MS) { - scheduleFallbackTimer(); - return; - } - - log.warn( - "No checkpoint completed within {}ms while schema change is pending. " - + "Applying deferred DDL via fallback timer.", - CHECKPOINT_STALL_TIMEOUT_MS); - - BufferedRecord head = advancePastDataRecords(); - if (head == null) { - return; - } - - applyNextPendingSchemaChange(true); - } - - private void scheduleFallbackTimer() { - if (fallbackScheduler == null || fallbackScheduler.isShutdown()) { - return; - } - ScheduledFuture existing = pendingFallbackFuture; - if (existing != null && !existing.isDone()) { - return; - } - pendingFallbackFuture = - fallbackScheduler.schedule( - () -> { - fallbackTimerFired = true; - registerProcessingTimeCallback(); - }, - CHECKPOINT_STALL_TIMEOUT_MS, - TimeUnit.MILLISECONDS); - } - - private void registerProcessingTimeCallback() { - try { - Object processingTimeService = getProcessingTimeService(); - Method getCurrentTimeMethod = null; - Method registerMethod = null; - Class callbackClass = null; - - for (Method method : processingTimeService.getClass().getMethods()) { - if ("getCurrentProcessingTime".equals(method.getName()) - && method.getParameterTypes().length == 0) { - getCurrentTimeMethod = method; - } else if ("registerTimer".equals(method.getName()) - && method.getParameterTypes().length == 2 - && method.getParameterTypes()[0] == long.class) { - registerMethod = method; - callbackClass = method.getParameterTypes()[1]; - } - } - - if (getCurrentTimeMethod == null || registerMethod == null || callbackClass == null) { - log.warn( - "Could not find required ProcessingTimeService methods, " - + "falling back to flag-based handling."); - return; - } - - getCurrentTimeMethod.setAccessible(true); - registerMethod.setAccessible(true); - long now = (long) getCurrentTimeMethod.invoke(processingTimeService); - - InvocationHandler handler = - (proxy, method, args) -> { - if ("onProcessingTime".equals(method.getName())) { - handleFallbackTimerOnTaskThread(); - } - return null; - }; - Object callbackProxy = - Proxy.newProxyInstance( - callbackClass.getClassLoader(), - new Class[] {callbackClass}, - handler); - - registerMethod.invoke(processingTimeService, now + 1, callbackProxy); - } catch (Exception e) { - log.warn("Failed to register processing time callback for fallback timer", e); - } + applyNextPendingSchemaChange(); } private BufferedRecord advancePastDataRecords() { @@ -401,8 +277,7 @@ private BufferedRecord advancePastDataRecords() { return head; } - private void applyNextPendingSchemaChange(boolean emitFlushSignalAfterDrain) - throws InterruptedException { + private void applyNextPendingSchemaChange() throws InterruptedException { BufferedRecord head = pendingQueue.peek(); if (head == null || !head.isSchemaChange) { return; @@ -419,7 +294,7 @@ private void applyNextPendingSchemaChange(boolean emitFlushSignalAfterDrain) lastProcessedEventTime); pendingQueue.poll(); firstSeenCheckpointId = -1L; - drainDataUntilNextSchemaChange(emitFlushSignalAfterDrain); + drainDataUntilNextSchemaChange(); return; } @@ -449,7 +324,7 @@ private void applyNextPendingSchemaChange(boolean emitFlushSignalAfterDrain) } lastProcessedEventTime = eventTime; - drainDataUntilNextSchemaChange(emitFlushSignalAfterDrain); + drainDataUntilNextSchemaChange(); log.info( "Schema change for table {} (epoch {}) processing complete. pendingQueue remaining: {}", @@ -458,7 +333,7 @@ private void applyNextPendingSchemaChange(boolean emitFlushSignalAfterDrain) pendingQueue.size()); } - private void drainDataUntilNextSchemaChange(boolean emitFlushSignalAfterDrain) { + private void drainDataUntilNextSchemaChange() { int released = 0; while (!pendingQueue.isEmpty()) { BufferedRecord record = pendingQueue.peek(); @@ -468,7 +343,6 @@ private void drainDataUntilNextSchemaChange(boolean emitFlushSignalAfterDrain) { "Released {} buffered data records. Another schema change pending, " + "waiting for next checkpoint.", released); - scheduleFallbackTimer(); return; } pendingQueue.poll(); @@ -479,18 +353,6 @@ private void drainDataUntilNextSchemaChange(boolean emitFlushSignalAfterDrain) { // queue is empty schemaChangePending = false; log.info("Released {} buffered data records. Normal data flow resumed.", released); - if (emitFlushSignalAfterDrain) { - sendFlushSignalToDownstream(); - } - } - - private void sendFlushSignalToDownstream() { - SeaTunnelRow flushRow = new SeaTunnelRow(0); - Map options = new HashMap<>(); - options.put("flush_signal", true); - flushRow.setOptions(options); - output.collect(new StreamRecord<>(flushRow)); - log.info("Sent flush signal to downstream sink after schema change completion."); } @Override @@ -634,9 +496,6 @@ private void sendSchemaChangeEventToDownstream(SchemaChangeEvent schemaChangeEve @Override public void close() throws Exception { - if (fallbackScheduler != null && !fallbackScheduler.isShutdown()) { - fallbackScheduler.shutdownNow(); - } super.close(); } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java index 30da6298d102..f75b9d539783 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java @@ -70,8 +70,6 @@ public class FlinkSinkWriter private long checkpointId; - private volatile boolean checkpointStalled; - private final SinkCommitter sinkCommitter; private final SinkAggregatedCommitter sinkAggregatedCommitter; @@ -145,15 +143,6 @@ public void write(InputT element, SinkWriter.Context context) throws IOException sinkWriteCount.inc(); sinkWriteBytes.inc(seaTunnelRow.getBytesSize()); sinkWriterQPS.markEvent(); - - if (checkpointStalled) { - try { - Optional commitInfo = sinkWriter.prepareCommit(checkpointId); - immediateCommitIfStalled(commitInfo); - } catch (Exception e) { - log.warn("Auto-flush after write failed (checkpoint stalled mode)", e); - } - } } private boolean handleControlMessage(Map options) throws IOException { @@ -168,38 +157,9 @@ private boolean handleControlMessage(Map options) throws IOExcep return true; } - if (options.containsKey("flush_signal")) { - flushBufferedData(); - return true; - } - return false; } - private void flushBufferedData() { - checkpointStalled = true; - try { - Optional commitInfo = sinkWriter.prepareCommit(checkpointId); - immediateCommitIfStalled(commitInfo); - log.info( - "Flushed buffered data in response to flush signal. " - + "Checkpoint-stalled mode enabled: subsequent writes will auto-flush."); - } catch (Exception e) { - log.warn("Failed to flush buffered data on flush signal", e); - } - } - - private void immediateCommitIfStalled(Optional commitInfo) { - if (!commitInfo.isPresent()) { - return; - } - try { - commitPreparedData(commitInfo); - } catch (Exception e) { - log.warn("Failed to immediately commit prepared data (checkpoint-stalled mode)", e); - } - } - private void handleSchemaChangeEvent( SchemaChangeEvent schemaChangeEvent, Map options) throws IOException { log.info( @@ -220,7 +180,6 @@ private void handleSchemaChangeEvent( boolean success = false; try { - flushBeforeSchemaChange(schemaChangeEvent); ((SupportSchemaEvolutionSinkWriter) sinkWriter).applySchemaChange(schemaChangeEvent); log.info( "FlinkSinkWriter successfully applied SchemaChangeEvent for table: {}", @@ -245,47 +204,6 @@ private void handleSchemaChangeEvent( } } - private void flushBeforeSchemaChange(SchemaChangeEvent schemaChangeEvent) throws IOException { - if (sinkCommitter == null && sinkAggregatedCommitter == null) { - return; - } - - try { - Optional commitInfo = sinkWriter.prepareCommit(checkpointId); - commitPreparedData(commitInfo); - log.info( - "Committed pending sink data before schema change for table: {}", - schemaChangeEvent.tableIdentifier()); - } catch (Exception e) { - throw new IOException( - String.format( - "Failed to commit pending sink data before schema change for table %s", - schemaChangeEvent.tableIdentifier()), - e); - } - } - - private void commitPreparedData(Optional commitInfo) throws Exception { - if (!commitInfo.isPresent()) { - return; - } - if (sinkCommitter != null) { - sinkCommitter.commit(Collections.singletonList(commitInfo.get())); - return; - } - if (sinkAggregatedCommitter != null) { - Object aggregatedCommitInfo = - sinkAggregatedCommitter.combine(Collections.singletonList(commitInfo.get())); - List retryCommitInfos = - sinkAggregatedCommitter.commit(Collections.singletonList(aggregatedCommitInfo)); - if (!retryCommitInfos.isEmpty()) { - log.warn( - "Aggregated committer returned {} retry commit(s).", - retryCommitInfos.size()); - } - } - } - private void sendSchemaChangeAck( SchemaChangeEvent schemaChangeEvent, long epoch, int subtaskId, boolean success) { if (subtaskId < 0) { @@ -366,14 +284,6 @@ public List> snapshotState() throws IOException { @Override public void close() throws Exception { - try { - if (checkpointStalled) { - Optional commitInfo = sinkWriter.prepareCommit(checkpointId); - immediateCommitIfStalled(commitInfo); - } - } catch (Exception e) { - log.warn("Failed to perform final stalled-mode flush before close", e); - } sinkWriter.close(); context.getEventListener().onEvent(new WriterCloseEvent()); try { diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java index 4c2a7b6d2e50..52a6d9c7a5de 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java @@ -49,6 +49,8 @@ public class FlinkSourceReader implements SourceReader> { + private static final String SOURCE_KEEP_ALIVE_CONFIG = "schema-changes.source-keep-alive"; + private final Logger LOGGER = LoggerFactory.getLogger(FlinkSourceReader.class); private final org.apache.seatunnel.api.source.SourceReader sourceReader; @@ -65,6 +67,8 @@ public class FlinkSourceReader private final ScheduledExecutorService scheduledExecutor; + private final boolean sourceKeepAliveEnabled; + public FlinkSourceReader( org.apache.seatunnel.api.source.SourceReader sourceReader, org.apache.seatunnel.api.source.SourceReader.Context context, @@ -81,6 +85,9 @@ public FlinkSourceReader( this.sourceReader = sourceReader; this.context = context; this.flinkRowCollector = new FlinkRowCollector(envConfig, context.getMetricsContext()); + this.sourceKeepAliveEnabled = + envConfig.hasPath(SOURCE_KEEP_ALIVE_CONFIG) + && envConfig.getBoolean(SOURCE_KEEP_ALIVE_CONFIG); } @Override @@ -108,8 +115,11 @@ public InputStatus pollNext(ReaderOutput output) throws Exception return InputStatus.NOTHING_AVAILABLE; } } else { - // reduce CPU idle - Thread.sleep(DEFAULT_WAIT_TIME_MILLIS); + if (sourceKeepAliveEnabled) { + // Flink 1.13 requires idle source subtasks to stay alive so checkpoints continue. + Thread.sleep(DEFAULT_WAIT_TIME_MILLIS); + return InputStatus.NOTHING_AVAILABLE; + } } return inputStatus; } @@ -132,6 +142,10 @@ public CompletableFuture isAvailable() { @Override public void addSplits(List> splits) { + if (!splits.isEmpty() && context instanceof FlinkSourceReaderContext) { + ((FlinkSourceReaderContext) context).resetNoMoreElementEvent(); + inputStatus = InputStatus.MORE_AVAILABLE; + } sourceReader.addSplits( splits.stream().map(SplitWrapper::getSourceSplit).collect(Collectors.toList())); } @@ -144,7 +158,8 @@ public void notifyNoMoreSplits() { @Override public void handleSourceEvents(SourceEvent sourceEvent) { if (sourceEvent instanceof NoMoreElementEvent) { - inputStatus = InputStatus.END_OF_INPUT; + inputStatus = + sourceKeepAliveEnabled ? InputStatus.MORE_AVAILABLE : InputStatus.END_OF_INPUT; } if (sourceEvent instanceof SourceEventWrapper) { sourceReader.handleSourceEvent((((SourceEventWrapper) sourceEvent).getSourceEvent())); diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReaderContext.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReaderContext.java index 2b20e0d4047f..458b240d4d28 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReaderContext.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReaderContext.java @@ -101,6 +101,10 @@ public boolean isSendNoMoreElementEvent() { return isSendNoMoreElementEvent.get(); } + public void resetNoMoreElementEvent() { + isSendNoMoreElementEvent.set(false); + } + @Override public EventListener getEventListener() { return eventListener; diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java index 5065e6be5fba..91086d41f7a4 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java @@ -177,36 +177,6 @@ void testCoordinationFailureKeepsBufferedRecordsBlocked() throws Exception { } @Test - void testFallbackTimerAppliesSchemaChangeAndFlushesBufferedData() throws Exception { - LocalSchemaCoordinator coordinator = Mockito.mock(LocalSchemaCoordinator.class); - Mockito.when( - coordinator.requestSchemaChange( - Mockito.any(), Mockito.anyLong(), Mockito.anyLong())) - .thenReturn(true); - - OperatorTestContext context = createOperator(false); - setField(context.operator, "coordinator", coordinator); - - AlterTableAddColumnEvent event = createSchemaChangeEvent(); - SeaTunnelRow row = createDataRow("row-released-after-fallback"); - - context.operator.processElement(new StreamRecord<>(createSchemaRow(event), 400L)); - context.operator.processElement(new StreamRecord<>(row, 401L)); - - invokeNoArgMethod(context.operator, "handleFallbackTimerOnTaskThread"); - - assertEquals(3, context.output.records.size()); - assertSchemaBroadcast(context.output.records.get(0), event); - assertEquals(row, context.output.records.get(1).getValue()); - assertTrue( - Boolean.TRUE.equals( - context.output.records.get(2).getValue().getOptions().get("flush_signal"))); - assertFalse(getBooleanField(context.operator, "schemaChangePending")); - assertTrue(getPendingQueue(context.operator).isEmpty()); - Mockito.verify(coordinator) - .requestSchemaChange(event.tableIdentifier(), event.getCreatedTime(), 300_000L); - } - private static OperatorTestContext createOperator(boolean restored) throws Exception { return createOperator(new OperatorStateStoreStub(), restored); } @@ -316,12 +286,6 @@ private static void setField(Object target, Class owner, String fieldName, Ob field.set(target, value); } - private static Object invokeNoArgMethod(Object target, String methodName) throws Exception { - java.lang.reflect.Method method = target.getClass().getDeclaredMethod(methodName); - method.setAccessible(true); - return method.invoke(target); - } - private static Field findField(Class type, String fieldName) throws NoSuchFieldException { Class current = type; while (current != null) { diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java index f9ce1cb29062..3922a7eb4c02 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java @@ -21,6 +21,10 @@ import org.apache.seatunnel.api.event.EventListener; import org.apache.seatunnel.api.sink.SinkCommitter; import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.sink.SupportSchemaEvolutionSinkWriter; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.schema.event.AlterTableAddColumnEvent; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.junit.jupiter.api.Assertions; @@ -28,7 +32,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; @@ -88,17 +91,29 @@ void testSnapshotStateWithoutPrepareCommitFallsBack() throws Exception { } @Test - void testFlushSignalEnablesImmediateCommitForStalledCheckpoint() throws Exception { - RecordingSinkWriter delegate = new RecordingSinkWriter(); + void testSchemaChangeEventDoesNotForceCommit() throws Exception { + SchemaAwareRecordingSinkWriter delegate = new SchemaAwareRecordingSinkWriter(); RecordingCommitter committer = new RecordingCommitter(); RecordingContext context = new RecordingContext(); FlinkSinkWriter flinkSinkWriter = new FlinkSinkWriter<>(delegate, 7L, context, committer, null); + AlterTableAddColumnEvent event = + AlterTableAddColumnEvent.add( + TableIdentifier.of("catalog", "database", "table"), + PhysicalColumn.of( + "added_col", + org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE, + 64L, + true, + null, + null)); + event.setJobId("job-under-test"); SeaTunnelRow flushSignal = new SeaTunnelRow(0); Map options = new LinkedHashMap<>(); - options.put("flush_signal", true); + options.put("schema_change_event", event); + options.put("schema_subtask_id", 0L); flushSignal.setOptions(options); flinkSinkWriter.write(flushSignal, null); @@ -107,15 +122,17 @@ void testFlushSignalEnablesImmediateCommitForStalledCheckpoint() throws Exceptio flinkSinkWriter.write(row, null); Assertions.assertEquals(1, delegate.writtenRows.size()); - Assertions.assertEquals(Arrays.asList(7L, 7L), delegate.prepareCommitCalls); - Assertions.assertEquals(Arrays.asList("commit-7", "commit-7"), committer.committed); + Assertions.assertEquals(Collections.emptyList(), delegate.prepareCommitCalls); + Assertions.assertEquals(1, delegate.appliedSchemaChanges.size()); + Assertions.assertEquals(event, delegate.appliedSchemaChanges.get(0)); + Assertions.assertEquals(Collections.emptyList(), committer.committed); } private static class RecordingSinkWriter implements SinkWriter { - private final List prepareCommitCalls = new ArrayList<>(); - private final List snapshotCalls = new ArrayList<>(); - private final List writtenRows = new ArrayList<>(); + protected final List prepareCommitCalls = new ArrayList<>(); + protected final List snapshotCalls = new ArrayList<>(); + protected final List writtenRows = new ArrayList<>(); @Override public void write(SeaTunnelRow element) throws IOException { @@ -147,6 +164,19 @@ public void abortPrepare() {} public void close() throws IOException {} } + private static class SchemaAwareRecordingSinkWriter extends RecordingSinkWriter + implements SupportSchemaEvolutionSinkWriter { + + private final List + appliedSchemaChanges = new ArrayList<>(); + + @Override + public void applySchemaChange( + org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent event) { + appliedSchemaChanges.add(event); + } + } + private static class RecordingCommitter implements SinkCommitter { private final List committed = new ArrayList<>(); From a70fc3c1d2f2f83c7748090466b5ab5606d798a8 Mon Sep 17 00:00:00 2001 From: CloverDew Date: Wed, 27 May 2026 21:47:47 +0800 Subject: [PATCH 4/7] fix --- .../flink/schema/SchemaOperator.java | 129 ++++++++++++++++++ .../coordinator/LocalSchemaCoordinator.java | 70 +++++++--- .../flink/schema/SchemaOperatorTest.java | 34 +++++ .../flink/sink/FlinkSinkWriterTest.java | 8 +- 4 files changed, 216 insertions(+), 25 deletions(-) diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java index d3499800d30a..8d6f9a43d60f 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java @@ -44,12 +44,19 @@ import lombok.extern.slf4j.Slf4j; import java.io.Serializable; +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Queue; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; /** * Operator placed after the source to handle schema evolution. @@ -64,6 +71,11 @@ * *

Per checkpoint cycle, at most ONE schema change is applied. If multiple DDLs arrive between * two checkpoints, they are processed across successive checkpoint cycles. + * + *

Flink 1.13 cannot continue checkpointing after some source subtasks have finished. When high + * parallelism CDC jobs hit that condition, pending schema changes would otherwise stay blocked + * forever. A lightweight fallback timer detects the checkpoint stall and re-enters the task thread + * through Flink's processing-time service so the deferred DDL can still be applied safely. */ @Slf4j public class SchemaOperator extends AbstractStreamOperator @@ -72,6 +84,7 @@ public class SchemaOperator extends AbstractStreamOperator private static final int MAX_BUFFERED_RECORDS = 100000; private static final long SCHEMA_CHANGE_TIMEOUT_MS = 300_000L; private static final int CHECKPOINT_WAIT_ROUNDS = 1; + private static final long CHECKPOINT_STALL_TIMEOUT_MS = 15_000L; private final Map localSchemaState; private String jobId; @@ -82,6 +95,10 @@ public class SchemaOperator extends AbstractStreamOperator private transient Queue pendingQueue; private volatile boolean schemaChangePending = false; private long firstSeenCheckpointId = -1L; + private transient ScheduledExecutorService fallbackScheduler; + private transient volatile ScheduledFuture pendingFallbackFuture; + private volatile long lastCheckpointCompletedMs = -1L; + private volatile boolean fallbackTimerFired = false; private transient ListState localSchemaStateStore; private transient ListState lastProcessedEventTimeState; @@ -107,6 +124,13 @@ public void open() throws Exception { this.pendingQueue = new LinkedList<>(); } this.coordinator = LocalSchemaCoordinator.getInstance(this.jobId); + this.fallbackScheduler = + Executors.newSingleThreadScheduledExecutor( + runnable -> { + Thread thread = new Thread(runnable, "schema-fallback-timer-" + jobId); + thread.setDaemon(true); + return thread; + }); log.info( "SchemaOperator opened for job: {}, schemaChangePending: {}, pendingQueue size: {}", @@ -125,6 +149,10 @@ public void processElement(StreamRecord streamRecord) return; } + if (fallbackTimerFired) { + handleFallbackTimerOnTaskThread(); + } + // detect schema change events if ("__SCHEMA_CHANGE_EVENT__".equals(element.getTableId()) && element.getOptions() != null) { @@ -167,6 +195,7 @@ private void handleSchemaChangeDetected(SchemaChangeEvent event) { pendingQueue.add(BufferedRecord.schemaChange(event)); schemaChangePending = true; + scheduleFallbackTimer(); } private void enqueueDataRecord(SeaTunnelRow row, long timestamp) { @@ -211,6 +240,7 @@ private TableIdentifier getPendingSchemaTableIdentifier() { @Override public void notifyCheckpointComplete(long checkpointId) throws Exception { super.notifyCheckpointComplete(checkpointId); + lastCheckpointCompletedMs = System.currentTimeMillis(); if (!schemaChangePending || pendingQueue.isEmpty()) { return; @@ -263,6 +293,101 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { applyNextPendingSchemaChange(); } + private void handleFallbackTimerOnTaskThread() throws InterruptedException { + fallbackTimerFired = false; + + if (!schemaChangePending || pendingQueue.isEmpty()) { + return; + } + + if (lastCheckpointCompletedMs > 0 + && System.currentTimeMillis() - lastCheckpointCompletedMs + < CHECKPOINT_STALL_TIMEOUT_MS) { + scheduleFallbackTimer(); + return; + } + + log.warn( + "No checkpoint completed within {}ms while schema change is pending. " + + "Applying deferred DDL via fallback timer. Note: data committed via " + + "normal Flink checkpoint lifecycle may be delayed until checkpoints resume.", + CHECKPOINT_STALL_TIMEOUT_MS); + + BufferedRecord head = advancePastDataRecords(); + if (head == null) { + return; + } + + applyNextPendingSchemaChange(); + } + + private void scheduleFallbackTimer() { + if (fallbackScheduler == null || fallbackScheduler.isShutdown()) { + return; + } + ScheduledFuture existing = pendingFallbackFuture; + if (existing != null && !existing.isDone()) { + return; + } + pendingFallbackFuture = + fallbackScheduler.schedule( + () -> { + fallbackTimerFired = true; + registerProcessingTimeCallback(); + }, + CHECKPOINT_STALL_TIMEOUT_MS, + TimeUnit.MILLISECONDS); + } + + private void registerProcessingTimeCallback() { + try { + Object processingTimeService = getProcessingTimeService(); + Method getCurrentTimeMethod = null; + Method registerMethod = null; + Class callbackClass = null; + + for (Method method : processingTimeService.getClass().getMethods()) { + if ("getCurrentProcessingTime".equals(method.getName()) + && method.getParameterTypes().length == 0) { + getCurrentTimeMethod = method; + } else if ("registerTimer".equals(method.getName()) + && method.getParameterTypes().length == 2 + && method.getParameterTypes()[0] == long.class) { + registerMethod = method; + callbackClass = method.getParameterTypes()[1]; + } + } + + if (getCurrentTimeMethod == null || registerMethod == null || callbackClass == null) { + log.warn( + "Could not find required ProcessingTimeService methods, " + + "falling back to flag-based handling."); + return; + } + + getCurrentTimeMethod.setAccessible(true); + registerMethod.setAccessible(true); + long now = (long) getCurrentTimeMethod.invoke(processingTimeService); + + InvocationHandler handler = + (proxy, method, args) -> { + if ("onProcessingTime".equals(method.getName())) { + handleFallbackTimerOnTaskThread(); + } + return null; + }; + Object callbackProxy = + Proxy.newProxyInstance( + callbackClass.getClassLoader(), + new Class[] {callbackClass}, + handler); + + registerMethod.invoke(processingTimeService, now + 1, callbackProxy); + } catch (Exception e) { + log.warn("Failed to register processing time callback for fallback timer", e); + } + } + private BufferedRecord advancePastDataRecords() { BufferedRecord head = pendingQueue.peek(); while (head != null && !head.isSchemaChange) { @@ -343,6 +468,7 @@ private void drainDataUntilNextSchemaChange() { "Released {} buffered data records. Another schema change pending, " + "waiting for next checkpoint.", released); + scheduleFallbackTimer(); return; } pendingQueue.poll(); @@ -496,6 +622,9 @@ private void sendSchemaChangeEventToDownstream(SchemaChangeEvent schemaChangeEve @Override public void close() throws Exception { + if (fallbackScheduler != null && !fallbackScheduler.isShutdown()) { + fallbackScheduler.shutdownNow(); + } super.close(); } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/coordinator/LocalSchemaCoordinator.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/coordinator/LocalSchemaCoordinator.java index 8d99f433a391..4edf628c539b 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/coordinator/LocalSchemaCoordinator.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/coordinator/LocalSchemaCoordinator.java @@ -41,6 +41,12 @@ * Local coordinator for schema change synchronization. This coordinator only manages temporary * communication between SchemaOperator and sink subtasks. All persistent state is managed by * BroadcastSchemaSinkOperator in Flink State. + * + *

Schema changes (DDL like ALTER TABLE) are database-level operations that only need to be + * executed once. In Flink's parallel execution model, SchemaOperator sends schema change events via + * output.collect() which routes to only ONE downstream subtask based on partitioning. Therefore, + * this coordinator completes schema change requests when ANY single subtask successfully applies + * the change, rather than waiting for all subtasks. */ @Slf4j public class LocalSchemaCoordinator { @@ -138,24 +144,27 @@ public void unregisterSinkSubtask(int subtaskId) { remaining, jobId); + // Check if any pending requests can now be completed + // (Since we only need 1 ACK for DDL, this typically won't change anything, + // but we keep it for edge cases where all subtasks close before any ACK) for (Map.Entry entry : pendingRequests.entrySet()) { String key = entry.getKey(); TimestampedPendingRequest request = entry.getValue(); Set applied = receivedAcks.get(key); - int expectedActive = Math.max(remaining, 1); - if (applied != null && applied.size() >= expectedActive) { + // If we already have at least 1 ACK, complete the request + if (applied != null && !applied.isEmpty()) { if (request.appliedPhaseCompleteAtomic.compareAndSet(false, true)) { - boolean allSuccess = request.allSuccess.get(); - request.future.complete(allSuccess); + boolean success = request.allSuccess.get(); + request.future.complete(success); log.info( - "After subtask {} unregistered, all {} active subtasks have applied " - + "schema change for table {} (epoch {}). Completing request with result: {}", + "After subtask {} unregistered, completing schema change request for " + + "table {} (epoch {}) with {} ACK(s). Result: {}", subtaskId, - expectedActive, request.tableId, request.epoch, - allSuccess); + applied.size(), + success); } } } @@ -218,8 +227,8 @@ public enum SchemaProcessingStatus { public boolean requestSchemaChange(TableIdentifier tableId, long epoch, long timeoutMs) throws InterruptedException, SchemaCoordinationException { String key = tableId.toString() + "#" + epoch; - int expectedAcks = activeSinkSubtasks.size(); - if (expectedAcks == 0) { + int totalSubtasks = activeSinkSubtasks.size(); + if (totalSubtasks == 0) { log.warn( "No active sink subtasks. Cannot coordinate schema change for table {} (epoch {}). " + "Assuming success to avoid deadlock.", @@ -227,11 +236,17 @@ public boolean requestSchemaChange(TableIdentifier tableId, long epoch, long tim epoch); return true; } + // Schema changes (DDL) are database-level operations that only need to execute once. + // Due to Flink's partitioning, only one subtask receives the schema change event, + // so we only need 1 ACK to confirm the DDL was applied successfully. + int expectedAcks = 1; log.info( - "Requesting schema change for table {} (epoch {}). Waiting for all {} sink subtasks to apply after checkpoint completion.", + "Requesting schema change for table {} (epoch {}). Waiting for at least {} of {} " + + "sink subtasks to apply the DDL (database-level operation).", tableId, epoch, - expectedAcks); + expectedAcks, + totalSubtasks); long now = System.currentTimeMillis(); TimestampedPendingRequest request = @@ -312,31 +327,42 @@ public void notifySchemaChangeApplied( } appliedSubtasks.add(subtaskId); - int currentExpected = Math.min(request.expectedAcks, activeSinkSubtasks.size()); - currentExpected = Math.max(currentExpected, 1); + // Schema changes only need 1 successful application since they're database-level operations + int requiredAcks = request.expectedAcks; // This is now 1 log.info( - "Subtask {} applied schema change for table {} (epoch {}), success: {}. {}/{} subtasks applied.", + "Subtask {} applied schema change for table {} (epoch {}), success: {}. " + + "{} subtask(s) applied (need {} for completion).", subtaskId, tableId, epoch, success, appliedSubtasks.size(), - currentExpected); + requiredAcks); if (!success) { request.allSuccess.set(false); } - if (appliedSubtasks.size() >= currentExpected) { + // Complete when we have at least 1 successful ACK (DDL only needs to run once) + if (appliedSubtasks.size() >= requiredAcks && success) { if (request.appliedPhaseCompleteAtomic.compareAndSet(false, true)) { - boolean allSuccess = request.allSuccess.get(); - request.future.complete(allSuccess); + request.future.complete(true); log.info( - "All {} active subtasks have applied schema change for table {} (epoch {}). Completing request with result: {}", - currentExpected, + "Schema change for table {} (epoch {}) successfully applied by subtask {}. " + + "DDL execution complete (database-level operation).", + tableId, + epoch, + subtaskId); + } + } else if (appliedSubtasks.size() >= requiredAcks && !success) { + // If the only ACK we got was a failure, complete with failure + if (request.appliedPhaseCompleteAtomic.compareAndSet(false, true)) { + request.future.complete(false); + log.error( + "Schema change for table {} (epoch {}) failed on subtask {}.", tableId, epoch, - allSuccess); + subtaskId); } } } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java index 91086d41f7a4..5c7e373c9b8a 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java @@ -177,6 +177,34 @@ void testCoordinationFailureKeepsBufferedRecordsBlocked() throws Exception { } @Test + void testFallbackTimerAppliesSchemaChangeWithoutFlushSignal() throws Exception { + LocalSchemaCoordinator coordinator = Mockito.mock(LocalSchemaCoordinator.class); + Mockito.when( + coordinator.requestSchemaChange( + Mockito.any(), Mockito.anyLong(), Mockito.anyLong())) + .thenReturn(true); + + OperatorTestContext context = createOperator(false); + setField(context.operator, "coordinator", coordinator); + + AlterTableAddColumnEvent event = createSchemaChangeEvent(); + SeaTunnelRow row = createDataRow("row-released-after-fallback"); + + context.operator.processElement(new StreamRecord<>(createSchemaRow(event), 400L)); + context.operator.processElement(new StreamRecord<>(row, 401L)); + + invokeNoArgMethod(context.operator, "handleFallbackTimerOnTaskThread"); + + // Should emit schema broadcast and buffered data, but NO flush_signal + assertEquals(2, context.output.records.size()); + assertSchemaBroadcast(context.output.records.get(0), event); + assertEquals(row, context.output.records.get(1).getValue()); + assertFalse(getBooleanField(context.operator, "schemaChangePending")); + assertTrue(getPendingQueue(context.operator).isEmpty()); + Mockito.verify(coordinator) + .requestSchemaChange(event.tableIdentifier(), event.getCreatedTime(), 300_000L); + } + private static OperatorTestContext createOperator(boolean restored) throws Exception { return createOperator(new OperatorStateStoreStub(), restored); } @@ -286,6 +314,12 @@ private static void setField(Object target, Class owner, String fieldName, Ob field.set(target, value); } + private static Object invokeNoArgMethod(Object target, String methodName) throws Exception { + java.lang.reflect.Method method = target.getClass().getDeclaredMethod(methodName); + method.setAccessible(true); + return method.invoke(target); + } + private static Field findField(Class type, String fieldName) throws NoSuchFieldException { Class current = type; while (current != null) { diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java index 3922a7eb4c02..2cc9b1b79243 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java @@ -110,17 +110,19 @@ void testSchemaChangeEventDoesNotForceCommit() throws Exception { null, null)); event.setJobId("job-under-test"); - SeaTunnelRow flushSignal = new SeaTunnelRow(0); + SeaTunnelRow schemaEvent = new SeaTunnelRow(0); Map options = new LinkedHashMap<>(); options.put("schema_change_event", event); options.put("schema_subtask_id", 0L); - flushSignal.setOptions(options); - flinkSinkWriter.write(flushSignal, null); + schemaEvent.setOptions(options); + flinkSinkWriter.write(schemaEvent, null); SeaTunnelRow row = new SeaTunnelRow(1); row.setField(0, "value"); flinkSinkWriter.write(row, null); + // Schema change should apply without forcing commit - commits happen via normal Flink + // lifecycle Assertions.assertEquals(1, delegate.writtenRows.size()); Assertions.assertEquals(Collections.emptyList(), delegate.prepareCommitCalls); Assertions.assertEquals(1, delegate.appliedSchemaChanges.size()); From ba39f9af360f2090f9c1c1a68e98cfabe298c6b7 Mon Sep 17 00:00:00 2001 From: CloverDew Date: Sun, 31 May 2026 20:49:17 +0800 Subject: [PATCH 5/7] fix --- .../flink/schema/SchemaOperator.java | 39 ++++++++++++++++--- .../flink/schema/SchemaOperatorTest.java | 33 +++++++++++++++- 2 files changed, 64 insertions(+), 8 deletions(-) diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java index 8d6f9a43d60f..f093a977ef37 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java @@ -293,6 +293,19 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { applyNextPendingSchemaChange(); } + /** + * Handles fallback timer firing on the task thread (Flink 1.13 checkpoint stall workaround). + * + *

IMPORTANT: This method still respects the checkpoint-completion safety fence. The DDL is + * only applied if at least one checkpoint has completed after the schema event was detected + * ({@code firstSeenCheckpointId >= 0}). This ensures XA transactions from the earlier + * checkpoint cycle have finished before ALTER TABLE runs. + * + *

If no checkpoint has completed yet, we reschedule the fallback timer and wait. The + * fallback only kicks in when checkpoints stall AFTER the first post-DDL checkpoint has + * completed, which is the scenario in Flink 1.13 high-parallelism CDC jobs where some source + * subtasks finish before others. + */ private void handleFallbackTimerOnTaskThread() throws InterruptedException { fallbackTimerFired = false; @@ -307,17 +320,31 @@ private void handleFallbackTimerOnTaskThread() throws InterruptedException { return; } - log.warn( - "No checkpoint completed within {}ms while schema change is pending. " - + "Applying deferred DDL via fallback timer. Note: data committed via " - + "normal Flink checkpoint lifecycle may be delayed until checkpoints resume.", - CHECKPOINT_STALL_TIMEOUT_MS); - BufferedRecord head = advancePastDataRecords(); if (head == null) { return; } + if (firstSeenCheckpointId < 0) { + log.info( + "Fallback timer fired but no checkpoint has completed after schema event " + + "for table {} (epoch {}). Rescheduling fallback to preserve " + + "checkpoint-completion safety fence.", + head.schemaEvent.tableIdentifier(), + head.schemaEvent.getCreatedTime()); + scheduleFallbackTimer(); + return; + } + + log.warn( + "Checkpoint stall detected after first post-DDL checkpoint {}. " + + "Applying deferred DDL for table {} (epoch {}) via fallback timer. " + + "Note: data committed via normal Flink checkpoint lifecycle may be " + + "delayed until checkpoints resume.", + firstSeenCheckpointId, + head.schemaEvent.tableIdentifier(), + head.schemaEvent.getCreatedTime()); + applyNextPendingSchemaChange(); } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java index 5c7e373c9b8a..f62728b4ef10 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java @@ -177,7 +177,7 @@ void testCoordinationFailureKeepsBufferedRecordsBlocked() throws Exception { } @Test - void testFallbackTimerAppliesSchemaChangeWithoutFlushSignal() throws Exception { + void testFallbackTimerRespectsCheckpointSafetyFence() throws Exception { LocalSchemaCoordinator coordinator = Mockito.mock(LocalSchemaCoordinator.class); Mockito.when( coordinator.requestSchemaChange( @@ -193,9 +193,38 @@ void testFallbackTimerAppliesSchemaChangeWithoutFlushSignal() throws Exception { context.operator.processElement(new StreamRecord<>(createSchemaRow(event), 400L)); context.operator.processElement(new StreamRecord<>(row, 401L)); + // Fallback timer fires but no checkpoint has completed yet (firstSeenCheckpointId < 0). + // The fallback should NOT apply DDL — it must reschedule itself to preserve the + // checkpoint-completion safety fence that protects against XA/MDL conflicts. + invokeNoArgMethod(context.operator, "handleFallbackTimerOnTaskThread"); + + assertTrue(context.output.records.isEmpty()); + assertTrue(getBooleanField(context.operator, "schemaChangePending")); + assertEquals(2, getPendingQueue(context.operator).size()); + assertEquals(-1L, getLongField(context.operator, "firstSeenCheckpointId")); + Mockito.verifyNoInteractions(coordinator); + + // Now complete the first checkpoint to set firstSeenCheckpointId. + context.operator.notifyCheckpointComplete(40L); + + assertTrue(context.output.records.isEmpty()); + assertEquals(40L, getLongField(context.operator, "firstSeenCheckpointId")); + assertTrue(getBooleanField(context.operator, "schemaChangePending")); + Mockito.verifyNoInteractions(coordinator); + + // Simulate checkpoint stall: set lastCheckpointCompletedMs to a time in the past + // (older than CHECKPOINT_STALL_TIMEOUT_MS = 15_000L). This simulates Flink 1.13 + // high-parallelism CDC jobs where checkpoints stop after some source subtasks finish. + setField( + context.operator, + "lastCheckpointCompletedMs", + System.currentTimeMillis() - 20_000L); + + // Fallback timer fires again. Now firstSeenCheckpointId >= 0 and stall detected, + // so the DDL can be applied (checkpoint safety fence is respected since at least one + // checkpoint completed after the DDL was detected). invokeNoArgMethod(context.operator, "handleFallbackTimerOnTaskThread"); - // Should emit schema broadcast and buffered data, but NO flush_signal assertEquals(2, context.output.records.size()); assertSchemaBroadcast(context.output.records.get(0), event); assertEquals(row, context.output.records.get(1).getValue()); From 5d5d2f483c6076c0ed0e7dc25621d38c6f026f0a Mon Sep 17 00:00:00 2001 From: CloverDew Date: Tue, 2 Jun 2026 11:46:06 +0800 Subject: [PATCH 6/7] fix --- .../execution/SourceExecuteProcessor.java | 71 ++++++ .../AbstractSourceExecuteProcessor.java | 238 ++++++++++++++++++ .../execution/SourceExecuteProcessor.java | 193 +------------- .../flink/schema/SchemaOperator13.java | 106 ++++++++ .../flink/schema/SchemaOperator.java | 173 ++++--------- .../coordinator/LocalSchemaCoordinator.java | 8 + .../flink/schema/SchemaOperatorTest.java | 32 ++- 7 files changed, 502 insertions(+), 319 deletions(-) create mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java create mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractSourceExecuteProcessor.java create mode 100644 seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-13/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator13.java diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java new file mode 100644 index 000000000000..981227491773 --- /dev/null +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.core.starter.flink.execution; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import org.apache.seatunnel.api.common.JobContext; +import org.apache.seatunnel.api.source.SupportSchemaEvolution; +import org.apache.seatunnel.translation.flink.schema.SchemaOperator; +import org.apache.seatunnel.translation.flink.schema.SchemaOperator13; + +import java.net.URL; +import java.util.List; + +/** + * Flink 1.13-specific source execution processor. Shadows the common {@code SourceExecuteProcessor} + * at runtime (same package, same class name) to provide two Flink 1.13-specific behaviours without + * using reflection: + * + *

    + *
  1. {@link #createSchemaOperator} returns {@link SchemaOperator13}, which registers the + * checkpoint-stall fallback timer via the strongly-typed {@code + * ProcessingTimeService.registerTimer} API instead of a background {@code + * ScheduledExecutorService} + reflection. + *
  2. {@link #supportsSinkFunctionFinish} hard-codes {@code false}: Flink 1.13's {@code + * SinkFunction} does not expose a {@code finish()} method, so this fact is known at compile + * time and no reflection is needed. + *
+ */ +@SuppressWarnings("unchecked,rawtypes") +public class SourceExecuteProcessor extends AbstractSourceExecuteProcessor { + + public SourceExecuteProcessor( + List jarPaths, + Config envConfig, + List pluginConfigs, + JobContext jobContext) { + super(jarPaths, envConfig, pluginConfigs, jobContext); + } + + @Override + protected SchemaOperator createSchemaOperator( + String jobId, SupportSchemaEvolution source, Config pluginConfig) { + return new SchemaOperator13(jobId, source, pluginConfig); + } + + /** + * Flink 1.13's {@code SinkFunction} does not have a {@code finish()} method, so source + * keep-alive must always be enabled when schema evolution is active. Returns {@code false} + * directly rather than using reflection. + */ + @Override + protected boolean supportsSinkFunctionFinish() { + return false; + } +} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractSourceExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractSourceExecuteProcessor.java new file mode 100644 index 000000000000..158570238a40 --- /dev/null +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractSourceExecuteProcessor.java @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.core.starter.flink.execution; + +import org.apache.seatunnel.shade.com.google.common.collect.Lists; +import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigValueFactory; + +import org.apache.seatunnel.api.common.JobContext; +import org.apache.seatunnel.api.common.PluginIdentifier; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.options.EnvCommonOptions; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.source.SupportSchemaEvolution; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.factory.FactoryUtil; +import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.constants.EngineType; +import org.apache.seatunnel.common.constants.PluginType; +import org.apache.seatunnel.core.starter.execution.SourceTableInfo; +import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelFactoryDiscovery; +import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSourcePluginDiscovery; +import org.apache.seatunnel.translation.flink.schema.SchemaOperator; +import org.apache.seatunnel.translation.flink.source.FlinkSource; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import scala.Tuple2; + +import java.io.Serializable; +import java.net.URL; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.function.Function; + +import static org.apache.seatunnel.api.options.ConnectorCommonOptions.PLUGIN_NAME; +import static org.apache.seatunnel.api.options.ConnectorCommonOptions.PLUGIN_OUTPUT; +import static org.apache.seatunnel.api.table.factory.FactoryUtil.ensureJobModeMatch; +import static org.apache.seatunnel.common.constants.JobMode.STREAMING; + +@SuppressWarnings("unchecked,rawtypes") +public abstract class AbstractSourceExecuteProcessor + extends FlinkAbstractPluginExecuteProcessor { + + private static final String SOURCE_KEEP_ALIVE_CONFIG = "schema-changes.source-keep-alive"; + + protected AbstractSourceExecuteProcessor( + List jarPaths, + Config envConfig, + List pluginConfigs, + JobContext jobContext) { + super(jarPaths, envConfig, pluginConfigs, jobContext); + } + + @Override + public List execute(List upstreamDataStreams) { + StreamExecutionEnvironment executionEnvironment = + flinkRuntimeEnvironment.getStreamExecutionEnvironment(); + List sources = new ArrayList<>(); + for (int i = 0; i < plugins.size(); i++) { + SourceTableInfo sourceTableInfo = plugins.get(i); + SeaTunnelSource internalSource = sourceTableInfo.getSource(); + Config pluginConfig = pluginConfigs.get(i); + + DataStreamSource sourceStream = + executionEnvironment.fromSource( + new FlinkSource<>( + internalSource, + enableSourceKeepAliveIfNeeded( + internalSource, pluginConfig, envConfig)), + WatermarkStrategy.noWatermarks(), + String.format("%s-Source", internalSource.getPluginName())); + + if (pluginConfig.hasPath(EnvCommonOptions.PARALLELISM.key())) { + int parallelism = pluginConfig.getInt(EnvCommonOptions.PARALLELISM.key()); + sourceStream.setParallelism(parallelism); + } + + boolean isStreaming = + envConfig.hasPath("job.mode") + && STREAMING + .toString() + .equalsIgnoreCase(envConfig.getString("job.mode")); + + boolean enableSchemaChange = false; + for (Config cfg : pluginConfigs) { + if (cfg.hasPath("schema-changes.enabled") + && cfg.getBoolean("schema-changes.enabled")) { + enableSchemaChange = true; + break; + } + } + // add schema evolution functionality to cdc source + DataStream evolvedStream = null; + if (isStreaming + && enableSchemaChange + && sourceTableInfo.getSource() instanceof SupportSchemaEvolution) { + evolvedStream = + sourceStream.transform( + "schema-evolution", + TypeInformation.of(SeaTunnelRow.class), + createSchemaOperator( + jobContext.getJobId(), + (SupportSchemaEvolution) sourceTableInfo.getSource(), + pluginConfig)); + } + + if (evolvedStream != null) { + sources.add( + new DataStreamTableInfo( + evolvedStream, + sourceTableInfo.getCatalogTables(), + ReadonlyConfig.fromConfig(pluginConfig).get(PLUGIN_OUTPUT))); + } else { + sources.add( + new DataStreamTableInfo( + sourceStream, + sourceTableInfo.getCatalogTables(), + ReadonlyConfig.fromConfig(pluginConfig).get(PLUGIN_OUTPUT))); + } + } + return sources; + } + + private Config enableSourceKeepAliveIfNeeded( + SeaTunnelSource source, Config pluginConfig, Config currentEnvConfig) { + boolean isStreaming = + currentEnvConfig.hasPath("job.mode") + && STREAMING + .toString() + .equalsIgnoreCase(currentEnvConfig.getString("job.mode")); + boolean enableSchemaChange = + pluginConfig.hasPath("schema-changes.enabled") + && pluginConfig.getBoolean("schema-changes.enabled"); + boolean shouldEnableKeepAlive = + isStreaming + && enableSchemaChange + && source instanceof SupportSchemaEvolution + && !supportsSinkFunctionFinish(); + if (!shouldEnableKeepAlive) { + return currentEnvConfig; + } + return currentEnvConfig.withValue( + SOURCE_KEEP_ALIVE_CONFIG, ConfigValueFactory.fromAnyRef(true)); + } + + /** + * Returns the {@link SchemaOperator} instance to attach after schema-evolution-capable sources. + * Subclasses may override to return a version-specific operator (e.g. {@code SchemaOperator13} + * for Flink 1.13) that uses the public {@code ProcessingTimeService} API instead of reflection. + */ + protected SchemaOperator createSchemaOperator( + String jobId, SupportSchemaEvolution source, Config pluginConfig) { + return new SchemaOperator(jobId, source, pluginConfig); + } + + /** + * Returns {@code true} if the current Flink runtime's {@code SinkFunction} exposes a {@code + * finish()} method (introduced in Flink 1.14). When {@code false}, source keep-alive is enabled + * so pending schema changes can still be applied after all source subtasks finish. Subclasses + * may override with a hard-coded value to avoid reflection. + */ + protected boolean supportsSinkFunctionFinish() { + for (java.lang.reflect.Method method : + org.apache.flink.streaming.api.functions.sink.SinkFunction.class.getMethods()) { + if ("finish".equals(method.getName()) && method.getParameterCount() == 0) { + return true; + } + } + return false; + } + + @Override + protected List initializePlugins( + List jarPaths, List pluginConfigs) { + SeaTunnelFactoryDiscovery factoryDiscovery = + new SeaTunnelFactoryDiscovery(TableSourceFactory.class, ADD_URL_TO_CLASSLOADER); + SeaTunnelSourcePluginDiscovery sourcePluginDiscovery = + new SeaTunnelSourcePluginDiscovery(ADD_URL_TO_CLASSLOADER); + Function fallbackCreateSource = + sourcePluginDiscovery::createPluginInstance; + + List sources = new ArrayList<>(); + Set jars = new HashSet<>(); + for (Config sourceConfig : pluginConfigs) { + PluginIdentifier pluginIdentifier = + PluginIdentifier.of( + EngineType.SEATUNNEL.getEngine(), + PluginType.SOURCE.getType(), + sourceConfig.getString(PLUGIN_NAME.key())); + jars.addAll( + sourcePluginDiscovery.getPluginJarAndDependencyPaths( + Lists.newArrayList(pluginIdentifier))); + + Tuple2, List> source = + FactoryUtil.createAndPrepareSource( + ReadonlyConfig.fromConfig(sourceConfig), + classLoader, + pluginIdentifier.getPluginName(), + fallbackCreateSource, + (TableSourceFactory) + factoryDiscovery + .createOptionalPluginInstance(pluginIdentifier) + .orElse(null), + null); + + source._1().setJobContext(jobContext); + ensureJobModeMatch(jobContext, source._1()); + + sources.add(new SourceTableInfo(source._1(), source._2())); + } + jarPaths.addAll(jars); + return sources; + } +} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java index ef7ea9ac632a..6cd1efcf2050 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java @@ -17,54 +17,22 @@ package org.apache.seatunnel.core.starter.flink.execution; -import org.apache.seatunnel.shade.com.google.common.collect.Lists; import org.apache.seatunnel.shade.com.typesafe.config.Config; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigValueFactory; import org.apache.seatunnel.api.common.JobContext; -import org.apache.seatunnel.api.common.PluginIdentifier; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.options.EnvCommonOptions; -import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.source.SourceSplit; -import org.apache.seatunnel.api.source.SupportSchemaEvolution; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.factory.FactoryUtil; -import org.apache.seatunnel.api.table.factory.TableSourceFactory; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.common.constants.EngineType; -import org.apache.seatunnel.common.constants.PluginType; -import org.apache.seatunnel.core.starter.execution.SourceTableInfo; -import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelFactoryDiscovery; -import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSourcePluginDiscovery; -import org.apache.seatunnel.translation.flink.schema.SchemaOperator; -import org.apache.seatunnel.translation.flink.source.FlinkSource; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; - -import scala.Tuple2; - -import java.io.Serializable; import java.net.URL; -import java.util.ArrayList; -import java.util.HashSet; import java.util.List; -import java.util.Set; -import java.util.function.Function; - -import static org.apache.seatunnel.api.options.ConnectorCommonOptions.PLUGIN_NAME; -import static org.apache.seatunnel.api.options.ConnectorCommonOptions.PLUGIN_OUTPUT; -import static org.apache.seatunnel.api.table.factory.FactoryUtil.ensureJobModeMatch; -import static org.apache.seatunnel.common.constants.JobMode.STREAMING; +/** + * Default (Flink 1.15+) source execution processor. Delegates entirely to {@link + * AbstractSourceExecuteProcessor}. For Flink 1.13, this class is shadowed at runtime by the version + * in {@code seatunnel-flink-13-starter}, which overrides {@link #createSchemaOperator} and {@link + * #supportsSinkFunctionFinish} with strongly-typed Flink 1.13 implementations that avoid + * reflection. + */ @SuppressWarnings("unchecked,rawtypes") -public class SourceExecuteProcessor extends FlinkAbstractPluginExecuteProcessor { - - private static final String SOURCE_KEEP_ALIVE_CONFIG = "schema-changes.source-keep-alive"; +public class SourceExecuteProcessor extends AbstractSourceExecuteProcessor { public SourceExecuteProcessor( List jarPaths, @@ -73,149 +41,4 @@ public SourceExecuteProcessor( JobContext jobContext) { super(jarPaths, envConfig, pluginConfigs, jobContext); } - - @Override - public List execute(List upstreamDataStreams) { - StreamExecutionEnvironment executionEnvironment = - flinkRuntimeEnvironment.getStreamExecutionEnvironment(); - List sources = new ArrayList<>(); - for (int i = 0; i < plugins.size(); i++) { - SourceTableInfo sourceTableInfo = plugins.get(i); - SeaTunnelSource internalSource = sourceTableInfo.getSource(); - Config pluginConfig = pluginConfigs.get(i); - - DataStreamSource sourceStream = - executionEnvironment.fromSource( - new FlinkSource<>( - internalSource, - enableSourceKeepAliveIfNeeded( - internalSource, pluginConfig, envConfig)), - WatermarkStrategy.noWatermarks(), - String.format("%s-Source", internalSource.getPluginName())); - - if (pluginConfig.hasPath(EnvCommonOptions.PARALLELISM.key())) { - int parallelism = pluginConfig.getInt(EnvCommonOptions.PARALLELISM.key()); - sourceStream.setParallelism(parallelism); - } - - boolean isStreaming = - envConfig.hasPath("job.mode") - && STREAMING - .toString() - .equalsIgnoreCase(envConfig.getString("job.mode")); - - boolean enableSchemaChange = false; - for (Config cfg : pluginConfigs) { - if (cfg.hasPath("schema-changes.enabled") - && cfg.getBoolean("schema-changes.enabled")) { - enableSchemaChange = true; - break; - } - } - // add schema evolution functionality to cdc source - DataStream evolvedStream = null; - if (isStreaming - && enableSchemaChange - && sourceTableInfo.getSource() instanceof SupportSchemaEvolution) { - evolvedStream = - sourceStream.transform( - "schema-evolution", - TypeInformation.of(SeaTunnelRow.class), - new SchemaOperator( - jobContext.getJobId(), - (SupportSchemaEvolution) sourceTableInfo.getSource(), - pluginConfig)); - } - - if (evolvedStream != null) { - sources.add( - new DataStreamTableInfo( - evolvedStream, - sourceTableInfo.getCatalogTables(), - ReadonlyConfig.fromConfig(pluginConfig).get(PLUGIN_OUTPUT))); - } else { - sources.add( - new DataStreamTableInfo( - sourceStream, - sourceTableInfo.getCatalogTables(), - ReadonlyConfig.fromConfig(pluginConfig).get(PLUGIN_OUTPUT))); - } - } - return sources; - } - - private Config enableSourceKeepAliveIfNeeded( - SeaTunnelSource source, Config pluginConfig, Config currentEnvConfig) { - boolean isStreaming = - currentEnvConfig.hasPath("job.mode") - && STREAMING - .toString() - .equalsIgnoreCase(currentEnvConfig.getString("job.mode")); - boolean enableSchemaChange = - pluginConfig.hasPath("schema-changes.enabled") - && pluginConfig.getBoolean("schema-changes.enabled"); - boolean shouldEnableKeepAlive = - isStreaming - && enableSchemaChange - && source instanceof SupportSchemaEvolution - && !supportsSinkFunctionFinish(); - if (!shouldEnableKeepAlive) { - return currentEnvConfig; - } - return currentEnvConfig.withValue( - SOURCE_KEEP_ALIVE_CONFIG, ConfigValueFactory.fromAnyRef(true)); - } - - private boolean supportsSinkFunctionFinish() { - for (java.lang.reflect.Method method : - org.apache.flink.streaming.api.functions.sink.SinkFunction.class.getMethods()) { - if ("finish".equals(method.getName()) && method.getParameterCount() == 0) { - return true; - } - } - return false; - } - - @Override - protected List initializePlugins( - List jarPaths, List pluginConfigs) { - SeaTunnelFactoryDiscovery factoryDiscovery = - new SeaTunnelFactoryDiscovery(TableSourceFactory.class, ADD_URL_TO_CLASSLOADER); - SeaTunnelSourcePluginDiscovery sourcePluginDiscovery = - new SeaTunnelSourcePluginDiscovery(ADD_URL_TO_CLASSLOADER); - Function fallbackCreateSource = - sourcePluginDiscovery::createPluginInstance; - - List sources = new ArrayList<>(); - Set jars = new HashSet<>(); - for (Config sourceConfig : pluginConfigs) { - PluginIdentifier pluginIdentifier = - PluginIdentifier.of( - EngineType.SEATUNNEL.getEngine(), - PluginType.SOURCE.getType(), - sourceConfig.getString(PLUGIN_NAME.key())); - jars.addAll( - sourcePluginDiscovery.getPluginJarAndDependencyPaths( - Lists.newArrayList(pluginIdentifier))); - - Tuple2, List> source = - FactoryUtil.createAndPrepareSource( - ReadonlyConfig.fromConfig(sourceConfig), - classLoader, - pluginIdentifier.getPluginName(), - fallbackCreateSource, - (TableSourceFactory) - factoryDiscovery - .createOptionalPluginInstance(pluginIdentifier) - .orElse(null), - null); - - source._1().setJobContext(jobContext); - ensureJobModeMatch(jobContext, source._1()); - - sources.add(new SourceTableInfo(source._1(), source._2())); - } - jarPaths.addAll(jars); - return sources; - } } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-13/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator13.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-13/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator13.java new file mode 100644 index 000000000000..3430c46b6188 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-13/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator13.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.translation.flink.schema; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import org.apache.seatunnel.api.source.SupportSchemaEvolution; + +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; + +import lombok.extern.slf4j.Slf4j; + +/** + * Flink 1.13-specific extension of {@link SchemaOperator} that resolves two issues present when the + * fallback timer is placed in the common module: + * + *
    + *
  1. No reflection: {@link ProcessingTimeService} and {@link ProcessingTimeCallback} are + * imported directly as strongly-typed Flink 1.13 APIs. There is no risk of silent breakage + * from method renames in future Flink versions. + *
  2. No dead flag path: the timer callback fires on the Flink task thread via + * {@code ProcessingTimeService.registerTimer}, so {@link #handleFallbackTimerOnTaskThread()} + * is always reachable even when no more source data arrives and {@code processElement} is + * never called again. This is the exact scenario this workaround targets on Flink 1.13. + *
+ * + *

The base {@link SchemaOperator} carries none of this timer infrastructure; Flink 1.15 and + * later use that base class directly because checkpointing behaves correctly there. + */ +@Slf4j +public class SchemaOperator13 extends SchemaOperator { + + /** + * Guards against double-registration. All accesses happen on the Flink task thread + * (processElement, timer callbacks, notifyCheckpointComplete) + */ + private boolean fallbackTimerPending = false; + + public SchemaOperator13(String jobId, SupportSchemaEvolution source, Config pluginConfig) { + super(jobId, source, pluginConfig); + } + + /** + * Registers a processing-time timer that will call {@link #handleFallbackTimerOnTaskThread()} + * on the Flink task thread after {@link #CHECKPOINT_STALL_TIMEOUT_MS} milliseconds. + * + *

Using {@link ProcessingTimeService#registerTimer} instead of a background {@code + * ScheduledExecutorService} achieves two goals: + * + *

    + *
  • The callback is delivered on the task thread, so {@code output.collect} and operator + * state are accessed safely without additional synchronisation. + *
  • No daemon thread overhead is introduced for Flink 1.14+ users who use the common + * module's no-op default. + *
+ * + *

If a timer is already pending this call is a no-op to prevent duplicate firings. + */ + @Override + protected void scheduleFallbackTimer() { + if (fallbackTimerPending) { + return; + } + fallbackTimerPending = true; + + ProcessingTimeService pts = getProcessingTimeService(); + long fireAt = pts.getCurrentProcessingTime() + CHECKPOINT_STALL_TIMEOUT_MS; + + pts.registerTimer( + fireAt, + (ProcessingTimeCallback) + timestamp -> { + fallbackTimerPending = false; + try { + handleFallbackTimerOnTaskThread(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + log.error( + "Fallback schema-change timer interrupted for job {}", + jobId, + e); + } + }); + + log.debug( + "Registered Flink processing-time fallback timer to fire in {}ms for job {}", + CHECKPOINT_STALL_TIMEOUT_MS, + jobId); + } +} diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java index f093a977ef37..0ce26072b3fa 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java @@ -1,12 +1,12 @@ /* * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -44,24 +44,17 @@ import lombok.extern.slf4j.Slf4j; import java.io.Serializable; -import java.lang.reflect.InvocationHandler; -import java.lang.reflect.Method; -import java.lang.reflect.Proxy; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Queue; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; /** * Operator placed after the source to handle schema evolution. * - *

schema change events are NOT processed synchronously in {@link #processElement}. Instead, they + *

Schema change events are NOT processed synchronously in {@link #processElement}. Instead, they * are buffered and deferred until an additional checkpoint cycle has completed after the first * checkpoint that observed the pending DDL. This wait ensures that when the sink executes ALTER * TABLE, all XA transactions from prior checkpoint cycles have been fully committed by the {@code @@ -72,10 +65,12 @@ *

Per checkpoint cycle, at most ONE schema change is applied. If multiple DDLs arrive between * two checkpoints, they are processed across successive checkpoint cycles. * - *

Flink 1.13 cannot continue checkpointing after some source subtasks have finished. When high - * parallelism CDC jobs hit that condition, pending schema changes would otherwise stay blocked - * forever. A lightweight fallback timer detects the checkpoint stall and re-enters the task thread - * through Flink's processing-time service so the deferred DDL can still be applied safely. + *

Flink 1.13 cannot continue checkpointing after some source subtasks have finished. When + * high-parallelism CDC jobs hit that condition, pending schema changes would otherwise stay blocked + * forever. Subclasses may override {@link #scheduleFallbackTimer()} to register a version-specific + * timer that detects the stall and re-enters the task thread via {@link + * #handleFallbackTimerOnTaskThread()} so the deferred DDL can still be applied safely. The base + * implementation is a no-op, keeping the common module free of version-specific overhead. */ @Slf4j public class SchemaOperator extends AbstractStreamOperator @@ -84,10 +79,15 @@ public class SchemaOperator extends AbstractStreamOperator private static final int MAX_BUFFERED_RECORDS = 100000; private static final long SCHEMA_CHANGE_TIMEOUT_MS = 300_000L; private static final int CHECKPOINT_WAIT_ROUNDS = 1; - private static final long CHECKPOINT_STALL_TIMEOUT_MS = 15_000L; + + /** Exposed to subclasses so version-specific fallback timers can use the same threshold. */ + protected static final long CHECKPOINT_STALL_TIMEOUT_MS = 15_000L; private final Map localSchemaState; - private String jobId; + + /** Exposed to subclasses for logging only. */ + protected String jobId; + private final SupportSchemaEvolution source; private final Config pluginConfig; private volatile Long lastProcessedEventTime; @@ -95,10 +95,13 @@ public class SchemaOperator extends AbstractStreamOperator private transient Queue pendingQueue; private volatile boolean schemaChangePending = false; private long firstSeenCheckpointId = -1L; - private transient ScheduledExecutorService fallbackScheduler; - private transient volatile ScheduledFuture pendingFallbackFuture; - private volatile long lastCheckpointCompletedMs = -1L; - private volatile boolean fallbackTimerFired = false; + + /** + * Timestamp of the most recently completed checkpoint. Updated in {@link + * #notifyCheckpointComplete} and read by {@link #handleFallbackTimerOnTaskThread} to detect + * whether checkpoints have stalled. + */ + protected volatile long lastCheckpointCompletedMs = -1L; private transient ListState localSchemaStateStore; private transient ListState lastProcessedEventTimeState; @@ -124,13 +127,6 @@ public void open() throws Exception { this.pendingQueue = new LinkedList<>(); } this.coordinator = LocalSchemaCoordinator.getInstance(this.jobId); - this.fallbackScheduler = - Executors.newSingleThreadScheduledExecutor( - runnable -> { - Thread thread = new Thread(runnable, "schema-fallback-timer-" + jobId); - thread.setDaemon(true); - return thread; - }); log.info( "SchemaOperator opened for job: {}, schemaChangePending: {}, pendingQueue size: {}", @@ -149,10 +145,6 @@ public void processElement(StreamRecord streamRecord) return; } - if (fallbackTimerFired) { - handleFallbackTimerOnTaskThread(); - } - // detect schema change events if ("__SCHEMA_CHANGE_EVENT__".equals(element.getTableId()) && element.getOptions() != null) { @@ -227,12 +219,12 @@ private TableIdentifier getPendingSchemaTableIdentifier() { * ensure safety: * *

    - *
  • first time seeing the DDL: record {@link #firstSeenCheckpointId} but do NOT - * broadcast the DDL yet. At this point the {@code FlinkGlobalCommitter} may still be - * running {@code XA COMMIT} for this checkpoint's prepared transactions, holding MDL - * locks on the sink table. - *
  • {@code checkpointId >= firstSeenCheckpointId + CHECKPOINT_WAIT_ROUNDS} : the XA - * COMMIT from the earlier checkpoint cycle is guaranteed to have finished (at least one + *
  • First time seeing the DDL: record {@link #firstSeenCheckpointId} but do NOT broadcast + * the DDL yet. At this point the {@code FlinkGlobalCommitter} may still be running {@code + * XA COMMIT} for this checkpoint's prepared transactions, holding MDL locks on the sink + * table. + *
  • {@code checkpointId >= firstSeenCheckpointId + CHECKPOINT_WAIT_ROUNDS}: the XA COMMIT + * from the earlier checkpoint cycle is guaranteed to have finished (at least one * additional checkpoint cycle has completed, which implies the committer ran). The sink's * ALTER TABLE will not encounter MDL lock, it is now safe to broadcast the DDL. *
@@ -294,21 +286,17 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { } /** - * Handles fallback timer firing on the task thread (Flink 1.13 checkpoint stall workaround). + * Handles a checkpoint-stall fallback on the task thread. Must be called from the Flink task + * thread (e.g. via {@code ProcessingTimeService.registerTimer} callback) to keep {@code + * output.collect} and operator state accesses thread-safe. * - *

IMPORTANT: This method still respects the checkpoint-completion safety fence. The DDL is - * only applied if at least one checkpoint has completed after the schema event was detected - * ({@code firstSeenCheckpointId >= 0}). This ensures XA transactions from the earlier - * checkpoint cycle have finished before ALTER TABLE runs. - * - *

If no checkpoint has completed yet, we reschedule the fallback timer and wait. The - * fallback only kicks in when checkpoints stall AFTER the first post-DDL checkpoint has - * completed, which is the scenario in Flink 1.13 high-parallelism CDC jobs where some source - * subtasks finish before others. + *

Safety fence: the DDL is applied only when at least one checkpoint has already completed + * after the schema event ({@code firstSeenCheckpointId >= 0}). This preserves the guarantee + * that XA transactions from the earlier checkpoint cycle have finished before ALTER TABLE runs. + * If that fence has not been crossed yet, the fallback reschedules itself by calling {@link + * #scheduleFallbackTimer()} and returns without applying anything. */ - private void handleFallbackTimerOnTaskThread() throws InterruptedException { - fallbackTimerFired = false; - + protected void handleFallbackTimerOnTaskThread() throws InterruptedException { if (!schemaChangePending || pendingQueue.isEmpty()) { return; } @@ -348,71 +336,16 @@ private void handleFallbackTimerOnTaskThread() throws InterruptedException { applyNextPendingSchemaChange(); } - private void scheduleFallbackTimer() { - if (fallbackScheduler == null || fallbackScheduler.isShutdown()) { - return; - } - ScheduledFuture existing = pendingFallbackFuture; - if (existing != null && !existing.isDone()) { - return; - } - pendingFallbackFuture = - fallbackScheduler.schedule( - () -> { - fallbackTimerFired = true; - registerProcessingTimeCallback(); - }, - CHECKPOINT_STALL_TIMEOUT_MS, - TimeUnit.MILLISECONDS); - } - - private void registerProcessingTimeCallback() { - try { - Object processingTimeService = getProcessingTimeService(); - Method getCurrentTimeMethod = null; - Method registerMethod = null; - Class callbackClass = null; - - for (Method method : processingTimeService.getClass().getMethods()) { - if ("getCurrentProcessingTime".equals(method.getName()) - && method.getParameterTypes().length == 0) { - getCurrentTimeMethod = method; - } else if ("registerTimer".equals(method.getName()) - && method.getParameterTypes().length == 2 - && method.getParameterTypes()[0] == long.class) { - registerMethod = method; - callbackClass = method.getParameterTypes()[1]; - } - } - - if (getCurrentTimeMethod == null || registerMethod == null || callbackClass == null) { - log.warn( - "Could not find required ProcessingTimeService methods, " - + "falling back to flag-based handling."); - return; - } - - getCurrentTimeMethod.setAccessible(true); - registerMethod.setAccessible(true); - long now = (long) getCurrentTimeMethod.invoke(processingTimeService); - - InvocationHandler handler = - (proxy, method, args) -> { - if ("onProcessingTime".equals(method.getName())) { - handleFallbackTimerOnTaskThread(); - } - return null; - }; - Object callbackProxy = - Proxy.newProxyInstance( - callbackClass.getClassLoader(), - new Class[] {callbackClass}, - handler); - - registerMethod.invoke(processingTimeService, now + 1, callbackProxy); - } catch (Exception e) { - log.warn("Failed to register processing time callback for fallback timer", e); - } + /** + * Schedules a fallback timer that will call {@link #handleFallbackTimerOnTaskThread()} if + * checkpoints stall before the pending schema change can be applied. + * + *

The base implementation is a no-op: version-specific subclasses (e.g. {@code + * SchemaOperator13}) override this to register a timer via {@code ProcessingTimeService}, + * keeping the common module free of version-specific timer infrastructure and reflection. + */ + protected void scheduleFallbackTimer() { + // no-op by default; overridden in version-specific subclasses } private BufferedRecord advancePastDataRecords() { @@ -647,14 +580,6 @@ private void sendSchemaChangeEventToDownstream(SchemaChangeEvent schemaChangeEve output.collect(new StreamRecord<>(broadcastRow)); } - @Override - public void close() throws Exception { - if (fallbackScheduler != null && !fallbackScheduler.isShutdown()) { - fallbackScheduler.shutdownNow(); - } - super.close(); - } - static class BufferedRecord { final boolean isSchemaChange; final SeaTunnelRow row; diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/coordinator/LocalSchemaCoordinator.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/coordinator/LocalSchemaCoordinator.java index 4edf628c539b..3cd96d43dc6f 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/coordinator/LocalSchemaCoordinator.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/coordinator/LocalSchemaCoordinator.java @@ -239,6 +239,14 @@ public boolean requestSchemaChange(TableIdentifier tableId, long epoch, long tim // Schema changes (DDL) are database-level operations that only need to execute once. // Due to Flink's partitioning, only one subtask receives the schema change event, // so we only need 1 ACK to confirm the DDL was applied successfully. + // + // Precondition: sink subtasks that do NOT receive the schema-change event directly + // (because Flink's partitioning routed it elsewhere) must have their local schema + // view refreshed through BroadcastSchemaSinkOperator's broadcast/state path. + // If that broadcast path is incomplete, those subtasks will silently apply the old + // schema to new-format rows — a data-corruption risk. Any change to the broadcast + // path must preserve this invariant, and a multi-table (≥2 tables, parallelism ≥2) + // E2E test should guard it so regressions are caught immediately. int expectedAcks = 1; log.info( "Requesting schema change for table {} (epoch {}). Waiting for at least {} of {} " diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java index f62728b4ef10..f1873556963c 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/schema/SchemaOperatorTest.java @@ -176,6 +176,18 @@ void testCoordinationFailureKeepsBufferedRecordsBlocked() throws Exception { assertTrue(pendingQueue.peek().isSchemaChange); } + /** + * Verifies that {@link SchemaOperator#handleFallbackTimerOnTaskThread()} correctly respects the + * checkpoint-completion safety fence even when called from a stall-detection timer. + * + *

The test invokes the handler directly (as if a processing-time timer fired) to keep the + * unit test independent of Flink's timer infrastructure. In production, the handler is called + * by {@link SchemaOperator13#scheduleFallbackTimer()} via {@code + * ProcessingTimeService.registerTimer}. + * + *

The base {@link SchemaOperator#scheduleFallbackTimer()} is a no-op; this test verifies + * only the handler logic, not the scheduling mechanism. + */ @Test void testFallbackTimerRespectsCheckpointSafetyFence() throws Exception { LocalSchemaCoordinator coordinator = Mockito.mock(LocalSchemaCoordinator.class); @@ -193,9 +205,9 @@ void testFallbackTimerRespectsCheckpointSafetyFence() throws Exception { context.operator.processElement(new StreamRecord<>(createSchemaRow(event), 400L)); context.operator.processElement(new StreamRecord<>(row, 401L)); - // Fallback timer fires but no checkpoint has completed yet (firstSeenCheckpointId < 0). - // The fallback should NOT apply DDL — it must reschedule itself to preserve the - // checkpoint-completion safety fence that protects against XA/MDL conflicts. + // Simulate timer firing before any checkpoint has completed (firstSeenCheckpointId < 0). + // The handler must NOT apply the DDL — it must call scheduleFallbackTimer() to wait for + // the checkpoint-completion safety fence (guards XA/MDL conflicts). invokeNoArgMethod(context.operator, "handleFallbackTimerOnTaskThread"); assertTrue(context.output.records.isEmpty()); @@ -204,7 +216,8 @@ void testFallbackTimerRespectsCheckpointSafetyFence() throws Exception { assertEquals(-1L, getLongField(context.operator, "firstSeenCheckpointId")); Mockito.verifyNoInteractions(coordinator); - // Now complete the first checkpoint to set firstSeenCheckpointId. + // Complete the first post-DDL checkpoint — sets firstSeenCheckpointId, not yet safe to + // apply (need one additional round, so notifyCheckpointComplete stops here). context.operator.notifyCheckpointComplete(40L); assertTrue(context.output.records.isEmpty()); @@ -212,17 +225,16 @@ void testFallbackTimerRespectsCheckpointSafetyFence() throws Exception { assertTrue(getBooleanField(context.operator, "schemaChangePending")); Mockito.verifyNoInteractions(coordinator); - // Simulate checkpoint stall: set lastCheckpointCompletedMs to a time in the past - // (older than CHECKPOINT_STALL_TIMEOUT_MS = 15_000L). This simulates Flink 1.13 - // high-parallelism CDC jobs where checkpoints stop after some source subtasks finish. + // Simulate checkpoint stall: move lastCheckpointCompletedMs into the past beyond + // CHECKPOINT_STALL_TIMEOUT_MS (15 s). This mirrors the Flink 1.13 behaviour where + // high-parallelism CDC jobs stop checkpointing after some source subtasks finish. setField( context.operator, "lastCheckpointCompletedMs", System.currentTimeMillis() - 20_000L); - // Fallback timer fires again. Now firstSeenCheckpointId >= 0 and stall detected, - // so the DDL can be applied (checkpoint safety fence is respected since at least one - // checkpoint completed after the DDL was detected). + // Simulate timer firing again. firstSeenCheckpointId >= 0 and checkpoint has stalled, + // so the safety fence is satisfied — the DDL can now be applied. invokeNoArgMethod(context.operator, "handleFallbackTimerOnTaskThread"); assertEquals(2, context.output.records.size()); From d69d5951a8129082ca4f98d1d9d934b40c20aca6 Mon Sep 17 00:00:00 2001 From: CloverDew Date: Wed, 3 Jun 2026 21:00:57 +0800 Subject: [PATCH 7/7] fix ci --- .../MysqlCDCWithFlinkSchemaChangeIT.java | 43 --------------- ...cdc_to_mysql_with_flink_schema_change.conf | 2 +- ...with_flink_schema_change_exactly_once.conf | 2 +- ...ma_change_exactly_once_multi_parallel.conf | 55 ------------------- ...th_flink_schema_change_multi_parallel.conf | 53 ------------------ .../translation/flink/sink/FlinkSink.java | 16 +----- .../flink/sink/FlinkSinkWriter.java | 37 +------------ .../flink/sink/FlinkSinkWriterTest.java | 22 +------- 8 files changed, 8 insertions(+), 222 deletions(-) delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once_multi_parallel.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_multi_parallel.conf diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCWithFlinkSchemaChangeIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCWithFlinkSchemaChangeIT.java index ec7dc4e67fc2..37fea77fe210 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCWithFlinkSchemaChangeIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/MysqlCDCWithFlinkSchemaChangeIT.java @@ -167,49 +167,6 @@ public void testMysqlCdcWithSchemaEvolutionCaseExactlyOnce(TestContainer contain assertSchemaEvolution(MYSQL_DATABASE, SOURCE_TABLE, SINK_TABLE2); } - @Order(3) - @TestTemplate - public void testMysqlCdcWithSchemaEvolutionCaseMultiParallel(TestContainer container) { - resetDatabaseToInitialState(); - - CompletableFuture.runAsync( - () -> { - try { - container.executeJob( - "/mysqlcdc_to_mysql_with_flink_schema_change_multi_parallel.conf"); - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - }); - - assertSchemaEvolutionForAddColumns(MYSQL_DATABASE, SOURCE_TABLE, SINK_TABLE); - shopDatabase.setTemplateName("drop_columns").createAndInitialize(); - assertTableStructureAndData(MYSQL_DATABASE, SOURCE_TABLE, SINK_TABLE); - shopDatabase.setTemplateName("change_columns").createAndInitialize(); - shopDatabase.setTemplateName("modify_columns").createAndInitialize(); - assertTableStructureAndData(MYSQL_DATABASE, SOURCE_TABLE, SINK_TABLE); - } - - @Order(4) - @TestTemplate - public void testMysqlCdcWithSchemaEvolutionCaseExactlyOnceMultiParallel( - TestContainer container) { - shopDatabase.setTemplateName("shop").createAndInitialize(); - CompletableFuture.runAsync( - () -> { - try { - container.executeJob( - "/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once_multi_parallel.conf"); - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - }); - - assertSchemaEvolution(MYSQL_DATABASE, SOURCE_TABLE, SINK_TABLE2); - } - private void assertSchemaEvolution(String database, String sourceTable, String sinkTable) { await().atMost(180000, TimeUnit.MILLISECONDS) .untilAsserted( diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change.conf index 9dd595f54785..7ffbf1c1c614 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change.conf @@ -20,7 +20,7 @@ env { # You can set engine configuration here - parallelism = 1 + parallelism = 5 job.mode = "STREAMING" checkpoint.interval = 5000 read_limit.bytes_per_second=7000000 diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once.conf index 09c88f312068..69816a41b9b6 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once.conf @@ -20,7 +20,7 @@ env { # You can set engine configuration here - parallelism = 1 + parallelism = 5 job.mode = "STREAMING" checkpoint.interval = 5000 read_limit.bytes_per_second=7000000 diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once_multi_parallel.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once_multi_parallel.conf deleted file mode 100644 index 69816a41b9b6..000000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_exactly_once_multi_parallel.conf +++ /dev/null @@ -1,55 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - # You can set engine configuration here - parallelism = 5 - job.mode = "STREAMING" - checkpoint.interval = 5000 - read_limit.bytes_per_second=7000000 - read_limit.rows_per_second=400 -} - -source { - MySQL-CDC { - server-id = 5652-5657 - username = "st_user_source" - password = "mysqlpw" - table-names = ["shop.products"] - url = "jdbc:mysql://mysql_cdc_e2e:3306/shop" - - schema-changes.enabled = true - } -} - -sink { - jdbc { - url = "jdbc:mysql://mysql_cdc_e2e:3306/shop" - driver = "com.mysql.cj.jdbc.Driver" - user = "st_user_sink" - password = "mysqlpw" - generate_sink_sql = true - database = shop - table = mysql_cdc_e2e_sink_table_with_schema_change_exactly_once - primary_keys = ["id"] - is_exactly_once = true - xa_data_source_class_name = "com.mysql.cj.jdbc.MysqlXADataSource" - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_multi_parallel.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_multi_parallel.conf deleted file mode 100644 index 7ffbf1c1c614..000000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql_with_flink_schema_change_multi_parallel.conf +++ /dev/null @@ -1,53 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - # You can set engine configuration here - parallelism = 5 - job.mode = "STREAMING" - checkpoint.interval = 5000 - read_limit.bytes_per_second=7000000 - read_limit.rows_per_second=400 -} - -source { - MySQL-CDC { - server-id = 5652-5657 - username = "st_user_source" - password = "mysqlpw" - table-names = ["shop.products"] - url = "jdbc:mysql://mysql_cdc_e2e:3306/shop" - - schema-changes.enabled = true - } -} - -sink { - jdbc { - url = "jdbc:mysql://mysql_cdc_e2e:3306/shop" - driver = "com.mysql.cj.jdbc.Driver" - user = "st_user_sink" - password = "mysqlpw" - generate_sink_sql = true - database = shop - table = mysql_cdc_e2e_sink_table_with_schema_change - primary_keys = ["id"] - } -} diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java index e76345eed1c8..f961b67b9266 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java @@ -18,8 +18,6 @@ package org.apache.seatunnel.translation.flink.sink; import org.apache.seatunnel.api.sink.SeaTunnelSink; -import org.apache.seatunnel.api.sink.SinkAggregatedCommitter; -import org.apache.seatunnel.api.sink.SinkCommitter; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.translation.flink.serialization.CommitWrapperSerializer; @@ -82,25 +80,15 @@ public SinkWriter, FlinkWriterState> throws IOException { org.apache.seatunnel.api.sink.SinkWriter.Context stContext = new FlinkSinkWriterContext(context, parallelism); - SinkCommitter sinkCommitter = sink.createCommitter().orElse(null); - SinkAggregatedCommitter sinkAggregatedCommitter = - sinkCommitter == null ? sink.createAggregatedCommitter().orElse(null) : null; if (states == null || states.isEmpty()) { - return new FlinkSinkWriter<>( - sink.createWriter(stContext), - 1, - stContext, - sinkCommitter, - sinkAggregatedCommitter); + return new FlinkSinkWriter<>(sink.createWriter(stContext), 1, stContext); } else { List restoredState = states.stream().map(FlinkWriterState::getState).collect(Collectors.toList()); return new FlinkSinkWriter<>( sink.restoreWriter(stContext, restoredState), states.get(0).getCheckpointId() + 1, - stContext, - sinkCommitter, - sinkAggregatedCommitter); + stContext); } } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java index f75b9d539783..f776520acded 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java @@ -22,8 +22,6 @@ import org.apache.seatunnel.api.common.metrics.MetricNames; import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.sink.MultiTableResourceManager; -import org.apache.seatunnel.api.sink.SinkAggregatedCommitter; -import org.apache.seatunnel.api.sink.SinkCommitter; import org.apache.seatunnel.api.sink.SupportResourceShare; import org.apache.seatunnel.api.sink.SupportSchemaEvolutionSinkWriter; import org.apache.seatunnel.api.sink.event.WriterCloseEvent; @@ -70,14 +68,8 @@ public class FlinkSinkWriter private long checkpointId; - private final SinkCommitter sinkCommitter; - - private final SinkAggregatedCommitter sinkAggregatedCommitter; - private MultiTableResourceManager resourceManager; - private MultiTableResourceManager aggregatedCommitterResourceManager; - /** * Cached writer states produced together with {@link #prepareCommit(boolean)}. * @@ -94,17 +86,10 @@ public class FlinkSinkWriter FlinkSinkWriter( org.apache.seatunnel.api.sink.SinkWriter sinkWriter, long checkpointId, - org.apache.seatunnel.api.sink.SinkWriter.Context context, - SinkCommitter sinkCommitter, - SinkAggregatedCommitter sinkAggregatedCommitter) { + org.apache.seatunnel.api.sink.SinkWriter.Context context) { this.context = context; this.sinkWriter = sinkWriter; this.checkpointId = checkpointId; - this.sinkCommitter = sinkCommitter; - this.sinkAggregatedCommitter = - sinkAggregatedCommitter == null - ? null - : (SinkAggregatedCommitter) sinkAggregatedCommitter; MetricsContext metricsContext = context.getMetricsContext(); this.sinkWriteCount = metricsContext.counter(MetricNames.SINK_WRITE_COUNT); this.sinkWriteBytes = metricsContext.counter(MetricNames.SINK_WRITE_BYTES); @@ -114,16 +99,6 @@ public class FlinkSinkWriter ((SupportResourceShare) sinkWriter).initMultiTableResourceManager(1, 1); ((SupportResourceShare) sinkWriter).setMultiTableResourceManager(resourceManager, 0); } - if (this.sinkAggregatedCommitter != null) { - this.sinkAggregatedCommitter.init(); - if (this.sinkAggregatedCommitter instanceof SupportResourceShare) { - aggregatedCommitterResourceManager = - ((SupportResourceShare) this.sinkAggregatedCommitter) - .initMultiTableResourceManager(1, 1); - ((SupportResourceShare) this.sinkAggregatedCommitter) - .setMultiTableResourceManager(aggregatedCommitterResourceManager, 0); - } - } } @Override @@ -293,15 +268,5 @@ public void close() throws Exception { } catch (Throwable e) { log.error("close resourceManager error", e); } - try { - if (sinkAggregatedCommitter != null) { - sinkAggregatedCommitter.close(); - } - if (aggregatedCommitterResourceManager != null) { - aggregatedCommitterResourceManager.close(); - } - } catch (Throwable e) { - log.error("close aggregated committer resource error", e); - } } } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java index 2cc9b1b79243..943d48b43d8f 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/test/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriterTest.java @@ -19,7 +19,6 @@ import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.event.EventListener; -import org.apache.seatunnel.api.sink.SinkCommitter; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportSchemaEvolutionSinkWriter; import org.apache.seatunnel.api.table.catalog.PhysicalColumn; @@ -46,7 +45,7 @@ void testPrepareCommitSnapshotsStateAndAdvancesCheckpoint() throws Exception { RecordingContext context = new RecordingContext(); FlinkSinkWriter flinkSinkWriter = - new FlinkSinkWriter<>(delegate, 1L, context, null, null); + new FlinkSinkWriter<>(delegate, 1L, context); // first checkpoint List> commits = flinkSinkWriter.prepareCommit(false); @@ -79,7 +78,7 @@ void testSnapshotStateWithoutPrepareCommitFallsBack() throws Exception { RecordingContext context = new RecordingContext(); FlinkSinkWriter flinkSinkWriter = - new FlinkSinkWriter<>(delegate, 3L, context, null, null); + new FlinkSinkWriter<>(delegate, 3L, context); // Direct snapshotState should call delegate.snapshotState with checkpointId 3 List> states = flinkSinkWriter.snapshotState(); @@ -93,11 +92,10 @@ void testSnapshotStateWithoutPrepareCommitFallsBack() throws Exception { @Test void testSchemaChangeEventDoesNotForceCommit() throws Exception { SchemaAwareRecordingSinkWriter delegate = new SchemaAwareRecordingSinkWriter(); - RecordingCommitter committer = new RecordingCommitter(); RecordingContext context = new RecordingContext(); FlinkSinkWriter flinkSinkWriter = - new FlinkSinkWriter<>(delegate, 7L, context, committer, null); + new FlinkSinkWriter<>(delegate, 7L, context); AlterTableAddColumnEvent event = AlterTableAddColumnEvent.add( @@ -127,7 +125,6 @@ void testSchemaChangeEventDoesNotForceCommit() throws Exception { Assertions.assertEquals(Collections.emptyList(), delegate.prepareCommitCalls); Assertions.assertEquals(1, delegate.appliedSchemaChanges.size()); Assertions.assertEquals(event, delegate.appliedSchemaChanges.get(0)); - Assertions.assertEquals(Collections.emptyList(), committer.committed); } private static class RecordingSinkWriter implements SinkWriter { @@ -179,19 +176,6 @@ public void applySchemaChange( } } - private static class RecordingCommitter implements SinkCommitter { - private final List committed = new ArrayList<>(); - - @Override - public List commit(List commitInfos) { - committed.addAll(commitInfos); - return Collections.emptyList(); - } - - @Override - public void abort(List commitInfos) {} - } - private static class RecordingContext implements SinkWriter.Context { @Override