|
@@ -30,12 +30,14 @@ import java.util.Random;
|
|
|
|
|
|
import javax.crypto.SecretKey;
|
|
|
|
|
|
+import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.io.FloatWritable;
|
|
|
+import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.io.NullWritable;
|
|
|
import org.apache.hadoop.io.Writable;
|
|
|
import org.apache.hadoop.io.WritableComparable;
|
|
@@ -52,6 +54,7 @@ import org.apache.hadoop.mapreduce.security.TokenCache;
|
|
|
import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
|
|
|
import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
import org.slf4j.Logger;
|
|
@@ -66,6 +69,7 @@ class Application<K1 extends WritableComparable, V1 extends Writable,
|
|
|
private static final Logger LOG =
|
|
|
LoggerFactory.getLogger(Application.class.getName());
|
|
|
private ServerSocket serverSocket;
|
|
|
+ private PingSocketCleaner socketCleaner;
|
|
|
private Process process;
|
|
|
private Socket clientSocket;
|
|
|
private OutputHandler<K2, V2> handler;
|
|
@@ -133,6 +137,13 @@ class Application<K1 extends WritableComparable, V1 extends Writable,
|
|
|
|
|
|
process = runClient(cmd, env);
|
|
|
clientSocket = serverSocket.accept();
|
|
|
+ // start ping socket cleaner
|
|
|
+ int soTimeout = conf.getInt(CommonConfigurationKeys.IPC_PING_INTERVAL_KEY,
|
|
|
+ CommonConfigurationKeys.IPC_PING_INTERVAL_DEFAULT);
|
|
|
+ socketCleaner = new PingSocketCleaner("ping-socket-cleaner", serverSocket,
|
|
|
+ soTimeout);
|
|
|
+ socketCleaner.setDaemon(true);
|
|
|
+ socketCleaner.start();
|
|
|
|
|
|
String challenge = getSecurityChallenge();
|
|
|
String digestToSend = createDigest(password, challenge);
|
|
@@ -237,6 +248,7 @@ class Application<K1 extends WritableComparable, V1 extends Writable,
|
|
|
serverSocket.close();
|
|
|
try {
|
|
|
downlink.close();
|
|
|
+ socketCleaner.interrupt();
|
|
|
} catch (InterruptedException ie) {
|
|
|
Thread.currentThread().interrupt();
|
|
|
}
|
|
@@ -266,4 +278,44 @@ class Application<K1 extends WritableComparable, V1 extends Writable,
|
|
|
return SecureShuffleUtils.hashFromString(data, key);
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ public static class PingSocketCleaner extends Thread {
|
|
|
+ private final ServerSocket serverSocket;
|
|
|
+ private final int soTimeout;
|
|
|
+
|
|
|
+ PingSocketCleaner(String name, ServerSocket serverSocket, int soTimeout) {
|
|
|
+ super(name);
|
|
|
+ this.serverSocket = serverSocket;
|
|
|
+ this.soTimeout = soTimeout;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ LOG.info("PingSocketCleaner started...");
|
|
|
+ while (!Thread.currentThread().isInterrupted()) {
|
|
|
+ Socket clientSocket = null;
|
|
|
+ try {
|
|
|
+ clientSocket = serverSocket.accept();
|
|
|
+ clientSocket.setSoTimeout(soTimeout);
|
|
|
+ LOG.debug("Connection received from {}",
|
|
|
+ clientSocket.getInetAddress());
|
|
|
+ int readData = 0;
|
|
|
+ while (readData != -1) {
|
|
|
+ readData = clientSocket.getInputStream().read();
|
|
|
+ }
|
|
|
+ LOG.debug("close socket cause client has closed.");
|
|
|
+ closeSocketInternal(clientSocket);
|
|
|
+ } catch (IOException exception) {
|
|
|
+ LOG.error("PingSocketCleaner exception", exception);
|
|
|
+ } finally {
|
|
|
+ closeSocketInternal(clientSocket);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ protected void closeSocketInternal(Socket clientSocket) {
|
|
|
+ IOUtils.closeSocket(clientSocket);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|