Skip to content

Commit 55be7d5

Browse files
committed
[server] Register signal handlers to log termination due to SIGTERM, SIGHUP and SIGINT
1 parent ae84521 commit 55be7d5

File tree

3 files changed

+101
-1
lines changed

3 files changed

+101
-1
lines changed

fluss-server/src/main/java/org/apache/fluss/server/ServerBase.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@
3232
import org.apache.fluss.server.utils.ConfigurationParserUtils;
3333
import org.apache.fluss.server.utils.FatalErrorHandler;
3434
import org.apache.fluss.server.utils.ShutdownHookUtil;
35+
import org.apache.fluss.server.utils.SignalHandler;
3536
import org.apache.fluss.utils.AutoCloseableAsync;
3637
import org.apache.fluss.utils.ExceptionUtils;
3738
import org.apache.fluss.utils.concurrent.FutureUtils;
@@ -113,6 +114,7 @@ protected static void startServer(ServerBase server) {
113114
}
114115

115116
public void start() throws Exception {
117+
SignalHandler.register(LOG);
116118
try {
117119
addShutDownHook();
118120

fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -284,7 +284,7 @@ protected void startServices() throws Exception {
284284
@Override
285285
protected CompletableFuture<Result> closeAsync(Result result) {
286286
if (isShutDown.compareAndSet(false, true)) {
287-
287+
LOG.info("Shutting down Tablet server ({}).", result);
288288
controlledShutDown();
289289

290290
CompletableFuture<Void> serviceShutdownFuture = stopServices();
Lines changed: 98 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,98 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.fluss.server.utils;
19+
20+
import org.apache.fluss.utils.OperatingSystem;
21+
22+
import org.slf4j.Logger;
23+
import sun.misc.Signal;
24+
25+
/* This file is based on source code of Apache Flink Project (https://flink.apache.org/), licensed by the Apache
26+
* Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for
27+
* additional information regarding copyright ownership. */
28+
29+
/**
30+
* This signal handler / signal logger is based on Apache Hadoop's
31+
* org.apache.hadoop.util.SignalLogger.
32+
*/
33+
public class SignalHandler {
34+
35+
private static boolean registered = false;
36+
37+
/** Our signal handler. */
38+
private static class Handler implements sun.misc.SignalHandler {
39+
40+
private final Logger log;
41+
private final sun.misc.SignalHandler prevHandler;
42+
43+
Handler(String name, Logger log) {
44+
this.log = log;
45+
prevHandler = Signal.handle(new Signal(name), this);
46+
}
47+
48+
/**
49+
* Handle an incoming signal.
50+
*
51+
* @param signal The incoming signal
52+
*/
53+
@Override
54+
public void handle(Signal signal) {
55+
log.warn(
56+
"RECEIVED SIGNAL {}: SIG{}. Shutting down as requested.",
57+
signal.getNumber(),
58+
signal.getName());
59+
prevHandler.handle(signal);
60+
}
61+
}
62+
63+
/**
64+
* Register some signal handlers.
65+
*
66+
* @param log The slf4j logger
67+
*/
68+
public static void register(final Logger log) {
69+
synchronized (SignalHandler.class) {
70+
if (registered) {
71+
return;
72+
}
73+
registered = true;
74+
75+
final String[] signals =
76+
OperatingSystem.isWindows()
77+
? new String[] {"TERM", "INT"}
78+
: new String[] {"TERM", "HUP", "INT"};
79+
80+
StringBuilder bld = new StringBuilder();
81+
bld.append("Registered UNIX signal handlers for [");
82+
83+
String separator = "";
84+
for (String signalName : signals) {
85+
try {
86+
new Handler(signalName, log);
87+
bld.append(separator);
88+
bld.append(signalName);
89+
separator = ", ";
90+
} catch (Exception e) {
91+
log.info("Error while registering signal handler", e);
92+
}
93+
}
94+
bld.append("]");
95+
log.info(bld.toString());
96+
}
97+
}
98+
}

0 commit comments

Comments
 (0)