Skip to content

Commit

Permalink
HDDS-10983. Fix testcase
Browse files Browse the repository at this point in the history
Change-Id: I44de245342a7622d1c3fd4b49dd1db4c2bcac5b4
  • Loading branch information
swamirishi committed Jun 7, 2024
1 parent cfc7415 commit 8edc907
Show file tree
Hide file tree
Showing 8 changed files with 35 additions and 27 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -450,7 +450,7 @@ private XceiverClientReply sendCommandWithRetry(
processForDebug(request), pipeline);
} else {
LOG.error(message + " on the pipeline {}.",
request.getCmdType(), pipeline);
request.getCmdType(), pipeline, new RuntimeException());
}
throw ioException;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -439,8 +439,8 @@ protected ByteBuffer[] readChunk(ChunkInfo readChunkInfo)
throws IOException {

ReadChunkResponseProto readChunkResponse =
ContainerProtocolCalls.readChunk(xceiverClient,
readChunkInfo, datanodeBlockID, validators, tokenSupplier.get());
ContainerProtocolCalls.readChunk(xceiverClient, readChunkInfo, datanodeBlockID, validators,
tokenSupplier.get());

if (readChunkResponse.hasData()) {
return readChunkResponse.getData().asReadOnlyByteBufferList()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -266,11 +266,8 @@ private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient,
* @return container protocol getLastCommittedBlockLength response
* @throws IOException if there is an I/O error while performing the call
*/
public static ContainerProtos.GetCommittedBlockLengthResponseProto
getCommittedBlockLength(
XceiverClientSpi xceiverClient, BlockID blockID,
Token<OzoneBlockTokenIdentifier> token)
throws IOException {
public static ContainerProtos.GetCommittedBlockLengthResponseProto getCommittedBlockLength(
XceiverClientSpi xceiverClient, BlockID blockID, Token<OzoneBlockTokenIdentifier> token) throws IOException {
ContainerProtos.GetCommittedBlockLengthRequestProto.Builder
getBlockLengthRequestBuilder =
ContainerProtos.GetCommittedBlockLengthRequestProto.newBuilder().
Expand Down Expand Up @@ -345,10 +342,8 @@ public static ContainerCommandRequestProto getPutBlockRequest(
* @return container protocol read chunk response
* @throws IOException if there is an I/O error while performing the call
*/
public static ContainerProtos.ReadChunkResponseProto readChunk(
XceiverClientSpi xceiverClient, ChunkInfo chunk,
DatanodeBlockID blockID, List<Validator> validators,
Token<? extends TokenIdentifier> token) throws IOException {
public static ContainerProtos.ReadChunkResponseProto readChunk(XceiverClientSpi xceiverClient, ChunkInfo chunk,
DatanodeBlockID blockID, List<Validator> validators, Token<? extends TokenIdentifier> token) throws IOException {
ReadChunkRequestProto.Builder readChunkRequest =
ReadChunkRequestProto.newBuilder()
.setBlockID(blockID)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
import org.apache.hadoop.security.token.Token;

import com.google.common.base.Preconditions;
import org.mockito.Mockito;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -511,7 +512,13 @@ public static ContainerCommandRequestProto getDeleteContainer(
}

public static BlockID getTestBlockID(long containerID) {
return new BlockID(containerID, UniqueId.next());
return getTestBlockID(containerID, null);
}

public static BlockID getTestBlockID(long containerID, Integer replicaIndex) {
BlockID blockID = Mockito.spy(new BlockID(containerID, UniqueId.next()));
Mockito.when(blockID.getReplicaIndex()).thenReturn(replicaIndex);
return blockID;
}

public static long getTestContainerID() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -247,8 +247,8 @@ public static void verifyBCSId(Container container, BlockID blockID)
*/
public static void verifyReplicaIdx(Container container, BlockID blockID)
throws IOException {
int containerReplicaIndex = container.getContainerData().getReplicaIndex();
if (containerReplicaIndex > 0 && containerReplicaIndex != blockID.getReplicaIndex()) {
Integer containerReplicaIndex = container.getContainerData().getReplicaIndex();
if (containerReplicaIndex > 0 && !containerReplicaIndex.equals(blockID.getReplicaIndex())) {
throw new StorageContainerException(
"Unable to find the Container with replicaIdx " + blockID.getReplicaIndex() + ". Container "
+ container.getContainerData().getContainerID() + " replicaIdx is "
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -189,10 +189,13 @@ private List<ContainerProtos.ChunkInfo> getChunkInfos(OmKeyLocationInfo
xceiverClientSpi =
getXceiverClientFactory().acquireClientForReadData(pipeline);

ContainerProtos.DatanodeBlockID datanodeBlockID = blockID
.getDatanodeBlockIDProtobuf();
ContainerProtos.DatanodeBlockID.Builder datanodeBlockID = blockID.getDatanodeBlockIDProtobufBuilder();
int replicaIndex = pipeline.getReplicaIndex(pipeline.getClosestNode());
if (replicaIndex > 0) {
datanodeBlockID.setReplicaIndex(replicaIndex);
}
ContainerProtos.GetBlockResponseProto response = ContainerProtocolCalls
.getBlock(xceiverClientSpi, datanodeBlockID, token);
.getBlock(xceiverClientSpi, datanodeBlockID.build(), token);

chunks = response.getBlockData().getChunksList();
} finally {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -149,13 +149,15 @@ protected List<ContainerProtos.ChunkInfo> getChunkInfos(
LOG.debug("Initializing BlockInputStream for get key to access {}",
blockID.getContainerID());
}
xceiverClientSpi =
getXceiverClientFactory().acquireClientForReadData(pipeline);
xceiverClientSpi = getXceiverClientFactory().acquireClientForReadData(pipeline);

ContainerProtos.DatanodeBlockID datanodeBlockID = blockID
.getDatanodeBlockIDProtobuf();
ContainerProtos.DatanodeBlockID.Builder datanodeBlockID = blockID.getDatanodeBlockIDProtobufBuilder();
int replicaIndex = pipeline.getReplicaIndex(pipeline.getClosestNode());
if (replicaIndex > 0) {
datanodeBlockID.setReplicaIndex(replicaIndex);
}
ContainerProtos.GetBlockResponseProto response = ContainerProtocolCalls
.getBlock(xceiverClientSpi, datanodeBlockID, token);
.getBlock(xceiverClientSpi, datanodeBlockID.build(), token);

chunks = response.getBlockData().getChunksList();
} finally {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -452,10 +452,10 @@ public void testCreateRecoveryContainer() throws Exception {
.generateToken(ANY_USER, container.containerID());
scm.getContainerManager().getContainerStateManager()
.addContainer(container.getProtobuf());

int replicaIndex = 4;
XceiverClientSpi dnClient = xceiverClientManager.acquireClient(
createSingleNodePipeline(newPipeline, newPipeline.getNodes().get(0),
2));
replicaIndex));
try {
// To create the actual situation, container would have been in closed
// state at SCM.
Expand All @@ -470,7 +470,7 @@ public void testCreateRecoveryContainer() throws Exception {
String encodedToken = cToken.encodeToUrlString();
ContainerProtocolCalls.createRecoveringContainer(dnClient,
container.containerID().getProtobuf().getId(),
encodedToken, 4);
encodedToken, replicaIndex);

BlockID blockID = ContainerTestHelper
.getTestBlockID(container.containerID().getProtobuf().getId());
Expand Down Expand Up @@ -511,7 +511,8 @@ public void testCreateRecoveryContainer() throws Exception {
readContainerResponseProto.getContainerData().getState());
ContainerProtos.ReadChunkResponseProto readChunkResponseProto =
ContainerProtocolCalls.readChunk(dnClient,
writeChunkRequest.getWriteChunk().getChunkData(), blockID.getDatanodeBlockIDProtobuf(), null,
writeChunkRequest.getWriteChunk().getChunkData(),
blockID.getDatanodeBlockIDProtobufBuilder().setReplicaIndex(replicaIndex).build(), null,
blockToken);
ByteBuffer[] readOnlyByteBuffersArray = BufferUtils
.getReadOnlyByteBuffersArray(
Expand Down

0 comments on commit 8edc907

Please sign in to comment.