HDDS-1224. Restructure code to validate the response from server in the Read path (#806)
This commit is contained in:
parent
ea3b0a1844
commit
e4b4fec66f
@ -23,7 +23,6 @@
|
|||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hdds.HddsUtils;
|
import org.apache.hadoop.hdds.HddsUtils;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
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.ContainerCommandRequestProto;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc;
|
||||||
@ -31,6 +30,7 @@
|
|||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
|
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
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.exception.SCMSecurityException;
|
||||||
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
||||||
import org.apache.hadoop.hdds.tracing.GrpcClientInterceptor;
|
import org.apache.hadoop.hdds.tracing.GrpcClientInterceptor;
|
||||||
@ -62,7 +62,6 @@
|
|||||||
import java.util.concurrent.Semaphore;
|
import java.util.concurrent.Semaphore;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A Client for the storageContainer protocol.
|
* A Client for the storageContainer protocol.
|
||||||
@ -101,7 +100,7 @@ public XceiverClientGrpc(Pipeline pipeline, Configuration config) {
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* To be used when grpc token is not enabled.
|
* To be used when grpc token is not enabled.
|
||||||
* */
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void connect() throws Exception {
|
public void connect() throws Exception {
|
||||||
// leader by default is the 1st datanode in the datanode list of pipleline
|
// leader by default is the 1st datanode in the datanode list of pipleline
|
||||||
@ -112,7 +111,7 @@ public void connect() throws Exception {
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* Passed encoded token to GRPC header when security is enabled.
|
* Passed encoded token to GRPC header when security is enabled.
|
||||||
* */
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void connect(String encodedToken) throws Exception {
|
public void connect(String encodedToken) throws Exception {
|
||||||
// leader by default is the 1st datanode in the datanode list of pipleline
|
// leader by default is the 1st datanode in the datanode list of pipleline
|
||||||
@ -132,11 +131,10 @@ private void connectToDatanode(DatanodeDetails dn, String encodedToken)
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Add credential context to the client call
|
// Add credential context to the client call
|
||||||
String userName = UserGroupInformation.getCurrentUser()
|
String userName = UserGroupInformation.getCurrentUser().getShortUserName();
|
||||||
.getShortUserName();
|
|
||||||
LOG.debug("Connecting to server Port : " + dn.getIpAddress());
|
LOG.debug("Connecting to server Port : " + dn.getIpAddress());
|
||||||
NettyChannelBuilder channelBuilder = NettyChannelBuilder.forAddress(dn
|
NettyChannelBuilder channelBuilder =
|
||||||
.getIpAddress(), port).usePlaintext()
|
NettyChannelBuilder.forAddress(dn.getIpAddress(), port).usePlaintext()
|
||||||
.maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
|
.maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
|
||||||
.intercept(new ClientCredentialInterceptor(userName, encodedToken),
|
.intercept(new ClientCredentialInterceptor(userName, encodedToken),
|
||||||
new GrpcClientInterceptor());
|
new GrpcClientInterceptor());
|
||||||
@ -149,8 +147,8 @@ private void connectToDatanode(DatanodeDetails dn, String encodedToken)
|
|||||||
if (trustCertCollectionFile != null) {
|
if (trustCertCollectionFile != null) {
|
||||||
sslContextBuilder.trustManager(trustCertCollectionFile);
|
sslContextBuilder.trustManager(trustCertCollectionFile);
|
||||||
}
|
}
|
||||||
if (secConfig.isGrpcMutualTlsRequired() && clientCertChainFile != null &&
|
if (secConfig.isGrpcMutualTlsRequired() && clientCertChainFile != null
|
||||||
privateKeyFile != null) {
|
&& privateKeyFile != null) {
|
||||||
sslContextBuilder.keyManager(clientCertChainFile, privateKeyFile);
|
sslContextBuilder.keyManager(clientCertChainFile, privateKeyFile);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -216,49 +214,45 @@ public ContainerCommandResponseProto sendCommand(
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public XceiverClientReply sendCommand(
|
public ContainerCommandResponseProto sendCommand(
|
||||||
ContainerCommandRequestProto request, List<DatanodeDetails> excludeDns)
|
ContainerCommandRequestProto request, List<CheckedBiFunction> validators)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Preconditions.checkState(HddsUtils.isReadOnly(request));
|
try {
|
||||||
return sendCommandWithTraceIDAndRetry(request, excludeDns);
|
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(
|
private XceiverClientReply sendCommandWithTraceIDAndRetry(
|
||||||
ContainerCommandRequestProto request, List<DatanodeDetails> excludeDns)
|
ContainerCommandRequestProto request, List<CheckedBiFunction> validators)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
try (Scope scope = GlobalTracer.get()
|
try (Scope scope = GlobalTracer.get()
|
||||||
.buildSpan("XceiverClientGrpc." + request.getCmdType().name())
|
.buildSpan("XceiverClientGrpc." + request.getCmdType().name())
|
||||||
.startActive(true)) {
|
.startActive(true)) {
|
||||||
ContainerCommandRequestProto finalPayload =
|
ContainerCommandRequestProto finalPayload =
|
||||||
ContainerCommandRequestProto.newBuilder(request)
|
ContainerCommandRequestProto.newBuilder(request)
|
||||||
.setTraceID(TracingUtil.exportCurrentSpan())
|
.setTraceID(TracingUtil.exportCurrentSpan()).build();
|
||||||
.build();
|
return sendCommandWithRetry(finalPayload, validators);
|
||||||
return sendCommandWithRetry(finalPayload, excludeDns);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private XceiverClientReply sendCommandWithRetry(
|
private XceiverClientReply sendCommandWithRetry(
|
||||||
ContainerCommandRequestProto request, List<DatanodeDetails> excludeDns)
|
ContainerCommandRequestProto request, List<CheckedBiFunction> validators)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
ContainerCommandResponseProto responseProto = null;
|
ContainerCommandResponseProto responseProto = null;
|
||||||
|
IOException ioException = null;
|
||||||
|
|
||||||
// In case of an exception or an error, we will try to read from the
|
// In case of an exception or an error, we will try to read from the
|
||||||
// datanodes in the pipeline in a round robin fashion.
|
// datanodes in the pipeline in a round robin fashion.
|
||||||
|
|
||||||
// TODO: cache the correct leader info in here, so that any subsequent calls
|
// TODO: cache the correct leader info in here, so that any subsequent calls
|
||||||
// should first go to leader
|
// 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);
|
XceiverClientReply reply = new XceiverClientReply(null);
|
||||||
for (DatanodeDetails dn : healthyDns) {
|
for (DatanodeDetails dn : pipeline.getNodes()) {
|
||||||
try {
|
try {
|
||||||
LOG.debug("Executing command " + request + " on datanode " + dn);
|
LOG.debug("Executing command " + request + " on datanode " + dn);
|
||||||
// In case the command gets retried on a 2nd datanode,
|
// In case the command gets retried on a 2nd datanode,
|
||||||
@ -266,17 +260,26 @@ private XceiverClientReply sendCommandWithRetry(
|
|||||||
// in case these don't exist for the specific datanode.
|
// in case these don't exist for the specific datanode.
|
||||||
reply.addDatanode(dn);
|
reply.addDatanode(dn);
|
||||||
responseProto = sendCommandAsync(request, dn).getResponse().get();
|
responseProto = sendCommandAsync(request, dn).getResponse().get();
|
||||||
if (responseProto.getResult() == ContainerProtos.Result.SUCCESS) {
|
if (validators != null && !validators.isEmpty()) {
|
||||||
break;
|
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
|
LOG.debug("Failed to execute command " + request + " on datanode " + dn
|
||||||
.getUuidString(), e);
|
.getUuidString(), e);
|
||||||
|
if (!(e instanceof IOException)) {
|
||||||
if (Status.fromThrowable(e.getCause()).getCode()
|
if (Status.fromThrowable(e.getCause()).getCode()
|
||||||
== Status.UNAUTHENTICATED.getCode()) {
|
== Status.UNAUTHENTICATED.getCode()) {
|
||||||
throw new SCMSecurityException("Failed to authenticate with "
|
throw new SCMSecurityException("Failed to authenticate with "
|
||||||
+ "GRPC XceiverServer with Ozone block token.");
|
+ "GRPC XceiverServer with Ozone block token.");
|
||||||
}
|
}
|
||||||
|
ioException = new IOException(e);
|
||||||
|
} else {
|
||||||
|
ioException = (IOException) e;
|
||||||
|
}
|
||||||
|
responseProto = null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -284,9 +287,10 @@ private XceiverClientReply sendCommandWithRetry(
|
|||||||
reply.setResponse(CompletableFuture.completedFuture(responseProto));
|
reply.setResponse(CompletableFuture.completedFuture(responseProto));
|
||||||
return reply;
|
return reply;
|
||||||
} else {
|
} else {
|
||||||
throw new IOException(
|
Preconditions.checkNotNull(ioException);
|
||||||
"Failed to execute command " + request + " on the pipeline "
|
LOG.error("Failed to execute command " + request + " on the pipeline "
|
||||||
+ pipeline.getId());
|
+ pipeline.getId());
|
||||||
|
throw ioException;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -21,12 +21,11 @@
|
|||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientReply;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers
|
import org.apache.hadoop.hdds.scm.container.common.helpers
|
||||||
.StorageContainerException;
|
.StorageContainerException;
|
||||||
import org.apache.hadoop.ozone.common.Checksum;
|
import org.apache.hadoop.ozone.common.Checksum;
|
||||||
import org.apache.hadoop.ozone.common.ChecksumData;
|
import org.apache.hadoop.ozone.common.ChecksumData;
|
||||||
|
import org.apache.hadoop.ozone.common.OzoneChecksumException;
|
||||||
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
|
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
|
||||||
import org.apache.hadoop.fs.Seekable;
|
import org.apache.hadoop.fs.Seekable;
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
||||||
@ -34,16 +33,17 @@
|
|||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||||
.ReadChunkResponseProto;
|
.ReadChunkResponseProto;
|
||||||
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
|
||||||
|
ContainerCommandResponseProto;
|
||||||
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
|
||||||
|
ContainerCommandRequestProto;
|
||||||
import org.apache.hadoop.hdds.client.BlockID;
|
import org.apache.hadoop.hdds.client.BlockID;
|
||||||
|
|
||||||
import java.io.EOFException;
|
import java.io.EOFException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.ExecutionException;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An {@link InputStream} used by the REST service in combination with the
|
* An {@link InputStream} used by the REST service in combination with the
|
||||||
@ -74,7 +74,7 @@ public class BlockInputStream extends InputStream implements Seekable {
|
|||||||
private List<ByteBuffer> buffers;
|
private List<ByteBuffer> buffers;
|
||||||
private int bufferIndex;
|
private int bufferIndex;
|
||||||
private long bufferPosition;
|
private long bufferPosition;
|
||||||
private final boolean verifyChecksum;
|
private boolean verifyChecksum;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new BlockInputStream.
|
* Creates a new BlockInputStream.
|
||||||
@ -323,41 +323,8 @@ private boolean chunksRemaining() {
|
|||||||
private synchronized void readChunkFromContainer() throws IOException {
|
private synchronized void readChunkFromContainer() throws IOException {
|
||||||
// Read the chunk at chunkIndex
|
// Read the chunk at chunkIndex
|
||||||
final ChunkInfo chunkInfo = chunks.get(chunkIndex);
|
final ChunkInfo chunkInfo = chunks.get(chunkIndex);
|
||||||
List<DatanodeDetails> excludeDns = null;
|
|
||||||
ByteString byteString;
|
ByteString byteString;
|
||||||
List<DatanodeDetails> dnList = getDatanodeList();
|
byteString = readChunk(chunkInfo);
|
||||||
while (true) {
|
|
||||||
List<DatanodeDetails> dnListFromReadChunkCall = new ArrayList<>();
|
|
||||||
byteString = readChunk(chunkInfo, excludeDns, dnListFromReadChunkCall);
|
|
||||||
try {
|
|
||||||
if (byteString.size() != chunkInfo.getLen()) {
|
|
||||||
// Bytes read from chunk should be equal to chunk size.
|
|
||||||
throw new IOException(String
|
|
||||||
.format("Inconsistent read for chunk=%s len=%d bytesRead=%d",
|
|
||||||
chunkInfo.getChunkName(), chunkInfo.getLen(),
|
|
||||||
byteString.size()));
|
|
||||||
}
|
|
||||||
ChecksumData checksumData =
|
|
||||||
ChecksumData.getFromProtoBuf(chunkInfo.getChecksumData());
|
|
||||||
if (verifyChecksum) {
|
|
||||||
Checksum.verifyChecksum(byteString, checksumData);
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
} catch (IOException ioe) {
|
|
||||||
// we will end up in this situation only if the checksum mismatch
|
|
||||||
// happens or the length of the chunk mismatches.
|
|
||||||
// In this case, read should be retried on a different replica.
|
|
||||||
// TODO: Inform SCM of a possible corrupt container replica here
|
|
||||||
if (excludeDns == null) {
|
|
||||||
excludeDns = new ArrayList<>();
|
|
||||||
}
|
|
||||||
excludeDns.addAll(dnListFromReadChunkCall);
|
|
||||||
if (excludeDns.size() == dnList.size()) {
|
|
||||||
throw ioe;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
buffers = byteString.asReadOnlyByteBufferList();
|
buffers = byteString.asReadOnlyByteBufferList();
|
||||||
bufferIndex = 0;
|
bufferIndex = 0;
|
||||||
chunkIndexOfCurrentBuffer = chunkIndex;
|
chunkIndexOfCurrentBuffer = chunkIndex;
|
||||||
@ -372,28 +339,20 @@ private synchronized void readChunkFromContainer() throws IOException {
|
|||||||
* Send RPC call to get the chunk from the container.
|
* Send RPC call to get the chunk from the container.
|
||||||
*/
|
*/
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
protected ByteString readChunk(final ChunkInfo chunkInfo,
|
protected ByteString readChunk(final ChunkInfo chunkInfo)
|
||||||
List<DatanodeDetails> excludeDns, List<DatanodeDetails> dnListFromReply)
|
|
||||||
throws IOException {
|
throws IOException {
|
||||||
XceiverClientReply reply;
|
ReadChunkResponseProto readChunkResponse;
|
||||||
ReadChunkResponseProto readChunkResponse = null;
|
|
||||||
try {
|
try {
|
||||||
reply = ContainerProtocolCalls
|
List<CheckedBiFunction> validators =
|
||||||
.readChunk(xceiverClient, chunkInfo, blockID, traceID, excludeDns);
|
ContainerProtocolCalls.getValidatorList();
|
||||||
ContainerProtos.ContainerCommandResponseProto response;
|
validators.add(validator);
|
||||||
response = reply.getResponse().get();
|
readChunkResponse = ContainerProtocolCalls
|
||||||
ContainerProtocolCalls.validateContainerResponse(response);
|
.readChunk(xceiverClient, chunkInfo, blockID, traceID, validators);
|
||||||
readChunkResponse = response.getReadChunk();
|
|
||||||
dnListFromReply.addAll(reply.getDatanodes());
|
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
if (e instanceof StorageContainerException) {
|
if (e instanceof StorageContainerException) {
|
||||||
throw e;
|
throw e;
|
||||||
}
|
}
|
||||||
throw new IOException("Unexpected OzoneException: " + e.toString(), e);
|
throw new IOException("Unexpected OzoneException: " + e.toString(), e);
|
||||||
} catch (ExecutionException | InterruptedException e) {
|
|
||||||
throw new IOException(
|
|
||||||
"Failed to execute ReadChunk command for chunk " + chunkInfo
|
|
||||||
.getChunkName(), e);
|
|
||||||
}
|
}
|
||||||
return readChunkResponse.getData();
|
return readChunkResponse.getData();
|
||||||
}
|
}
|
||||||
@ -403,6 +362,26 @@ protected List<DatanodeDetails> getDatanodeList() {
|
|||||||
return xceiverClient.getPipeline().getNodes();
|
return xceiverClient.getPipeline().getNodes();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private CheckedBiFunction<ContainerCommandRequestProto,
|
||||||
|
ContainerCommandResponseProto, IOException> validator =
|
||||||
|
(request, response) -> {
|
||||||
|
ReadChunkResponseProto readChunkResponse = response.getReadChunk();
|
||||||
|
final ChunkInfo chunkInfo = readChunkResponse.getChunkData();
|
||||||
|
ByteString byteString = readChunkResponse.getData();
|
||||||
|
if (byteString.size() != chunkInfo.getLen()) {
|
||||||
|
// Bytes read from chunk should be equal to chunk size.
|
||||||
|
throw new OzoneChecksumException(String
|
||||||
|
.format("Inconsistent read for chunk=%s len=%d bytesRead=%d",
|
||||||
|
chunkInfo.getChunkName(), chunkInfo.getLen(),
|
||||||
|
byteString.size()));
|
||||||
|
}
|
||||||
|
ChecksumData checksumData =
|
||||||
|
ChecksumData.getFromProtoBuf(chunkInfo.getChecksumData());
|
||||||
|
if (verifyChecksum) {
|
||||||
|
Checksum.verifyChecksum(byteString, checksumData);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized void seek(long pos) throws IOException {
|
public synchronized void seek(long pos) throws IOException {
|
||||||
if (pos < 0 || (chunks.size() == 0 && pos > 0)
|
if (pos < 0 || (chunks.size() == 0 && pos > 0)
|
||||||
|
@ -114,8 +114,7 @@ private static class DummyBlockInputStream extends BlockInputStream {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ByteString readChunk(final ChunkInfo chunkInfo,
|
protected ByteString readChunk(final ChunkInfo chunkInfo)
|
||||||
List<DatanodeDetails> excludeDns, List<DatanodeDetails> dnListFromReply)
|
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return getByteString(chunkInfo.getChunkName(), (int) chunkInfo.getLen());
|
return getByteString(chunkInfo.getChunkName(), (int) chunkInfo.getLen());
|
||||||
}
|
}
|
||||||
|
@ -25,13 +25,13 @@
|
|||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
|
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.ContainerProtos.ContainerCommandResponseProto;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
import org.apache.hadoop.hdds.scm.storage.CheckedBiFunction;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A Client for the storageContainer protocol.
|
* A Client for the storageContainer protocol.
|
||||||
@ -118,18 +118,21 @@ public ContainerCommandResponseProto sendCommand(
|
|||||||
* Sends a given command to server and gets the reply back along with
|
* Sends a given command to server and gets the reply back along with
|
||||||
* the server associated info.
|
* the server associated info.
|
||||||
* @param request Request
|
* @param request Request
|
||||||
* @param excludeDns list of servers on which the command won't be sent to.
|
* @param validators functions to validate the response
|
||||||
* @return Response to the command
|
* @return Response to the command
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public XceiverClientReply sendCommand(
|
public ContainerCommandResponseProto sendCommand(
|
||||||
ContainerCommandRequestProto request, List<DatanodeDetails> excludeDns)
|
ContainerCommandRequestProto request, List<CheckedBiFunction> validators)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
try {
|
try {
|
||||||
XceiverClientReply reply;
|
XceiverClientReply reply;
|
||||||
reply = sendCommandAsync(request);
|
reply = sendCommandAsync(request);
|
||||||
reply.getResponse().get();
|
ContainerCommandResponseProto responseProto = reply.getResponse().get();
|
||||||
return reply;
|
for (CheckedBiFunction function : validators) {
|
||||||
|
function.apply(request, responseProto);
|
||||||
|
}
|
||||||
|
return responseProto;
|
||||||
} catch (ExecutionException | InterruptedException e) {
|
} catch (ExecutionException | InterruptedException e) {
|
||||||
throw new IOException("Failed to command " + request, e);
|
throw new IOException("Failed to command " + request, e);
|
||||||
}
|
}
|
||||||
@ -156,7 +159,7 @@ public XceiverClientReply sendCommand(
|
|||||||
/**
|
/**
|
||||||
* Check if an specfic commitIndex is replicated to majority/all servers.
|
* Check if an specfic commitIndex is replicated to majority/all servers.
|
||||||
* @param index index to watch for
|
* @param index index to watch for
|
||||||
* @param timeout timeout provided for the watch ipeartion to complete
|
* @param timeout timeout provided for the watch operation to complete
|
||||||
* @return reply containing the min commit index replicated to all or majority
|
* @return reply containing the min commit index replicated to all or majority
|
||||||
* servers in case of a failure
|
* servers in case of a failure
|
||||||
* @throws InterruptedException
|
* @throws InterruptedException
|
||||||
|
@ -0,0 +1,31 @@
|
|||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.hdds.scm.storage;
|
||||||
|
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Defines a functional interface having two inputs which throws IOException.
|
||||||
|
*/
|
||||||
|
@FunctionalInterface
|
||||||
|
public interface CheckedBiFunction<LEFT, RIGHT,
|
||||||
|
THROWABLE extends IOException> {
|
||||||
|
void apply(LEFT left, RIGHT right) throws THROWABLE;
|
||||||
|
}
|
@ -18,7 +18,6 @@
|
|||||||
|
|
||||||
package org.apache.hadoop.hdds.scm.storage;
|
package org.apache.hadoop.hdds.scm.storage;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientReply;
|
import org.apache.hadoop.hdds.scm.XceiverClientReply;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers
|
import org.apache.hadoop.hdds.scm.container.common.helpers
|
||||||
.BlockNotCommittedException;
|
.BlockNotCommittedException;
|
||||||
@ -72,6 +71,7 @@
|
|||||||
import org.apache.hadoop.hdds.client.BlockID;
|
import org.apache.hadoop.hdds.client.BlockID;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
|
|
||||||
@ -116,9 +116,8 @@ public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient,
|
|||||||
}
|
}
|
||||||
|
|
||||||
ContainerCommandRequestProto request = builder.build();
|
ContainerCommandRequestProto request = builder.build();
|
||||||
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
ContainerCommandResponseProto response =
|
||||||
validateContainerResponse(response);
|
xceiverClient.sendCommand(request, getValidatorList());
|
||||||
|
|
||||||
return response.getGetBlock();
|
return response.getGetBlock();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -153,8 +152,8 @@ public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient,
|
|||||||
builder.setEncodedToken(encodedToken);
|
builder.setEncodedToken(encodedToken);
|
||||||
}
|
}
|
||||||
ContainerCommandRequestProto request = builder.build();
|
ContainerCommandRequestProto request = builder.build();
|
||||||
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
ContainerCommandResponseProto response =
|
||||||
validateContainerResponse(response);
|
xceiverClient.sendCommand(request, getValidatorList());
|
||||||
return response.getGetCommittedBlockLength();
|
return response.getGetCommittedBlockLength();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -184,8 +183,8 @@ public static ContainerProtos.PutBlockResponseProto putBlock(
|
|||||||
builder.setEncodedToken(encodedToken);
|
builder.setEncodedToken(encodedToken);
|
||||||
}
|
}
|
||||||
ContainerCommandRequestProto request = builder.build();
|
ContainerCommandRequestProto request = builder.build();
|
||||||
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
ContainerCommandResponseProto response =
|
||||||
validateContainerResponse(response);
|
xceiverClient.sendCommand(request, getValidatorList());
|
||||||
return response.getPutBlock();
|
return response.getPutBlock();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -228,35 +227,31 @@ public static XceiverClientReply putBlockAsync(
|
|||||||
* @param chunk information about chunk to read
|
* @param chunk information about chunk to read
|
||||||
* @param blockID ID of the block
|
* @param blockID ID of the block
|
||||||
* @param traceID container protocol call args
|
* @param traceID container protocol call args
|
||||||
* @param excludeDns datamode to exclude while executing the command
|
* @param validators functions to validate the response
|
||||||
* @return container protocol read chunk response
|
* @return container protocol read chunk response
|
||||||
* @throws IOException if there is an I/O error while performing the call
|
* @throws IOException if there is an I/O error while performing the call
|
||||||
*/
|
*/
|
||||||
public static XceiverClientReply readChunk(XceiverClientSpi xceiverClient,
|
public static ContainerProtos.ReadChunkResponseProto readChunk(
|
||||||
ChunkInfo chunk, BlockID blockID, String traceID,
|
XceiverClientSpi xceiverClient, ChunkInfo chunk, BlockID blockID,
|
||||||
List<DatanodeDetails> excludeDns)
|
String traceID, List<CheckedBiFunction> validators) throws IOException {
|
||||||
throws IOException {
|
ReadChunkRequestProto.Builder readChunkRequest =
|
||||||
ReadChunkRequestProto.Builder readChunkRequest = ReadChunkRequestProto
|
ReadChunkRequestProto.newBuilder()
|
||||||
.newBuilder()
|
|
||||||
.setBlockID(blockID.getDatanodeBlockIDProtobuf())
|
.setBlockID(blockID.getDatanodeBlockIDProtobuf())
|
||||||
.setChunkData(chunk);
|
.setChunkData(chunk);
|
||||||
String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
|
String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
|
||||||
ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto
|
ContainerCommandRequestProto.Builder builder =
|
||||||
.newBuilder()
|
ContainerCommandRequestProto.newBuilder().setCmdType(Type.ReadChunk)
|
||||||
.setCmdType(Type.ReadChunk)
|
.setContainerID(blockID.getContainerID()).setTraceID(traceID)
|
||||||
.setContainerID(blockID.getContainerID())
|
.setDatanodeUuid(id).setReadChunk(readChunkRequest);
|
||||||
.setTraceID(traceID)
|
|
||||||
.setDatanodeUuid(id)
|
|
||||||
.setReadChunk(readChunkRequest);
|
|
||||||
String encodedToken = getEncodedBlockToken(new Text(blockID.
|
String encodedToken = getEncodedBlockToken(new Text(blockID.
|
||||||
getContainerBlockID().toString()));
|
getContainerBlockID().toString()));
|
||||||
if (encodedToken != null) {
|
if (encodedToken != null) {
|
||||||
builder.setEncodedToken(encodedToken);
|
builder.setEncodedToken(encodedToken);
|
||||||
}
|
}
|
||||||
ContainerCommandRequestProto request = builder.build();
|
ContainerCommandRequestProto request = builder.build();
|
||||||
XceiverClientReply reply =
|
ContainerCommandResponseProto reply =
|
||||||
xceiverClient.sendCommand(request, excludeDns);
|
xceiverClient.sendCommand(request, validators);
|
||||||
return reply;
|
return reply.getReadChunk();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -291,8 +286,7 @@ public static void writeChunk(XceiverClientSpi xceiverClient, ChunkInfo chunk,
|
|||||||
builder.setEncodedToken(encodedToken);
|
builder.setEncodedToken(encodedToken);
|
||||||
}
|
}
|
||||||
ContainerCommandRequestProto request = builder.build();
|
ContainerCommandRequestProto request = builder.build();
|
||||||
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
xceiverClient.sendCommand(request, getValidatorList());
|
||||||
validateContainerResponse(response);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -384,8 +378,8 @@ public static PutSmallFileResponseProto writeSmallFile(
|
|||||||
builder.setEncodedToken(encodedToken);
|
builder.setEncodedToken(encodedToken);
|
||||||
}
|
}
|
||||||
ContainerCommandRequestProto request = builder.build();
|
ContainerCommandRequestProto request = builder.build();
|
||||||
ContainerCommandResponseProto response = client.sendCommand(request);
|
ContainerCommandResponseProto response =
|
||||||
validateContainerResponse(response);
|
client.sendCommand(request, getValidatorList());
|
||||||
return response.getPutSmallFile();
|
return response.getPutSmallFile();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -416,9 +410,7 @@ public static void createContainer(XceiverClientSpi client, long containerID,
|
|||||||
request.setCreateContainer(createRequest.build());
|
request.setCreateContainer(createRequest.build());
|
||||||
request.setDatanodeUuid(id);
|
request.setDatanodeUuid(id);
|
||||||
request.setTraceID(traceID);
|
request.setTraceID(traceID);
|
||||||
ContainerCommandResponseProto response = client.sendCommand(
|
client.sendCommand(request.build(), getValidatorList());
|
||||||
request.build());
|
|
||||||
validateContainerResponse(response);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -447,9 +439,7 @@ public static void deleteContainer(XceiverClientSpi client, long containerID,
|
|||||||
if (encodedToken != null) {
|
if (encodedToken != null) {
|
||||||
request.setEncodedToken(encodedToken);
|
request.setEncodedToken(encodedToken);
|
||||||
}
|
}
|
||||||
ContainerCommandResponseProto response =
|
client.sendCommand(request.build(), getValidatorList());
|
||||||
client.sendCommand(request.build());
|
|
||||||
validateContainerResponse(response);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -476,9 +466,7 @@ public static void closeContainer(XceiverClientSpi client,
|
|||||||
if(encodedToken != null) {
|
if(encodedToken != null) {
|
||||||
request.setEncodedToken(encodedToken);
|
request.setEncodedToken(encodedToken);
|
||||||
}
|
}
|
||||||
ContainerCommandResponseProto response =
|
client.sendCommand(request.build(), getValidatorList());
|
||||||
client.sendCommand(request.build());
|
|
||||||
validateContainerResponse(response);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -505,8 +493,7 @@ public static ReadContainerResponseProto readContainer(
|
|||||||
request.setEncodedToken(encodedToken);
|
request.setEncodedToken(encodedToken);
|
||||||
}
|
}
|
||||||
ContainerCommandResponseProto response =
|
ContainerCommandResponseProto response =
|
||||||
client.sendCommand(request.build());
|
client.sendCommand(request.build(), getValidatorList());
|
||||||
validateContainerResponse(response);
|
|
||||||
|
|
||||||
return response.getReadContainer();
|
return response.getReadContainer();
|
||||||
}
|
}
|
||||||
@ -544,9 +531,8 @@ public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client,
|
|||||||
builder.setEncodedToken(encodedToken);
|
builder.setEncodedToken(encodedToken);
|
||||||
}
|
}
|
||||||
ContainerCommandRequestProto request = builder.build();
|
ContainerCommandRequestProto request = builder.build();
|
||||||
ContainerCommandResponseProto response = client.sendCommand(request);
|
ContainerCommandResponseProto response =
|
||||||
validateContainerResponse(response);
|
client.sendCommand(request, getValidatorList());
|
||||||
|
|
||||||
return response.getGetSmallFile();
|
return response.getGetSmallFile();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -598,4 +584,13 @@ private static Text getService(DatanodeBlockID blockId) {
|
|||||||
.append(blockId.getLocalID())
|
.append(blockId.getLocalID())
|
||||||
.toString());
|
.toString());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static List<CheckedBiFunction> getValidatorList() {
|
||||||
|
List<CheckedBiFunction> validators = new ArrayList<>(1);
|
||||||
|
CheckedBiFunction<ContainerProtos.ContainerCommandRequestProto,
|
||||||
|
ContainerProtos.ContainerCommandResponseProto, IOException>
|
||||||
|
validator = (request, response) -> validateContainerResponse(response);
|
||||||
|
validators.add(validator);
|
||||||
|
return validators;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -871,7 +871,7 @@ private void readCorruptedKey(String volumeName, String bucketName,
|
|||||||
fail("Reading corrupted data should fail, as verify checksum is " +
|
fail("Reading corrupted data should fail, as verify checksum is " +
|
||||||
"enabled");
|
"enabled");
|
||||||
}
|
}
|
||||||
} catch (OzoneChecksumException e) {
|
} catch (IOException e) {
|
||||||
if (!verifyChecksum) {
|
if (!verifyChecksum) {
|
||||||
fail("Reading corrupted data should not fail, as verify checksum is " +
|
fail("Reading corrupted data should not fail, as verify checksum is " +
|
||||||
"disabled");
|
"disabled");
|
||||||
@ -1022,7 +1022,7 @@ public void testReadKeyWithCorruptedData() throws IOException {
|
|||||||
OzoneInputStream is = bucket.readKey(keyName);
|
OzoneInputStream is = bucket.readKey(keyName);
|
||||||
is.read(new byte[100]);
|
is.read(new byte[100]);
|
||||||
fail("Reading corrupted data should fail.");
|
fail("Reading corrupted data should fail.");
|
||||||
} catch (OzoneChecksumException e) {
|
} catch (IOException e) {
|
||||||
GenericTestUtils.assertExceptionContains("Checksum mismatch", e);
|
GenericTestUtils.assertExceptionContains("Checksum mismatch", e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -1103,7 +1103,7 @@ public void testReadKeyWithCorruptedDataWithMutiNodes() throws IOException {
|
|||||||
byte[] b = new byte[data.length];
|
byte[] b = new byte[data.length];
|
||||||
is.read(b);
|
is.read(b);
|
||||||
fail("Reading corrupted data should fail.");
|
fail("Reading corrupted data should fail.");
|
||||||
} catch (OzoneChecksumException e) {
|
} catch (IOException e) {
|
||||||
GenericTestUtils.assertExceptionContains("Checksum mismatch", e);
|
GenericTestUtils.assertExceptionContains("Checksum mismatch", e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -204,9 +204,8 @@ public void testPutKeyAndGetKeyThreeNodes()
|
|||||||
readKey(bucket, keyName, value);
|
readKey(bucket, keyName, value);
|
||||||
fail("Expected exception not thrown");
|
fail("Expected exception not thrown");
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
Assert.assertTrue(e.getMessage().contains("Failed to execute command"));
|
// it should throw an ioException as none of the servers
|
||||||
Assert.assertTrue(
|
// are available
|
||||||
e.getMessage().contains("on the pipeline " + pipeline.getId()));
|
|
||||||
}
|
}
|
||||||
manager.releaseClient(clientSpi, false);
|
manager.releaseClient(clientSpi, false);
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user