Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
import org.apache.flink.cdc.connectors.mysql.source.reader.MySqlSourceReader;
import org.apache.flink.cdc.connectors.mysql.source.reader.MySqlSourceReaderContext;
import org.apache.flink.cdc.connectors.mysql.source.reader.MySqlSplitReader;
import org.apache.flink.cdc.connectors.mysql.source.reader.async.PartitionedDeserializationScheduler;
import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSplit;
import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSplitSerializer;
import org.apache.flink.cdc.connectors.mysql.source.split.SourceRecords;
Expand Down Expand Up @@ -127,11 +128,35 @@ public static <T> MySqlSourceBuilder<T> builder() {
this(
configFactory,
deserializationSchema,
(sourceReaderMetrics, sourceConfig) ->
new MySqlRecordEmitter<>(
(sourceReaderMetrics, sourceConfig) -> {
if (!sourceConfig.isParallelDeserializeEnabled()) {
return new MySqlRecordEmitter<>(
deserializationSchema,
sourceReaderMetrics,
sourceConfig.isIncludeSchemaChanges()));
sourceConfig.isIncludeSchemaChanges());
}
final int cores = Math.max(1, Runtime.getRuntime().availableProcessors());
int pkWorkers =
sourceConfig.getParallelDeserializePkWorkers() > 0
? sourceConfig.getParallelDeserializePkWorkers()
: Math.max(1, cores - 1);
int deserThreads =
sourceConfig.getParallelDeserializeThreads() > 0
? sourceConfig.getParallelDeserializeThreads()
: Math.max(0, pkWorkers / 2);
int queueCapacity =
Math.max(1, sourceConfig.getParallelDeserializeQueueCapacity());
return new MySqlRecordEmitter<>(
deserializationSchema,
sourceReaderMetrics,
sourceConfig.isIncludeSchemaChanges(),
new PartitionedDeserializationScheduler<>(
deserializationSchema,
deserThreads,
pkWorkers,
0,
queueCapacity));
});
}

MySqlSource(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,12 @@ public class MySqlSourceConfig implements Serializable {
public static boolean useLegacyJsonFormat = true;
private final boolean assignUnboundedChunkFirst;

// ---------------- Parallel deserialize options ----------------
private final boolean parallelDeserializeEnabled;
private final int parallelDeserializePkWorkers;
private final int parallelDeserializeThreads;
private final int parallelDeserializeQueueCapacity;

// --------------------------------------------------------------------------------------------
// Debezium Configurations
// --------------------------------------------------------------------------------------------
Expand Down Expand Up @@ -108,7 +114,11 @@ public class MySqlSourceConfig implements Serializable {
boolean parseOnLineSchemaChanges,
boolean treatTinyInt1AsBoolean,
boolean useLegacyJsonFormat,
boolean assignUnboundedChunkFirst) {
boolean assignUnboundedChunkFirst,
boolean parallelDeserializeEnabled,
int parallelDeserializePkWorkers,
int parallelDeserializeThreads,
int parallelDeserializeQueueCapacity) {
this.hostname = checkNotNull(hostname);
this.port = port;
this.username = checkNotNull(username);
Expand Down Expand Up @@ -152,6 +162,10 @@ public class MySqlSourceConfig implements Serializable {
this.treatTinyInt1AsBoolean = treatTinyInt1AsBoolean;
this.useLegacyJsonFormat = useLegacyJsonFormat;
this.assignUnboundedChunkFirst = assignUnboundedChunkFirst;
this.parallelDeserializeEnabled = parallelDeserializeEnabled;
this.parallelDeserializePkWorkers = parallelDeserializePkWorkers;
this.parallelDeserializeThreads = parallelDeserializeThreads;
this.parallelDeserializeQueueCapacity = parallelDeserializeQueueCapacity;
}

public String getHostname() {
Expand Down Expand Up @@ -243,6 +257,23 @@ public boolean isAssignUnboundedChunkFirst() {
return assignUnboundedChunkFirst;
}

// ---------------- Parallel deserialize getters ----------------
public boolean isParallelDeserializeEnabled() {
return parallelDeserializeEnabled;
}

public int getParallelDeserializePkWorkers() {
return parallelDeserializePkWorkers;
}

public int getParallelDeserializeThreads() {
return parallelDeserializeThreads;
}

public int getParallelDeserializeQueueCapacity() {
return parallelDeserializeQueueCapacity;
}

public Properties getDbzProperties() {
return dbzProperties;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,11 @@ public class MySqlSourceConfigFactory implements Serializable {
private Duration heartbeatInterval = MySqlSourceOptions.HEARTBEAT_INTERVAL.defaultValue();
private Properties dbzProperties;
private Map<ObjectPath, String> chunkKeyColumns = new HashMap<>();
// ---------------- Parallel deserialize options ----------------
private boolean parallelDeserializeEnabled = false;
private int parallelDeserializePkWorkers = 0;
private int parallelDeserializeThreads = 0;
private int parallelDeserializeQueueCapacity = 65536;
private boolean skipSnapshotBackfill = false;
private boolean parseOnLineSchemaChanges = false;
private boolean treatTinyInt1AsBoolean = true;
Expand Down Expand Up @@ -324,6 +329,31 @@ public MySqlSourceConfigFactory assignUnboundedChunkFirst(boolean assignUnbounde
return this;
}

// ---------------- scan.parallel-deserialize.* options ----------------
/** Enable parallel deserialization and ordered delivery to output. */
public MySqlSourceConfigFactory parallelDeserializeEnabled(boolean enabled) {
this.parallelDeserializeEnabled = enabled;
return this;
}

/** Number of per-key workers. 0 = auto from CPU. */
public MySqlSourceConfigFactory parallelDeserializePkWorkers(int pkWorkers) {
this.parallelDeserializePkWorkers = pkWorkers;
return this;
}

/** Number of shared deserialization threads. 0 = auto from CPU. */
public MySqlSourceConfigFactory parallelDeserializeThreads(int threads) {
this.parallelDeserializeThreads = threads;
return this;
}

/** Bounded capacity per-partition queue. */
public MySqlSourceConfigFactory parallelDeserializeQueueCapacity(int capacity) {
this.parallelDeserializeQueueCapacity = capacity;
return this;
}

/** Creates a new {@link MySqlSourceConfig} for the given subtask {@code subtaskId}. */
public MySqlSourceConfig createConfig(int subtaskId) {
// hard code server name, because we don't need to distinguish it, docs:
Expand Down Expand Up @@ -421,6 +451,10 @@ public MySqlSourceConfig createConfig(int subtaskId, String serverName) {
parseOnLineSchemaChanges,
treatTinyInt1AsBoolean,
useLegacyJsonFormat,
assignUnboundedChunkFirst);
assignUnboundedChunkFirst,
parallelDeserializeEnabled,
parallelDeserializePkWorkers,
parallelDeserializeThreads,
parallelDeserializeQueueCapacity);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.flink.api.connector.source.SourceOutput;
import org.apache.flink.cdc.connectors.mysql.source.metrics.MySqlSourceReaderMetrics;
import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset;
import org.apache.flink.cdc.connectors.mysql.source.reader.async.AsyncScheduler;
import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSplit;
import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSplitState;
import org.apache.flink.cdc.connectors.mysql.source.split.SourceRecords;
Expand All @@ -37,6 +38,7 @@
import org.slf4j.LoggerFactory;

import java.util.Iterator;
import java.util.concurrent.atomic.AtomicInteger;

/**
* The {@link RecordEmitter} implementation for {@link MySqlSourceReader}.
Expand All @@ -54,6 +56,8 @@ public class MySqlRecordEmitter<T> implements RecordEmitter<SourceRecords, T, My
private final MySqlSourceReaderMetrics sourceReaderMetrics;
private final boolean includeSchemaChanges;
private final OutputCollector<T> outputCollector;
/** Optional: parallel scheduler; null means parallelization is disabled. */
private final AsyncScheduler<T> scheduler;

public MySqlRecordEmitter(
DebeziumDeserializationSchema<T> debeziumDeserializationSchema,
Expand All @@ -63,16 +67,63 @@ public MySqlRecordEmitter(
this.sourceReaderMetrics = sourceReaderMetrics;
this.includeSchemaChanges = includeSchemaChanges;
this.outputCollector = new OutputCollector<>();
this.scheduler = null;
}

/** Constructor that injects a parallel scheduler. */
public MySqlRecordEmitter(
DebeziumDeserializationSchema<T> debeziumDeserializationSchema,
MySqlSourceReaderMetrics sourceReaderMetrics,
boolean includeSchemaChanges,
AsyncScheduler<T> scheduler) {
this.debeziumDeserializationSchema = debeziumDeserializationSchema;
this.sourceReaderMetrics = sourceReaderMetrics;
this.includeSchemaChanges = includeSchemaChanges;
this.outputCollector = new OutputCollector<>();
this.scheduler = scheduler;
}

@Override
public void emitRecord(
SourceRecords sourceRecords, SourceOutput<T> output, MySqlSplitState splitState)
throws Exception {
final Iterator<SourceRecord> elementIterator = sourceRecords.iterator();
if (scheduler == null || !scheduler.isEnabled()) {
while (elementIterator.hasNext()) {
processElement(elementIterator.next(), output, splitState);
}
return;
}

// Parallel path: partitioned deserialization + source-thread replay + advance offset after
// replay
final AtomicInteger pendingPartitionTasks = new AtomicInteger(0);
while (elementIterator.hasNext()) {
processElement(elementIterator.next(), output, splitState);
SourceRecord next = elementIterator.next();
if (RecordUtils.isDataChangeRecord(next)) {
reportMetrics(next);
scheduler.schedulePartitioned(next, pendingPartitionTasks);
} else {
// Control / non-DML events: keep inline processing (or switch to global-async if
// needed)
processElement(next, output, splitState);
}
scheduler.drainRound(
output,
(offset) -> {
if (offset != null && splitState.isBinlogSplitState()) {
splitState.asBinlogSplitState().setStartingOffset(offset);
}
});
}
scheduler.waitAndDrainAll(
output,
pendingPartitionTasks,
(offset) -> {
if (offset != null && splitState.isBinlogSplitState()) {
splitState.asBinlogSplitState().setStartingOffset(offset);
}
});
}

protected void processElement(
Expand Down
Original file line number Diff line number Diff line change
@@ -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.
*/

package org.apache.flink.cdc.connectors.mysql.source.reader.async;

import org.apache.flink.api.connector.source.SourceOutput;
import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset;

import org.apache.kafka.connect.source.SourceRecord;

import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;

/**
* Parallel scheduler interface: performs thread-safe in-source deserialization and advances offsets
* when replaying on the source thread.
*/
public interface AsyncScheduler<T> {

/** Whether parallelization is enabled (false = fall back to single-threaded path). */
boolean isEnabled();

/**
* Schedule data-change events by primary-key partition; pendingTasks counts the number of tasks
* that are enqueued but not yet replayed.
*/
void schedulePartitioned(SourceRecord record, AtomicInteger pendingTasks);

/** Schedule global async work (e.g., control or non-DML events). */
void scheduleGlobalAsync(SourceRecord record);

/** Perform one replay round on the source thread; advance offset via onAfterEmit. */
void drainRound(SourceOutput<T> output, Consumer<BinlogOffset> onAfterEmit);

/** Wait until all scheduled tasks have completed and been replayed. */
void waitAndDrainAll(
SourceOutput<T> output, AtomicInteger pendingTasks, Consumer<BinlogOffset> onAfterEmit)
throws InterruptedException;
}
Loading