dybyte commented on code in PR #10448: URL: https://github.com/apache/seatunnel/pull/10448#discussion_r2768197280
########## seatunnel-engine/seatunnel-engine-server/src/test/resources/stream_fake_to_console_checkpoint_barrier_trigger_error.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 { + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 1000 + checkpoint.timeout = 60000 +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + plugin_output = "fake1" + row.num = 1000 + split.num = 100 + split.read-interval = 3000 + parallelism = 1 + schema = { + fields { + name = "string" + age = "int" + } + } + parallelism = 1 Review Comment: ```suggestion ``` ########## seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointBarrierTriggerErrorTest.java: ########## @@ -0,0 +1,125 @@ +/* + * 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.engine.server.checkpoint; + +import org.apache.seatunnel.engine.common.job.JobStatus; +import org.apache.seatunnel.engine.common.utils.PassiveCompletableFuture; +import org.apache.seatunnel.engine.core.dag.logical.LogicalDag; +import org.apache.seatunnel.engine.core.job.JobImmutableInformation; +import org.apache.seatunnel.engine.server.AbstractSeaTunnelServerTest; +import org.apache.seatunnel.engine.server.TestUtils; +import org.apache.seatunnel.engine.server.checkpoint.operation.CheckpointBarrierTriggerOperation; +import org.apache.seatunnel.engine.server.master.JobMaster; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; + +import com.hazelcast.internal.serialization.Data; + +import java.lang.reflect.Field; +import java.util.Collections; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.awaitility.Awaitility.await; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; + +public class CheckpointBarrierTriggerErrorTest extends AbstractSeaTunnelServerTest { + + private static final String CONF_PATH = + "stream_fake_to_console_checkpoint_barrier_trigger_error.conf"; + private static final AtomicInteger COUNTER = new AtomicInteger(0); + + @Test + public void testCheckpointBarrierTriggerError() + throws NoSuchFieldException, IllegalAccessException { + long jobId = System.currentTimeMillis(); + startJob(System.currentTimeMillis(), CONF_PATH); Review Comment: ```suggestion long jobId = System.currentTimeMillis(); startJob(jobId, CONF_PATH); ``` ########## seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java: ########## @@ -683,9 +683,16 @@ private void startTriggerPendingCheckpoint( try { CompletableFuture.allOf(completableFutureArray).get(); } catch (InterruptedException e) { + handleCoordinatorError( + "triggering checkpoint barrier has been interrupted", + e, + CheckpointCloseReason.CHECKPOINT_INSIDE_ERROR); throw new RuntimeException(e); Review Comment: ```suggestion return; ``` ########## seatunnel-engine/seatunnel-engine-server/src/test/resources/stream_fake_to_console_checkpoint_barrier_trigger_error.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 { + parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 1000 + checkpoint.timeout = 60000 +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + plugin_output = "fake1" + row.num = 1000 + split.num = 100 + split.read-interval = 3000 + parallelism = 1 + schema = { + fields { + name = "string" + age = "int" + } + } + parallelism = 1 + } +} + +transform { +} + +sink { + console { + log.print.delay.ms=0 Review Comment: ```suggestion ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
