|
@@ -20,14 +20,17 @@ package org.apache.hadoop.scm;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
-import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandRequestProto;
|
|
|
|
-import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandResponseProto;
|
|
|
|
|
|
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
|
|
|
+ .ContainerCommandRequestProto;
|
|
|
|
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
|
|
|
|
+ .ContainerCommandResponseProto;
|
|
import org.apache.ratis.RatisHelper;
|
|
import org.apache.ratis.RatisHelper;
|
|
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
|
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
|
|
import org.apache.ratis.client.RaftClient;
|
|
import org.apache.ratis.client.RaftClient;
|
|
import org.apache.ratis.protocol.RaftClientReply;
|
|
import org.apache.ratis.protocol.RaftClientReply;
|
|
import org.apache.ratis.rpc.RpcType;
|
|
import org.apache.ratis.rpc.RpcType;
|
|
import org.apache.ratis.rpc.SupportedRpcType;
|
|
import org.apache.ratis.rpc.SupportedRpcType;
|
|
|
|
+import org.apache.ratis.shaded.com.google.protobuf.ByteString;
|
|
import org.apache.ratis.shaded.com.google.protobuf.ShadedProtoUtil;
|
|
import org.apache.ratis.shaded.com.google.protobuf.ShadedProtoUtil;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -94,13 +97,48 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
|
return Objects.requireNonNull(client.get(), "client is null");
|
|
return Objects.requireNonNull(client.get(), "client is null");
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private boolean isReadOnly(ContainerCommandRequestProto proto) {
|
|
|
|
+ switch (proto.getCmdType()) {
|
|
|
|
+ case ReadContainer:
|
|
|
|
+ case ReadChunk:
|
|
|
|
+ case ListKey:
|
|
|
|
+ case GetKey:
|
|
|
|
+ case GetSmallFile:
|
|
|
|
+ case ListContainer:
|
|
|
|
+ case ListChunk:
|
|
|
|
+ return true;
|
|
|
|
+ case CloseContainer:
|
|
|
|
+ case WriteChunk:
|
|
|
|
+ case UpdateContainer:
|
|
|
|
+ case CompactChunk:
|
|
|
|
+ case CreateContainer:
|
|
|
|
+ case DeleteChunk:
|
|
|
|
+ case DeleteContainer:
|
|
|
|
+ case DeleteKey:
|
|
|
|
+ case PutKey:
|
|
|
|
+ case PutSmallFile:
|
|
|
|
+ default:
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private RaftClientReply sendRequest(ContainerCommandRequestProto request)
|
|
|
|
+ throws IOException {
|
|
|
|
+ boolean isReadOnlyRequest = isReadOnly(request);
|
|
|
|
+ ByteString byteString =
|
|
|
|
+ ShadedProtoUtil.asShadedByteString(request.toByteArray());
|
|
|
|
+ LOG.debug("sendCommand {} {}", isReadOnlyRequest, request);
|
|
|
|
+ final RaftClientReply reply = isReadOnlyRequest ?
|
|
|
|
+ getClient().sendReadOnly(() -> byteString) :
|
|
|
|
+ getClient().send(() -> byteString);
|
|
|
|
+ LOG.debug("reply {} {}", isReadOnlyRequest, reply);
|
|
|
|
+ return reply;
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public ContainerCommandResponseProto sendCommand(
|
|
public ContainerCommandResponseProto sendCommand(
|
|
ContainerCommandRequestProto request) throws IOException {
|
|
ContainerCommandRequestProto request) throws IOException {
|
|
- LOG.debug("sendCommand {}", request);
|
|
|
|
- final RaftClientReply reply = getClient().send(
|
|
|
|
- () -> ShadedProtoUtil.asShadedByteString(request.toByteArray()));
|
|
|
|
- LOG.debug("reply {}", reply);
|
|
|
|
|
|
+ final RaftClientReply reply = sendRequest(request);
|
|
Preconditions.checkState(reply.isSuccess());
|
|
Preconditions.checkState(reply.isSuccess());
|
|
return ContainerCommandResponseProto.parseFrom(
|
|
return ContainerCommandResponseProto.parseFrom(
|
|
ShadedProtoUtil.asByteString(reply.getMessage().getContent()));
|
|
ShadedProtoUtil.asByteString(reply.getMessage().getContent()));
|