Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Fix] Fix MultiTableSink restore failed when add new table #5746

Merged
merged 5 commits into from
Nov 3, 2023
Merged
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 @@ -217,13 +217,13 @@ public SnapshotPhaseState snapshotState(long checkpointId) {
new SnapshotPhaseState(
alreadyProcessedTables,
remainingSplits.isEmpty()
? Collections.emptyList()
Copy link
Member Author

Choose a reason for hiding this comment

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

Use unmutable list will throw exception when restore with add new table. So I think we should use mutable list for now. cc @hailin0

Copy link
Member

Choose a reason for hiding this comment

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

good

? new ArrayList<>()
: new ArrayList<>(remainingSplits),
assignedSplits,
splitCompletedOffsets,
assignerCompleted,
remainingTables.isEmpty()
? Collections.emptyList()
? new ArrayList<>()
: new ArrayList<>(remainingTables),
isTableIdCaseSensitive,
true);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.seatunnel.api.table.type.SeaTunnelRow;

import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -94,15 +95,21 @@ public SinkWriter<SeaTunnelRow, MultiTableCommitInfo, MultiTableState> restoreWr
SinkIdentifier sinkIdentifier = SinkIdentifier.of(tableIdentifier, index);
List<?> state =
states.stream()
.flatMap(
.map(
multiTableState ->
multiTableState.getStates().get(sinkIdentifier)
.stream())
multiTableState.getStates().get(sinkIdentifier))
.filter(Objects::nonNull)
.flatMap(Collection::stream)
.collect(Collectors.toList());
writers.put(
sinkIdentifier,
sink.restoreWriter(new SinkContextProxy(index, context), state));
if (state.isEmpty()) {
writers.put(
sinkIdentifier,
sink.createWriter(new SinkContextProxy(index, context)));
} else {
writers.put(
sinkIdentifier,
sink.restoreWriter(new SinkContextProxy(index, context), state));
}
}
}
return new MultiTableSinkWriter(writers, replicaNum);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -196,8 +196,12 @@ public Optional<MultiTableCommitInfo> prepareCommit() throws IOException {

@Override
public void abortPrepare() {
checkQueueRemain();
Throwable firstE = null;
try {
checkQueueRemain();
} catch (Exception e) {
firstE = e;
}
for (int i = 0; i < sinkWritersWithIndex.size(); i++) {
synchronized (runnable.get(i)) {
for (SinkWriter<SeaTunnelRow, ?, ?> sinkWriter :
Expand All @@ -220,9 +224,13 @@ public void abortPrepare() {

@Override
public void close() throws IOException {
checkQueueRemain();
executorService.shutdownNow();
Throwable firstE = null;
try {
checkQueueRemain();
} catch (Exception e) {
firstE = e;
}
executorService.shutdownNow();
for (int i = 0; i < sinkWritersWithIndex.size(); i++) {
synchronized (runnable.get(i)) {
for (SinkWriter<SeaTunnelRow, ?, ?> sinkWriter :
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@

import lombok.extern.slf4j.Slf4j;

import java.io.IOException;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.ResultSet;
Expand All @@ -47,6 +48,8 @@
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Stream;

import static org.awaitility.Awaitility.await;
Expand Down Expand Up @@ -184,7 +187,8 @@ public void testMysqlCdcMultiTableE2e(TestContainer container) {
CompletableFuture.supplyAsync(
() -> {
try {
container.executeJob("/mysqlcdc_to_mysql_with_multi_table.conf");
container.executeJob(
"/mysqlcdc_to_mysql_with_multi_table_mode_two_table.conf");
} catch (Exception e) {
log.error("Commit task exception :" + e.getMessage());
throw new RuntimeException(e);
Expand Down Expand Up @@ -223,6 +227,104 @@ public void testMysqlCdcMultiTableE2e(TestContainer container) {
SOURCE_TABLE_2)))));
}

@TestTemplate
@DisabledOnContainer(
value = {},
type = {EngineType.SPARK, EngineType.FLINK},
disabledReason = "Currently SPARK and FLINK do not support multi table")
public void testMultiTableWithRestore(TestContainer container)
throws IOException, InterruptedException {
// Clear related content to ensure that multiple operations are not affected
clearTable(MYSQL_DATABASE, SOURCE_TABLE_1);
clearTable(MYSQL_DATABASE, SOURCE_TABLE_2);
clearTable(MYSQL_DATABASE2, SOURCE_TABLE_1);
clearTable(MYSQL_DATABASE2, SOURCE_TABLE_2);

CompletableFuture.supplyAsync(
() -> {
try {
return container.executeJob(
"/mysqlcdc_to_mysql_with_multi_table_mode_one_table.conf");
} catch (Exception e) {
log.error("Commit task exception :" + e.getMessage());
throw new RuntimeException(e);
}
});

// insert update delete
upsertDeleteSourceTable(MYSQL_DATABASE, SOURCE_TABLE_1);

// stream stage
await().atMost(60000, TimeUnit.MILLISECONDS)
.untilAsserted(
() ->
Assertions.assertAll(
() ->
Assertions.assertIterableEquals(
query(
getSourceQuerySQL(
MYSQL_DATABASE,
SOURCE_TABLE_1)),
query(
getSourceQuerySQL(
MYSQL_DATABASE2,
SOURCE_TABLE_1)))));

Pattern jobIdPattern =
Pattern.compile(
".*Init JobMaster for Job SeaTunnel_Job \\(([0-9]*)\\).*", Pattern.DOTALL);
Matcher matcher = jobIdPattern.matcher(container.getServerLogs());
String jobId;
if (matcher.matches()) {
jobId = matcher.group(1);
} else {
throw new RuntimeException("Can not find jobId");
}

Assertions.assertEquals(0, container.savepointJob(jobId).getExitCode());

// Restore job with add a new table
CompletableFuture.supplyAsync(
() -> {
try {
container.restoreJob(
"/mysqlcdc_to_mysql_with_multi_table_mode_two_table.conf", jobId);
} catch (Exception e) {
log.error("Commit task exception :" + e.getMessage());
throw new RuntimeException(e);
}
return null;
});

upsertDeleteSourceTable(MYSQL_DATABASE, SOURCE_TABLE_2);

// stream stage
await().atMost(60000, TimeUnit.MILLISECONDS)
.untilAsserted(
() ->
Assertions.assertAll(
() ->
Assertions.assertIterableEquals(
query(
getSourceQuerySQL(
MYSQL_DATABASE,
SOURCE_TABLE_1)),
query(
getSourceQuerySQL(
MYSQL_DATABASE2,
SOURCE_TABLE_1))),
() ->
Assertions.assertIterableEquals(
query(
getSourceQuerySQL(
MYSQL_DATABASE,
SOURCE_TABLE_2)),
query(
getSourceQuerySQL(
MYSQL_DATABASE2,
SOURCE_TABLE_2)))));
}

private Connection getJdbcConnection() throws SQLException {
return DriverManager.getConnection(
MYSQL_CONTAINER.getJdbcUrl(),
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
#
# 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
execution.parallelism = 1
job.mode = "STREAMING"
checkpoint.interval = 5000
}

source {
MySQL-CDC {
result_table_name = "customers_mysql_cdc"
server-id = 5652
username = "mysqluser"
password = "mysqlpw"
table-names = ["mysql_cdc.mysql_cdc_e2e_source_table"]
base-url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc"
}
}

transform {
}

sink {
jdbc {
source_table_name = "customers_mysql_cdc"
url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc2"
driver = "com.mysql.cj.jdbc.Driver"
user = "mysqluser"
password = "mysqlpw"
database = "mysql_cdc2"
generate_sink_sql = true
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,10 @@ public AbstractTestContainer() {

protected abstract String getConnectorType();

protected abstract String getSavePointCommand();

protected abstract String getRestoreCommand();

protected abstract String getConnectorNamePrefix();

protected abstract List<String> getExtraStartShellCommands();
Expand Down Expand Up @@ -100,59 +104,94 @@ protected Container.ExecResult executeJob(GenericContainer<?> container, String
getConnectorNamePrefix(),
getConnectorType(),
SEATUNNEL_HOME);
return executeCommand(container, confInContainerPath);
final List<String> command = new ArrayList<>();
String binPath = Paths.get(SEATUNNEL_HOME, "bin", getStartShellName()).toString();
// base command
command.add(adaptPathForWin(binPath));
command.add("--config");
command.add(adaptPathForWin(confInContainerPath));
command.addAll(getExtraStartShellCommands());
return executeCommand(container, command);
}

protected Container.ExecResult executeCommand(GenericContainer<?> container, String configPath)
protected Container.ExecResult savepointJob(GenericContainer<?> container, String jobId)
throws IOException, InterruptedException {
final List<String> command = new ArrayList<>();
String binPath = Paths.get(SEATUNNEL_HOME, "bin", getStartShellName()).toString();
// base command
command.add(adaptPathForWin(binPath));
command.add(getSavePointCommand());
command.add(jobId);
command.addAll(getExtraStartShellCommands());
return executeCommand(container, command);
}

protected Container.ExecResult restoreJob(
GenericContainer<?> container, String confFile, String jobId)
throws IOException, InterruptedException {
final String confInContainerPath = copyConfigFileToContainer(container, confFile);
// copy connectors
copyConnectorJarToContainer(
container,
confFile,
getConnectorModulePath(),
getConnectorNamePrefix(),
getConnectorType(),
SEATUNNEL_HOME);
final List<String> command = new ArrayList<>();
String binPath = Paths.get(SEATUNNEL_HOME, "bin", getStartShellName()).toString();
// base command
command.add(adaptPathForWin(binPath));
command.add("--config");
command.add(adaptPathForWin(configPath));
command.add(adaptPathForWin(confInContainerPath));
command.add(getRestoreCommand());
command.add(jobId);
command.addAll(getExtraStartShellCommands());
return executeCommand(container, command);
}

protected Container.ExecResult executeCommand(
GenericContainer<?> container, List<String> command)
throws IOException, InterruptedException {
String commandStr = String.join(" ", command);
LOG.info(
"Execute config file: {} to Container[{}] "
"Execute command in container[{}] "
+ "\n==================== Shell Command start ====================\n"
+ "{}"
+ "\n==================== Shell Command end ====================",
configPath,
container.getDockerImageName(),
String.join(" ", command));
Container.ExecResult execResult =
container.execInContainer("bash", "-c", String.join(" ", command));
commandStr);
Container.ExecResult execResult = container.execInContainer("bash", "-c", commandStr);

if (execResult.getStdout() != null && execResult.getStdout().length() > 0) {
if (execResult.getStdout() != null && !execResult.getStdout().isEmpty()) {
LOG.info(
"Execute config file: {} to Container[{}] STDOUT:"
"Container[{}] command {} STDOUT:"
+ "\n==================== STDOUT start ====================\n"
+ "{}"
+ "\n==================== STDOUT end ====================",
configPath,
container.getDockerImageName(),
commandStr,
execResult.getStdout());
}
if (execResult.getStderr() != null && execResult.getStderr().length() > 0) {
if (execResult.getStderr() != null && !execResult.getStderr().isEmpty()) {
LOG.error(
"Execute config file: {} to Container[{}] STDERR:"
"Container[{}] command {} STDERR:"
+ "\n==================== STDERR start ====================\n"
+ "{}"
+ "\n==================== STDERR end ====================",
configPath,
container.getDockerImageName(),
commandStr,
execResult.getStderr());
}

if (execResult.getExitCode() != 0) {
LOG.info(
"Execute config file: {} to Container[{}] Server Log:"
"Container[{}] command {} Server Log:"
+ "\n==================== Server Log start ====================\n"
+ "{}"
+ "\n==================== Server Log end ====================",
configPath,
container.getDockerImageName(),
commandStr,
container.getLogs());
}

Expand Down
Loading