Skip to content

Commit 2beeee3

Browse files
committed
HADOOP-19329. Remove directly usage of sun.misc.Signal
1 parent 6eae158 commit 2beeee3

File tree

3 files changed

+165
-17
lines changed

3 files changed

+165
-17
lines changed

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/IrqHandler.java

Lines changed: 7 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -21,10 +21,9 @@
2121
import java.util.concurrent.atomic.AtomicInteger;
2222

2323
import org.apache.hadoop.util.Preconditions;
24+
import org.apache.hadoop.util.SignalUtil;
2425
import org.slf4j.Logger;
2526
import org.slf4j.LoggerFactory;
26-
import sun.misc.Signal;
27-
import sun.misc.SignalHandler;
2827

2928
import org.apache.hadoop.classification.InterfaceAudience;
3029
import org.apache.hadoop.classification.InterfaceStability;
@@ -39,7 +38,7 @@
3938
@InterfaceAudience.Private
4039
@InterfaceStability.Unstable
4140
@SuppressWarnings("UseOfSunClasses")
42-
public final class IrqHandler implements SignalHandler {
41+
public final class IrqHandler implements SignalUtil.Handler {
4342
private static final Logger LOG = LoggerFactory.getLogger(IrqHandler.class);
4443

4544
/**
@@ -68,7 +67,7 @@ public final class IrqHandler implements SignalHandler {
6867
/**
6968
* Stored signal.
7069
*/
71-
private Signal signal;
70+
private SignalUtil.Signal signal;
7271

7372
/**
7473
* Create an IRQ handler bound to the specific interrupt.
@@ -89,8 +88,8 @@ public IrqHandler(String name, Interrupted handler) {
8988
public void bind() {
9089
Preconditions.checkState(signal == null, "Handler already bound");
9190
try {
92-
signal = new Signal(name);
93-
Signal.handle(signal, this);
91+
signal = new SignalUtil.Signal(name);
92+
SignalUtil.handle(signal, this);
9493
} catch (IllegalArgumentException e) {
9594
throw new IllegalArgumentException(
9695
"Could not set handler for signal \"" + name + "\"."
@@ -110,7 +109,7 @@ public String getName() {
110109
* Raise the signal.
111110
*/
112111
public void raise() {
113-
Signal.raise(signal);
112+
SignalUtil.raise(signal);
114113
}
115114

116115
@Override
@@ -123,7 +122,7 @@ public String toString() {
123122
* @param s signal raised
124123
*/
125124
@Override
126-
public void handle(Signal s) {
125+
public void handle(SignalUtil.Signal s) {
127126
signalCount.incrementAndGet();
128127
InterruptData data = new InterruptData(s.getName(), s.getNumber());
129128
LOG.info("Interrupted: {}", data);

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SignalLogger.java

Lines changed: 6 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -19,8 +19,6 @@
1919
package org.apache.hadoop.util;
2020

2121
import org.slf4j.Logger;
22-
import sun.misc.Signal;
23-
import sun.misc.SignalHandler;
2422

2523
import org.apache.hadoop.classification.InterfaceAudience;
2624
import org.apache.hadoop.classification.InterfaceStability;
@@ -41,13 +39,13 @@ public enum SignalLogger {
4139
/**
4240
* Our signal handler.
4341
*/
44-
private static class Handler implements SignalHandler {
42+
private static class Handler implements SignalUtil.Handler {
4543
final private Logger log;
46-
final private SignalHandler prevHandler;
44+
final private SignalUtil.Handler prevHandler;
4745

4846
Handler(String name, Logger log) {
4947
this.log = log;
50-
prevHandler = Signal.handle(new Signal(name), this);
48+
prevHandler = SignalUtil.handle(new SignalUtil.Signal(name), this);
5149
}
5250

5351
/**
@@ -56,9 +54,8 @@ private static class Handler implements SignalHandler {
5654
* @param signal The incoming signal
5755
*/
5856
@Override
59-
public void handle(Signal signal) {
60-
log.error("RECEIVED SIGNAL " + signal.getNumber() +
61-
": SIG" + signal.getName());
57+
public void handle(SignalUtil.Signal signal) {
58+
log.error("RECEIVED SIGNAL {}: SIG{}", signal.getNumber(), signal.getName());
6259
prevHandler.handle(signal);
6360
}
6461
}
@@ -75,7 +72,7 @@ public void register(final Logger log) {
7572
registered = true;
7673
StringBuilder bld = new StringBuilder();
7774
bld.append("registered UNIX signal handlers for [");
78-
final String SIGNALS[] = { "TERM", "HUP", "INT" };
75+
final String[] SIGNALS = { "TERM", "HUP", "INT" };
7976
String separator = "";
8077
for (String signalName : SIGNALS) {
8178
try {
Lines changed: 152 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,152 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.hadoop.util;
20+
21+
import org.apache.hadoop.classification.InterfaceAudience;
22+
23+
import org.apache.hadoop.util.dynamic.BindingUtils;
24+
import org.apache.hadoop.util.dynamic.DynConstructors;
25+
import org.apache.hadoop.util.dynamic.DynMethods;
26+
27+
import java.lang.reflect.InvocationHandler;
28+
import java.lang.reflect.Proxy;
29+
30+
@InterfaceAudience.Private
31+
public class SignalUtil {
32+
33+
static final Class<?> jdkSignalClazz =
34+
BindingUtils.loadClassSafely("sun.misc.Signal");
35+
static final Class<?> jdkSignalHandlerClazz =
36+
BindingUtils.loadClassSafely("sun.misc.SignalHandler");
37+
38+
static DynConstructors.Ctor<?> jdkSignalCtor =
39+
new DynConstructors.Builder()
40+
.impl(jdkSignalClazz, String.class)
41+
.build();
42+
43+
static DynMethods.StaticMethod jdkSignalHandleStaticMethod =
44+
new DynMethods.Builder("handle")
45+
.impl(jdkSignalClazz, jdkSignalClazz, jdkSignalHandlerClazz)
46+
.buildStatic();
47+
48+
static DynMethods.StaticMethod jdkSignalRaiseStaticMethod =
49+
new DynMethods.Builder("raise")
50+
.impl(jdkSignalClazz, jdkSignalClazz)
51+
.buildStatic();
52+
53+
static DynMethods.UnboundMethod jdkSignalHandlerHandleMethod =
54+
new DynMethods.Builder("handle")
55+
.impl(jdkSignalHandlerClazz, jdkSignalClazz)
56+
.build();
57+
58+
@InterfaceAudience.Private
59+
public static class Signal {
60+
private final static DynMethods.UnboundMethod getNumberMethod =
61+
new DynMethods.Builder("getNumber").impl(jdkSignalClazz).build();
62+
63+
private final static DynMethods.UnboundMethod getNameMethod =
64+
new DynMethods.Builder("getName").impl(jdkSignalClazz).build();
65+
66+
private final Object delegate;
67+
68+
public Signal(String name) {
69+
this.delegate = jdkSignalCtor.newInstance(name);
70+
}
71+
72+
public Signal(Object delegate) {
73+
Preconditions.checkArgument(jdkSignalClazz.isInstance(delegate),
74+
String.format("Expected class is '%s', but actual class is '%s'",
75+
jdkSignalClazz.getName(), delegate.getClass().getName()));
76+
this.delegate = delegate;
77+
}
78+
79+
public int getNumber() {
80+
return getNumberMethod.bind(delegate).invoke();
81+
}
82+
83+
public String getName() {
84+
return getNameMethod.bind(delegate).invoke();
85+
}
86+
87+
public boolean equals(Object obj) {
88+
if (this == obj) {
89+
return true;
90+
}
91+
if (obj instanceof Signal) {
92+
return delegate.equals(((Signal)obj).delegate);
93+
}
94+
return false;
95+
}
96+
97+
public int hashCode() {
98+
return delegate.hashCode();
99+
}
100+
101+
public String toString() {
102+
return delegate.toString();
103+
}
104+
}
105+
106+
@InterfaceAudience.Private
107+
public interface Handler {
108+
void handle(Signal sig);
109+
}
110+
111+
static class JdkSignalHandlerImpl implements Handler {
112+
113+
private final Object delegate;
114+
115+
JdkSignalHandlerImpl(Handler handler) {
116+
this.delegate = Proxy.newProxyInstance(
117+
getClass().getClassLoader(),
118+
new Class<?>[] { jdkSignalHandlerClazz },
119+
(proxyObj, method, args) -> {
120+
if ("handle".equals(method.getName()) && args.length == 1 && jdkSignalClazz.isInstance(args[0])) {
121+
handler.handle(new Signal(args[0]));
122+
return null;
123+
} else {
124+
return InvocationHandler.invokeDefault(proxyObj, method, args);
125+
}
126+
}
127+
);
128+
}
129+
130+
JdkSignalHandlerImpl(Object delegate) {
131+
Preconditions.checkArgument(jdkSignalHandlerClazz.isInstance(delegate),
132+
String.format("Expected class is '%s', but actual class is '%s'",
133+
jdkSignalHandlerClazz.getName(), delegate.getClass().getName()));
134+
this.delegate = delegate;
135+
}
136+
137+
@Override
138+
public void handle(Signal sig) {
139+
jdkSignalHandlerHandleMethod.bind(delegate).invoke(sig.delegate);
140+
}
141+
}
142+
143+
public static Handler handle(Signal sig, Handler handler) {
144+
Object preHandle = jdkSignalHandleStaticMethod.invoke(
145+
sig.delegate, new JdkSignalHandlerImpl(handler).delegate);
146+
return new JdkSignalHandlerImpl(preHandle);
147+
}
148+
149+
public static void raise(Signal sig) throws IllegalArgumentException {
150+
jdkSignalRaiseStaticMethod.invoke(sig.delegate);
151+
}
152+
}

0 commit comments

Comments
 (0)