HDDS-1224. Restructure code to validate the response from server in the Read path (#806)

This commit is contained in:
bshashikant 2019-06-04 23:07:02 +05:30 committed by Hanisha Koneru
parent ea3b0a1844
commit e4b4fec66f
8 changed files with 168 additions and 158 deletions

View File

@ -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;
} }
} }

View File

@ -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)

View File

@ -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());
} }

View File

@ -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

View File

@ -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;
}

View File

@ -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;
}
} }

View File

@ -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);
} }
} }

View File

@ -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);
} }