|
@@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
|
|
|
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -40,6 +41,7 @@ import java.io.IOException;
|
|
|
import java.util.concurrent.CompletableFuture;
|
|
|
import java.util.concurrent.ExecutionException;
|
|
|
import java.util.List;
|
|
|
+import java.util.concurrent.Semaphore;
|
|
|
|
|
|
/**
|
|
|
* A Client for the storageContainer protocol.
|
|
@@ -51,6 +53,7 @@ public class XceiverClient extends XceiverClientSpi {
|
|
|
private Channel channel;
|
|
|
private Bootstrap b;
|
|
|
private EventLoopGroup group;
|
|
|
+ private final Semaphore semaphore;
|
|
|
|
|
|
/**
|
|
|
* Constructs a client that can communicate with the Container framework on
|
|
@@ -65,6 +68,8 @@ public class XceiverClient extends XceiverClientSpi {
|
|
|
Preconditions.checkNotNull(config);
|
|
|
this.pipeline = pipeline;
|
|
|
this.config = config;
|
|
|
+ this.semaphore =
|
|
|
+ new Semaphore(OzoneClientUtils.getMaxOutstandingRequests(config));
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -78,7 +83,7 @@ public class XceiverClient extends XceiverClientSpi {
|
|
|
b.group(group)
|
|
|
.channel(NioSocketChannel.class)
|
|
|
.handler(new LoggingHandler(LogLevel.INFO))
|
|
|
- .handler(new XceiverClientInitializer(this.pipeline));
|
|
|
+ .handler(new XceiverClientInitializer(this.pipeline, semaphore));
|
|
|
DatanodeID leader = this.pipeline.getLeader();
|
|
|
|
|
|
// read port from the data node, on failure use default configured
|
|
@@ -116,8 +121,7 @@ public class XceiverClient extends XceiverClientSpi {
|
|
|
|
|
|
@Override
|
|
|
public ContainerProtos.ContainerCommandResponseProto sendCommand(
|
|
|
- ContainerProtos.ContainerCommandRequestProto request)
|
|
|
- throws IOException {
|
|
|
+ ContainerProtos.ContainerCommandRequestProto request) throws IOException {
|
|
|
try {
|
|
|
if ((channel == null) || (!channel.isActive())) {
|
|
|
throw new IOException("This channel is not connected.");
|
|
@@ -127,7 +131,20 @@ public class XceiverClient extends XceiverClientSpi {
|
|
|
|
|
|
return handler.sendCommand(request);
|
|
|
} catch (ExecutionException | InterruptedException e) {
|
|
|
- throw new IOException("Unexpected exception during execution", e);
|
|
|
+ /**
|
|
|
+ * In case the netty channel handler throws an exception,
|
|
|
+ * the exception thrown will be wrapped within {@link ExecutionException}.
|
|
|
+ * Unwarpping here so that original exception gets passed
|
|
|
+ * to to the client.
|
|
|
+ */
|
|
|
+ if (e instanceof ExecutionException) {
|
|
|
+ Throwable cause = e.getCause();
|
|
|
+ if (cause instanceof IOException) {
|
|
|
+ throw (IOException) cause;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ throw new IOException(
|
|
|
+ "Unexpected exception during execution:" + e.getMessage());
|
|
|
}
|
|
|
}
|
|
|
|