|
@@ -23,7 +23,6 @@ import com.google.common.base.Preconditions;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdds.HddsUtils;
|
|
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
|
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
|
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
|
|
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
|
|
|
import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc;
|
|
@@ -31,6 +30,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServi
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|
|
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
|
|
|
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|
|
+import org.apache.hadoop.hdds.scm.storage.CheckedBiFunction;
|
|
|
import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
|
|
|
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
|
|
import org.apache.hadoop.hdds.tracing.GrpcClientInterceptor;
|
|
@@ -62,7 +62,6 @@ import java.util.concurrent.ExecutionException;
|
|
|
import java.util.concurrent.Semaphore;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
|
-import java.util.stream.Collectors;
|
|
|
|
|
|
/**
|
|
|
* A Client for the storageContainer protocol.
|
|
@@ -83,7 +82,7 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
|
|
* data nodes.
|
|
|
*
|
|
|
* @param pipeline - Pipeline that defines the machines.
|
|
|
- * @param config -- Ozone Config
|
|
|
+ * @param config -- Ozone Config
|
|
|
*/
|
|
|
public XceiverClientGrpc(Pipeline pipeline, Configuration config) {
|
|
|
super();
|
|
@@ -91,7 +90,7 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
|
|
Preconditions.checkNotNull(config);
|
|
|
this.pipeline = pipeline;
|
|
|
this.config = config;
|
|
|
- this.secConfig = new SecurityConfig(config);
|
|
|
+ this.secConfig = new SecurityConfig(config);
|
|
|
this.semaphore =
|
|
|
new Semaphore(HddsClientUtils.getMaxOutstandingRequests(config));
|
|
|
this.metrics = XceiverClientManager.getXceiverClientMetrics();
|
|
@@ -101,7 +100,7 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
|
|
|
|
|
/**
|
|
|
* To be used when grpc token is not enabled.
|
|
|
- * */
|
|
|
+ */
|
|
|
@Override
|
|
|
public void connect() throws Exception {
|
|
|
// leader by default is the 1st datanode in the datanode list of pipleline
|
|
@@ -112,7 +111,7 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
|
|
|
|
|
/**
|
|
|
* Passed encoded token to GRPC header when security is enabled.
|
|
|
- * */
|
|
|
+ */
|
|
|
@Override
|
|
|
public void connect(String encodedToken) throws Exception {
|
|
|
// leader by default is the 1st datanode in the datanode list of pipleline
|
|
@@ -132,11 +131,10 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
|
|
}
|
|
|
|
|
|
// Add credential context to the client call
|
|
|
- String userName = UserGroupInformation.getCurrentUser()
|
|
|
- .getShortUserName();
|
|
|
+ String userName = UserGroupInformation.getCurrentUser().getShortUserName();
|
|
|
LOG.debug("Connecting to server Port : " + dn.getIpAddress());
|
|
|
- NettyChannelBuilder channelBuilder = NettyChannelBuilder.forAddress(dn
|
|
|
- .getIpAddress(), port).usePlaintext()
|
|
|
+ NettyChannelBuilder channelBuilder =
|
|
|
+ NettyChannelBuilder.forAddress(dn.getIpAddress(), port).usePlaintext()
|
|
|
.maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
|
|
|
.intercept(new ClientCredentialInterceptor(userName, encodedToken),
|
|
|
new GrpcClientInterceptor());
|
|
@@ -149,8 +147,8 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
|
|
if (trustCertCollectionFile != null) {
|
|
|
sslContextBuilder.trustManager(trustCertCollectionFile);
|
|
|
}
|
|
|
- if (secConfig.isGrpcMutualTlsRequired() && clientCertChainFile != null &&
|
|
|
- privateKeyFile != null) {
|
|
|
+ if (secConfig.isGrpcMutualTlsRequired() && clientCertChainFile != null
|
|
|
+ && privateKeyFile != null) {
|
|
|
sslContextBuilder.keyManager(clientCertChainFile, privateKeyFile);
|
|
|
}
|
|
|
|
|
@@ -216,49 +214,45 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public XceiverClientReply sendCommand(
|
|
|
- ContainerCommandRequestProto request, List<DatanodeDetails> excludeDns)
|
|
|
+ public ContainerCommandResponseProto sendCommand(
|
|
|
+ ContainerCommandRequestProto request, List<CheckedBiFunction> validators)
|
|
|
throws IOException {
|
|
|
- Preconditions.checkState(HddsUtils.isReadOnly(request));
|
|
|
- return sendCommandWithTraceIDAndRetry(request, excludeDns);
|
|
|
+ try {
|
|
|
+ XceiverClientReply reply;
|
|
|
+ reply = sendCommandWithTraceIDAndRetry(request, validators);
|
|
|
+ ContainerCommandResponseProto responseProto = reply.getResponse().get();
|
|
|
+ return responseProto;
|
|
|
+ } catch (ExecutionException | InterruptedException e) {
|
|
|
+ throw new IOException("Failed to execute command " + request, e);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private XceiverClientReply sendCommandWithTraceIDAndRetry(
|
|
|
- ContainerCommandRequestProto request, List<DatanodeDetails> excludeDns)
|
|
|
+ ContainerCommandRequestProto request, List<CheckedBiFunction> validators)
|
|
|
throws IOException {
|
|
|
try (Scope scope = GlobalTracer.get()
|
|
|
.buildSpan("XceiverClientGrpc." + request.getCmdType().name())
|
|
|
.startActive(true)) {
|
|
|
ContainerCommandRequestProto finalPayload =
|
|
|
ContainerCommandRequestProto.newBuilder(request)
|
|
|
- .setTraceID(TracingUtil.exportCurrentSpan())
|
|
|
- .build();
|
|
|
- return sendCommandWithRetry(finalPayload, excludeDns);
|
|
|
+ .setTraceID(TracingUtil.exportCurrentSpan()).build();
|
|
|
+ return sendCommandWithRetry(finalPayload, validators);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
private XceiverClientReply sendCommandWithRetry(
|
|
|
- ContainerCommandRequestProto request, List<DatanodeDetails> excludeDns)
|
|
|
+ ContainerCommandRequestProto request, List<CheckedBiFunction> validators)
|
|
|
throws IOException {
|
|
|
ContainerCommandResponseProto responseProto = null;
|
|
|
+ IOException ioException = null;
|
|
|
|
|
|
// In case of an exception or an error, we will try to read from the
|
|
|
// datanodes in the pipeline in a round robin fashion.
|
|
|
|
|
|
// TODO: cache the correct leader info in here, so that any subsequent calls
|
|
|
// should first go to leader
|
|
|
- List<DatanodeDetails> dns = pipeline.getNodes();
|
|
|
- List<DatanodeDetails> healthyDns =
|
|
|
- excludeDns != null ? dns.stream().filter(dnId -> {
|
|
|
- for (DatanodeDetails excludeId : excludeDns) {
|
|
|
- if (dnId.equals(excludeId)) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- }
|
|
|
- return true;
|
|
|
- }).collect(Collectors.toList()) : dns;
|
|
|
XceiverClientReply reply = new XceiverClientReply(null);
|
|
|
- for (DatanodeDetails dn : healthyDns) {
|
|
|
+ for (DatanodeDetails dn : pipeline.getNodes()) {
|
|
|
try {
|
|
|
LOG.debug("Executing command " + request + " on datanode " + dn);
|
|
|
// In case the command gets retried on a 2nd datanode,
|
|
@@ -266,17 +260,26 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
|
|
// in case these don't exist for the specific datanode.
|
|
|
reply.addDatanode(dn);
|
|
|
responseProto = sendCommandAsync(request, dn).getResponse().get();
|
|
|
- if (responseProto.getResult() == ContainerProtos.Result.SUCCESS) {
|
|
|
- break;
|
|
|
+ if (validators != null && !validators.isEmpty()) {
|
|
|
+ for (CheckedBiFunction validator : validators) {
|
|
|
+ validator.apply(request, responseProto);
|
|
|
+ }
|
|
|
}
|
|
|
- } catch (ExecutionException | InterruptedException e) {
|
|
|
+ break;
|
|
|
+ } catch (ExecutionException | InterruptedException | IOException e) {
|
|
|
LOG.debug("Failed to execute command " + request + " on datanode " + dn
|
|
|
.getUuidString(), e);
|
|
|
- if (Status.fromThrowable(e.getCause()).getCode()
|
|
|
- == Status.UNAUTHENTICATED.getCode()) {
|
|
|
- throw new SCMSecurityException("Failed to authenticate with "
|
|
|
- + "GRPC XceiverServer with Ozone block token.");
|
|
|
+ if (!(e instanceof IOException)) {
|
|
|
+ if (Status.fromThrowable(e.getCause()).getCode()
|
|
|
+ == Status.UNAUTHENTICATED.getCode()) {
|
|
|
+ throw new SCMSecurityException("Failed to authenticate with "
|
|
|
+ + "GRPC XceiverServer with Ozone block token.");
|
|
|
+ }
|
|
|
+ ioException = new IOException(e);
|
|
|
+ } else {
|
|
|
+ ioException = (IOException) e;
|
|
|
}
|
|
|
+ responseProto = null;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -284,9 +287,10 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
|
|
reply.setResponse(CompletableFuture.completedFuture(responseProto));
|
|
|
return reply;
|
|
|
} else {
|
|
|
- throw new IOException(
|
|
|
- "Failed to execute command " + request + " on the pipeline "
|
|
|
- + pipeline.getId());
|
|
|
+ Preconditions.checkNotNull(ioException);
|
|
|
+ LOG.error("Failed to execute command " + request + " on the pipeline "
|
|
|
+ + pipeline.getId());
|
|
|
+ throw ioException;
|
|
|
}
|
|
|
}
|
|
|
|