Skip to content

Commit

Permalink
[Fix] Fix MultiTableWriterRunnable can not catch Throwable error (apa…
Browse files Browse the repository at this point in the history
  • Loading branch information
Hisoka-X authored and chl-wxp committed May 10, 2024
1 parent 1a9bf92 commit 7d6c3c2
Show file tree
Hide file tree
Showing 10 changed files with 92 additions and 12 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ public void run() {
// exception.
throwable = e;
break;
} catch (Exception e) {
} catch (Throwable e) {
log.error("MultiTableWriterRunnable error", e);
throwable = e;
break;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -360,7 +360,8 @@ public void testMultiTableWithRestore(TestContainer container)

Pattern jobIdPattern =
Pattern.compile(
".*Init JobMaster for Job SeaTunnel_Job \\(([0-9]*)\\).*", Pattern.DOTALL);
".*Init JobMaster for Job mysqlcdc_to_mysql_with_multi_table_mode_one_table.conf \\(([0-9]*)\\).*",
Pattern.DOTALL);
Matcher matcher = jobIdPattern.matcher(container.getServerLogs());
String jobId;
if (matcher.matches()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -405,7 +405,8 @@ public void testMultiTableWithRestore(TestContainer container)

Pattern jobIdPattern =
Pattern.compile(
".*Init JobMaster for Job SeaTunnel_Job \\(([0-9]*)\\).*", Pattern.DOTALL);
".*Init JobMaster for Job oraclecdc_to_oracle_with_multi_table_mode_one_table.conf \\(([0-9]*)\\).*",
Pattern.DOTALL);
Matcher matcher = jobIdPattern.matcher(container.getServerLogs());
String jobId;
if (matcher.matches()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -307,7 +307,7 @@ public void testMultiTableWithRestore(TestContainer container)

Pattern jobIdPattern =
Pattern.compile(
".*Init JobMaster for Job SeaTunnel_Job \\(([0-9]*)\\).*",
".*Init JobMaster for Job pgcdc_to_pg_with_multi_table_mode_one_table.conf \\(([0-9]*)\\).*",
Pattern.DOTALL);
Matcher matcher = jobIdPattern.matcher(container.getServerLogs());
String jobId;
Expand Down Expand Up @@ -412,7 +412,7 @@ public void testAddFiledWithRestore(TestContainer container)

Pattern jobIdPattern =
Pattern.compile(
".*Init JobMaster for Job SeaTunnel_Job \\(([0-9]*)\\).*",
".*Init JobMaster for Job postgrescdc_to_postgres_test_add_Filed.conf \\(([0-9]*)\\).*",
Pattern.DOTALL);
Matcher matcher = jobIdPattern.matcher(container.getServerLogs());
String jobId;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,8 @@ protected Container.ExecResult executeJob(
command.add(adaptPathForWin(binPath));
command.add("--config");
command.add(adaptPathForWin(confInContainerPath));
command.add("--name");
command.add(new File(confInContainerPath).getName());
List<String> extraStartShellCommands = new ArrayList<>(getExtraStartShellCommands());
if (variables != null && !variables.isEmpty()) {
variables.forEach(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,9 @@ public class InMemorySinkFactory

public static final Option<Boolean> THROW_EXCEPTION =
Options.key("throw_exception").booleanType().defaultValue(false);

public static final Option<Boolean> THROW_OUT_OF_MEMORY =
Options.key("throw_out_of_memory").booleanType().defaultValue(false);
public static final Option<Boolean> CHECKPOINT_SLEEP =
Options.key("checkpoint_sleep").booleanType().defaultValue(false);

Expand All @@ -45,7 +48,9 @@ public String factoryIdentifier() {

@Override
public OptionRule optionRule() {
return OptionRule.builder().optional(THROW_EXCEPTION, CHECKPOINT_SLEEP).build();
return OptionRule.builder()
.optional(THROW_EXCEPTION, THROW_OUT_OF_MEMORY, CHECKPOINT_SLEEP)
.build();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,11 @@ public InMemorySinkWriter(ReadonlyConfig config) {
private InMemoryMultiTableResourceManager resourceManager;

@Override
public void write(SeaTunnelRow element) throws IOException {}
public void write(SeaTunnelRow element) throws IOException {
if (config.get(InMemorySinkFactory.THROW_OUT_OF_MEMORY)) {
throw new OutOfMemoryError();
}
}

@Override
public Optional<InMemoryCommitInfo> prepareCommit() throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,7 @@ public void testZetaStreamingCheckpointInterval(TestContainer container)
() -> {
Pattern jobIdPattern =
Pattern.compile(
".*Init JobMaster for Job SeaTunnel_Job \\(([0-9]*)\\).*",
".*Init JobMaster for Job stream_fakesource_to_localfile_interval.conf \\(([0-9]*)\\).*",
Pattern.DOTALL);
Matcher matcher = jobIdPattern.matcher(container.getServerLogs());
if (matcher.matches()) {
Expand Down Expand Up @@ -181,7 +181,7 @@ public void testZetaStreamingCheckpointNoInterval(TestContainer container)
() -> {
Pattern jobIdPattern =
Pattern.compile(
".*Init JobMaster for Job SeaTunnel_Job \\(([0-9]*)\\).*",
".*Init JobMaster for Job stream_fakesource_to_localfile.conf \\(([0-9]*)\\).*",
Pattern.DOTALL);
Matcher matcher = jobIdPattern.matcher(container.getServerLogs());
if (matcher.matches()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,13 +45,29 @@ public void testJobRetryTimes() throws IOException, InterruptedException {
Container.ExecResult execResult =
executeJob(server, "/retry-times/stream_fake_to_inmemory_with_error_retry_1.conf");
Assertions.assertNotEquals(0, execResult.getExitCode());
Assertions.assertTrue(server.getLogs().contains("Restore time 1, pipeline"));
Assertions.assertFalse(server.getLogs().contains("Restore time 3, pipeline"));
Assertions.assertTrue(
server.getLogs()
.contains(
"Restore time 1, pipeline Job stream_fake_to_inmemory_with_error_retry_1.conf"));
Assertions.assertFalse(
server.getLogs()
.contains(
"Restore time 3, pipeline Job stream_fake_to_inmemory_with_error_retry_1.conf"));

Container.ExecResult execResult2 =
executeJob(server, "/retry-times/stream_fake_to_inmemory_with_error.conf");
Assertions.assertNotEquals(0, execResult2.getExitCode());
Assertions.assertTrue(server.getLogs().contains("Restore time 3, pipeline"));
Assertions.assertTrue(
server.getLogs()
.contains(
"Restore time 3, pipeline Job stream_fake_to_inmemory_with_error.conf"));
}

@Test
public void testMultiTableSinkFailedWithThrowable() throws IOException, InterruptedException {
Container.ExecResult execResult =
executeJob(server, "/stream_fake_to_inmemory_with_throwable_error.conf");
Assertions.assertNotEquals(0, execResult.getExitCode());
}

@Test
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
#
# 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 {
parallelism = 1
job.mode = "STREAMING"
checkpoint.interval = 5000
}

source {
# This is a example source plugin **only for test and demonstrate the feature source plugin**
FakeSource {
result_table_name = "fake"
row.num = 100
split.num = 5
schema = {
fields {
name = "string"
age = "int"
}
}
parallelism = 1
}
}

transform {
}

sink {
InMemory {
source_table_name="fake"
throw_out_of_memory=true
}
}

0 comments on commit 7d6c3c2

Please sign in to comment.