Skip to content

Commit

Permalink
[FLINK-36690][runtime] Fix schema operator hanging under extreme para…
Browse files Browse the repository at this point in the history
…llelized pressure

Signed-off-by: yuxiqian <34335406+yuxiqian@users.noreply.github.com>
  • Loading branch information
yuxiqian committed Dec 13, 2024
1 parent 0037c43 commit cf4c706
Show file tree
Hide file tree
Showing 25 changed files with 764 additions and 205 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -28,14 +28,25 @@ public class FlushEvent implements Event {
/** The schema changes from which table. */
private final TableId tableId;

public FlushEvent(TableId tableId) {
/**
* Nonce code to distinguish flush events corresponding to each schema change event from
* different subTasks.
*/
private final long nonce;

public FlushEvent(TableId tableId, long nonce) {
this.tableId = tableId;
this.nonce = nonce;
}

public TableId getTableId() {
return tableId;
}

public long getNonce() {
return nonce;
}

@Override
public boolean equals(Object o) {
if (this == o) {
Expand All @@ -45,11 +56,16 @@ public boolean equals(Object o) {
return false;
}
FlushEvent that = (FlushEvent) o;
return Objects.equals(tableId, that.tableId);
return Objects.equals(tableId, that.tableId) && Objects.equals(nonce, that.nonce);
}

@Override
public int hashCode() {
return Objects.hash(tableId);
return Objects.hash(tableId, nonce);
}

@Override
public String toString() {
return "FlushEvent{" + "tableId=" + tableId + ", nonce=" + nonce + '}';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,20 @@ limitations under the License.
<type>test-jar</type>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-cdc-composer</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-cdc-pipeline-connector-values</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-test-util</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,207 @@
/*
* 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;

import org.apache.flink.cdc.common.configuration.Configuration;
import org.apache.flink.cdc.common.pipeline.PipelineOptions;
import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior;
import org.apache.flink.cdc.composer.PipelineExecution;
import org.apache.flink.cdc.composer.definition.PipelineDef;
import org.apache.flink.cdc.composer.definition.SinkDef;
import org.apache.flink.cdc.composer.definition.SourceDef;
import org.apache.flink.cdc.composer.flink.FlinkPipelineComposer;
import org.apache.flink.cdc.connectors.mysql.factory.MySqlDataSourceFactory;
import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase;
import org.apache.flink.cdc.connectors.values.ValuesDatabase;
import org.apache.flink.cdc.connectors.values.factory.ValuesDataFactory;
import org.apache.flink.cdc.connectors.values.sink.ValuesDataSinkOptions;

import org.junit.After;
import org.junit.Before;
import org.junit.Test;

import java.io.ByteArrayOutputStream;
import java.io.PrintStream;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.SQLException;
import java.sql.Statement;
import java.time.Duration;
import java.util.Collections;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;

import static org.apache.flink.cdc.connectors.mysql.testutils.MySqSourceTestUtils.TEST_PASSWORD;
import static org.apache.flink.cdc.connectors.mysql.testutils.MySqSourceTestUtils.TEST_USER;
import static org.apache.flink.cdc.connectors.mysql.testutils.MySqSourceTestUtils.getServerId;
import static org.apache.flink.cdc.connectors.mysql.testutils.MySqSourceTestUtils.loopCheck;
import static org.apache.flink.configuration.CoreOptions.ALWAYS_PARENT_FIRST_LOADER_PATTERNS_ADDITIONAL;
import static org.assertj.core.api.Assertions.assertThat;

/** Parallelized Integration test for MySQL connector. */
public class MySqlParallelizedPipelineITCase extends MySqlSourceTestBase {

private static final int PARALLELISM = 4;
private static final int TEST_TABLE_NUMBER = 100;

// Always use parent-first classloader for CDC classes.
// The reason is that ValuesDatabase uses static field for holding data, we need to make sure
// the class is loaded by AppClassloader so that we can verify data in the test case.
private static final org.apache.flink.configuration.Configuration MINI_CLUSTER_CONFIG =
new org.apache.flink.configuration.Configuration();

static {
MINI_CLUSTER_CONFIG.set(
ALWAYS_PARENT_FIRST_LOADER_PATTERNS_ADDITIONAL,
Collections.singletonList("org.apache.flink.cdc"));
}

private final PrintStream standardOut = System.out;
private final ByteArrayOutputStream outCaptor = new ByteArrayOutputStream();

private final UniqueDatabase parallelismDatabase =
new UniqueDatabase(
MYSQL_CONTAINER, "extreme_parallelism_test_database", TEST_USER, TEST_PASSWORD);

@Before
public void init() {
// Take over STDOUT as we need to check the output of values sink
System.setOut(new PrintStream(outCaptor));
// Initialize in-memory database
ValuesDatabase.clear();
}

@After
public void cleanup() {
System.setOut(standardOut);
}

@Test
public void testExtremeParallelizedSchemaChange() throws Exception {
final String databaseName = parallelismDatabase.getDatabaseName();
try (Connection conn =
DriverManager.getConnection(
MYSQL_CONTAINER.getJdbcUrl(), TEST_USER, TEST_PASSWORD);
Statement stat = conn.createStatement()) {
stat.execute(String.format("CREATE DATABASE %s;", databaseName));
stat.execute(String.format("USE %s;", databaseName));
for (int i = 1; i <= TEST_TABLE_NUMBER; i++) {
stat.execute(String.format("DROP TABLE IF EXISTS TABLE%d;", i));
stat.execute(
String.format(
"CREATE TABLE TABLE%d (ID INT NOT NULL PRIMARY KEY,VERSION VARCHAR(17));",
i));
stat.execute(String.format("INSERT INTO TABLE%d VALUES (%d, 'No.%d');", i, i, i));
}
} catch (SQLException e) {
LOG.error("Initialize table failed.", e);
throw e;
}
LOG.info("Table initialized successfully.");

FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster();

// Setup MySQL source
Configuration sourceConfig = new Configuration();
sourceConfig.set(MySqlDataSourceOptions.HOSTNAME, MYSQL_CONTAINER.getHost());
sourceConfig.set(MySqlDataSourceOptions.PORT, MYSQL_CONTAINER.getDatabasePort());
sourceConfig.set(MySqlDataSourceOptions.USERNAME, TEST_USER);
sourceConfig.set(MySqlDataSourceOptions.PASSWORD, TEST_PASSWORD);
sourceConfig.set(MySqlDataSourceOptions.SERVER_TIME_ZONE, "UTC");
sourceConfig.set(MySqlDataSourceOptions.TABLES, "\\.*.\\.*");
sourceConfig.set(MySqlDataSourceOptions.SERVER_ID, getServerId(PARALLELISM));

SourceDef sourceDef =
new SourceDef(MySqlDataSourceFactory.IDENTIFIER, "MySQL Source", sourceConfig);

// Setup value sink
Configuration sinkConfig = new Configuration();
sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true);
SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig);

// Setup pipeline
Configuration pipelineConfig = new Configuration();
pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, PARALLELISM);
pipelineConfig.set(
PipelineOptions.PIPELINE_SCHEMA_CHANGE_BEHAVIOR, SchemaChangeBehavior.EVOLVE);
PipelineDef pipelineDef =
new PipelineDef(
sourceDef,
sinkDef,
Collections.emptyList(),
Collections.emptyList(),
Collections.emptyList(),
pipelineConfig);

// Execute the pipeline
PipelineExecution execution = composer.compose(pipelineDef);
Thread executeThread =
new Thread(
() -> {
try {
execution.execute();
} catch (Exception e) {
throw new RuntimeException(e);
}
});

executeThread.start();

try {
loopCheck(
() ->
outCaptor.toString().trim().split("\n").length
>= TEST_TABLE_NUMBER * (PARALLELISM + 1),
"collect enough rows",
Duration.ofSeconds(120),
Duration.ofSeconds(1));
} finally {
executeThread.interrupt();
}

// Check the order and content of all received events
String outputEvents = outCaptor.toString();
assertThat(outputEvents)
.contains(
IntStream.rangeClosed(1, TEST_TABLE_NUMBER)
.boxed()
.flatMap(
i ->
Stream.concat(
IntStream.range(0, PARALLELISM)
.boxed()
.map(
subTaskId ->
String.format(
"%d> CreateTableEvent{tableId=%s.TABLE%d, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17)}, primaryKeys=ID, options=()}",
subTaskId,
parallelismDatabase
.getDatabaseName(),
i)),
Stream.of(
String.format(
"> DataChangeEvent{tableId=%s.TABLE%d, before=[], after=[%d, No.%d], op=INSERT, meta=()}",
parallelismDatabase
.getDatabaseName(),
i,
i,
i))))
.collect(Collectors.toList()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,13 @@
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.cdc.common.event.CreateTableEvent;

import java.time.Duration;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Random;
import java.util.concurrent.TimeoutException;
import java.util.function.Supplier;

/** Test utilities for MySQL event source. */
public class MySqSourceTestUtils {
Expand Down Expand Up @@ -63,5 +66,19 @@ public static String getServerId(int parallelism) {
return serverId + "-" + (serverId + parallelism);
}

public static void loopCheck(
Supplier<Boolean> runnable, String description, Duration timeout, Duration interval)
throws Exception {
long deadline = System.currentTimeMillis() + timeout.toMillis();
while (System.currentTimeMillis() < deadline) {
if (runnable.get()) {
return;
}
Thread.sleep(interval.toMillis());
}
throw new TimeoutException(
"Ran out of time when waiting for " + description + " to success.");
}

private MySqSourceTestUtils() {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -124,7 +124,9 @@ public void processElement(StreamRecord<Event> streamRecord) throws Exception {
output.collect(
new StreamRecord<>(
new BucketWrapperFlushEvent(
currentTaskNumber, ((FlushEvent) event).getTableId())));
currentTaskNumber,
((FlushEvent) event).getTableId(),
((FlushEvent) event).getNonce())));
return;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,7 @@ public void serialize(Event event, DataOutputView dataOutputView) throws IOExcep
BucketWrapperFlushEvent bucketWrapperFlushEvent = (BucketWrapperFlushEvent) event;
dataOutputView.writeInt(bucketWrapperFlushEvent.getBucket());
tableIdSerializer.serialize(bucketWrapperFlushEvent.getTableId(), dataOutputView);
dataOutputView.writeLong(bucketWrapperFlushEvent.getNonce());
}
}

Expand All @@ -90,7 +91,7 @@ public Event deserialize(DataInputView source) throws IOException {
EventClass eventClass = enumSerializer.deserialize(source);
if (eventClass.equals(EventClass.BUCKET_WRAPPER_FLUSH_EVENT)) {
return new BucketWrapperFlushEvent(
source.readInt(), tableIdSerializer.deserialize(source));
source.readInt(), tableIdSerializer.deserialize(source), source.readLong());
} else {
return new BucketWrapperChangeEvent(
source.readInt(), (ChangeEvent) eventSerializer.deserialize(source));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,8 @@ public class BucketWrapperFlushEvent extends FlushEvent implements BucketWrapper

private final int bucket;

public BucketWrapperFlushEvent(int bucket, TableId tableId) {
super(tableId);
public BucketWrapperFlushEvent(int bucket, TableId tableId, long nonce) {
super(tableId, nonce);
this.bucket = bucket;
}

Expand Down
Loading

0 comments on commit cf4c706

Please sign in to comment.