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

[Improve] [Zeta] Fix Client Have Error Can't be Shutdown #4099

Merged
merged 2 commits into from
Feb 10, 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 @@ -45,7 +45,7 @@
import java.time.Duration;
import java.time.LocalDateTime;
import java.util.Random;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
Expand All @@ -59,17 +59,16 @@ public class ClientExecuteCommand implements Command<ClientCommandArgs> {
private final ClientCommandArgs clientCommandArgs;

private JobStatus jobStatus;

private SeaTunnelClient engineClient;
private HazelcastInstance instance;
private ScheduledExecutorService executorService;
public ClientExecuteCommand(ClientCommandArgs clientCommandArgs) {
this.clientCommandArgs = clientCommandArgs;
}

@SuppressWarnings({"checkstyle:RegexpSingleline", "checkstyle:MagicNumber"})
@Override
public void execute() throws CommandExecuteException {
HazelcastInstance instance = null;
SeaTunnelClient engineClient = null;
ScheduledExecutorService executorService = null;
JobMetricsRunner.JobMetricsSummary jobMetricsSummary = null;
LocalDateTime startTime = LocalDateTime.now();
LocalDateTime endTime = LocalDateTime.now();
Expand Down Expand Up @@ -115,15 +114,17 @@ public void execute() throws CommandExecuteException {
// create job proxy
ClientJobProxy clientJobProxy = jobExecutionEnv.execute();
// register cancelJob hook
if (clientCommandArgs.isCloseJob()) {
Runtime.getRuntime().addShutdownHook(new Thread(() -> {
try {
shutdownHook(clientJobProxy);
} catch (Exception e) {
log.error("Cancel job failed.", e);
}
}));
}
Runtime.getRuntime().addShutdownHook(new Thread(() -> {
CompletableFuture<Void> future = CompletableFuture.runAsync(() -> {
log.info("run shutdown hook because get close signal");
shutdownHook(clientJobProxy);
});
try {
future.get(15, TimeUnit.SECONDS);
} catch (Exception e) {
log.error("Cancel job failed.", e);
}
}));
// get job id
long jobId = clientJobProxy.getJobId();
JobMetricsRunner jobMetricsRunner = new JobMetricsRunner(engineClient, jobId);
Expand All @@ -137,18 +138,9 @@ public void execute() throws CommandExecuteException {
// get job statistic information when job finished
jobMetricsSummary = engineClient.getJobMetricsSummary(jobId);
}
} catch (ExecutionException | InterruptedException e) {
} catch (Exception e) {
throw new CommandExecuteException("SeaTunnel job executed failed", e);
} finally {
if (engineClient != null) {
engineClient.close();
}
if (instance != null) {
instance.shutdown();
}
if (executorService != null) {
executorService.shutdown();
}
if (jobMetricsSummary != null) {
// print job statistics information when job finished
log.info(StringFormatUtils.formatTable(
Expand All @@ -171,6 +163,19 @@ public void execute() throws CommandExecuteException {
"Total Failed Count",
jobMetricsSummary.getSourceReadCount() - jobMetricsSummary.getSinkWriteCount()));
}
closeClient();
}
}

private void closeClient() {
if (engineClient != null) {
engineClient.close();
}
if (instance != null) {
instance.shutdown();
}
if (executorService != null) {
executorService.shutdownNow();
}
}

Expand All @@ -189,9 +194,11 @@ private String creatRandomClusterName(String namePrefix) {
}

private void shutdownHook(ClientJobProxy clientJobProxy) {
if (jobStatus == null || !jobStatus.isEndState()) {
log.warn("Task will be closed due to client shutdown.");
clientJobProxy.cancelJob();
if (clientCommandArgs.isCloseJob()) {
if (jobStatus == null || !jobStatus.isEndState()) {
log.warn("Task will be closed due to client shutdown.");
clientJobProxy.cancelJob();
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,8 +40,11 @@ public class SeaTunnelHazelcastClient {
private final HazelcastClientInstanceImpl hazelcastClient;
private final SerializationService serializationService;

private static final int CONNECT_TIMEOUT = 3000;

public SeaTunnelHazelcastClient(@NonNull ClientConfig clientConfig) {
Preconditions.checkNotNull(clientConfig, "config");
clientConfig.getConnectionStrategyConfig().getConnectionRetryConfig().setClusterConnectTimeoutMillis(CONNECT_TIMEOUT);
this.hazelcastClient =
((HazelcastClientProxy) com.hazelcast.client.HazelcastClient.newHazelcastClient(
clientConfig)).client;
Expand Down