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

[Hotfix][Zeta] Fix zeta scheduler bug #6050

Merged
merged 8 commits into from
Dec 26, 2023
Original file line number Diff line number Diff line change
@@ -0,0 +1,145 @@
/*
* 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.e2e;

import org.apache.seatunnel.common.config.Common;
import org.apache.seatunnel.common.config.DeployMode;
import org.apache.seatunnel.engine.client.SeaTunnelClient;
import org.apache.seatunnel.engine.client.job.ClientJobExecutionEnvironment;
import org.apache.seatunnel.engine.client.job.ClientJobProxy;
import org.apache.seatunnel.engine.common.config.ConfigProvider;
import org.apache.seatunnel.engine.common.config.JobConfig;
import org.apache.seatunnel.engine.common.config.SeaTunnelConfig;
import org.apache.seatunnel.engine.core.job.JobStatus;
import org.apache.seatunnel.engine.server.SeaTunnelServerStarter;

import org.awaitility.Awaitility;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;

import com.hazelcast.client.config.ClientConfig;
import com.hazelcast.instance.impl.HazelcastInstanceImpl;

import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;

public class SeaTunnelSlotIT {
@Test
public void testSlotNotEnough() throws Exception {
HazelcastInstanceImpl node1 = null;
SeaTunnelClient engineClient = null;

try {
String testClusterName = "testSlotNotEnough";
SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig();
seaTunnelConfig.getHazelcastConfig().setClusterName(testClusterName);
// slot num is 3
seaTunnelConfig.getEngineConfig().getSlotServiceConfig().setDynamicSlot(false);
seaTunnelConfig.getEngineConfig().getSlotServiceConfig().setSlotNum(3);

node1 = SeaTunnelServerStarter.createHazelcastInstance(seaTunnelConfig);

// client config
Common.setDeployMode(DeployMode.CLIENT);
String filePath = TestUtils.getResource("batch_slot_not_enough.conf");
JobConfig jobConfig = new JobConfig();
jobConfig.setName(testClusterName);

ClientConfig clientConfig = ConfigProvider.locateAndGetClientConfig();
clientConfig.setClusterName(testClusterName);
engineClient = new SeaTunnelClient(clientConfig);
ClientJobExecutionEnvironment jobExecutionEnv =
engineClient.createExecutionContext(filePath, jobConfig, seaTunnelConfig);

final ClientJobProxy clientJobProxy = jobExecutionEnv.execute();
CompletableFuture<JobStatus> objectCompletableFuture =
CompletableFuture.supplyAsync(clientJobProxy::waitForJobComplete);
Awaitility.await()
.atMost(600000, TimeUnit.MILLISECONDS)
.untilAsserted(
() -> {
Thread.sleep(2000);
Assertions.assertTrue(
objectCompletableFuture.isDone()
&& JobStatus.FAILED.equals(
objectCompletableFuture.get()));
});

} finally {
if (engineClient != null) {
engineClient.shutdown();
}

if (node1 != null) {
node1.shutdown();
}
}
}

@Test
public void testSlotEnough() throws Exception {
HazelcastInstanceImpl node1 = null;
SeaTunnelClient engineClient = null;

try {
String testClusterName = "testSlotEnough";
SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig();
seaTunnelConfig.getHazelcastConfig().setClusterName(testClusterName);
// slot num is 3
EricJoy2048 marked this conversation as resolved.
Show resolved Hide resolved
seaTunnelConfig.getEngineConfig().getSlotServiceConfig().setDynamicSlot(false);
seaTunnelConfig.getEngineConfig().getSlotServiceConfig().setSlotNum(10);

node1 = SeaTunnelServerStarter.createHazelcastInstance(seaTunnelConfig);

// client config
Common.setDeployMode(DeployMode.CLIENT);
String filePath = TestUtils.getResource("batch_slot_not_enough.conf");
JobConfig jobConfig = new JobConfig();
jobConfig.setName(testClusterName);

ClientConfig clientConfig = ConfigProvider.locateAndGetClientConfig();
clientConfig.setClusterName(testClusterName);
engineClient = new SeaTunnelClient(clientConfig);
ClientJobExecutionEnvironment jobExecutionEnv =
engineClient.createExecutionContext(filePath, jobConfig, seaTunnelConfig);

final ClientJobProxy clientJobProxy = jobExecutionEnv.execute();
CompletableFuture<JobStatus> objectCompletableFuture =
CompletableFuture.supplyAsync(clientJobProxy::waitForJobComplete);
Awaitility.await()
.atMost(600000, TimeUnit.MILLISECONDS)
.untilAsserted(
() -> {
Thread.sleep(2000);
Assertions.assertTrue(
objectCompletableFuture.isDone()
&& JobStatus.FINISHED.equals(
objectCompletableFuture.get()));
});

} finally {
if (engineClient != null) {
engineClient.shutdown();
}

if (node1 != null) {
node1.shutdown();
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
#
# 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
job.mode = "BATCH"
#execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint"
}

source {
# This is a example source plugin **only for test and demonstrate the feature source plugin**
FakeSource {
result_table_name = "fake"
parallelism = 4
schema = {
fields {
name = "string"
age = "int"
}
}
}
}

transform {
}

sink {
console {
source_table_name="fake"
}
}
Original file line number Diff line number Diff line change
@@ -33,8 +33,6 @@
import org.apache.seatunnel.engine.core.protocol.codec.SeaTunnelSubmitJobCodec;
import org.apache.seatunnel.engine.core.protocol.codec.SeaTunnelWaitForJobCompleteCodec;

import org.apache.commons.lang3.StringUtils;

import com.hazelcast.client.impl.protocol.ClientMessage;
import com.hazelcast.logging.ILogger;
import com.hazelcast.logging.Logger;
@@ -117,10 +115,6 @@ public JobResult waitForJobCompleteV2() {
throw new RuntimeException(e);
}
LOGGER.info(String.format("Job (%s) end with state %s", jobId, jobResult.getStatus()));
if (StringUtils.isNotEmpty(jobResult.getError())
|| jobResult.getStatus().equals(JobStatus.FAILED)) {
throw new SeaTunnelEngineException(jobResult.getError());
}
Comment on lines -120 to -123
Copy link
Member

Choose a reason for hiding this comment

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

If client never throw exception even job failed. The shell status code always return 0, does not meet shell standards. Also client side will not get job result error msg. We need test case to cover it.

Copy link
Member Author

Choose a reason for hiding this comment

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

If client never throw exception even job failed. The shell status code always return 0, does not meet shell standards. Also client side will not get job result error msg. We need test case to cover it.

Done, I have added test cases.

return jobResult;
}

Original file line number Diff line number Diff line change
@@ -312,6 +312,10 @@ public synchronized void updatePipelineState(@NonNull PipelineStatus targetState
return;
}

if (PipelineStatus.FAILING.equals(current) && targetState.isEndState()) {
targetState = PipelineStatus.FAILED;
}
EricJoy2048 marked this conversation as resolved.
Show resolved Hide resolved

// consistency check
if (current.isEndState()) {
String message = "Pipeline is trying to leave terminal state " + current;
@@ -322,10 +326,11 @@ public synchronized void updatePipelineState(@NonNull PipelineStatus targetState
// now do the actual state transition
// we must update runningJobStateTimestampsIMap first and then can update
// runningJobStateIMap
PipelineStatus finalTargetState = targetState;
RetryUtils.retryWithException(
() -> {
updateStateTimestamps(targetState);
runningJobStateIMap.set(pipelineLocation, targetState);
updateStateTimestamps(finalTargetState);
runningJobStateIMap.set(pipelineLocation, finalTargetState);
return null;
},
new RetryUtils.RetryMaterial(
@@ -614,11 +619,13 @@ private synchronized void stateProcess() {
case CANCELING:
coordinatorVertexList.forEach(
task -> {
task.startPhysicalVertex();
task.cancel();
});

physicalVertexList.forEach(
task -> {
task.startPhysicalVertex();
task.cancel();
});
break;