|
@@ -18,10 +18,10 @@
|
|
|
|
|
|
package org.apache.hadoop.ipc;
|
|
|
|
|
|
+import org.apache.commons.lang3.tuple.Pair;
|
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.classification.VisibleForTesting;
|
|
|
import org.apache.hadoop.util.Preconditions;
|
|
|
-import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
@@ -166,73 +166,6 @@ public class Client implements AutoCloseable {
|
|
|
private final int maxAsyncCalls;
|
|
|
private final AtomicInteger asyncCallCounter = new AtomicInteger(0);
|
|
|
|
|
|
- /**
|
|
|
- * Executor on which IPC calls' parameters are sent.
|
|
|
- * Deferring the sending of parameters to a separate
|
|
|
- * thread isolates them from thread interruptions in the
|
|
|
- * calling code.
|
|
|
- */
|
|
|
- private final ExecutorService sendParamsExecutor;
|
|
|
- private final static ClientExecutorServiceFactory clientExcecutorFactory =
|
|
|
- new ClientExecutorServiceFactory();
|
|
|
-
|
|
|
- private static class ClientExecutorServiceFactory {
|
|
|
- private int executorRefCount = 0;
|
|
|
- private ExecutorService clientExecutor = null;
|
|
|
-
|
|
|
- /**
|
|
|
- * Get Executor on which IPC calls' parameters are sent.
|
|
|
- * If the internal reference counter is zero, this method
|
|
|
- * creates the instance of Executor. If not, this method
|
|
|
- * just returns the reference of clientExecutor.
|
|
|
- *
|
|
|
- * @return An ExecutorService instance
|
|
|
- */
|
|
|
- synchronized ExecutorService refAndGetInstance() {
|
|
|
- if (executorRefCount == 0) {
|
|
|
- clientExecutor = Executors.newCachedThreadPool(
|
|
|
- new ThreadFactoryBuilder()
|
|
|
- .setDaemon(true)
|
|
|
- .setNameFormat("IPC Parameter Sending Thread #%d")
|
|
|
- .build());
|
|
|
- }
|
|
|
- executorRefCount++;
|
|
|
-
|
|
|
- return clientExecutor;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Cleanup Executor on which IPC calls' parameters are sent.
|
|
|
- * If reference counter is zero, this method discards the
|
|
|
- * instance of the Executor. If not, this method
|
|
|
- * just decrements the internal reference counter.
|
|
|
- *
|
|
|
- * @return An ExecutorService instance if it exists.
|
|
|
- * Null is returned if not.
|
|
|
- */
|
|
|
- synchronized ExecutorService unrefAndCleanup() {
|
|
|
- executorRefCount--;
|
|
|
- assert(executorRefCount >= 0);
|
|
|
-
|
|
|
- if (executorRefCount == 0) {
|
|
|
- clientExecutor.shutdown();
|
|
|
- try {
|
|
|
- if (!clientExecutor.awaitTermination(1, TimeUnit.MINUTES)) {
|
|
|
- clientExecutor.shutdownNow();
|
|
|
- }
|
|
|
- } catch (InterruptedException e) {
|
|
|
- LOG.warn("Interrupted while waiting for clientExecutor" +
|
|
|
- " to stop");
|
|
|
- clientExecutor.shutdownNow();
|
|
|
- Thread.currentThread().interrupt();
|
|
|
- }
|
|
|
- clientExecutor = null;
|
|
|
- }
|
|
|
-
|
|
|
- return clientExecutor;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* set the ping interval value in configuration
|
|
|
*
|
|
@@ -301,11 +234,6 @@ public class Client implements AutoCloseable {
|
|
|
conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_TIMEOUT_KEY, timeout);
|
|
|
}
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
- public static final ExecutorService getClientExecutor() {
|
|
|
- return Client.clientExcecutorFactory.clientExecutor;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Increment this client's reference count
|
|
|
*/
|
|
@@ -462,8 +390,10 @@ public class Client implements AutoCloseable {
|
|
|
private AtomicLong lastActivity = new AtomicLong();// last I/O activity time
|
|
|
private AtomicBoolean shouldCloseConnection = new AtomicBoolean(); // indicate if the connection is closed
|
|
|
private IOException closeException; // close reason
|
|
|
-
|
|
|
- private final Object sendRpcRequestLock = new Object();
|
|
|
+
|
|
|
+ private final Thread rpcRequestThread;
|
|
|
+ private final SynchronousQueue<Pair<Call, ResponseBuffer>> rpcRequestQueue =
|
|
|
+ new SynchronousQueue<>(true);
|
|
|
|
|
|
private AtomicReference<Thread> connectingThread = new AtomicReference<>();
|
|
|
private final Consumer<Connection> removeMethod;
|
|
@@ -472,6 +402,9 @@ public class Client implements AutoCloseable {
|
|
|
Consumer<Connection> removeMethod) {
|
|
|
this.remoteId = remoteId;
|
|
|
this.server = remoteId.getAddress();
|
|
|
+ this.rpcRequestThread = new Thread(new RpcRequestSender(),
|
|
|
+ "IPC Parameter Sending Thread for " + remoteId);
|
|
|
+ this.rpcRequestThread.setDaemon(true);
|
|
|
|
|
|
this.maxResponseLength = remoteId.conf.getInt(
|
|
|
CommonConfigurationKeys.IPC_MAXIMUM_RESPONSE_LENGTH,
|
|
@@ -1150,6 +1083,10 @@ public class Client implements AutoCloseable {
|
|
|
|
|
|
@Override
|
|
|
public void run() {
|
|
|
+ // Don't start the ipc parameter sending thread until we start this
|
|
|
+ // thread, because the shutdown logic only gets triggered if this
|
|
|
+ // thread is started.
|
|
|
+ rpcRequestThread.start();
|
|
|
if (LOG.isDebugEnabled())
|
|
|
LOG.debug(getName() + ": starting, having connections "
|
|
|
+ connections.size());
|
|
@@ -1173,9 +1110,52 @@ public class Client implements AutoCloseable {
|
|
|
+ connections.size());
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * A thread to write rpc requests to the socket.
|
|
|
+ */
|
|
|
+ private class RpcRequestSender implements Runnable {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ while (!shouldCloseConnection.get()) {
|
|
|
+ ResponseBuffer buf = null;
|
|
|
+ try {
|
|
|
+ Pair<Call, ResponseBuffer> pair =
|
|
|
+ rpcRequestQueue.poll(maxIdleTime, TimeUnit.MILLISECONDS);
|
|
|
+ if (pair == null || shouldCloseConnection.get()) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ buf = pair.getRight();
|
|
|
+ synchronized (ipcStreams.out) {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ Call call = pair.getLeft();
|
|
|
+ LOG.debug(getName() + "{} sending #{} {}", getName(), call.id,
|
|
|
+ call.rpcRequest);
|
|
|
+ }
|
|
|
+ // RpcRequestHeader + RpcRequest
|
|
|
+ ipcStreams.sendRequest(buf.toByteArray());
|
|
|
+ ipcStreams.flush();
|
|
|
+ }
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ // stop this thread
|
|
|
+ return;
|
|
|
+ } catch (IOException e) {
|
|
|
+ // exception at this point would leave the connection in an
|
|
|
+ // unrecoverable state (eg half a call left on the wire).
|
|
|
+ // So, close the connection, killing any outstanding calls
|
|
|
+ markClosed(e);
|
|
|
+ } finally {
|
|
|
+ //the buffer is just an in-memory buffer, but it is still polite to
|
|
|
+ // close early
|
|
|
+ IOUtils.closeStream(buf);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/** Initiates a rpc call by sending the rpc request to the remote server.
|
|
|
- * Note: this is not called from the Connection thread, but by other
|
|
|
- * threads.
|
|
|
+ * Note: this is not called from the current thread, but by another
|
|
|
+ * thread, so that if the current thread is interrupted that the socket
|
|
|
+ * state isn't corrupted with a partially written message.
|
|
|
* @param call - the rpc request
|
|
|
*/
|
|
|
public void sendRpcRequest(final Call call)
|
|
@@ -1185,8 +1165,7 @@ public class Client implements AutoCloseable {
|
|
|
}
|
|
|
|
|
|
// Serialize the call to be sent. This is done from the actual
|
|
|
- // caller thread, rather than the sendParamsExecutor thread,
|
|
|
-
|
|
|
+ // caller thread, rather than the rpcRequestThread in the connection,
|
|
|
// so that if the serialization throws an error, it is reported
|
|
|
// properly. This also parallelizes the serialization.
|
|
|
//
|
|
@@ -1203,51 +1182,7 @@ public class Client implements AutoCloseable {
|
|
|
final ResponseBuffer buf = new ResponseBuffer();
|
|
|
header.writeDelimitedTo(buf);
|
|
|
RpcWritable.wrap(call.rpcRequest).writeTo(buf);
|
|
|
-
|
|
|
- synchronized (sendRpcRequestLock) {
|
|
|
- Future<?> senderFuture = sendParamsExecutor.submit(new Runnable() {
|
|
|
- @Override
|
|
|
- public void run() {
|
|
|
- try {
|
|
|
- synchronized (ipcStreams.out) {
|
|
|
- if (shouldCloseConnection.get()) {
|
|
|
- return;
|
|
|
- }
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug(getName() + " sending #" + call.id
|
|
|
- + " " + call.rpcRequest);
|
|
|
- }
|
|
|
- // RpcRequestHeader + RpcRequest
|
|
|
- ipcStreams.sendRequest(buf.toByteArray());
|
|
|
- ipcStreams.flush();
|
|
|
- }
|
|
|
- } catch (IOException e) {
|
|
|
- // exception at this point would leave the connection in an
|
|
|
- // unrecoverable state (eg half a call left on the wire).
|
|
|
- // So, close the connection, killing any outstanding calls
|
|
|
- markClosed(e);
|
|
|
- } finally {
|
|
|
- //the buffer is just an in-memory buffer, but it is still polite to
|
|
|
- // close early
|
|
|
- IOUtils.closeStream(buf);
|
|
|
- }
|
|
|
- }
|
|
|
- });
|
|
|
-
|
|
|
- try {
|
|
|
- senderFuture.get();
|
|
|
- } catch (ExecutionException e) {
|
|
|
- Throwable cause = e.getCause();
|
|
|
-
|
|
|
- // cause should only be a RuntimeException as the Runnable above
|
|
|
- // catches IOException
|
|
|
- if (cause instanceof RuntimeException) {
|
|
|
- throw (RuntimeException) cause;
|
|
|
- } else {
|
|
|
- throw new RuntimeException("unexpected checked exception", cause);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
+ rpcRequestQueue.put(Pair.of(call, buf));
|
|
|
}
|
|
|
|
|
|
/* Receive a response.
|
|
@@ -1396,7 +1331,6 @@ public class Client implements AutoCloseable {
|
|
|
CommonConfigurationKeys.IPC_CLIENT_BIND_WILDCARD_ADDR_DEFAULT);
|
|
|
|
|
|
this.clientId = ClientId.getClientId();
|
|
|
- this.sendParamsExecutor = clientExcecutorFactory.refAndGetInstance();
|
|
|
this.maxAsyncCalls = conf.getInt(
|
|
|
CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
|
|
|
CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_DEFAULT);
|
|
@@ -1440,6 +1374,7 @@ public class Client implements AutoCloseable {
|
|
|
// wake up all connections
|
|
|
for (Connection conn : connections.values()) {
|
|
|
conn.interrupt();
|
|
|
+ conn.rpcRequestThread.interrupt();
|
|
|
conn.interruptConnectingThread();
|
|
|
}
|
|
|
|
|
@@ -1456,7 +1391,6 @@ public class Client implements AutoCloseable {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
- clientExcecutorFactory.unrefAndCleanup();
|
|
|
}
|
|
|
|
|
|
/**
|