|
@@ -18,52 +18,54 @@
|
|
|
|
|
|
package org.apache.hadoop.hdds.scm;
|
|
|
|
|
|
-import com.google.common.annotations.VisibleForTesting;
|
|
|
+import java.io.IOException;
|
|
|
+import java.util.Collection;
|
|
|
+import java.util.List;
|
|
|
+import java.util.Objects;
|
|
|
+import java.util.OptionalLong;
|
|
|
+import java.util.UUID;
|
|
|
+import java.util.concurrent.CompletableFuture;
|
|
|
+import java.util.concurrent.CompletionException;
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
|
+import java.util.concurrent.ExecutionException;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.concurrent.TimeoutException;
|
|
|
+import java.util.concurrent.atomic.AtomicReference;
|
|
|
+import java.util.stream.Collectors;
|
|
|
+
|
|
|
+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.security.x509.SecurityConfig;
|
|
|
-
|
|
|
-import io.opentracing.Scope;
|
|
|
-import io.opentracing.util.GlobalTracer;
|
|
|
-import org.apache.hadoop.util.Time;
|
|
|
-import org.apache.ratis.grpc.GrpcTlsConfig;
|
|
|
-import org.apache.ratis.proto.RaftProtos;
|
|
|
-import org.apache.ratis.protocol.GroupMismatchException;
|
|
|
-import org.apache.ratis.protocol.RaftRetryFailureException;
|
|
|
-import org.apache.ratis.retry.RetryPolicy;
|
|
|
-import org.apache.ratis.thirdparty.com.google.protobuf
|
|
|
- .InvalidProtocolBufferException;
|
|
|
-import org.apache.hadoop.conf.Configuration;
|
|
|
+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.proto.HddsProtos;
|
|
|
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
|
|
|
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|
|
-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.proto.HddsProtos;
|
|
|
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
|
|
import org.apache.hadoop.hdds.tracing.TracingUtil;
|
|
|
-
|
|
|
+import org.apache.hadoop.util.Time;
|
|
|
import org.apache.ratis.RatisHelper;
|
|
|
import org.apache.ratis.client.RaftClient;
|
|
|
+import org.apache.ratis.grpc.GrpcTlsConfig;
|
|
|
+import org.apache.ratis.proto.RaftProtos;
|
|
|
+import org.apache.ratis.protocol.GroupMismatchException;
|
|
|
import org.apache.ratis.protocol.RaftClientReply;
|
|
|
+import org.apache.ratis.protocol.RaftException;
|
|
|
+import org.apache.ratis.retry.RetryPolicy;
|
|
|
import org.apache.ratis.rpc.RpcType;
|
|
|
import org.apache.ratis.rpc.SupportedRpcType;
|
|
|
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
|
|
|
+import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
|
|
|
import org.apache.ratis.util.TimeDuration;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
-import java.io.IOException;
|
|
|
-import java.util.*;
|
|
|
-import java.util.concurrent.CompletableFuture;
|
|
|
-import java.util.concurrent.CompletionException;
|
|
|
-import java.util.concurrent.TimeUnit;
|
|
|
-import java.util.concurrent.ExecutionException;
|
|
|
-import java.util.concurrent.TimeoutException;
|
|
|
-import java.util.concurrent.atomic.AtomicReference;
|
|
|
-import java.util.concurrent.ConcurrentHashMap;
|
|
|
-import java.util.stream.Collectors;
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
+
|
|
|
+import io.opentracing.Scope;
|
|
|
+import io.opentracing.util.GlobalTracer;
|
|
|
|
|
|
/**
|
|
|
* An abstract implementation of {@link XceiverClientSpi} using Ratis.
|
|
@@ -309,10 +311,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
|
|
Time.monotonicNowNanos() - requestTime);
|
|
|
}).thenApply(reply -> {
|
|
|
try {
|
|
|
- // we need to handle RaftRetryFailure Exception
|
|
|
- RaftRetryFailureException raftRetryFailureException =
|
|
|
- reply.getRetryFailureException();
|
|
|
- if (raftRetryFailureException != null) {
|
|
|
+ if (!reply.isSuccess()) {
|
|
|
// in case of raft retry failure, the raft client is
|
|
|
// not able to connect to the leader hence the pipeline
|
|
|
// can not be used but this instance of RaftClient will close
|
|
@@ -324,7 +323,10 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
|
|
// to SCM as in this case, it is the raft client which is not
|
|
|
// able to connect to leader in the pipeline, though the
|
|
|
// pipeline can still be functional.
|
|
|
- throw new CompletionException(raftRetryFailureException);
|
|
|
+ RaftException exception = reply.getException();
|
|
|
+ Preconditions.checkNotNull(exception, "Raft reply failure but " +
|
|
|
+ "no exception propagated.");
|
|
|
+ throw new CompletionException(exception);
|
|
|
}
|
|
|
ContainerCommandResponseProto response =
|
|
|
ContainerCommandResponseProto
|