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][Rest-API] Submit or stop job from an inactive master node #6217

Merged
merged 18 commits into from
Jan 17, 2024
Original file line number Diff line number Diff line change
Expand Up @@ -32,13 +32,14 @@
import org.apache.seatunnel.engine.server.CoordinatorService;
import org.apache.seatunnel.engine.server.SeaTunnelServer;
import org.apache.seatunnel.engine.server.log.Log4j2HttpPostCommandProcessor;
import org.apache.seatunnel.engine.server.operation.CancelJobOperation;
import org.apache.seatunnel.engine.server.operation.SavePointJobOperation;
import org.apache.seatunnel.engine.server.operation.SubmitJobOperation;
import org.apache.seatunnel.engine.server.utils.NodeEngineUtil;
import org.apache.seatunnel.engine.server.utils.RestUtil;

import org.apache.commons.lang.StringUtils;

import com.hazelcast.core.Hazelcast;
import com.hazelcast.core.HazelcastInstance;
import com.hazelcast.instance.impl.HazelcastInstanceProxy;
import com.hazelcast.internal.ascii.TextCommandService;
import com.hazelcast.internal.ascii.rest.HttpCommandProcessor;
import com.hazelcast.internal.ascii.rest.HttpPostCommand;
Expand Down Expand Up @@ -102,20 +103,7 @@ private SeaTunnelServer getSeaTunnelServer() {
SeaTunnelServer seaTunnelServer =
(SeaTunnelServer) extensionServices.get(Constant.SEATUNNEL_SERVICE_NAME);
if (!seaTunnelServer.isMasterNode()) {
for (HazelcastInstance hazelcastInstance : Hazelcast.getAllHazelcastInstances()) {
seaTunnelServer =
(SeaTunnelServer)
((HazelcastInstanceProxy) hazelcastInstance)
.getOriginal()
.node
.getNodeExtension()
.createExtensionServices()
.get(Constant.SEATUNNEL_SERVICE_NAME);

if (seaTunnelServer.isMasterNode()) {
return seaTunnelServer;
}
}
return null;
}
return seaTunnelServer;
}
Expand Down Expand Up @@ -146,14 +134,21 @@ private void handleSubmitJob(HttpPostCommand httpPostCommand, String uri)
? Long.parseLong(requestParams.get(RestConstant.JOB_ID))
: null);
JobImmutableInformation jobImmutableInformation = restJobExecutionEnvironment.build();

Long jobId = jobImmutableInformation.getJobId();
SeaTunnelServer seaTunnelServer = getSeaTunnelServer();
Long jobId =
submitJob(
seaTunnelServer,
jobImmutableInformation,
jobConfig,
restJobExecutionEnvironment);
if (seaTunnelServer == null) {

NodeEngineUtil.sendOperationToMasterNode(
getNode().nodeEngine,
new SubmitJobOperation(
jobImmutableInformation.getJobId(),
getNode().nodeEngine.toData(jobImmutableInformation)))
.join();

} else {

submitJob(seaTunnelServer, jobImmutableInformation, jobConfig);
}

this.prepareResponse(
httpPostCommand,
Expand All @@ -174,12 +169,26 @@ private void handleStopJob(HttpPostCommand httpPostCommand, String uri) {
Boolean.parseBoolean(map.get(RestConstant.IS_STOP_WITH_SAVE_POINT).toString());
}

CoordinatorService coordinatorService = getSeaTunnelServer().getCoordinatorService();
SeaTunnelServer seaTunnelServer = getSeaTunnelServer();
if (seaTunnelServer == null) {
if (isStopWithSavePoint) {
NodeEngineUtil.sendOperationToMasterNode(
getNode().nodeEngine, new SavePointJobOperation(jobId))
.join();
} else {
NodeEngineUtil.sendOperationToMasterNode(
getNode().nodeEngine, new CancelJobOperation(jobId))
.join();
}

if (isStopWithSavePoint) {
coordinatorService.savePoint(jobId);
} else {
coordinatorService.cancelJob(jobId);
CoordinatorService coordinatorService = getSeaTunnelServer().getCoordinatorService();

if (isStopWithSavePoint) {
coordinatorService.savePoint(jobId);
} else {
coordinatorService.cancelJob(jobId);
}
}

this.prepareResponse(
Expand Down Expand Up @@ -215,11 +224,10 @@ private JsonNode requestHandle(HttpPostCommand httpPostCommand) {
return requestBodyJsonNode;
}

private Long submitJob(
private void submitJob(
SeaTunnelServer seaTunnelServer,
JobImmutableInformation jobImmutableInformation,
JobConfig jobConfig,
RestJobExecutionEnvironment restJobExecutionEnvironment) {
JobConfig jobConfig) {
CoordinatorService coordinatorService = seaTunnelServer.getCoordinatorService();
Data data =
textCommandService
Expand All @@ -231,7 +239,5 @@ private Long submitJob(
coordinatorService.submitJob(
Long.parseLong(jobConfig.getJobContext().getJobId()), data);
voidPassiveCompletableFuture.join();

return restJobExecutionEnvironment.getJobId();
}
}