From a1d5c4af4ebea0b0c713e21cc504d9c1f566c578 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 5 Jun 2024 20:40:31 -0700 Subject: [PATCH 01/21] HDDS-10983. EC Key read corruption when the replica index of container in DN mismatches Change-Id: Ic88575f31305bde9d78b0e4d0c7bbf25c53a7ccb --- .../hdds/scm/storage/ChunkInputStream.java | 19 +++++++- .../apache/hadoop/hdds/client/BlockID.java | 48 +++++++++++++++---- .../scm/storage/ContainerProtocolCalls.java | 13 ++++- .../keyvalue/impl/BlockManagerImpl.java | 9 ++++ 4 files changed, 76 insertions(+), 13 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java index b30f555795b2..529a6967343b 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java @@ -26,6 +26,8 @@ import org.apache.hadoop.fs.CanUnbuffer; import org.apache.hadoop.fs.Seekable; import org.apache.hadoop.hdds.client.BlockID; +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.ChunkInfo; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; @@ -60,6 +62,7 @@ public class ChunkInputStream extends InputStream private final ChunkInfo chunkInfo; private final long length; private final BlockID blockID; + private ContainerProtos.DatanodeBlockID datanodeBlockID; private final XceiverClientFactory xceiverClientFactory; private XceiverClientSpi xceiverClient; private final Supplier pipelineSupplier; @@ -290,6 +293,19 @@ protected synchronized void releaseClient() { } } + /** + * Updates DatanodeBlockId which based on blockId. + */ + private void updateDatanodeBlockId() throws IOException { + DatanodeDetails closestNode = pipelineSupplier.get().getClosestNode(); + int replicaIdx = pipelineSupplier.get().getReplicaIndex(closestNode); + ContainerProtos.DatanodeBlockID.Builder builder = blockID.getDatanodeBlockIDProtobufBuilder(); + if (replicaIdx > 0) { + builder.setReplicaIndex(replicaIdx); + } + datanodeBlockID = builder.build(); + } + /** * Acquire new client if previous one was released. */ @@ -297,6 +313,7 @@ protected synchronized void acquireClient() throws IOException { if (xceiverClientFactory != null && xceiverClient == null) { xceiverClient = xceiverClientFactory.acquireClientForReadData( pipelineSupplier.get()); + updateDatanodeBlockId(); } } @@ -423,7 +440,7 @@ protected ByteBuffer[] readChunk(ChunkInfo readChunkInfo) ReadChunkResponseProto readChunkResponse = ContainerProtocolCalls.readChunk(xceiverClient, - readChunkInfo, blockID, validators, tokenSupplier.get()); + readChunkInfo, blockID, datanodeBlockID, validators, tokenSupplier.get()); if (readChunkResponse.hasData()) { return readChunkResponse.getData().asReadOnlyByteBufferList() diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java index 8540a0c5ab83..de3844191b50 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java @@ -23,30 +23,38 @@ import java.util.Objects; /** - * BlockID of Ozone (containerID + localID + blockCommitSequenceId). + * BlockID of Ozone (containerID + localID + blockCommitSequenceId + replicaIndex). */ public class BlockID { private final ContainerBlockID containerBlockID; private long blockCommitSequenceId; + private int replicaIndex; public BlockID(long containerID, long localID) { - this(containerID, localID, 0); + this(containerID, localID, 0, 0); } - private BlockID(long containerID, long localID, long bcsID) { + private BlockID(long containerID, long localID, long bcsID, int repIndex) { containerBlockID = new ContainerBlockID(containerID, localID); blockCommitSequenceId = bcsID; + this.replicaIndex = repIndex; + } + + public BlockID(BlockID blockID) { + this(blockID.getContainerID(), blockID.getLocalID(), blockID.getBlockCommitSequenceId(), + blockID.getReplicaIndex()); } public BlockID(ContainerBlockID containerBlockID) { - this(containerBlockID, 0); + this(containerBlockID, 0, 0); } - private BlockID(ContainerBlockID containerBlockID, long bcsId) { + private BlockID(ContainerBlockID containerBlockID, long bcsId, int repIndex) { this.containerBlockID = containerBlockID; blockCommitSequenceId = bcsId; + this.replicaIndex = repIndex; } public long getContainerID() { @@ -65,6 +73,14 @@ public void setBlockCommitSequenceId(long blockCommitSequenceId) { this.blockCommitSequenceId = blockCommitSequenceId; } + public int getReplicaIndex() { + return replicaIndex; + } + + public void setReplicaIndex(int replicaIndex) { + this.replicaIndex = replicaIndex; + } + public ContainerBlockID getContainerBlockID() { return containerBlockID; } @@ -79,21 +95,32 @@ public String toString() { public void appendTo(StringBuilder sb) { containerBlockID.appendTo(sb); sb.append(" bcsId: ").append(blockCommitSequenceId); + sb.append(" replicaIndex: ").append(replicaIndex); } @JsonIgnore public ContainerProtos.DatanodeBlockID getDatanodeBlockIDProtobuf() { + ContainerProtos.DatanodeBlockID.Builder blockID = getDatanodeBlockIDProtobufBuilder(); + if (replicaIndex > 0) { + blockID.setReplicaIndex(replicaIndex); + } + return blockID.build(); + } + + @JsonIgnore + public ContainerProtos.DatanodeBlockID.Builder getDatanodeBlockIDProtobufBuilder() { return ContainerProtos.DatanodeBlockID.newBuilder(). setContainerID(containerBlockID.getContainerID()) .setLocalID(containerBlockID.getLocalID()) - .setBlockCommitSequenceId(blockCommitSequenceId).build(); + .setBlockCommitSequenceId(blockCommitSequenceId); } @JsonIgnore public static BlockID getFromProtobuf( ContainerProtos.DatanodeBlockID blockID) { return new BlockID(blockID.getContainerID(), - blockID.getLocalID(), blockID.getBlockCommitSequenceId()); + blockID.getLocalID(), blockID.getBlockCommitSequenceId(), blockID.hasReplicaIndex() ? + blockID.getReplicaIndex() : 0); } @JsonIgnore @@ -107,7 +134,7 @@ public HddsProtos.BlockID getProtobuf() { public static BlockID getFromProtobuf(HddsProtos.BlockID blockID) { return new BlockID( ContainerBlockID.getFromProtobuf(blockID.getContainerBlockID()), - blockID.getBlockCommitSequenceId()); + blockID.getBlockCommitSequenceId(), 0); } @Override @@ -120,13 +147,14 @@ public boolean equals(Object o) { } BlockID blockID = (BlockID) o; return containerBlockID.equals(blockID.getContainerBlockID()) - && blockCommitSequenceId == blockID.getBlockCommitSequenceId(); + && blockCommitSequenceId == blockID.getBlockCommitSequenceId() + && replicaIndex == blockID.getReplicaIndex(); } @Override public int hashCode() { return Objects .hash(containerBlockID.getContainerID(), containerBlockID.getLocalID(), - blockCommitSequenceId); + blockCommitSequenceId, replicaIndex); } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 66c8459a01a3..7fc233b94940 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -312,12 +312,21 @@ public static ContainerCommandRequestProto getPutBlockRequest( return builder.build(); } + /** + * Calls the container protocol to read a chunk. + */ + public static ContainerProtos.ReadChunkResponseProto readChunk( + XceiverClientSpi xceiverClient, ChunkInfo chunk, BlockID blockID, + List validators, Token token) throws IOException { + return readChunk(xceiverClient, chunk, blockID, blockID.getDatanodeBlockIDProtobuf(), validators, token); + } /** * Calls the container protocol to read a chunk. * * @param xceiverClient client to perform call * @param chunk information about chunk to read * @param blockID ID of the block + * @param dnBlockId dnBlock of the blockId * @param validators functions to validate the response * @param token a token for this block (may be null) * @return container protocol read chunk response @@ -325,11 +334,11 @@ public static ContainerCommandRequestProto getPutBlockRequest( */ public static ContainerProtos.ReadChunkResponseProto readChunk( XceiverClientSpi xceiverClient, ChunkInfo chunk, BlockID blockID, - List validators, + DatanodeBlockID dnBlockId, List validators, Token token) throws IOException { ReadChunkRequestProto.Builder readChunkRequest = ReadChunkRequestProto.newBuilder() - .setBlockID(blockID.getDatanodeBlockIDProtobuf()) + .setBlockID(dnBlockId) .setChunkData(chunk) .setReadChunkVersion(ContainerProtos.ReadChunkVersion.V1); ContainerCommandRequestProto.Builder builder = diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java index af515c33e299..ea2f8d08057b 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java @@ -38,6 +38,7 @@ import com.google.common.base.Preconditions; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.BCSID_MISMATCH; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_NOT_FOUND; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_BLOCK; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNKNOWN_BCSID; import org.slf4j.Logger; @@ -218,6 +219,14 @@ public BlockData getBlock(Container container, BlockID blockID) KeyValueContainerData containerData = (KeyValueContainerData) container .getContainerData(); + int containerReplicaIndex = containerData.getReplicaIndex(); + if (containerReplicaIndex != blockID.getReplicaIndex()) { + throw new StorageContainerException( + "Unable to find the Container with replicaIdx " + blockID.getReplicaIndex() + ". Container " + + containerData.getContainerID() + " replicaIdx is " + + containerReplicaIndex + ".", CONTAINER_NOT_FOUND); + } + long containerBCSId = containerData.getBlockCommitSequenceId(); if (containerBCSId < bcsId) { throw new StorageContainerException( From 881594024de523177a91179f6d7bb257f3c0048c Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Thu, 6 Jun 2024 00:39:20 -0700 Subject: [PATCH 02/21] HDDS-10983. Add Testcase Change-Id: I4386a0de5d61d1cec51f63ed7f75d531fa389e9c --- .../hdds/scm/storage/ChunkInputStream.java | 2 +- .../scm/storage/ContainerProtocolCalls.java | 19 +- .../ozone/container/ContainerTestHelper.java | 9 +- ...KeyValueHandlerWithUnhealthyContainer.java | 29 +++ .../hdds/scm/TestXceiverClientGrpc.java | 2 +- .../scm/storage/TestContainerCommandsEC.java | 2 +- .../container/TestContainerReplication.java | 220 ++++++++++++++++++ .../hadoop/ozone/container/TestHelper.java | 2 +- 8 files changed, 265 insertions(+), 20 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java index 529a6967343b..172887e56417 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java @@ -440,7 +440,7 @@ protected ByteBuffer[] readChunk(ChunkInfo readChunkInfo) ReadChunkResponseProto readChunkResponse = ContainerProtocolCalls.readChunk(xceiverClient, - readChunkInfo, blockID, datanodeBlockID, validators, tokenSupplier.get()); + readChunkInfo, datanodeBlockID, validators, tokenSupplier.get()); if (readChunkResponse.hasData()) { return readChunkResponse.getData().asReadOnlyByteBufferList() diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 7fc233b94940..f5ad0a015d6e 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -312,33 +312,24 @@ public static ContainerCommandRequestProto getPutBlockRequest( return builder.build(); } - /** - * Calls the container protocol to read a chunk. - */ - public static ContainerProtos.ReadChunkResponseProto readChunk( - XceiverClientSpi xceiverClient, ChunkInfo chunk, BlockID blockID, - List validators, Token token) throws IOException { - return readChunk(xceiverClient, chunk, blockID, blockID.getDatanodeBlockIDProtobuf(), validators, token); - } /** * Calls the container protocol to read a chunk. * * @param xceiverClient client to perform call * @param chunk information about chunk to read * @param blockID ID of the block - * @param dnBlockId dnBlock of the blockId * @param validators functions to validate the response * @param token a token for this block (may be null) * @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, BlockID blockID, - DatanodeBlockID dnBlockId, List validators, + XceiverClientSpi xceiverClient, ChunkInfo chunk, + DatanodeBlockID blockID, List validators, Token token) throws IOException { ReadChunkRequestProto.Builder readChunkRequest = ReadChunkRequestProto.newBuilder() - .setBlockID(dnBlockId) + .setBlockID(blockID) .setChunkData(chunk) .setReadChunkVersion(ContainerProtos.ReadChunkVersion.V1); ContainerCommandRequestProto.Builder builder = @@ -365,7 +356,7 @@ public static ContainerProtos.ReadChunkResponseProto readChunk( } private static ContainerProtos.ReadChunkResponseProto readChunk( - XceiverClientSpi xceiverClient, ChunkInfo chunk, BlockID blockID, + XceiverClientSpi xceiverClient, ChunkInfo chunk, DatanodeBlockID blockID, List validators, ContainerCommandRequestProto.Builder builder, DatanodeDetails d) throws IOException { @@ -387,7 +378,7 @@ private static ContainerProtos.ReadChunkResponseProto readChunk( return response; } - static String toErrorMessage(ChunkInfo chunk, BlockID blockId, + static String toErrorMessage(ChunkInfo chunk, DatanodeBlockID blockId, DatanodeDetails d) { return String.format("Failed to read chunk %s (len=%s) %s from %s", chunk.getChunkName(), chunk.getLen(), blockId, d); diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java index 7ade596add7a..73150bd47778 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java @@ -540,6 +540,11 @@ public static byte[] generateData(int length, boolean random) { return data; } + public static ContainerCommandRequestProto getDummyCommandRequestProto( + ContainerProtos.Type cmdType) { + return getDummyCommandRequestProto(cmdType, 0); + } + /** * Construct fake protobuf messages for various types of requests. * This is tedious, however necessary to test. Protobuf classes are final @@ -549,7 +554,7 @@ public static byte[] generateData(int length, boolean random) { * @return */ public static ContainerCommandRequestProto getDummyCommandRequestProto( - ContainerProtos.Type cmdType) { + ContainerProtos.Type cmdType, int replicaIndex) { final Builder builder = ContainerCommandRequestProto.newBuilder() .setCmdType(cmdType) @@ -558,7 +563,7 @@ public static ContainerCommandRequestProto getDummyCommandRequestProto( final DatanodeBlockID fakeBlockId = DatanodeBlockID.newBuilder() - .setContainerID(DUMMY_CONTAINER_ID).setLocalID(1) + .setContainerID(DUMMY_CONTAINER_ID).setLocalID(1).setReplicaIndex(replicaIndex) .setBlockCommitSequenceId(101).build(); final ContainerProtos.ChunkInfo fakeChunkInfo = diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java index d9b85a7ce806..9c4209e1a471 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java @@ -34,6 +34,8 @@ import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,6 +97,22 @@ public void testGetBlock() { assertEquals(UNKNOWN_BCSID, response.getResult()); } + @ParameterizedTest + @ValueSource(ints = {0, 1, 2, 3, 4, 5}) + public void testGetBlockWithReplicaIndexMismatch(int replicaIndex) { + KeyValueContainer container = getMockContainerWithReplicaIndex(replicaIndex); + KeyValueHandler handler = getDummyHandler(); + for (int rid = 0; rid <= 5; rid++) { + ContainerProtos.ContainerCommandResponseProto response = + handler.handleGetBlock( + getDummyCommandRequestProto(ContainerProtos.Type.GetBlock, rid), + container); + assertEquals(rid != replicaIndex ? ContainerProtos.Result.CONTAINER_NOT_FOUND : UNKNOWN_BCSID, + response.getResult()); + } + + } + @Test public void testGetCommittedBlockLength() { KeyValueContainer container = getMockUnhealthyContainer(); @@ -204,4 +222,15 @@ private KeyValueContainer getMockUnhealthyContainer() { .ContainerDataProto.newBuilder().setContainerID(1).build()); return new KeyValueContainer(containerData, new OzoneConfiguration()); } + + private KeyValueContainer getMockContainerWithReplicaIndex(int replicaIndex) { + KeyValueContainerData containerData = mock(KeyValueContainerData.class); + when(containerData.getState()).thenReturn( + ContainerProtos.ContainerDataProto.State.CLOSED); + when(containerData.getBlockCommitSequenceId()).thenReturn(100L); + when(containerData.getReplicaIndex()).thenReturn(replicaIndex); + when(containerData.getProtoBufMessage()).thenReturn(ContainerProtos + .ContainerDataProto.newBuilder().setContainerID(1).build()); + return new KeyValueContainer(containerData, new OzoneConfiguration()); + } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientGrpc.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientGrpc.java index 99095f55b008..bab979618efd 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientGrpc.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientGrpc.java @@ -259,7 +259,7 @@ private void invokeXceiverClientReadChunk(XceiverClientSpi client) .setLen(-1) .setOffset(0) .build(), - bid, + bid.getDatanodeBlockIDProtobuf(), null, null); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java index b4814d7b5e5d..c8ffe5d011ac 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java @@ -511,7 +511,7 @@ public void testCreateRecoveryContainer() throws Exception { readContainerResponseProto.getContainerData().getState()); ContainerProtos.ReadChunkResponseProto readChunkResponseProto = ContainerProtocolCalls.readChunk(dnClient, - writeChunkRequest.getWriteChunk().getChunkData(), blockID, null, + writeChunkRequest.getWriteChunk().getChunkData(), blockID.getDatanodeBlockIDProtobuf(), null, blockToken); ByteBuffer[] readOnlyByteBuffersArray = BufferUtils .getReadOnlyByteBuffersArray( diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java index 810a57254924..f184acf68958 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java @@ -22,6 +22,7 @@ import static java.util.Arrays.asList; import static java.util.Collections.emptyMap; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_KEY; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DEADNODE_INTERVAL; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL; @@ -30,36 +31,63 @@ import static org.apache.ozone.test.GenericTestUtils.setLogLevel; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.mockito.Mockito.any; import java.io.IOException; import java.io.OutputStream; import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; import java.util.LinkedList; import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import com.google.common.base.Functions; +import com.google.common.collect.ImmutableMap; +import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.RatisReplicationConfig; +import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.scm.PlacementPolicy; +import org.apache.hadoop.hdds.scm.container.ContainerReplica; +import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicyFactory; +import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementMetrics; +import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRackScatter; import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager.ReplicationManagerConfiguration; import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementCapacity; import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRackAware; import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRandom; +import org.apache.hadoop.hdds.scm.net.NetworkTopology; +import org.apache.hadoop.hdds.scm.node.NodeManager; +import org.apache.hadoop.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.client.ObjectStore; import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.client.OzoneClientFactory; import org.apache.hadoop.ozone.client.OzoneVolume; +import org.apache.hadoop.ozone.client.io.OzoneInputStream; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.MockedStatic; +import org.mockito.Mockito; import org.slf4j.event.Level; /** @@ -159,6 +187,21 @@ private void createTestData(OzoneClient client) throws IOException { } } + private byte[] createTestData(OzoneClient client, int size) throws IOException { + ObjectStore objectStore = client.getObjectStore(); + objectStore.createVolume(VOLUME); + OzoneVolume volume = objectStore.getVolume(VOLUME); + volume.createBucket(BUCKET); + OzoneBucket bucket = volume.getBucket(BUCKET); + try (OutputStream out = bucket.createKey(KEY, 0, new ECReplicationConfig("RS-3-2-1k"), + new HashMap<>())) { + byte[] b = new byte[size]; + new Random().nextBytes(b); + out.write(b); + return b; + } + } + private static List lookupKey(MiniOzoneCluster cluster) throws IOException { OmKeyArgs keyArgs = new OmKeyArgs.Builder() @@ -172,4 +215,181 @@ private static List lookupKey(MiniOzoneCluster cluster) return locations.getLocationList(); } + private static OmKeyLocationInfo lookupKeyFirstLocation(MiniOzoneCluster cluster) + throws IOException { + OmKeyArgs keyArgs = new OmKeyArgs.Builder() + .setVolumeName(VOLUME) + .setBucketName(BUCKET) + .setKeyName(KEY) + .build(); + OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs); + OmKeyLocationInfoGroup locations = keyInfo.getLatestVersionLocations(); + Assertions.assertNotNull(locations); + return locations.getLocationList().get(0); + } + + + public void assertState(MiniOzoneCluster cluster, Map expectedReplicaMap) + throws IOException { + OmKeyLocationInfo keyLocation = lookupKeyFirstLocation(cluster); + Map replicaMap = + keyLocation.getPipeline().getNodes().stream().collect(Collectors.toMap( + dn -> keyLocation.getPipeline().getReplicaIndex(dn), Functions.identity())); + Assertions.assertEquals(expectedReplicaMap, replicaMap); + } + + private OzoneInputStream createInputStream(OzoneClient client) throws IOException { + ObjectStore objectStore = client.getObjectStore(); + OzoneVolume volume = objectStore.getVolume(VOLUME); + OzoneBucket bucket = volume.getBucket(BUCKET); + return bucket.readKey(KEY); + } + + + @Test + public void testECContainerReplication() throws Exception { + OzoneConfiguration conf = createConfiguration(false); + final AtomicReference mockedDatanodeToRemove = new AtomicReference<>(); + + // Overiding Config to support 1k Chunk size + conf.set("ozone.replication.allowed-configs", "(^((STANDALONE|RATIS)/(ONE|THREE))|(EC/(3-2|6-3|10-4)-" + + "(512|1024|2048|4096|1)k)$)"); + conf.set(OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_KEY, SCMContainerPlacementRackScatter.class.getCanonicalName()); + try (MockedStatic mocked = + Mockito.mockStatic(ContainerPlacementPolicyFactory.class, Mockito.CALLS_REAL_METHODS)) { + mocked.when(() -> ContainerPlacementPolicyFactory.getECPolicy(any(ConfigurationSource.class), + any(NodeManager.class), any(NetworkTopology.class), Mockito.anyBoolean(), + any(SCMContainerPlacementMetrics.class))).thenAnswer(i -> { + PlacementPolicy placementPolicy = (PlacementPolicy) Mockito.spy(i.callRealMethod()); + Mockito.doAnswer(args -> { + Set containerReplica = ((Set) args.getArgument(0)).stream() + .filter(r -> r.getDatanodeDetails().equals(mockedDatanodeToRemove.get())) + .collect(Collectors.toSet()); + return containerReplica; + }).when(placementPolicy).replicasToRemoveToFixOverreplication(Mockito.anySet(), Mockito.anyInt()); + return placementPolicy; + }); + + // Creating Cluster with 6 Nodes + try (MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(6).build()) { + cluster.waitForClusterToBeReady(); + try (OzoneClient client = OzoneClientFactory.getRpcClient(conf)) { + Set allNodes = + cluster.getHddsDatanodes().stream().map(HddsDatanodeService::getDatanodeDetails).collect( + Collectors.toSet()); + List initialNodesWithData = new ArrayList<>(); + DatanodeDetails extraNode = null; + // Keeping 5 DNs and stopping the 6th Node here it is kept in the var extraNode + for (DatanodeDetails dn : allNodes) { + if (initialNodesWithData.size() < 5) { + initialNodesWithData.add(dn); + } else { + extraNode = dn; + cluster.shutdownHddsDatanode(dn); + } + } + + // Creating 2 stripes with Chunk Size 1k + int size = 6 * 1024; + byte[] originalData = createTestData(client, size); + + // Getting latest location of the key + final OmKeyLocationInfo keyLocation = lookupKeyFirstLocation(cluster); + long containerID = keyLocation.getContainerID(); + waitForContainerClose(cluster, containerID); + + // Forming Replica Index Map + Map replicaIndexMap = + initialNodesWithData.stream().map(dn -> new Object[]{dn, keyLocation.getPipeline().getReplicaIndex(dn)}) + .collect( + Collectors.toMap(x -> (Integer) x[1], x -> (DatanodeDetails) x[0])); + + //Reading through file and comparing with input data. + byte[] readData = new byte[size]; + try (OzoneInputStream inputStream = createInputStream(client)) { + inputStream.read(readData); + Assertions.assertTrue(Arrays.equals(readData, originalData)); + } + + //Opening a new stream before we make changes to the blocks. + try (OzoneInputStream inputStream = createInputStream(client)) { + int firstReadLen = 1024 * 3; + Arrays.fill(readData, (byte)0); + inputStream.read(readData, 0, firstReadLen); + //Checking the initial state as per the latest location. + assertState(cluster, ImmutableMap.of(1, replicaIndexMap.get(1), 2, replicaIndexMap.get(2), + 3, replicaIndexMap.get(3), 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); + + // Shutting down DN1 and waiting for underreplication + cluster.shutdownHddsDatanode(replicaIndexMap.get(1)); + waitForReplicaCount(containerID, 4, cluster); + assertState(cluster, ImmutableMap.of(2, replicaIndexMap.get(2), 3, replicaIndexMap.get(3), + 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); + + //Starting up ExtraDN. RM should run and create Replica Index 1 to ExtraDN + cluster.restartHddsDatanode(extraNode, false); + waitForReplicaCount(containerID, 5, cluster); + assertState(cluster, ImmutableMap.of(1, extraNode, 2, replicaIndexMap.get(2), + 3, replicaIndexMap.get(3), 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); + + //Stopping RM and starting DN1, this should lead to overreplication of ReplicaIndex 1 + cluster.getStorageContainerManager().getReplicationManager().stop(); + cluster.restartHddsDatanode(replicaIndexMap.get(1), true); + waitForReplicaCount(containerID, 6, cluster); + + //Mocking Overreplication processor to remove replica from DN1. Final Replica1 should be in extraNode + mockedDatanodeToRemove.set(replicaIndexMap.get(1)); + cluster.getStorageContainerManager().getReplicationManager().start(); + waitForReplicaCount(containerID, 5, cluster); + assertState(cluster, ImmutableMap.of(1, extraNode, 2, replicaIndexMap.get(2), + 3, replicaIndexMap.get(3), 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); + + //Stopping DN3 and waiting for underreplication + cluster.getStorageContainerManager().getReplicationManager().stop(); + cluster.shutdownHddsDatanode(replicaIndexMap.get(3)); + waitForReplicaCount(containerID, 4, cluster); + assertState(cluster, ImmutableMap.of(1, extraNode, 2, replicaIndexMap.get(2), + 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); + + //Starting RM, Under replication processor should create Replica 3 in DN1 + cluster.getStorageContainerManager().getReplicationManager().start(); + waitForReplicaCount(containerID, 5, cluster); + assertState(cluster, ImmutableMap.of(1, extraNode, 2, replicaIndexMap.get(2), + 3, replicaIndexMap.get(1), 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); + + //Starting DN3 leading to overreplication of replica 3 + cluster.getStorageContainerManager().getReplicationManager().stop(); + cluster.restartHddsDatanode(replicaIndexMap.get(3), true); + waitForReplicaCount(containerID, 6, cluster); + + //Mocking Overreplication processor to remove data from DN3 leading to Replica3 to stay in DN1. + mockedDatanodeToRemove.set(replicaIndexMap.get(3)); + cluster.getStorageContainerManager().getReplicationManager().start(); + waitForReplicaCount(containerID, 5, cluster); + assertState(cluster, ImmutableMap.of(1, extraNode, 2, replicaIndexMap.get(2), + 3, replicaIndexMap.get(1), 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); + + //Stopping Extra DN leading to underreplication of Replica 1 + cluster.getStorageContainerManager().getReplicationManager().stop(); + cluster.shutdownHddsDatanode(extraNode); + waitForReplicaCount(containerID, 4, cluster); + assertState(cluster, ImmutableMap.of(2, replicaIndexMap.get(2), 3, replicaIndexMap.get(1), + 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); + + + //RM should fix underreplication and write data to DN3 + cluster.getStorageContainerManager().getReplicationManager().start(); + waitForReplicaCount(containerID, 5, cluster); + assertState(cluster, ImmutableMap.of(1, replicaIndexMap.get(3), 2, replicaIndexMap.get(2), + 3, replicaIndexMap.get(1), 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); + + // Reading the pre initialized inputStream. This leads to swap of the block1 & block3. + inputStream.read(readData, firstReadLen, size - firstReadLen); + Assertions.assertTrue(Arrays.equals(readData, originalData)); + } + } + } + } + } + } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java index 2a33ddc5677f..d9d24f4ae499 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java @@ -442,6 +442,6 @@ public static int countReplicas(long containerID, MiniOzoneCluster cluster) { public static void waitForReplicaCount(long containerID, int count, MiniOzoneCluster cluster) throws TimeoutException, InterruptedException { GenericTestUtils.waitFor(() -> countReplicas(containerID, cluster) == count, - 200, 30000); + 200, 300000); } } From 428727049f64214f5b97f799999f1deb2dc2a255 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Thu, 6 Jun 2024 19:55:10 -0700 Subject: [PATCH 03/21] HDDS-10983. Make the read api compatible with older clients Change-Id: Ic1175d69441957382fb9213f7e37e3047d284806 --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 7 +-- .../scm/storage/BlockDataStreamOutput.java | 3 +- .../ozone/client/io/ECBlockInputStream.java | 5 +- .../apache/hadoop/hdds/client/BlockID.java | 43 +++++++------ .../ratis/ContainerCommandRequestMessage.java | 5 +- .../scm/storage/ContainerProtocolCalls.java | 60 ++++++++++++------- .../apache/hadoop/ozone/ClientVersion.java | 7 ++- .../TestContainerCommandRequestMessage.java | 6 +- .../ozone/container/ContainerTestHelper.java | 39 +++++------- .../container/common/impl/HddsDispatcher.java | 3 +- .../CloseContainerCommandHandler.java | 5 +- .../server/ratis/ContainerStateMachine.java | 10 ++-- .../container/keyvalue/KeyValueHandler.java | 13 ++-- .../keyvalue/helpers/BlockUtils.java | 23 ++++++- .../keyvalue/impl/BlockManagerImpl.java | 31 ++-------- .../keyvalue/interfaces/BlockManager.java | 15 ++++- .../common/impl/TestHddsDispatcher.java | 6 +- .../keyvalue/TestKeyValueHandler.java | 10 ++-- ...KeyValueHandlerWithUnhealthyContainer.java | 14 +++++ .../impl/TestKeyValueStreamDataChannel.java | 3 +- .../client/io/ECBlockOutputStreamEntry.java | 3 +- .../rpc/TestContainerReplicationEndToEnd.java | 4 +- .../TestContainerStateMachineFailures.java | 15 ++--- .../rpc/TestDeleteWithInAdequateDN.java | 4 +- .../container/TestContainerReplication.java | 37 +++++++++--- .../ozone/freon/DatanodeBlockPutter.java | 5 +- .../ozone/freon/DatanodeChunkGenerator.java | 5 +- .../ozone/freon/DatanodeChunkValidator.java | 13 ++-- .../FollowerAppendLogEntryGenerator.java | 5 +- .../freon/LeaderAppendLogEntryGenerator.java | 6 +- 30 files changed, 246 insertions(+), 159 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 2b5854ca2086..39a69b35d5dc 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -71,6 +71,7 @@ import org.slf4j.LoggerFactory; import static org.apache.hadoop.hdds.HddsUtils.processForDebug; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; /** * {@link XceiverClientSpi} implementation, the standalone client. @@ -337,8 +338,7 @@ private XceiverClientReply sendCommandWithTraceIDAndRetry( return TracingUtil.executeInNewSpan(spanName, () -> { - ContainerCommandRequestProto finalPayload = - ContainerCommandRequestProto.newBuilder(request) + ContainerCommandRequestProto finalPayload = getContainerCommandRequestProtoBuilder(request) .setTraceID(TracingUtil.exportCurrentSpan()).build(); return sendCommandWithRetry(finalPayload, validators); }); @@ -490,8 +490,7 @@ public XceiverClientReply sendCommandAsync( try (Scope ignored = GlobalTracer.get().activateSpan(span)) { - ContainerCommandRequestProto finalPayload = - ContainerCommandRequestProto.newBuilder(request) + ContainerCommandRequestProto finalPayload = getContainerCommandRequestProtoBuilder(request) .setTraceID(TracingUtil.exportCurrentSpan()) .build(); XceiverClientReply asyncReply = diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java index d5423d4ec0bb..c9174537ba70 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java @@ -62,6 +62,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync; /** @@ -204,7 +205,7 @@ private DataStreamOutput setupStream(Pipeline pipeline) throws IOException { // it or remove it completely if possible String id = pipeline.getFirstNode().getUuidString(); ContainerProtos.ContainerCommandRequestProto.Builder builder = - ContainerProtos.ContainerCommandRequestProto.newBuilder() + getContainerCommandRequestProtoBuilder() .setCmdType(ContainerProtos.Type.StreamInit) .setContainerID(blockID.get().getContainerID()) .setDatanodeUuid(id).setWriteChunk(writeChunkRequest); diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java index 8dc07f129b9c..f2cbbf0863cb 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java @@ -176,8 +176,8 @@ protected BlockExtendedInputStream getOrOpenStream(int locationIndex) { .setReplicationConfig(StandaloneReplicationConfig.getInstance( HddsProtos.ReplicationFactor.ONE)) .setNodes(Arrays.asList(dataLocation)) - .setId(PipelineID.valueOf(dataLocation.getUuid())).setReplicaIndexes( - ImmutableMap.of(dataLocation, locationIndex + 1)) + .setId(PipelineID.valueOf(dataLocation.getUuid())) + .setReplicaIndexes(ImmutableMap.of(dataLocation, locationIndex + 1)) .setState(Pipeline.PipelineState.CLOSED) .build(); @@ -228,6 +228,7 @@ protected Function ecPipelineRefreshFunction( HddsProtos.ReplicationFactor.ONE)) .setNodes(Collections.singletonList(curIndexNode)) .setId(PipelineID.randomId()) + .setReplicaIndexes(Collections.singletonMap(curIndexNode, replicaIndex)) .setState(Pipeline.PipelineState.CLOSED) .build(); blockLocationInfo.setPipeline(pipeline); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java index de3844191b50..e59a412cdb60 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java @@ -20,7 +20,9 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import java.util.Arrays; import java.util.Objects; +import java.util.stream.Collectors; /** * BlockID of Ozone (containerID + localID + blockCommitSequenceId + replicaIndex). @@ -30,16 +32,21 @@ public class BlockID { private final ContainerBlockID containerBlockID; private long blockCommitSequenceId; - private int replicaIndex; + // null value when not set with private constructor.(This is to avoid confusion of replica index 0 & null value). + // This value would be only set when deserializing from ContainerProtos.DatanodeBlockID or copying from another + // BlockID object. + private Integer replicaIndex; + private StackTraceElement[] stackTraceElements; public BlockID(long containerID, long localID) { - this(containerID, localID, 0, 0); + this(containerID, localID, 0, null); } - private BlockID(long containerID, long localID, long bcsID, int repIndex) { + private BlockID(long containerID, long localID, long bcsID, Integer repIndex) { containerBlockID = new ContainerBlockID(containerID, localID); blockCommitSequenceId = bcsID; this.replicaIndex = repIndex; + this.stackTraceElements = Thread.currentThread().getStackTrace(); } public BlockID(BlockID blockID) { @@ -48,13 +55,14 @@ public BlockID(BlockID blockID) { } public BlockID(ContainerBlockID containerBlockID) { - this(containerBlockID, 0, 0); + this(containerBlockID, 0, null); } - private BlockID(ContainerBlockID containerBlockID, long bcsId, int repIndex) { + private BlockID(ContainerBlockID containerBlockID, long bcsId, Integer repIndex) { this.containerBlockID = containerBlockID; blockCommitSequenceId = bcsId; this.replicaIndex = repIndex; + this.stackTraceElements = Thread.currentThread().getStackTrace(); } public long getContainerID() { @@ -73,11 +81,12 @@ public void setBlockCommitSequenceId(long blockCommitSequenceId) { this.blockCommitSequenceId = blockCommitSequenceId; } - public int getReplicaIndex() { + // Can return a null value in case it is not set. + public Integer getReplicaIndex() { return replicaIndex; } - public void setReplicaIndex(int replicaIndex) { + public void setReplicaIndex(Integer replicaIndex) { this.replicaIndex = replicaIndex; } @@ -96,12 +105,13 @@ public void appendTo(StringBuilder sb) { containerBlockID.appendTo(sb); sb.append(" bcsId: ").append(blockCommitSequenceId); sb.append(" replicaIndex: ").append(replicaIndex); + sb.append(Arrays.stream(stackTraceElements).map(StackTraceElement::toString).collect(Collectors.joining("\n"))); } @JsonIgnore public ContainerProtos.DatanodeBlockID getDatanodeBlockIDProtobuf() { ContainerProtos.DatanodeBlockID.Builder blockID = getDatanodeBlockIDProtobufBuilder(); - if (replicaIndex > 0) { + if (replicaIndex != null) { blockID.setReplicaIndex(replicaIndex); } return blockID.build(); @@ -116,11 +126,11 @@ public ContainerProtos.DatanodeBlockID.Builder getDatanodeBlockIDProtobufBuilder } @JsonIgnore - public static BlockID getFromProtobuf( - ContainerProtos.DatanodeBlockID blockID) { + public static BlockID getFromProtobuf(ContainerProtos.DatanodeBlockID blockID) { return new BlockID(blockID.getContainerID(), - blockID.getLocalID(), blockID.getBlockCommitSequenceId(), blockID.hasReplicaIndex() ? - blockID.getReplicaIndex() : 0); + blockID.getLocalID(), + blockID.getBlockCommitSequenceId(), + blockID.hasReplicaIndex() ? blockID.getReplicaIndex() : null); } @JsonIgnore @@ -134,7 +144,7 @@ public HddsProtos.BlockID getProtobuf() { public static BlockID getFromProtobuf(HddsProtos.BlockID blockID) { return new BlockID( ContainerBlockID.getFromProtobuf(blockID.getContainerBlockID()), - blockID.getBlockCommitSequenceId(), 0); + blockID.getBlockCommitSequenceId(), null); } @Override @@ -148,13 +158,12 @@ public boolean equals(Object o) { BlockID blockID = (BlockID) o; return containerBlockID.equals(blockID.getContainerBlockID()) && blockCommitSequenceId == blockID.getBlockCommitSequenceId() - && replicaIndex == blockID.getReplicaIndex(); + && Objects.equals(replicaIndex, blockID.getReplicaIndex()); } @Override public int hashCode() { - return Objects - .hash(containerBlockID.getContainerID(), containerBlockID.getLocalID(), - blockCommitSequenceId, replicaIndex); + return Objects.hash(containerBlockID.getContainerID(), containerBlockID.getLocalID(), + blockCommitSequenceId, replicaIndex); } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/ContainerCommandRequestMessage.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/ContainerCommandRequestMessage.java index e1ebde25198c..98eba583be3a 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/ContainerCommandRequestMessage.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/ContainerCommandRequestMessage.java @@ -32,6 +32,8 @@ import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.apache.ratis.util.JavaUtils; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; + /** * Implementing the {@link Message} interface * for {@link ContainerCommandRequestProto}. @@ -39,8 +41,7 @@ public final class ContainerCommandRequestMessage implements Message { public static ContainerCommandRequestMessage toMessage( ContainerCommandRequestProto request, String traceId) { - final ContainerCommandRequestProto.Builder b - = ContainerCommandRequestProto.newBuilder(request); + final ContainerCommandRequestProto.Builder b = getContainerCommandRequestProtoBuilder(request); if (traceId != null) { b.setTraceID(traceId); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index f5ad0a015d6e..2026e78e137d 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -70,6 +70,7 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; import org.apache.hadoop.hdds.tracing.TracingUtil; +import org.apache.hadoop.ozone.ClientVersion; import org.apache.hadoop.ozone.common.Checksum; import org.apache.hadoop.ozone.common.ChecksumData; import org.apache.hadoop.security.token.Token; @@ -97,6 +98,28 @@ public final class ContainerProtocolCalls { private ContainerProtocolCalls() { } + /** + * Creates a ContainerCommandRequestProto with version set. + */ + public static ContainerCommandRequestProto.Builder getContainerCommandRequestProtoBuilder(int version) { + return getContainerCommandRequestProtoBuilder(null, version); + } + + public static ContainerCommandRequestProto.Builder getContainerCommandRequestProtoBuilder() { + return getContainerCommandRequestProtoBuilder(ClientVersion.CURRENT.toProtoValue()); + } + + public static ContainerCommandRequestProto.Builder getContainerCommandRequestProtoBuilder( + ContainerCommandRequestProto req, int version) { + return (req == null ? + ContainerCommandRequestProto.newBuilder() : ContainerCommandRequestProto.newBuilder(req)).setVersion(version); + } + + public static ContainerCommandRequestProto.Builder getContainerCommandRequestProtoBuilder( + ContainerCommandRequestProto req) { + return getContainerCommandRequestProtoBuilder(req, ClientVersion.CURRENT.toProtoValue()); + } + /** * Calls the container protocol to list blocks in container. * @@ -125,7 +148,7 @@ public static ListBlockResponseProto listBlock(XceiverClientSpi xceiverClient, xceiverClient.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder builder = - ContainerCommandRequestProto.newBuilder() + getContainerCommandRequestProtoBuilder() .setCmdType(Type.ListBlock) .setContainerID(containerID) .setDatanodeUuid(datanodeID) @@ -195,8 +218,7 @@ public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, GetBlockRequestProto.Builder readBlockRequest = GetBlockRequestProto .newBuilder() .setBlockID(datanodeBlockID); - ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto - .newBuilder() + ContainerCommandRequestProto.Builder builder = getContainerCommandRequestProtoBuilder() .setCmdType(Type.GetBlock) .setContainerID(datanodeBlockID.getContainerID()) .setGetBlock(readBlockRequest); @@ -255,7 +277,7 @@ private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, setBlockID(blockID.getDatanodeBlockIDProtobuf()); String id = xceiverClient.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder builder = - ContainerCommandRequestProto.newBuilder() + getContainerCommandRequestProtoBuilder() .setCmdType(Type.GetCommittedBlockLength) .setContainerID(blockID.getContainerID()) .setDatanodeUuid(id) @@ -301,11 +323,11 @@ public static ContainerCommandRequestProto getPutBlockRequest( .setBlockData(containerBlockData) .setEof(eof); final String id = pipeline.getFirstNode().getUuidString(); - ContainerCommandRequestProto.Builder builder = - ContainerCommandRequestProto.newBuilder().setCmdType(Type.PutBlock) - .setContainerID(containerBlockData.getBlockID().getContainerID()) - .setDatanodeUuid(id) - .setPutBlock(createBlockRequest); + ContainerCommandRequestProto.Builder builder = getContainerCommandRequestProtoBuilder() + .setCmdType(Type.PutBlock) + .setContainerID(containerBlockData.getBlockID().getContainerID()) + .setDatanodeUuid(id) + .setPutBlock(createBlockRequest); if (tokenString != null) { builder.setEncodedToken(tokenString); } @@ -333,7 +355,7 @@ public static ContainerProtos.ReadChunkResponseProto readChunk( .setChunkData(chunk) .setReadChunkVersion(ContainerProtos.ReadChunkVersion.V1); ContainerCommandRequestProto.Builder builder = - ContainerCommandRequestProto.newBuilder().setCmdType(Type.ReadChunk) + getContainerCommandRequestProtoBuilder().setCmdType(Type.ReadChunk) .setContainerID(blockID.getContainerID()) .setReadChunk(readChunkRequest); if (token != null) { @@ -421,7 +443,7 @@ public static XceiverClientReply writeChunkAsync( .setData(data); String id = xceiverClient.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder builder = - ContainerCommandRequestProto.newBuilder() + getContainerCommandRequestProtoBuilder() .setCmdType(Type.WriteChunk) .setContainerID(blockID.getContainerID()) .setDatanodeUuid(id) @@ -479,7 +501,7 @@ public static PutSmallFileResponseProto writeSmallFile( String id = client.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder builder = - ContainerCommandRequestProto.newBuilder() + getContainerCommandRequestProtoBuilder() .setCmdType(Type.PutSmallFile) .setContainerID(blockID.getContainerID()) .setDatanodeUuid(id) @@ -546,7 +568,7 @@ public static void createContainer(XceiverClientSpi client, String id = client.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder request = - ContainerCommandRequestProto.newBuilder(); + getContainerCommandRequestProtoBuilder(); if (encodedToken != null) { request.setEncodedToken(encodedToken); } @@ -576,7 +598,7 @@ public static void deleteContainer(XceiverClientSpi client, long containerID, String id = client.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder request = - ContainerCommandRequestProto.newBuilder(); + getContainerCommandRequestProtoBuilder(); request.setCmdType(ContainerProtos.Type.DeleteContainer); request.setContainerID(containerID); request.setDeleteContainer(deleteRequest); @@ -602,7 +624,7 @@ public static void closeContainer(XceiverClientSpi client, String id = client.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder request = - ContainerCommandRequestProto.newBuilder(); + getContainerCommandRequestProtoBuilder(); request.setCmdType(Type.CloseContainer); request.setContainerID(containerID); request.setCloseContainer(CloseContainerRequestProto.getDefaultInstance()); @@ -629,7 +651,7 @@ public static ReadContainerResponseProto readContainer( String id = client.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder request = - ContainerCommandRequestProto.newBuilder(); + getContainerCommandRequestProtoBuilder(); request.setCmdType(Type.ReadContainer); request.setContainerID(containerID); request.setReadContainer(ReadContainerRequestProto.getDefaultInstance()); @@ -667,8 +689,7 @@ public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client, .build(); String id = client.getPipeline().getClosestNode().getUuidString(); - ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto - .newBuilder() + ContainerCommandRequestProto.Builder builder = getContainerCommandRequestProtoBuilder() .setCmdType(Type.GetSmallFile) .setContainerID(blockID.getContainerID()) .setDatanodeUuid(id) @@ -778,8 +799,7 @@ public static List toValidatorList(Validator validator) { HashMap datanodeToResponseMap = new HashMap<>(); String id = xceiverClient.getPipeline().getFirstNode().getUuidString(); - ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto - .newBuilder() + ContainerCommandRequestProto.Builder builder = getContainerCommandRequestProtoBuilder() .setCmdType(Type.GetBlock) .setContainerID(datanodeBlockID.getContainerID()) .setDatanodeUuid(id) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java index cc6695dc7d68..82f5afd4fedc 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java @@ -28,7 +28,7 @@ /** * Versioning for protocol clients. */ -public enum ClientVersion implements ComponentVersion { +public enum ClientVersion implements ComponentVersion, Comparable { DEFAULT_VERSION(0, "Initial version"), @@ -42,6 +42,9 @@ public enum ClientVersion implements ComponentVersion { "This client version has support for Object Store and File " + "System Optimized Bucket Layouts."), + ERASURE_CODING_READ_CHUNK_CORRUPTION_FIX(4, + "This client version fixes for fixing read corruption in case for EC block reads."), + FUTURE_VERSION(-1, "Used internally when the server side is older and an" + " unknown client version has arrived from the client."); @@ -79,4 +82,6 @@ private static ClientVersion latest() { return versions[versions.length - 2]; } + + } diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/TestContainerCommandRequestMessage.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/TestContainerCommandRequestMessage.java index ef65335a6c01..b0f5bc4f4541 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/TestContainerCommandRequestMessage.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/TestContainerCommandRequestMessage.java @@ -40,6 +40,8 @@ import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; + /** Testing {@link ContainerCommandRequestMessage}. */ public class TestContainerCommandRequestMessage { static final Random RANDOM = new Random(); @@ -86,7 +88,7 @@ static ContainerCommandRequestProto newPutSmallFile( .setBlock(putBlockRequest) .setData(data) .build(); - return ContainerCommandRequestProto.newBuilder() + return getContainerCommandRequestProtoBuilder() .setCmdType(Type.PutSmallFile) .setContainerID(blockID.getContainerID()) .setDatanodeUuid(UUID.randomUUID().toString()) @@ -108,7 +110,7 @@ static ContainerCommandRequestProto newWriteChunk( .setBlockID(blockID.getDatanodeBlockIDProtobuf()) .setChunkData(chunk) .setData(data); - return ContainerCommandRequestProto.newBuilder() + return getContainerCommandRequestProtoBuilder() .setCmdType(Type.WriteChunk) .setContainerID(blockID.getContainerID()) .setDatanodeUuid(UUID.randomUUID().toString()) diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java index 73150bd47778..0ab75818dab0 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java @@ -52,6 +52,8 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.fail; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; + /** * Helpers for container tests. */ @@ -130,7 +132,7 @@ public static ContainerCommandRequestProto getWriteChunkRequest( public static ContainerCommandRequestProto getListBlockRequest( ContainerCommandRequestProto writeChunkRequest) { - return ContainerCommandRequestProto.newBuilder() + return getContainerCommandRequestProtoBuilder() .setContainerID(writeChunkRequest.getContainerID()) .setCmdType(ContainerProtos.Type.ListBlock) .setDatanodeUuid(writeChunkRequest.getDatanodeUuid()) @@ -146,7 +148,7 @@ public static ContainerCommandRequestProto getPutBlockRequest( .setSize(writeChunkRequest.getWriteChunk().getChunkData().getLen()) .setBlockID(writeChunkRequest.getWriteChunk().getBlockID()) .addChunks(writeChunkRequest.getWriteChunk().getChunkData()); - return ContainerCommandRequestProto.newBuilder() + return getContainerCommandRequestProtoBuilder() .setContainerID(writeChunkRequest.getContainerID()) .setCmdType(ContainerProtos.Type.PutBlock) .setDatanodeUuid(writeChunkRequest.getDatanodeUuid()) @@ -179,8 +181,7 @@ public static Builder newWriteChunkRequestBuilder( writeRequest.setChunkData(info.getProtoBufMessage()); writeRequest.setData(data.toByteString()); - Builder request = - ContainerCommandRequestProto.newBuilder(); + Builder request = getContainerCommandRequestProtoBuilder(); request.setCmdType(ContainerProtos.Type.WriteChunk); request.setContainerID(blockID.getContainerID()); request.setWriteChunk(writeRequest); @@ -220,8 +221,7 @@ public static ContainerCommandRequestProto getWriteSmallFileRequest( smallFileRequest.setData(data.toByteString()); smallFileRequest.setBlock(putRequest); - Builder request = - ContainerCommandRequestProto.newBuilder(); + Builder request = getContainerCommandRequestProtoBuilder(); request.setCmdType(ContainerProtos.Type.PutSmallFile); request.setContainerID(blockID.getContainerID()); request.setPutSmallFile(smallFileRequest); @@ -238,8 +238,7 @@ public static ContainerCommandRequestProto getReadSmallFileRequest( ContainerCommandRequestProto getKey = getBlockRequest(pipeline, putKey); smallFileRequest.setBlock(getKey.getGetBlock()); - Builder request = - ContainerCommandRequestProto.newBuilder(); + Builder request = getContainerCommandRequestProtoBuilder(); request.setCmdType(ContainerProtos.Type.GetSmallFile); request.setContainerID(getKey.getGetBlock().getBlockID().getContainerID()); request.setGetSmallFile(smallFileRequest); @@ -272,8 +271,7 @@ public static Builder newReadChunkRequestBuilder(Pipeline pipeline, readRequest.setChunkData(writeChunk.getChunkData()); readRequest.setReadChunkVersion(ContainerProtos.ReadChunkVersion.V1); - Builder newRequest = - ContainerCommandRequestProto.newBuilder(); + Builder newRequest = getContainerCommandRequestProtoBuilder(); newRequest.setCmdType(ContainerProtos.Type.ReadChunk); newRequest.setContainerID(readRequest.getBlockID().getContainerID()); newRequest.setReadChunk(readRequest); @@ -295,8 +293,7 @@ public static ContainerCommandRequestProto getCreateContainerRequest( private static Builder getContainerCommandRequestBuilder(long containerID, Pipeline pipeline) throws IOException { - Builder request = - ContainerCommandRequestProto.newBuilder(); + Builder request = getContainerCommandRequestProtoBuilder(); request.setCmdType(ContainerProtos.Type.CreateContainer); request.setContainerID(containerID); request.setCreateContainer( @@ -342,8 +339,7 @@ public static ContainerCommandRequestProto getUpdateContainerRequest( Pipeline pipeline = MockPipeline.createSingleNodePipeline(); - Builder request = - ContainerCommandRequestProto.newBuilder(); + Builder request = getContainerCommandRequestProtoBuilder(); request.setCmdType(ContainerProtos.Type.UpdateContainer); request.setContainerID(containerID); request.setUpdateContainer(updateRequestBuilder.build()); @@ -399,8 +395,7 @@ public static Builder newPutBlockRequestBuilder(Pipeline pipeline, blockData.setBlockCommitSequenceId(0); putRequest.setBlockData(blockData.getProtoBufMessage()); - Builder request = - ContainerCommandRequestProto.newBuilder(); + Builder request = getContainerCommandRequestProtoBuilder(); request.setCmdType(ContainerProtos.Type.PutBlock); request.setContainerID(blockData.getContainerID()); request.setPutBlock(putRequest); @@ -430,8 +425,7 @@ public static Builder newGetBlockRequestBuilder( ContainerProtos.GetBlockRequestProto.newBuilder(); getRequest.setBlockID(blockID); - Builder request = - ContainerCommandRequestProto.newBuilder(); + Builder request = getContainerCommandRequestProtoBuilder(); request.setCmdType(ContainerProtos.Type.GetBlock); request.setContainerID(blockID.getContainerID()); request.setGetBlock(getRequest); @@ -459,7 +453,7 @@ public static Builder newGetCommittedBlockLengthBuilder(Pipeline pipeline, ContainerProtos.GetCommittedBlockLengthRequestProto.newBuilder() .setBlockID(blockID); - return ContainerCommandRequestProto.newBuilder() + return getContainerCommandRequestProtoBuilder() .setCmdType(ContainerProtos.Type.GetCommittedBlockLength) .setContainerID(blockID.getContainerID()) .setDatanodeUuid(pipeline.getFirstNode().getUuidString()) @@ -475,7 +469,7 @@ public static Builder newGetCommittedBlockLengthBuilder(Pipeline pipeline, */ public static ContainerCommandRequestProto getCloseContainer( Pipeline pipeline, long containerID, Token token) throws IOException { - Builder builder = ContainerCommandRequestProto.newBuilder() + Builder builder = getContainerCommandRequestProtoBuilder() .setCmdType(ContainerProtos.Type.CloseContainer) .setContainerID(containerID) .setCloseContainer( @@ -506,7 +500,7 @@ public static ContainerCommandRequestProto getDeleteContainer( ContainerProtos.DeleteContainerRequestProto deleteRequest = ContainerProtos.DeleteContainerRequestProto.newBuilder(). setForceDelete(forceDelete).build(); - return ContainerCommandRequestProto.newBuilder() + return getContainerCommandRequestProtoBuilder() .setCmdType(ContainerProtos.Type.DeleteContainer) .setContainerID(containerID) .setDeleteContainer( @@ -555,8 +549,7 @@ public static ContainerCommandRequestProto getDummyCommandRequestProto( */ public static ContainerCommandRequestProto getDummyCommandRequestProto( ContainerProtos.Type cmdType, int replicaIndex) { - final Builder builder = - ContainerCommandRequestProto.newBuilder() + final Builder builder = getContainerCommandRequestProtoBuilder() .setCmdType(cmdType) .setContainerID(DUMMY_CONTAINER_ID) .setDatanodeUuid(DATANODE_UUID); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java index e494243ccc1c..f140846b79ba 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java @@ -79,6 +79,7 @@ import static org.apache.hadoop.ozone.audit.AuditLogger.PerformanceStringBuilder; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.malformedRequest; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.unsupportedRequest; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult; /** @@ -479,7 +480,7 @@ ContainerCommandResponseProto createContainer( } ContainerCommandRequestProto.Builder requestBuilder = - ContainerCommandRequestProto.newBuilder() + getContainerCommandRequestProtoBuilder() .setCmdType(Type.CreateContainer) .setContainerID(containerRequest.getContainerID()) .setCreateContainer(createRequest.build()) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java index 8533f7384d41..b3c4f745bfa4 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java @@ -48,6 +48,8 @@ import java.io.IOException; import java.util.concurrent.atomic.AtomicLong; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; + /** * Handler for close container command received from SCM. */ @@ -164,8 +166,7 @@ public void handle(SCMCommand command, OzoneContainer ozoneContainer, private ContainerCommandRequestProto getContainerCommandRequestProto( final DatanodeDetails datanodeDetails, final long containerId, final String encodedToken) { - final ContainerCommandRequestProto.Builder command = - ContainerCommandRequestProto.newBuilder(); + final ContainerCommandRequestProto.Builder command = getContainerCommandRequestProtoBuilder(); command.setCmdType(ContainerProtos.Type.CloseContainer); command.setTraceID(TracingUtil.exportCurrentSpan()); command.setContainerID(containerId); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java index 6351c746fe5c..ffb4d505ff8b 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java @@ -100,6 +100,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; + /** * A {@link StateMachine} for containers, * which is responsible for handling different types of container requests. @@ -387,7 +389,7 @@ public TransactionContext startTransaction(LogEntryProto entry, RaftPeerRole rol final ContainerCommandRequestProto requestProto; if (logProto.getCmdType() == Type.WriteChunk) { // combine state machine data - requestProto = ContainerCommandRequestProto.newBuilder(logProto) + requestProto = getContainerCommandRequestProtoBuilder(logProto) .setWriteChunk(WriteChunkRequestProto.newBuilder(logProto.getWriteChunk()) .setData(stateMachineLogEntry.getStateMachineEntry().getStateMachineData())) .build(); @@ -424,7 +426,7 @@ public TransactionContext startTransaction(RaftClientRequest request) } // once the token is verified, clear it from the proto - final ContainerCommandRequestProto.Builder protoBuilder = ContainerCommandRequestProto.newBuilder(proto) + final ContainerCommandRequestProto.Builder protoBuilder = getContainerCommandRequestProtoBuilder(proto) .clearEncodedToken(); if (proto.getCmdType() == Type.WriteChunk) { final WriteChunkRequestProto write = proto.getWriteChunk(); @@ -451,7 +453,7 @@ private static ContainerCommandRequestProto getContainerCommandRequestProto( // TODO: We can avoid creating new builder and set pipeline Id if // the client is already sending the pipeline id, then we just have to // validate the pipeline Id. - return ContainerCommandRequestProto.newBuilder( + return getContainerCommandRequestProtoBuilder( ContainerCommandRequestProto.parseFrom(request)) .setPipelineID(id.getUuid().toString()).build(); } @@ -726,7 +728,7 @@ private ByteString readStateMachineData( .setChunkData(chunkInfo) .setReadChunkVersion(ContainerProtos.ReadChunkVersion.V1); ContainerCommandRequestProto dataContainerCommandProto = - ContainerCommandRequestProto.newBuilder(requestProto) + getContainerCommandRequestProtoBuilder(requestProto) .setCmdType(Type.ReadChunk).setReadChunk(readChunkRequestProto) .build(); final DispatcherContext context = DispatcherContext diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java index e575a93de270..ae56da92ab80 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java @@ -116,6 +116,7 @@ import static org.apache.hadoop.hdds.scm.utils.ClientCommandsUtils.getReadChunkVersion; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos .ContainerDataProto.State.RECOVERING; +import static org.apache.hadoop.ozone.ClientVersion.ERASURE_CODING_READ_CHUNK_CORRUPTION_FIX; import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult; import org.apache.ratis.statemachine.StateMachine; @@ -588,8 +589,9 @@ ContainerCommandResponseProto handleGetBlock( try { BlockID blockID = BlockID.getFromProtobuf( request.getGetBlock().getBlockID()); - responseData = blockManager.getBlock(kvContainer, blockID) - .getProtoBufMessage(); + responseData = blockManager.getBlock(kvContainer, blockID, + request.hasVersion() && request.getVersion() >= + ERASURE_CODING_READ_CHUNK_CORRUPTION_FIX.toProtoValue()).getProtoBufMessage(); final long numBytes = responseData.getSerializedSize(); metrics.incContainerBytesStats(Type.GetBlock, numBytes); @@ -691,7 +693,6 @@ ContainerCommandResponseProto handleDeleteBlock( ContainerCommandResponseProto handleReadChunk( ContainerCommandRequestProto request, KeyValueContainer kvContainer, DispatcherContext dispatcherContext) { - if (!request.hasReadChunk()) { if (LOG.isDebugEnabled()) { LOG.debug("Malformed Read Chunk request. trace ID: {}", @@ -699,7 +700,6 @@ ContainerCommandResponseProto handleReadChunk( } return malformedRequest(request); } - ChunkBuffer data; try { BlockID blockID = BlockID.getFromProtobuf( @@ -707,8 +707,11 @@ ContainerCommandResponseProto handleReadChunk( ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(request.getReadChunk() .getChunkData()); Preconditions.checkNotNull(chunkInfo); - + if (request.hasVersion() && request.getVersion() >= ERASURE_CODING_READ_CHUNK_CORRUPTION_FIX.toProtoValue()) { + BlockUtils.verifyReplicaIdx(kvContainer, blockID); + } BlockUtils.verifyBCSId(kvContainer, blockID); + if (dispatcherContext == null) { dispatcherContext = DispatcherContext.getHandleReadChunk(); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java index 376285c4c72a..a38fe2569bf5 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java @@ -28,12 +28,12 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.container.common.helpers.BlockData; +import org.apache.hadoop.ozone.container.common.interfaces.Container; import org.apache.hadoop.ozone.container.common.interfaces.DBHandle; import org.apache.hadoop.ozone.container.common.utils.ContainerCache; import org.apache.hadoop.ozone.container.common.utils.DatanodeStoreCache; import org.apache.hadoop.ozone.container.common.utils.RawDB; import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB; -import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer; import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData; import com.google.common.base.Preconditions; @@ -42,6 +42,7 @@ import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaThreeImpl; import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaTwoImpl; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_NOT_FOUND; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.EXPORT_CONTAINER_METADATA_FAILED; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.IMPORT_CONTAINER_METADATA_FAILED; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_BLOCK; @@ -220,7 +221,7 @@ public static BlockData getBlockData(byte[] bytes) throws IOException { * @param blockID requested block info * @throws IOException if cannot support block's blockCommitSequenceId */ - public static void verifyBCSId(KeyValueContainer container, BlockID blockID) + public static void verifyBCSId(Container container, BlockID blockID) throws IOException { long bcsId = blockID.getBlockCommitSequenceId(); Preconditions.checkNotNull(blockID, @@ -237,6 +238,24 @@ public static void verifyBCSId(KeyValueContainer container, BlockID blockID) } } + /** + * Verify if request block BCSID is supported. + * + * @param container container object. + * @param blockID requested block info + * @throws IOException if cannot support block's blockCommitSequenceId + */ + public static void verifyReplicaIdx(Container container, BlockID blockID) + throws IOException { + int containerReplicaIndex = container.getContainerData().getReplicaIndex(); + if (containerReplicaIndex > 0 && containerReplicaIndex != blockID.getReplicaIndex()) { + throw new StorageContainerException( + "Unable to find the Container with replicaIdx " + blockID.getReplicaIndex() + ". Container " + + container.getContainerData().getContainerID() + " replicaIdx is " + + containerReplicaIndex + ".", CONTAINER_NOT_FOUND); + } + } + /** * Remove container KV metadata from per-disk db store. * @param containerData diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java index ea2f8d08057b..3a2ac7a49319 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java @@ -38,9 +38,7 @@ import com.google.common.base.Preconditions; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.BCSID_MISMATCH; -import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_NOT_FOUND; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.NO_SUCH_BLOCK; -import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNKNOWN_BCSID; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -209,32 +207,15 @@ public static long persistPutBlock(KeyValueContainer container, } @Override - public BlockData getBlock(Container container, BlockID blockID) + public BlockData getBlock(Container container, BlockID blockID, boolean isReplicaCheckRequired) throws IOException { - long bcsId = blockID.getBlockCommitSequenceId(); - Preconditions.checkNotNull(blockID, - "BlockID cannot be null in GetBlock request"); - Preconditions.checkNotNull(container, - "Container cannot be null"); - + if (isReplicaCheckRequired) { + BlockUtils.verifyReplicaIdx(container, blockID); + } + BlockUtils.verifyBCSId(container, blockID); KeyValueContainerData containerData = (KeyValueContainerData) container .getContainerData(); - int containerReplicaIndex = containerData.getReplicaIndex(); - if (containerReplicaIndex != blockID.getReplicaIndex()) { - throw new StorageContainerException( - "Unable to find the Container with replicaIdx " + blockID.getReplicaIndex() + ". Container " - + containerData.getContainerID() + " replicaIdx is " - + containerReplicaIndex + ".", CONTAINER_NOT_FOUND); - } - - long containerBCSId = containerData.getBlockCommitSequenceId(); - if (containerBCSId < bcsId) { - throw new StorageContainerException( - "Unable to find the block with bcsID " + bcsId + ". Container " - + containerData.getContainerID() + " bcsId is " - + containerBCSId + ".", UNKNOWN_BCSID); - } - + long bcsId = blockID.getBlockCommitSequenceId(); try (DBHandle db = BlockUtils.getDB(containerData, config)) { // This is a post condition that acts as a hint to the user. // Should never fail. diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java index 02b7e93d50f4..6ba50d7d4492 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java @@ -55,12 +55,25 @@ long putBlock(Container container, BlockData data, boolean endOfBlock) * * @param container - Container from which block needs to be fetched. * @param blockID - BlockID of the Block. + * @param isReplicaCheckRequired - if true will match container data's replica index with provided BlockID. * @return Block Data. * @throws IOException when BcsId is unknown or mismatched */ - BlockData getBlock(Container container, BlockID blockID) + BlockData getBlock(Container container, BlockID blockID, boolean isReplicaCheckRequired) throws IOException; + /** + * Gets an existing block. + * + * @param container - Container from which block needs to be fetched. + * @param blockID - BlockID of the Block. + * @return Block Data. + * @throws IOException when BcsId is unknown or mismatched + */ + default BlockData getBlock(Container container, BlockID blockID) throws IOException { + return getBlock(container, blockID, false); + } + /** * Deletes an existing block. * diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java index 1cbd6ee4706d..222fa810ea3c 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java @@ -87,6 +87,7 @@ import static org.apache.hadoop.hdds.fs.MockSpaceUsagePersistence.inMemory; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getContainerCommandResponse; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; import static org.apache.hadoop.ozone.container.common.ContainerTestUtils.COMMIT_STAGE; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -584,8 +585,7 @@ private ContainerCommandRequestProto getWriteChunkRequest( .setChunkData(chunk) .setData(data); - return ContainerCommandRequestProto - .newBuilder() + return getContainerCommandRequestProtoBuilder() .setContainerID(containerId) .setCmdType(ContainerProtos.Type.WriteChunk) .setDatanodeUuid(datanodeId) @@ -686,7 +686,7 @@ private ContainerCommandRequestProto getReadChunkRequest( .setBlockID(writeChunk.getBlockID()) .setChunkData(writeChunk.getChunkData()) .setReadChunkVersion(ContainerProtos.ReadChunkVersion.V1); - return ContainerCommandRequestProto.newBuilder() + return getContainerCommandRequestProtoBuilder() .setCmdType(ContainerProtos.Type.ReadChunk) .setContainerID(writeChunk.getBlockID().getContainerID()) .setTraceID(writeChunkRequest.getTraceID()) diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index b9c8feae16ce..f2f09824e313 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -57,6 +57,7 @@ import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DATANODE_VOLUME_CHOOSING_POLICY; import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_METADATA_DIRS; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -121,8 +122,7 @@ public void setup() throws StorageContainerException { public void testHandlerCommandHandling() throws Exception { reset(handler); // Test Create Container Request handling - ContainerCommandRequestProto createContainerRequest = - ContainerProtos.ContainerCommandRequestProto.newBuilder() + ContainerCommandRequestProto createContainerRequest = getContainerCommandRequestProtoBuilder() .setCmdType(ContainerProtos.Type.CreateContainer) .setContainerID(DUMMY_CONTAINER_ID) .setDatanodeUuid(DATANODE_UUID) @@ -296,7 +296,7 @@ public void testVolumeSetInKeyValueHandler() throws Exception { private ContainerCommandRequestProto getDummyCommandRequestProto( ContainerProtos.Type cmdType) { - return ContainerCommandRequestProto.newBuilder() + return getContainerCommandRequestProtoBuilder() .setCmdType(cmdType) .setContainerID(DUMMY_CONTAINER_ID) .setDatanodeUuid(DATANODE_UUID) @@ -317,7 +317,7 @@ public void testCloseInvalidContainer(ContainerLayoutVersion layoutVersion) // Create Close container request ContainerCommandRequestProto closeContainerRequest = - ContainerProtos.ContainerCommandRequestProto.newBuilder() + getContainerCommandRequestProtoBuilder() .setCmdType(ContainerProtos.Type.CloseContainer) .setContainerID(DUMMY_CONTAINER_ID) .setDatanodeUuid(DATANODE_UUID) @@ -435,7 +435,7 @@ public void testDeleteContainer() throws IOException { private static ContainerCommandRequestProto createContainerRequest( String datanodeId, long containerID) { - return ContainerCommandRequestProto.newBuilder() + return getContainerCommandRequestProtoBuilder() .setCmdType(ContainerProtos.Type.CreateContainer) .setDatanodeUuid(datanodeId).setCreateContainer( ContainerProtos.CreateContainerRequestProto.newBuilder() diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java index 9c4209e1a471..05b71184bac9 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java @@ -139,6 +139,20 @@ public void testReadChunk() { assertEquals(UNKNOWN_BCSID, response.getResult()); } + @ParameterizedTest + @ValueSource(ints = {0, 1, 2, 3, 4, 5}) + public void testReadChunkWithReplicaIndexMismatch(int replicaIndex) { + KeyValueContainer container = getMockContainerWithReplicaIndex(replicaIndex); + KeyValueHandler handler = getDummyHandler(); + for (int rid = 0; rid <= 5; rid++) { + ContainerProtos.ContainerCommandResponseProto response = + handler.handleReadChunk(getDummyCommandRequestProto(ContainerProtos.Type.ReadChunk, rid), container, null); + assertEquals(rid != replicaIndex ? ContainerProtos.Result.CONTAINER_NOT_FOUND : UNKNOWN_BCSID, + response.getResult()); + } + + } + @Test public void testGetSmallFile() { KeyValueContainer container = getMockUnhealthyContainer(); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestKeyValueStreamDataChannel.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestKeyValueStreamDataChannel.java index c97040d1b376..ee977f53b3ca 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestKeyValueStreamDataChannel.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestKeyValueStreamDataChannel.java @@ -58,6 +58,7 @@ import static org.apache.hadoop.hdds.scm.storage.BlockDataStreamOutput.PUT_BLOCK_REQUEST_LENGTH_MAX; import static org.apache.hadoop.hdds.scm.storage.BlockDataStreamOutput.executePutBlockClose; import static org.apache.hadoop.hdds.scm.storage.BlockDataStreamOutput.getProtoLength; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; import static org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel.closeBuffers; import static org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel.readPutBlockRequest; import static org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel.writeBuffers; @@ -71,7 +72,7 @@ public class TestKeyValueStreamDataChannel { LoggerFactory.getLogger(TestKeyValueStreamDataChannel.class); static final ContainerCommandRequestProto PUT_BLOCK_PROTO - = ContainerCommandRequestProto.newBuilder() + = getContainerCommandRequestProtoBuilder() .setCmdType(Type.PutBlock) .setPutBlock(PutBlockRequestProto.newBuilder().setBlockData( BlockData.newBuilder().setBlockID(DatanodeBlockID.newBuilder() diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java index 241754a57f19..660c38979762 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java @@ -248,8 +248,7 @@ Collection getFailedServers() { @VisibleForTesting Pipeline createSingleECBlockPipeline(Pipeline ecPipeline, DatanodeDetails node, int replicaIndex) { - Map indiciesForSinglePipeline = new HashMap<>(); - indiciesForSinglePipeline.put(node, replicaIndex); + Map indiciesForSinglePipeline = Collections.singletonMap(node, replicaIndex); return Pipeline.newBuilder() .setId(ecPipeline.getId()) .setReplicationConfig(ecPipeline.getReplicationConfig()) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerReplicationEndToEnd.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerReplicationEndToEnd.java index 78a4e78647eb..cd34d94d2569 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerReplicationEndToEnd.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerReplicationEndToEnd.java @@ -67,6 +67,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertSame; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; /** * Tests delete key operation with a slow follower in the datanode @@ -194,8 +195,7 @@ public void testContainerReplication() throws Exception { Thread.sleep(2 * containerReportInterval); DatanodeDetails oldReplicaNode = pipeline.getFirstNode(); // now move the container to the closed on the datanode. - ContainerProtos.ContainerCommandRequestProto.Builder request = - ContainerProtos.ContainerCommandRequestProto.newBuilder(); + ContainerProtos.ContainerCommandRequestProto.Builder request = getContainerCommandRequestProtoBuilder(); request.setDatanodeUuid(pipeline.getFirstNode().getUuidString()); request.setCmdType(ContainerProtos.Type.CloseContainer); request.setContainerID(containerID); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java index b6eaca8e80d0..de2b351c4885 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java @@ -104,13 +104,12 @@ import org.apache.ratis.protocol.exceptions.StateMachineException; import org.apache.ratis.server.storage.FileInfo; import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage; - import org.apache.ratis.statemachine.impl.StatemachineImplTestUtil; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; - +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; /** * Tests the containerStateMachine failure handling. */ @@ -391,8 +390,7 @@ public void testUnhealthyContainer() throws Exception { .getDatanodeStateMachine().getContainer(); HddsDispatcher dispatcher = (HddsDispatcher) ozoneContainer .getDispatcher(); - ContainerProtos.ContainerCommandRequestProto.Builder request = - ContainerProtos.ContainerCommandRequestProto.newBuilder(); + ContainerProtos.ContainerCommandRequestProto.Builder request = getContainerCommandRequestProtoBuilder(); request.setCmdType(ContainerProtos.Type.CloseContainer); request.setContainerID(containerID); request.setCloseContainer( @@ -451,8 +449,7 @@ public void testApplyTransactionFailure() throws Exception { .getContainerWithPipeline(containerID).getPipeline(); XceiverClientSpi xceiverClient = xceiverClientManager.acquireClient(pipeline); - ContainerProtos.ContainerCommandRequestProto.Builder request = - ContainerProtos.ContainerCommandRequestProto.newBuilder(); + ContainerProtos.ContainerCommandRequestProto.Builder request = getContainerCommandRequestProtoBuilder(); request.setDatanodeUuid(pipeline.getFirstNode().getUuidString()); request.setCmdType(ContainerProtos.Type.CloseContainer); request.setContainerID(containerID); @@ -536,8 +533,7 @@ void testApplyTransactionIdempotencyWithClosedContainer() .getContainerWithPipeline(containerID).getPipeline(); XceiverClientSpi xceiverClient = xceiverClientManager.acquireClient(pipeline); - ContainerProtos.ContainerCommandRequestProto.Builder request = - ContainerProtos.ContainerCommandRequestProto.newBuilder(); + ContainerProtos.ContainerCommandRequestProto.Builder request = getContainerCommandRequestProtoBuilder(); request.setDatanodeUuid(pipeline.getFirstNode().getUuidString()); request.setCmdType(ContainerProtos.Type.CloseContainer); request.setContainerID(containerID); @@ -628,8 +624,7 @@ void testWriteStateMachineDataIdempotencyWithClosedContainer() AtomicInteger failCount = new AtomicInteger(0); Runnable r1 = () -> { try { - ContainerProtos.ContainerCommandRequestProto.Builder request = - ContainerProtos.ContainerCommandRequestProto.newBuilder(); + ContainerProtos.ContainerCommandRequestProto.Builder request = getContainerCommandRequestProtoBuilder(); request.setDatanodeUuid(pipeline.getFirstNode().getUuidString()); request.setCmdType(ContainerProtos.Type.CloseContainer); request.setContainerID(containerID); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestDeleteWithInAdequateDN.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestDeleteWithInAdequateDN.java index d4ff85736273..2d3b7881d585 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestDeleteWithInAdequateDN.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestDeleteWithInAdequateDN.java @@ -79,6 +79,7 @@ import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; /** * Tests delete key operation with inadequate datanodes. @@ -249,8 +250,7 @@ void testDeleteKeyWithInAdequateDN() throws Exception { key.close(); // now move the container to the closed on the datanode. - ContainerProtos.ContainerCommandRequestProto.Builder request = - ContainerProtos.ContainerCommandRequestProto.newBuilder(); + ContainerProtos.ContainerCommandRequestProto.Builder request = getContainerCommandRequestProtoBuilder(); request.setDatanodeUuid(pipeline.getFirstNode().getUuidString()); request.setCmdType(ContainerProtos.Type.CloseContainer); request.setContainerID(containerID); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java index f184acf68958..6dc98c5bf28f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java @@ -31,6 +31,7 @@ import static org.apache.ozone.test.GenericTestUtils.setLogLevel; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.mockito.ArgumentMatchers.anyList; import static org.mockito.Mockito.any; import java.io.IOException; @@ -44,7 +45,9 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -55,6 +58,7 @@ import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.scm.PlacementPolicy; import org.apache.hadoop.hdds.scm.container.ContainerReplica; import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicyFactory; @@ -66,6 +70,7 @@ import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRandom; import org.apache.hadoop.hdds.scm.net.NetworkTopology; import org.apache.hadoop.hdds.scm.node.NodeManager; +import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; import org.apache.hadoop.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.client.ObjectStore; @@ -88,6 +93,8 @@ import org.junit.jupiter.params.provider.MethodSource; import org.mockito.MockedStatic; import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import org.slf4j.event.Level; /** @@ -250,14 +257,16 @@ private OzoneInputStream createInputStream(OzoneClient client) throws IOExceptio public void testECContainerReplication() throws Exception { OzoneConfiguration conf = createConfiguration(false); final AtomicReference mockedDatanodeToRemove = new AtomicReference<>(); - + final Map failedReadChunkCountMap = new ConcurrentHashMap<>(); // Overiding Config to support 1k Chunk size conf.set("ozone.replication.allowed-configs", "(^((STANDALONE|RATIS)/(ONE|THREE))|(EC/(3-2|6-3|10-4)-" + "(512|1024|2048|4096|1)k)$)"); conf.set(OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_KEY, SCMContainerPlacementRackScatter.class.getCanonicalName()); - try (MockedStatic mocked = - Mockito.mockStatic(ContainerPlacementPolicyFactory.class, Mockito.CALLS_REAL_METHODS)) { - mocked.when(() -> ContainerPlacementPolicyFactory.getECPolicy(any(ConfigurationSource.class), + try (MockedStatic mockedPlacementFactory = + Mockito.mockStatic(ContainerPlacementPolicyFactory.class, Mockito.CALLS_REAL_METHODS); + MockedStatic mockedContainerProtocolCalls = + Mockito.mockStatic(ContainerProtocolCalls.class, Mockito.CALLS_REAL_METHODS);) { + mockedPlacementFactory.when(() -> ContainerPlacementPolicyFactory.getECPolicy(any(ConfigurationSource.class), any(NodeManager.class), any(NetworkTopology.class), Mockito.anyBoolean(), any(SCMContainerPlacementMetrics.class))).thenAnswer(i -> { PlacementPolicy placementPolicy = (PlacementPolicy) Mockito.spy(i.callRealMethod()); @@ -269,7 +278,17 @@ public void testECContainerReplication() throws Exception { }).when(placementPolicy).replicasToRemoveToFixOverreplication(Mockito.anySet(), Mockito.anyInt()); return placementPolicy; }); - + mockedContainerProtocolCalls.when(() -> ContainerProtocolCalls.readChunk(any(), any(), any(), anyList(), any())) + .thenAnswer(invocation -> { + int replicaIndex = ((ContainerProtos.DatanodeBlockID)invocation.getArgument(2)).getReplicaIndex(); + try { + return invocation.callRealMethod(); + } catch (Throwable e) { + failedReadChunkCountMap.compute(replicaIndex, + (replicaIdx, totalCount) -> totalCount == null ? 1 : (totalCount+1)); + throw e; + } + }); // Creating Cluster with 6 Nodes try (MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(6).build()) { cluster.waitForClusterToBeReady(); @@ -308,14 +327,15 @@ public void testECContainerReplication() throws Exception { byte[] readData = new byte[size]; try (OzoneInputStream inputStream = createInputStream(client)) { inputStream.read(readData); - Assertions.assertTrue(Arrays.equals(readData, originalData)); + Assertions.assertArrayEquals(readData, originalData); } - + Assertions.assertEquals(0, failedReadChunkCountMap.size()); //Opening a new stream before we make changes to the blocks. try (OzoneInputStream inputStream = createInputStream(client)) { int firstReadLen = 1024 * 3; Arrays.fill(readData, (byte)0); inputStream.read(readData, 0, firstReadLen); + Assertions.assertEquals(0, failedReadChunkCountMap.size()); //Checking the initial state as per the latest location. assertState(cluster, ImmutableMap.of(1, replicaIndexMap.get(1), 2, replicaIndexMap.get(2), 3, replicaIndexMap.get(3), 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); @@ -385,7 +405,8 @@ public void testECContainerReplication() throws Exception { // Reading the pre initialized inputStream. This leads to swap of the block1 & block3. inputStream.read(readData, firstReadLen, size - firstReadLen); - Assertions.assertTrue(Arrays.equals(readData, originalData)); + Assertions.assertEquals(ImmutableMap.of(1, 1, 3, 1), failedReadChunkCountMap); + Assertions.assertArrayEquals(readData, originalData); } } } diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeBlockPutter.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeBlockPutter.java index f6a5c5965018..8a6aac621245 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeBlockPutter.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeBlockPutter.java @@ -43,6 +43,8 @@ import picocli.CommandLine.Command; import picocli.CommandLine.Option; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; + /** * Datanode test for block creation. */ @@ -146,8 +148,7 @@ private void putBlock(long stepNo) throws Exception { String id = client.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder builder = - ContainerCommandRequestProto - .newBuilder() + getContainerCommandRequestProtoBuilder() .setCmdType(Type.PutBlock) .setContainerID(blockId.getContainerID()) .setDatanodeUuid(id) diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkGenerator.java index 6362f32d0499..4d252522f8b8 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkGenerator.java @@ -52,6 +52,8 @@ import picocli.CommandLine.Command; import picocli.CommandLine.Option; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; + /** * Data generator to use pure datanode XCeiver interface. */ @@ -234,8 +236,7 @@ private void sendWriteChunkRequest(DatanodeBlockID blockId, String id = datanodeDetails.getUuidString(); ContainerCommandRequestProto.Builder builder = - ContainerCommandRequestProto - .newBuilder() + getContainerCommandRequestProtoBuilder() .setCmdType(Type.WriteChunk) .setContainerID(blockId.getContainerID()) .setDatanodeUuid(id) diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java index 2bbf8b6d5b24..d5f2955d30ad 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java @@ -39,6 +39,8 @@ import picocli.CommandLine.Command; import picocli.CommandLine.Option; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; + /** * Data validator of chunks to use pure datanode XCeiver interface. */ @@ -179,12 +181,11 @@ private ContainerCommandRequestProto createReadChunkRequest(long stepNo) String id = xceiverClient.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder builder = - ContainerCommandRequestProto - .newBuilder() - .setCmdType(ContainerProtos.Type.ReadChunk) - .setContainerID(blockId.getContainerID()) - .setDatanodeUuid(id) - .setReadChunk(readChunkRequest); + getContainerCommandRequestProtoBuilder() + .setCmdType(ContainerProtos.Type.ReadChunk) + .setContainerID(blockId.getContainerID()) + .setDatanodeUuid(id) + .setReadChunk(readChunkRequest); return builder.build(); } diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/FollowerAppendLogEntryGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/FollowerAppendLogEntryGenerator.java index d76081fd29ea..e0146852c734 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/FollowerAppendLogEntryGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/FollowerAppendLogEntryGenerator.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumData; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; @@ -73,6 +72,8 @@ import picocli.CommandLine.Command; import picocli.CommandLine.Option; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; + /** * Freon test to test one single datanode with a fake leader (this test). *

@@ -254,7 +255,7 @@ private AppendEntriesRequestProto createAppendLogEntry(long sequence, long blockId = chunkId / 1000; long containerId = blockId / 1000; //ozone specific - ByteString payload = ContainerCommandRequestProto.newBuilder() + ByteString payload = getContainerCommandRequestProtoBuilder() .setContainerID(containerId) .setCmdType(Type.WriteChunk) .setDatanodeUuid(serverId) diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/LeaderAppendLogEntryGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/LeaderAppendLogEntryGenerator.java index 40f670260326..c21df661cf98 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/LeaderAppendLogEntryGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/LeaderAppendLogEntryGenerator.java @@ -68,6 +68,8 @@ import picocli.CommandLine.Command; import picocli.CommandLine.Option; +import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; + /** * Test isolated LEADER datanodes. *

@@ -204,7 +206,7 @@ private XceiverClientRatis createXceiverClient(OzoneConfiguration conf) { private ContainerCommandRequestProto createContainerRequest( long containerId) { - return ContainerCommandRequestProto.newBuilder() + return getContainerCommandRequestProtoBuilder() .setContainerID(containerId) .setCmdType(Type.CreateContainer) .setDatanodeUuid(serverId) @@ -219,7 +221,7 @@ private ContainerCommandRequestProto createChunkWriteRequest(long containerId, long chunkId) { long blockId = getPrefix().hashCode() + chunkId / 1000; - return ContainerCommandRequestProto.newBuilder() + return getContainerCommandRequestProtoBuilder() .setContainerID(containerId) .setCmdType(Type.WriteChunk) .setDatanodeUuid(serverId) From 2bb1b9d5cd91f2dad6689dc9c3231d4821db7fa9 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Fri, 7 Jun 2024 07:48:43 -0700 Subject: [PATCH 04/21] HDDS-10983. Remove stacktrace Change-Id: Ifd60cb93ccb5e6ff7c81995734459719f6d5ecc3 --- .../src/main/java/org/apache/hadoop/hdds/client/BlockID.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java index e59a412cdb60..a06bf08971fa 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java @@ -36,7 +36,6 @@ public class BlockID { // This value would be only set when deserializing from ContainerProtos.DatanodeBlockID or copying from another // BlockID object. private Integer replicaIndex; - private StackTraceElement[] stackTraceElements; public BlockID(long containerID, long localID) { this(containerID, localID, 0, null); @@ -46,7 +45,6 @@ private BlockID(long containerID, long localID, long bcsID, Integer repIndex) { containerBlockID = new ContainerBlockID(containerID, localID); blockCommitSequenceId = bcsID; this.replicaIndex = repIndex; - this.stackTraceElements = Thread.currentThread().getStackTrace(); } public BlockID(BlockID blockID) { @@ -62,7 +60,6 @@ private BlockID(ContainerBlockID containerBlockID, long bcsId, Integer repIndex) this.containerBlockID = containerBlockID; blockCommitSequenceId = bcsId; this.replicaIndex = repIndex; - this.stackTraceElements = Thread.currentThread().getStackTrace(); } public long getContainerID() { @@ -105,7 +102,6 @@ public void appendTo(StringBuilder sb) { containerBlockID.appendTo(sb); sb.append(" bcsId: ").append(blockCommitSequenceId); sb.append(" replicaIndex: ").append(replicaIndex); - sb.append(Arrays.stream(stackTraceElements).map(StackTraceElement::toString).collect(Collectors.joining("\n"))); } @JsonIgnore From 5886cf426a841d74b4caf2b5e3c233780005b037 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Fri, 7 Jun 2024 07:58:33 -0700 Subject: [PATCH 05/21] HDDS-10983. Fix checkstyle Change-Id: I8f6e02e917dbc4d99294ca3c6d61109b1dafb897 --- .../apache/hadoop/hdds/client/BlockID.java | 2 - .../client/io/ECBlockOutputStreamEntry.java | 1 - .../container/TestContainerReplication.java | 63 ++++++++++--------- 3 files changed, 35 insertions(+), 31 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java index a06bf08971fa..8f94b9931f4e 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java @@ -20,9 +20,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import java.util.Arrays; import java.util.Objects; -import java.util.stream.Collectors; /** * BlockID of Ozone (containerID + localID + blockCommitSequenceId + replicaIndex). diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java index 660c38979762..2cf2ab0cf9c2 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java @@ -37,7 +37,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java index 6dc98c5bf28f..74aad64f1a7f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java @@ -47,7 +47,6 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -93,8 +92,6 @@ import org.junit.jupiter.params.provider.MethodSource; import org.mockito.MockedStatic; import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import org.slf4j.event.Level; /** @@ -252,12 +249,43 @@ private OzoneInputStream createInputStream(OzoneClient client) throws IOExceptio return bucket.readKey(KEY); } + private void mockContainerPlacementPolicy(final MockedStatic mockedPlacementFactory, + final AtomicReference mockedDatanodeToRemove) { + mockedPlacementFactory.when(() -> ContainerPlacementPolicyFactory.getECPolicy(any(ConfigurationSource.class), + any(NodeManager.class), any(NetworkTopology.class), Mockito.anyBoolean(), + any(SCMContainerPlacementMetrics.class))).thenAnswer(i -> { + PlacementPolicy placementPolicy = (PlacementPolicy) Mockito.spy(i.callRealMethod()); + Mockito.doAnswer(args -> { + Set containerReplica = ((Set) args.getArgument(0)).stream() + .filter(r -> r.getDatanodeDetails().equals(mockedDatanodeToRemove.get())) + .collect(Collectors.toSet()); + return containerReplica; + }).when(placementPolicy).replicasToRemoveToFixOverreplication(Mockito.anySet(), Mockito.anyInt()); + return placementPolicy; + }); + } + + private void mockContainerProtocolCalls(final MockedStatic mockedContainerProtocolCalls, + final Map failedReadChunkCountMap) { + mockedContainerProtocolCalls.when(() -> ContainerProtocolCalls.readChunk(any(), any(), any(), anyList(), any())) + .thenAnswer(invocation -> { + int replicaIndex = ((ContainerProtos.DatanodeBlockID) invocation.getArgument(2)).getReplicaIndex(); + try { + return invocation.callRealMethod(); + } catch (Throwable e) { + failedReadChunkCountMap.compute(replicaIndex, + (replicaIdx, totalCount) -> totalCount == null ? 1 : (totalCount + 1)); + throw e; + } + }); + } + @Test public void testECContainerReplication() throws Exception { OzoneConfiguration conf = createConfiguration(false); final AtomicReference mockedDatanodeToRemove = new AtomicReference<>(); - final Map failedReadChunkCountMap = new ConcurrentHashMap<>(); + final Map failedReadChunkCountMap = new ConcurrentHashMap<>(); // Overiding Config to support 1k Chunk size conf.set("ozone.replication.allowed-configs", "(^((STANDALONE|RATIS)/(ONE|THREE))|(EC/(3-2|6-3|10-4)-" + "(512|1024|2048|4096|1)k)$)"); @@ -266,29 +294,8 @@ public void testECContainerReplication() throws Exception { Mockito.mockStatic(ContainerPlacementPolicyFactory.class, Mockito.CALLS_REAL_METHODS); MockedStatic mockedContainerProtocolCalls = Mockito.mockStatic(ContainerProtocolCalls.class, Mockito.CALLS_REAL_METHODS);) { - mockedPlacementFactory.when(() -> ContainerPlacementPolicyFactory.getECPolicy(any(ConfigurationSource.class), - any(NodeManager.class), any(NetworkTopology.class), Mockito.anyBoolean(), - any(SCMContainerPlacementMetrics.class))).thenAnswer(i -> { - PlacementPolicy placementPolicy = (PlacementPolicy) Mockito.spy(i.callRealMethod()); - Mockito.doAnswer(args -> { - Set containerReplica = ((Set) args.getArgument(0)).stream() - .filter(r -> r.getDatanodeDetails().equals(mockedDatanodeToRemove.get())) - .collect(Collectors.toSet()); - return containerReplica; - }).when(placementPolicy).replicasToRemoveToFixOverreplication(Mockito.anySet(), Mockito.anyInt()); - return placementPolicy; - }); - mockedContainerProtocolCalls.when(() -> ContainerProtocolCalls.readChunk(any(), any(), any(), anyList(), any())) - .thenAnswer(invocation -> { - int replicaIndex = ((ContainerProtos.DatanodeBlockID)invocation.getArgument(2)).getReplicaIndex(); - try { - return invocation.callRealMethod(); - } catch (Throwable e) { - failedReadChunkCountMap.compute(replicaIndex, - (replicaIdx, totalCount) -> totalCount == null ? 1 : (totalCount+1)); - throw e; - } - }); + mockContainerPlacementPolicy(mockedPlacementFactory, mockedDatanodeToRemove); + mockContainerProtocolCalls(mockedContainerProtocolCalls, failedReadChunkCountMap); // Creating Cluster with 6 Nodes try (MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(6).build()) { cluster.waitForClusterToBeReady(); @@ -333,7 +340,7 @@ public void testECContainerReplication() throws Exception { //Opening a new stream before we make changes to the blocks. try (OzoneInputStream inputStream = createInputStream(client)) { int firstReadLen = 1024 * 3; - Arrays.fill(readData, (byte)0); + Arrays.fill(readData, (byte) 0); inputStream.read(readData, 0, firstReadLen); Assertions.assertEquals(0, failedReadChunkCountMap.size()); //Checking the initial state as per the latest location. From cfc7415ca058bab414d1537770a406aa87e7d8e2 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Fri, 7 Jun 2024 11:41:04 -0700 Subject: [PATCH 06/21] HDDS-10983. Address review comments Change-Id: If2af2959ab79ee0ffd275c21909c9dc86e1c2c38 --- .../hadoop/hdds/scm/storage/ChunkInputStream.java | 10 +++++----- .../java/org/apache/hadoop/hdds/client/BlockID.java | 6 +----- .../ozone/container/keyvalue/helpers/BlockUtils.java | 4 ++-- .../ozone/container/TestContainerReplication.java | 3 ++- 4 files changed, 10 insertions(+), 13 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java index 172887e56417..f8ea7b81f821 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java @@ -296,8 +296,8 @@ protected synchronized void releaseClient() { /** * Updates DatanodeBlockId which based on blockId. */ - private void updateDatanodeBlockId() throws IOException { - DatanodeDetails closestNode = pipelineSupplier.get().getClosestNode(); + private void updateDatanodeBlockId(Pipeline pipeline) throws IOException { + DatanodeDetails closestNode = pipeline.getClosestNode(); int replicaIdx = pipelineSupplier.get().getReplicaIndex(closestNode); ContainerProtos.DatanodeBlockID.Builder builder = blockID.getDatanodeBlockIDProtobufBuilder(); if (replicaIdx > 0) { @@ -311,9 +311,9 @@ private void updateDatanodeBlockId() throws IOException { */ protected synchronized void acquireClient() throws IOException { if (xceiverClientFactory != null && xceiverClient == null) { - xceiverClient = xceiverClientFactory.acquireClientForReadData( - pipelineSupplier.get()); - updateDatanodeBlockId(); + Pipeline pipeline = pipelineSupplier.get(); + xceiverClient = xceiverClientFactory.acquireClientForReadData(pipeline); + updateDatanodeBlockId(pipeline); } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java index 8f94b9931f4e..a777cdecb974 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java @@ -33,7 +33,7 @@ public class BlockID { // null value when not set with private constructor.(This is to avoid confusion of replica index 0 & null value). // This value would be only set when deserializing from ContainerProtos.DatanodeBlockID or copying from another // BlockID object. - private Integer replicaIndex; + private final Integer replicaIndex; public BlockID(long containerID, long localID) { this(containerID, localID, 0, null); @@ -81,10 +81,6 @@ public Integer getReplicaIndex() { return replicaIndex; } - public void setReplicaIndex(Integer replicaIndex) { - this.replicaIndex = replicaIndex; - } - public ContainerBlockID getContainerBlockID() { return containerBlockID; } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java index a38fe2569bf5..acfb3489a724 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java @@ -239,11 +239,11 @@ public static void verifyBCSId(Container container, BlockID blockID) } /** - * Verify if request block BCSID is supported. + * Verify if request's replicaIndex matches with containerData. * * @param container container object. * @param blockID requested block info - * @throws IOException if cannot support block's blockCommitSequenceId + * @throws IOException if replicaIndex mismatches. */ public static void verifyReplicaIdx(Container container, BlockID blockID) throws IOException { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java index 74aad64f1a7f..ff3e757f0c20 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java @@ -200,7 +200,8 @@ private byte[] createTestData(OzoneClient client, int size) throws IOException { try (OutputStream out = bucket.createKey(KEY, 0, new ECReplicationConfig("RS-3-2-1k"), new HashMap<>())) { byte[] b = new byte[size]; - new Random().nextBytes(b); + Random random = new Random(); + random.nextBytes(b); out.write(b); return b; } From 8edc907ccd1c447a430598c5f2ee42802d6df635 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Fri, 7 Jun 2024 15:34:55 -0700 Subject: [PATCH 07/21] HDDS-10983. Fix testcase Change-Id: I44de245342a7622d1c3fd4b49dd1db4c2bcac5b4 --- .../apache/hadoop/hdds/scm/XceiverClientGrpc.java | 2 +- .../hadoop/hdds/scm/storage/ChunkInputStream.java | 4 ++-- .../hdds/scm/storage/ContainerProtocolCalls.java | 13 ++++--------- .../hadoop/ozone/container/ContainerTestHelper.java | 9 ++++++++- .../container/keyvalue/helpers/BlockUtils.java | 4 ++-- .../ozone/client/checksum/ECFileChecksumHelper.java | 9 ++++++--- .../checksum/ReplicatedFileChecksumHelper.java | 12 +++++++----- .../hdds/scm/storage/TestContainerCommandsEC.java | 9 +++++---- 8 files changed, 35 insertions(+), 27 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 39a69b35d5dc..38172e99a2ad 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -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; } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java index f8ea7b81f821..8b5f7934e66e 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java @@ -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() diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 2026e78e137d..092c050740a0 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -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 token) - throws IOException { + public static ContainerProtos.GetCommittedBlockLengthResponseProto getCommittedBlockLength( + XceiverClientSpi xceiverClient, BlockID blockID, Token token) throws IOException { ContainerProtos.GetCommittedBlockLengthRequestProto.Builder getBlockLengthRequestBuilder = ContainerProtos.GetCommittedBlockLengthRequestProto.newBuilder(). @@ -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 validators, - Token token) throws IOException { + public static ContainerProtos.ReadChunkResponseProto readChunk(XceiverClientSpi xceiverClient, ChunkInfo chunk, + DatanodeBlockID blockID, List validators, Token token) throws IOException { ReadChunkRequestProto.Builder readChunkRequest = ReadChunkRequestProto.newBuilder() .setBlockID(blockID) diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java index 0ab75818dab0..ee084a15e94b 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java @@ -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; @@ -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() { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java index acfb3489a724..7773b54f7942 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/BlockUtils.java @@ -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 " diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ECFileChecksumHelper.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ECFileChecksumHelper.java index 3da23688418f..e178e8061bc2 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ECFileChecksumHelper.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ECFileChecksumHelper.java @@ -189,10 +189,13 @@ private List 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 { diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedFileChecksumHelper.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedFileChecksumHelper.java index 09f9c7d037e9..177ed6699715 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedFileChecksumHelper.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedFileChecksumHelper.java @@ -149,13 +149,15 @@ protected List 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 { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java index c8ffe5d011ac..8842cf34aa1f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java @@ -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. @@ -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()); @@ -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( From 1476d596119dada3fbc1ab1ea247d3bf54cc24bd Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Fri, 7 Jun 2024 19:26:54 -0700 Subject: [PATCH 08/21] HDDS-10983. Add client version test Change-Id: I9a8905dad8f4db731b110644fa03e9e80e1df859 --- .../ozone/container/ContainerTestHelper.java | 7 ++-- ...KeyValueHandlerWithUnhealthyContainer.java | 34 ++++++++++++++----- 2 files changed, 29 insertions(+), 12 deletions(-) diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java index ee084a15e94b..f4272f506049 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.utils.UniqueId; +import org.apache.hadoop.ozone.ClientVersion; import org.apache.hadoop.ozone.common.Checksum; import org.apache.hadoop.ozone.common.ChunkBuffer; import org.apache.hadoop.ozone.common.OzoneChecksumException; @@ -543,7 +544,7 @@ public static byte[] generateData(int length, boolean random) { public static ContainerCommandRequestProto getDummyCommandRequestProto( ContainerProtos.Type cmdType) { - return getDummyCommandRequestProto(cmdType, 0); + return getDummyCommandRequestProto(ClientVersion.CURRENT, cmdType, 0); } /** @@ -555,8 +556,8 @@ public static ContainerCommandRequestProto getDummyCommandRequestProto( * @return */ public static ContainerCommandRequestProto getDummyCommandRequestProto( - ContainerProtos.Type cmdType, int replicaIndex) { - final Builder builder = getContainerCommandRequestProtoBuilder() + ClientVersion clientVersion, ContainerProtos.Type cmdType, int replicaIndex) { + final Builder builder = getContainerCommandRequestProtoBuilder(clientVersion.toProtoValue()) .setCmdType(cmdType) .setContainerID(DUMMY_CONTAINER_ID) .setDatanodeUuid(DATANODE_UUID); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java index 05b71184bac9..b353a4d0fe83 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; +import org.apache.hadoop.ozone.ClientVersion; import org.apache.hadoop.ozone.container.common.ContainerTestUtils; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; @@ -35,12 +36,16 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.Arrays; import java.util.UUID; +import java.util.stream.IntStream; +import java.util.stream.Stream; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.CONTAINER_INTERNAL_ERROR; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.SUCCESS; @@ -97,17 +102,25 @@ public void testGetBlock() { assertEquals(UNKNOWN_BCSID, response.getResult()); } + private static Stream getAllClientVersions() { + return Arrays.stream(ClientVersion.values()).flatMap(client -> IntStream.range(0, 6) + .mapToObj(rid -> Arguments.of(client, rid))); + } + + @ParameterizedTest - @ValueSource(ints = {0, 1, 2, 3, 4, 5}) - public void testGetBlockWithReplicaIndexMismatch(int replicaIndex) { + @MethodSource("getAllClientVersions") + public void testGetBlockWithReplicaIndexMismatch(ClientVersion clientVersion, int replicaIndex) { KeyValueContainer container = getMockContainerWithReplicaIndex(replicaIndex); KeyValueHandler handler = getDummyHandler(); for (int rid = 0; rid <= 5; rid++) { ContainerProtos.ContainerCommandResponseProto response = handler.handleGetBlock( - getDummyCommandRequestProto(ContainerProtos.Type.GetBlock, rid), + getDummyCommandRequestProto(clientVersion, ContainerProtos.Type.GetBlock, rid), container); - assertEquals(rid != replicaIndex ? ContainerProtos.Result.CONTAINER_NOT_FOUND : UNKNOWN_BCSID, + assertEquals((replicaIndex > 0 && rid != replicaIndex && clientVersion.toProtoValue() >= + ClientVersion.ERASURE_CODING_READ_CHUNK_CORRUPTION_FIX.toProtoValue()) ? + ContainerProtos.Result.CONTAINER_NOT_FOUND : UNKNOWN_BCSID, response.getResult()); } @@ -140,14 +153,17 @@ public void testReadChunk() { } @ParameterizedTest - @ValueSource(ints = {0, 1, 2, 3, 4, 5}) - public void testReadChunkWithReplicaIndexMismatch(int replicaIndex) { + @MethodSource("getAllClientVersions") + public void testReadChunkWithReplicaIndexMismatch(ClientVersion clientVersion, int replicaIndex) { KeyValueContainer container = getMockContainerWithReplicaIndex(replicaIndex); KeyValueHandler handler = getDummyHandler(); for (int rid = 0; rid <= 5; rid++) { ContainerProtos.ContainerCommandResponseProto response = - handler.handleReadChunk(getDummyCommandRequestProto(ContainerProtos.Type.ReadChunk, rid), container, null); - assertEquals(rid != replicaIndex ? ContainerProtos.Result.CONTAINER_NOT_FOUND : UNKNOWN_BCSID, + handler.handleReadChunk(getDummyCommandRequestProto(clientVersion, ContainerProtos.Type.ReadChunk, rid), + container, null); + assertEquals((replicaIndex > 0 && rid != replicaIndex && + clientVersion.toProtoValue() >= ClientVersion.ERASURE_CODING_READ_CHUNK_CORRUPTION_FIX.toProtoValue()) ? + ContainerProtos.Result.CONTAINER_NOT_FOUND : UNKNOWN_BCSID, response.getResult()); } From 437594d16b12b9587308cc348505913d166540ee Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Sat, 8 Jun 2024 17:16:01 -0700 Subject: [PATCH 09/21] HDDS-10983. Fix test cases Change-Id: I822685eb14a347cd4dfa256a5c14d58ccad6b12e --- .../hdds/scm/storage/BlockInputStream.java | 32 ++++++++----------- .../client/io/BlockInputStreamFactory.java | 3 +- .../io/BlockInputStreamFactoryImpl.java | 3 +- .../ozone/client/io/ECBlockInputStream.java | 2 +- .../scm/storage/DummyBlockInputStream.java | 2 +- .../DummyBlockInputStreamWithRetry.java | 2 +- .../scm/storage/TestBlockInputStream.java | 2 +- .../io/TestBlockInputStreamFactoryImpl.java | 5 +-- .../apache/hadoop/hdds/client/BlockID.java | 6 ++-- .../hadoop/hdds/scm/pipeline/Pipeline.java | 5 ++- .../scm/storage/ContainerProtocolCalls.java | 32 ++++++++----------- .../ozone/container/ContainerTestHelper.java | 10 +++--- .../common/helpers/TestBlockData.java | 2 +- .../client/checksum/ECFileChecksumHelper.java | 10 ++---- .../ReplicatedFileChecksumHelper.java | 8 +---- .../ozone/client/io/KeyInputStream.java | 8 ++--- .../hdds/scm/TestXceiverClientGrpc.java | 4 +-- .../hadoop/ozone/om/TestChunkStreams.java | 5 +-- 18 files changed, 65 insertions(+), 76 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java index c5487832826a..908cd0ae9ab3 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java @@ -24,9 +24,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; +import java.util.stream.Collectors; import com.google.common.base.Preconditions; import org.apache.hadoop.hdds.client.BlockID; @@ -116,7 +118,7 @@ public BlockInputStream(BlockID blockId, long blockLen, Pipeline pipeline, Token token, XceiverClientFactory xceiverClientFactory, Function refreshFunction, - OzoneClientConfig config) { + OzoneClientConfig config) throws IOException { this.blockID = blockId; this.length = blockLen; setPipeline(pipeline); @@ -133,7 +135,7 @@ public BlockInputStream(BlockID blockId, long blockLen, Pipeline pipeline, Token token, XceiverClientFactory xceiverClientFactory, OzoneClientConfig config - ) { + ) throws IOException { this(blockId, blockLen, pipeline, token, xceiverClientFactory, null, config); } @@ -244,33 +246,27 @@ protected List getChunkInfoList() throws IOException { @VisibleForTesting protected List getChunkInfoListUsingClient() throws IOException { - final Pipeline pipeline = xceiverClient.getPipeline(); - if (LOG.isDebugEnabled()) { - LOG.debug("Initializing BlockInputStream for get key to access {}", - blockID.getContainerID()); - } - - DatanodeBlockID.Builder blkIDBuilder = - DatanodeBlockID.newBuilder().setContainerID(blockID.getContainerID()) - .setLocalID(blockID.getLocalID()) - .setBlockCommitSequenceId(blockID.getBlockCommitSequenceId()); - - int replicaIndex = pipeline.getReplicaIndex(pipeline.getClosestNode()); - if (replicaIndex > 0) { - blkIDBuilder.setReplicaIndex(replicaIndex); + LOG.debug("Initializing BlockInputStream for get key to access {} with pipeline {}.", + blockID.getContainerID(), xceiverClient.getPipeline()); } GetBlockResponseProto response = ContainerProtocolCalls.getBlock( - xceiverClient, VALIDATORS, blkIDBuilder.build(), tokenRef.get()); + xceiverClient, VALIDATORS, blockID, tokenRef.get()); return response.getBlockData().getChunksList(); } - private void setPipeline(Pipeline pipeline) { + private void setPipeline(Pipeline pipeline) throws IOException { if (pipeline == null) { return; } + Set replicaIndexes = + pipeline.getNodes().stream().map(pipeline::getReplicaIndex).collect(Collectors.toSet()); + if (replicaIndexes.size() != 1) { + throw new IOException(String.format("Pipeline: %s has nodes containing different replica indexes.", + pipeline.toString())); + } // irrespective of the container state, we will always read via Standalone // protocol. diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java index 6f8a744f762d..d347dee85121 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactory.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; import org.apache.hadoop.security.token.Token; +import java.io.IOException; import java.util.function.Function; /** @@ -52,6 +53,6 @@ BlockExtendedInputStream create(ReplicationConfig repConfig, Token token, XceiverClientFactory xceiverFactory, Function refreshFunction, - OzoneClientConfig config); + OzoneClientConfig config) throws IOException; } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java index 6bcdc3c48114..19ea76fa7064 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockInputStreamFactoryImpl.java @@ -32,6 +32,7 @@ import org.apache.hadoop.io.ElasticByteBufferPool; import org.apache.hadoop.security.token.Token; +import java.io.IOException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.function.Function; @@ -80,7 +81,7 @@ public BlockExtendedInputStream create(ReplicationConfig repConfig, Token token, XceiverClientFactory xceiverFactory, Function refreshFunction, - OzoneClientConfig config) { + OzoneClientConfig config) throws IOException { if (repConfig.getReplicationType().equals(HddsProtos.ReplicationType.EC)) { return new ECBlockInputStreamProxy((ECReplicationConfig)repConfig, blockInfo, xceiverFactory, refreshFunction, diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java index f2cbbf0863cb..6342de2c3381 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockInputStream.java @@ -164,7 +164,7 @@ protected int currentStreamIndex() { * stream if it has not been opened already. * @return BlockInput stream to read from. */ - protected BlockExtendedInputStream getOrOpenStream(int locationIndex) { + protected BlockExtendedInputStream getOrOpenStream(int locationIndex) throws IOException { BlockExtendedInputStream stream = blockStreams[locationIndex]; if (stream == null) { // To read an EC block, we create a STANDALONE pipeline that contains the diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStream.java index a89097533d2a..46ce89da0f8a 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStream.java @@ -49,7 +49,7 @@ class DummyBlockInputStream extends BlockInputStream { Function refreshFunction, List chunkList, Map chunks, - OzoneClientConfig config) { + OzoneClientConfig config) throws IOException { super(blockId, blockLen, pipeline, token, xceiverClientManager, refreshFunction, config); this.chunkDataMap = chunks; diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java index 6d12614228f9..6ab31f8c3723 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java @@ -56,7 +56,7 @@ final class DummyBlockInputStreamWithRetry List chunkList, Map chunkMap, AtomicBoolean isRerfreshed, IOException ioException, - OzoneClientConfig config) { + OzoneClientConfig config) throws IOException { super(blockId, blockLen, pipeline, token, xceiverClientManager, blockID -> { isRerfreshed.set(true); diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java index 21b088ce85fd..3aabe5f67ad4 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java @@ -355,7 +355,7 @@ private static ChunkInputStream throwingChunkInputStream(IOException ex, } private BlockInputStream createSubject(BlockID blockID, Pipeline pipeline, - ChunkInputStream stream) { + ChunkInputStream stream) throws IOException { OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); clientConfig.setChecksumVerify(false); return new DummyBlockInputStream(blockID, blockSize, pipeline, null, diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java index 623f7a4f86f1..6ea6dc67ca60 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; import org.junit.jupiter.api.Test; +import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.Map; @@ -48,7 +49,7 @@ public class TestBlockInputStreamFactoryImpl { private OzoneConfiguration conf = new OzoneConfiguration(); @Test - public void testNonECGivesBlockInputStream() { + public void testNonECGivesBlockInputStream() throws IOException { BlockInputStreamFactory factory = new BlockInputStreamFactoryImpl(); ReplicationConfig repConfig = RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE); @@ -68,7 +69,7 @@ public void testNonECGivesBlockInputStream() { } @Test - public void testECGivesECBlockInputStream() { + public void testECGivesECBlockInputStream() throws IOException { BlockInputStreamFactory factory = new BlockInputStreamFactoryImpl(); ReplicationConfig repConfig = new ECReplicationConfig(3, 2); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java index a777cdecb974..e3e3c3fa9ec1 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java @@ -146,9 +146,9 @@ public boolean equals(Object o) { return false; } BlockID blockID = (BlockID) o; - return containerBlockID.equals(blockID.getContainerBlockID()) - && blockCommitSequenceId == blockID.getBlockCommitSequenceId() - && Objects.equals(replicaIndex, blockID.getReplicaIndex()); + return this.getContainerBlockID().equals(blockID.getContainerBlockID()) + && this.getBlockCommitSequenceId() == blockID.getBlockCommitSequenceId() + && Objects.equals(this.getReplicaIndex(), blockID.getReplicaIndex()); } @Override diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java index 6ea92f74c193..d7ffcaf337d1 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java @@ -509,7 +509,10 @@ public String toString() { new StringBuilder(getClass().getSimpleName()).append("["); b.append(" Id: ").append(id.getId()); b.append(", Nodes: "); - nodeStatus.keySet().forEach(b::append); + for (DatanodeDetails datanodeDetails : nodeStatus.keySet()) { + b.append(datanodeDetails); + b.append(" ReplicaIndex: ").append(this.getReplicaIndex(datanodeDetails)); + } b.append(", ReplicationConfig: ").append(replicationConfig); b.append(", State:").append(getPipelineState()); b.append(", leaderId:").append(leaderId != null ? leaderId.toString() : ""); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 092c050740a0..4f73d588f5c2 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -206,52 +206,48 @@ static T tryEachDatanode(Pipeline pipeline, * * @param xceiverClient client to perform call * @param validators functions to validate the response - * @param datanodeBlockID blockID to identify container + * @param blockID blockID to identify container * @param token a token for this block (may be null) * @return container protocol get block response * @throws IOException if there is an I/O error while performing the call */ public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, - List validators, - DatanodeBlockID datanodeBlockID, - Token token) throws IOException { - GetBlockRequestProto.Builder readBlockRequest = GetBlockRequestProto - .newBuilder() - .setBlockID(datanodeBlockID); + List validators, BlockID blockID, Token token) throws IOException { ContainerCommandRequestProto.Builder builder = getContainerCommandRequestProtoBuilder() .setCmdType(Type.GetBlock) - .setContainerID(datanodeBlockID.getContainerID()) - .setGetBlock(readBlockRequest); + .setContainerID(blockID.getContainerID()); if (token != null) { builder.setEncodedToken(token.encodeToUrlString()); } return tryEachDatanode(xceiverClient.getPipeline(), - d -> getBlock(xceiverClient, validators, builder, d), - d -> toErrorMessage(datanodeBlockID, d)); + d -> getBlock(xceiverClient, validators, builder, blockID, d), + d -> toErrorMessage(blockID, d)); } - static String toErrorMessage(DatanodeBlockID blockId, DatanodeDetails d) { + static String toErrorMessage(BlockID blockId, DatanodeDetails d) { return String.format("Failed to get block #%s in container #%s from %s", blockId.getLocalID(), blockId.getContainerID(), d); } public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, - DatanodeBlockID datanodeBlockID, + BlockID datanodeBlockID, Token token) throws IOException { return getBlock(xceiverClient, getValidatorList(), datanodeBlockID, token); } - private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, - List validators, - ContainerCommandRequestProto.Builder builder, - DatanodeDetails datanode) throws IOException { + private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, List validators, + ContainerCommandRequestProto.Builder builder, BlockID blockID, DatanodeDetails datanode) throws IOException { String traceId = TracingUtil.exportCurrentSpan(); if (traceId != null) { builder.setTraceID(traceId); } + final DatanodeBlockID datanodeBlockID = blockID.getDatanodeBlockIDProtobufBuilder() + .setReplicaIndex(xceiverClient.getPipeline().getReplicaIndex(datanode)).build(); + final GetBlockRequestProto.Builder readBlockRequest = GetBlockRequestProto.newBuilder().setBlockID(datanodeBlockID); final ContainerCommandRequestProto request = builder - .setDatanodeUuid(datanode.getUuidString()).build(); + .setDatanodeUuid(datanode.getUuidString()) + .setGetBlock(readBlockRequest).build(); ContainerCommandResponseProto response = xceiverClient.sendCommand(request, validators); return response.getGetBlock(); diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java index f4272f506049..2fa48fde8844 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java @@ -47,7 +47,6 @@ 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; @@ -517,9 +516,12 @@ public static BlockID getTestBlockID(long containerID) { } 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; + DatanodeBlockID.Builder datanodeBlockID = DatanodeBlockID.newBuilder().setContainerID(containerID) + .setLocalID(UniqueId.next()); + if (replicaIndex != null) { + datanodeBlockID.setReplicaIndex(replicaIndex); + } + return BlockID.getFromProtobuf(datanodeBlockID.build()); } public static long getTestContainerID() { diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestBlockData.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestBlockData.java index 5aa4f0d9bf57..11cae324226e 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestBlockData.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/helpers/TestBlockData.java @@ -134,7 +134,7 @@ public void testToString() { final BlockID blockID = new BlockID(5, 123); blockID.setBlockCommitSequenceId(42); final BlockData subject = new BlockData(blockID); - assertEquals("[blockId=conID: 5 locID: 123 bcsId: 42, size=0]", + assertEquals("[blockId=conID: 5 locID: 123 bcsId: 42 replicaIndex: null, size=0]", subject.toString()); } } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ECFileChecksumHelper.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ECFileChecksumHelper.java index e178e8061bc2..7243e4baacf2 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ECFileChecksumHelper.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ECFileChecksumHelper.java @@ -186,16 +186,10 @@ private List getChunkInfos(OmKeyLocationInfo LOG.debug("Initializing BlockInputStream for get key to access {}", blockID.getContainerID()); } - xceiverClientSpi = - getXceiverClientFactory().acquireClientForReadData(pipeline); + xceiverClientSpi = getXceiverClientFactory().acquireClientForReadData(pipeline); - 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.build(), token); + .getBlock(xceiverClientSpi, blockID, token); chunks = response.getBlockData().getChunksList(); } finally { diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedFileChecksumHelper.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedFileChecksumHelper.java index 177ed6699715..652b57040e6c 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedFileChecksumHelper.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedFileChecksumHelper.java @@ -150,14 +150,8 @@ protected List getChunkInfos( blockID.getContainerID()); } xceiverClientSpi = getXceiverClientFactory().acquireClientForReadData(pipeline); - - 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.build(), token); + .getBlock(xceiverClientSpi, blockID, token); chunks = response.getBlockData().getChunksList(); } finally { diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java index 2d40841ee499..76fa1e394f6c 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java @@ -61,7 +61,7 @@ private static List createStreams( XceiverClientFactory xceiverClientFactory, Function retryFunction, BlockInputStreamFactory blockStreamFactory, - OzoneClientConfig config) { + OzoneClientConfig config) throws IOException { List partStreams = new ArrayList<>(); for (OmKeyLocationInfo omKeyLocationInfo : blockInfos) { if (LOG.isDebugEnabled()) { @@ -121,7 +121,7 @@ private static LengthInputStream getFromOmKeyInfo( Function retryFunction, BlockInputStreamFactory blockStreamFactory, List locationInfos, - OzoneClientConfig config) { + OzoneClientConfig config) throws IOException { List streams = createStreams(keyInfo, locationInfos, xceiverClientFactory, retryFunction, blockStreamFactory, config); @@ -137,7 +137,7 @@ public static LengthInputStream getFromOmKeyInfo(OmKeyInfo keyInfo, XceiverClientFactory xceiverClientFactory, Function retryFunction, BlockInputStreamFactory blockStreamFactory, - OzoneClientConfig config) { + OzoneClientConfig config) throws IOException { List keyLocationInfos = keyInfo .getLatestVersionLocations().getBlocksLatestVersionOnly(); @@ -150,7 +150,7 @@ public static List getStreamsFromKeyInfo(OmKeyInfo keyInfo, XceiverClientFactory xceiverClientFactory, Function retryFunction, BlockInputStreamFactory blockStreamFactory, - OzoneClientConfig config) { + OzoneClientConfig config) throws IOException { List keyLocationInfos = keyInfo .getLatestVersionLocations().getBlocksLatestVersionOnly(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientGrpc.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientGrpc.java index bab979618efd..fd74aba837b3 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientGrpc.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientGrpc.java @@ -238,11 +238,11 @@ public XceiverClientReply sendCommandAsync( private void invokeXceiverClientGetBlock(XceiverClientSpi client) throws IOException { ContainerProtocolCalls.getBlock(client, - ContainerProtos.DatanodeBlockID.newBuilder() + BlockID.getFromProtobuf(ContainerProtos.DatanodeBlockID.newBuilder() .setContainerID(1) .setLocalID(1) .setBlockCommitSequenceId(1) - .build(), null); + .build()), null); } private void invokeXceiverClientReadChunk(XceiverClientSpi client) diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestChunkStreams.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestChunkStreams.java index e0d5ef4084da..9e2077593ce5 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestChunkStreams.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestChunkStreams.java @@ -25,6 +25,7 @@ import org.junit.jupiter.api.Test; import java.io.ByteArrayInputStream; +import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -81,7 +82,7 @@ public void testErrorReadGroupInputStream() throws Exception { } @Nonnull - private List createInputStreams(String dataString) { + private List createInputStreams(String dataString) throws IOException { byte[] buf = dataString.getBytes(UTF_8); List streams = new ArrayList<>(); int offset = 0; @@ -93,7 +94,7 @@ private List createInputStreams(String dataString) { return streams; } - private BlockInputStream createStream(byte[] buf, int offset) { + private BlockInputStream createStream(byte[] buf, int offset) throws IOException { OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); clientConfig.setChecksumVerify(true); return new BlockInputStream(null, 100, null, null, null, From 65994f34788261c311610e7c3e8ad3c9f460cba0 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Sun, 9 Jun 2024 18:32:03 -0700 Subject: [PATCH 10/21] HDDS-10983. Fix issues Change-Id: If2ca384a0e026617fa8893c5dd306c3470971931 --- .../apache/hadoop/hdds/scm/storage/BlockInputStream.java | 4 ++-- .../ozone/client/io/TestBlockInputStreamFactoryImpl.java | 6 +++++- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java index 908cd0ae9ab3..dc01d7c3c0d8 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java @@ -263,9 +263,9 @@ private void setPipeline(Pipeline pipeline) throws IOException { } Set replicaIndexes = pipeline.getNodes().stream().map(pipeline::getReplicaIndex).collect(Collectors.toSet()); - if (replicaIndexes.size() != 1) { + if (replicaIndexes.size() > 1) { throw new IOException(String.format("Pipeline: %s has nodes containing different replica indexes.", - pipeline.toString())); + pipeline)); } // irrespective of the container state, we will always read via Standalone diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java index 6ea6dc67ca60..dda631372cbe 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/ozone/client/io/TestBlockInputStreamFactoryImpl.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hdds.scm.storage.BlockInputStream; import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; import org.junit.jupiter.api.Test; +import org.mockito.Mockito; import java.io.IOException; import java.util.ArrayList; @@ -40,6 +41,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.mockito.ArgumentMatchers.any; /** * Tests for BlockInputStreamFactoryImpl. @@ -56,7 +58,9 @@ public void testNonECGivesBlockInputStream() throws IOException { BlockLocationInfo blockInfo = createKeyLocationInfo(repConfig, 3, 1024 * 1024 * 10); - + Pipeline pipeline = Mockito.spy(blockInfo.getPipeline()); + blockInfo.setPipeline(pipeline); + Mockito.when(pipeline.getReplicaIndex(any(DatanodeDetails.class))).thenReturn(1); OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); clientConfig.setChecksumVerify(true); BlockExtendedInputStream stream = From 731107c1bedee360fd24aa71a97505051095a0c9 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Sun, 9 Jun 2024 20:14:43 -0700 Subject: [PATCH 11/21] HDDS-10983. Fix testcases Change-Id: I9744461813f5b6729039becb0d02d3934a8908c4 --- .../hdds/scm/storage/ContainerProtocolCalls.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 4f73d588f5c2..1ed47841ec66 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -242,9 +242,13 @@ private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, Li if (traceId != null) { builder.setTraceID(traceId); } - final DatanodeBlockID datanodeBlockID = blockID.getDatanodeBlockIDProtobufBuilder() - .setReplicaIndex(xceiverClient.getPipeline().getReplicaIndex(datanode)).build(); - final GetBlockRequestProto.Builder readBlockRequest = GetBlockRequestProto.newBuilder().setBlockID(datanodeBlockID); + final DatanodeBlockID.Builder datanodeBlockID = blockID.getDatanodeBlockIDProtobufBuilder(); + int replicaIndex = xceiverClient.getPipeline().getReplicaIndex(datanode); + if (replicaIndex > 0) { + datanodeBlockID.setReplicaIndex(replicaIndex); + } + final GetBlockRequestProto.Builder readBlockRequest = GetBlockRequestProto.newBuilder() + .setBlockID(datanodeBlockID.build()); final ContainerCommandRequestProto request = builder .setDatanodeUuid(datanode.getUuidString()) .setGetBlock(readBlockRequest).build(); From 585756789ed487404638ac52de4ee4454a653b03 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Sun, 9 Jun 2024 21:29:57 -0700 Subject: [PATCH 12/21] HDDS-10983. Fix checkstyle Change-Id: Ib28d150af433ab1f298d7892cd3e243c7b509522 --- .../apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 1ed47841ec66..988f10d26567 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -245,7 +245,7 @@ private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, Li final DatanodeBlockID.Builder datanodeBlockID = blockID.getDatanodeBlockIDProtobufBuilder(); int replicaIndex = xceiverClient.getPipeline().getReplicaIndex(datanode); if (replicaIndex > 0) { - datanodeBlockID.setReplicaIndex(replicaIndex); + datanodeBlockID.setReplicaIndex(replicaIndex); } final GetBlockRequestProto.Builder readBlockRequest = GetBlockRequestProto.newBuilder() .setBlockID(datanodeBlockID.build()); From dd11beb03f614de626c3a1c17f905ab5dab5c1d9 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Tue, 11 Jun 2024 16:40:08 -0700 Subject: [PATCH 13/21] HDDS-10983. Fix Acceptance test Change-Id: Id81b80a6c4d6137ac287022cd9386ad29ca70194 --- .../hdds/scm/storage/BlockInputStream.java | 5 +++-- .../hadoop/hdds/scm/pipeline/Pipeline.java | 10 ++++++++++ .../hdds/scm/storage/ContainerProtocolCalls.java | 16 +++++++++------- .../client/checksum/ECFileChecksumHelper.java | 2 +- .../checksum/ReplicatedFileChecksumHelper.java | 2 +- .../hadoop/hdds/scm/TestXceiverClientGrpc.java | 2 +- 6 files changed, 25 insertions(+), 12 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java index dc01d7c3c0d8..9b232f19c5e7 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java @@ -246,13 +246,14 @@ protected List getChunkInfoList() throws IOException { @VisibleForTesting protected List getChunkInfoListUsingClient() throws IOException { + Pipeline pipeline = pipelineRef.get(); if (LOG.isDebugEnabled()) { LOG.debug("Initializing BlockInputStream for get key to access {} with pipeline {}.", - blockID.getContainerID(), xceiverClient.getPipeline()); + blockID.getContainerID(), pipeline); } GetBlockResponseProto response = ContainerProtocolCalls.getBlock( - xceiverClient, VALIDATORS, blockID, tokenRef.get()); + xceiverClient, VALIDATORS, blockID, tokenRef.get(), pipeline.getReplicaIndexes()); return response.getBlockData().getChunksList(); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java index d7ffcaf337d1..1486f05f55c0 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java @@ -31,6 +31,8 @@ import java.util.Optional; import java.util.Set; import java.util.UUID; +import java.util.function.Function; +import java.util.stream.Collectors; import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.common.collect.ImmutableList; @@ -239,6 +241,14 @@ public int getReplicaIndex(DatanodeDetails dn) { return replicaIndexes.getOrDefault(dn, 0); } + /** + * Get the replicaIndex Map. + * @return + */ + public Map getReplicaIndexes() { + return this.getNodes().stream().collect(Collectors.toMap(Function.identity(), this::getReplicaIndex)); + } + /** * Returns the leader if found else defaults to closest node. * diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 988f10d26567..508ebf3c61c9 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -212,7 +212,8 @@ static T tryEachDatanode(Pipeline pipeline, * @throws IOException if there is an I/O error while performing the call */ public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, - List validators, BlockID blockID, Token token) throws IOException { + List validators, BlockID blockID, Token token, + Map replicaIndexes) throws IOException { ContainerCommandRequestProto.Builder builder = getContainerCommandRequestProtoBuilder() .setCmdType(Type.GetBlock) .setContainerID(blockID.getContainerID()); @@ -221,7 +222,7 @@ public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, } return tryEachDatanode(xceiverClient.getPipeline(), - d -> getBlock(xceiverClient, validators, builder, blockID, d), + d -> getBlock(xceiverClient, validators, builder, blockID, d, replicaIndexes), d -> toErrorMessage(blockID, d)); } @@ -231,19 +232,20 @@ static String toErrorMessage(BlockID blockId, DatanodeDetails d) { } public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, - BlockID datanodeBlockID, - Token token) throws IOException { - return getBlock(xceiverClient, getValidatorList(), datanodeBlockID, token); + BlockID datanodeBlockID, Token token, + Map replicaIndexes) throws IOException { + return getBlock(xceiverClient, getValidatorList(), datanodeBlockID, token, replicaIndexes); } private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, List validators, - ContainerCommandRequestProto.Builder builder, BlockID blockID, DatanodeDetails datanode) throws IOException { + ContainerCommandRequestProto.Builder builder, BlockID blockID, DatanodeDetails datanode, + Map replicaIndexes) throws IOException { String traceId = TracingUtil.exportCurrentSpan(); if (traceId != null) { builder.setTraceID(traceId); } final DatanodeBlockID.Builder datanodeBlockID = blockID.getDatanodeBlockIDProtobufBuilder(); - int replicaIndex = xceiverClient.getPipeline().getReplicaIndex(datanode); + int replicaIndex = replicaIndexes.getOrDefault(datanode, 0); if (replicaIndex > 0) { datanodeBlockID.setReplicaIndex(replicaIndex); } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ECFileChecksumHelper.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ECFileChecksumHelper.java index 7243e4baacf2..13ba57169878 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ECFileChecksumHelper.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ECFileChecksumHelper.java @@ -189,7 +189,7 @@ private List getChunkInfos(OmKeyLocationInfo xceiverClientSpi = getXceiverClientFactory().acquireClientForReadData(pipeline); ContainerProtos.GetBlockResponseProto response = ContainerProtocolCalls - .getBlock(xceiverClientSpi, blockID, token); + .getBlock(xceiverClientSpi, blockID, token, pipeline.getReplicaIndexes()); chunks = response.getBlockData().getChunksList(); } finally { diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedFileChecksumHelper.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedFileChecksumHelper.java index 652b57040e6c..016121ce1a9b 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedFileChecksumHelper.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/checksum/ReplicatedFileChecksumHelper.java @@ -151,7 +151,7 @@ protected List getChunkInfos( } xceiverClientSpi = getXceiverClientFactory().acquireClientForReadData(pipeline); ContainerProtos.GetBlockResponseProto response = ContainerProtocolCalls - .getBlock(xceiverClientSpi, blockID, token); + .getBlock(xceiverClientSpi, blockID, token, pipeline.getReplicaIndexes()); chunks = response.getBlockData().getChunksList(); } finally { diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientGrpc.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientGrpc.java index fd74aba837b3..405a92dc33bb 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientGrpc.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientGrpc.java @@ -242,7 +242,7 @@ private void invokeXceiverClientGetBlock(XceiverClientSpi client) .setContainerID(1) .setLocalID(1) .setBlockCommitSequenceId(1) - .build()), null); + .build()), null, client.getPipeline().getReplicaIndexes()); } private void invokeXceiverClientReadChunk(XceiverClientSpi client) From 0d769da26701e4ceb83002ee4a15372c8adc07ca Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Tue, 11 Jun 2024 16:48:14 -0700 Subject: [PATCH 14/21] HDDS-10983. Fix checkstyle Change-Id: Icb4405b0f899b0a83babd1ea49221f4ffdedf2cd --- .../apache/hadoop/ozone/container/TestContainerReplication.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java index 5774709ed7a1..d1af619690e3 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java @@ -65,7 +65,6 @@ import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicyFactory; import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementMetrics; import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRackScatter; -import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementMetrics; import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager.ReplicationManagerConfiguration; import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementCapacity; import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRackAware; From 45279a3bd5aef26b46f4197bb3291bf0a696e357 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 12 Jun 2024 18:07:30 -0700 Subject: [PATCH 15/21] HDDS-10983. change pipeline supplier usage Change-Id: I9fda20fc747f5989cc571ce7d088b5b449a2734f --- .../org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java index 8b5f7934e66e..983bb74989ad 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java @@ -298,7 +298,7 @@ protected synchronized void releaseClient() { */ private void updateDatanodeBlockId(Pipeline pipeline) throws IOException { DatanodeDetails closestNode = pipeline.getClosestNode(); - int replicaIdx = pipelineSupplier.get().getReplicaIndex(closestNode); + int replicaIdx = pipeline.getReplicaIndex(closestNode); ContainerProtos.DatanodeBlockID.Builder builder = blockID.getDatanodeBlockIDProtobufBuilder(); if (replicaIdx > 0) { builder.setReplicaIndex(replicaIdx); From 58b089dc0a3bb5d0fe1cad6c055f77bfabb038ba Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Thu, 13 Jun 2024 17:49:44 -0700 Subject: [PATCH 16/21] HDDS-10983. Address review comments and simplify testcase Change-Id: I371912e557e4770b0bbd0f25e823a186224b79c5 --- .../hadoop/hdds/scm/XceiverClientGrpc.java | 2 +- .../hdds/scm/storage/BlockInputStream.java | 8 +- .../container/TestContainerReplication.java | 125 +++++------------- .../hadoop/ozone/container/TestHelper.java | 2 +- 4 files changed, 41 insertions(+), 96 deletions(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 38172e99a2ad..39a69b35d5dc 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -450,7 +450,7 @@ private XceiverClientReply sendCommandWithRetry( processForDebug(request), pipeline); } else { LOG.error(message + " on the pipeline {}.", - request.getCmdType(), pipeline, new RuntimeException()); + request.getCmdType(), pipeline); } throw ioException; } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java index 9b232f19c5e7..70154228768d 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java @@ -24,11 +24,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; -import java.util.stream.Collectors; import com.google.common.base.Preconditions; import org.apache.hadoop.hdds.client.BlockID; @@ -262,9 +260,9 @@ private void setPipeline(Pipeline pipeline) throws IOException { if (pipeline == null) { return; } - Set replicaIndexes = - pipeline.getNodes().stream().map(pipeline::getReplicaIndex).collect(Collectors.toSet()); - if (replicaIndexes.size() > 1) { + long replicaIndexes = pipeline.getNodes().stream().map(pipeline::getReplicaIndex).distinct().count(); + + if (replicaIndexes > 1) { throw new IOException(String.format("Pipeline: %s has nodes containing different replica indexes.", pipeline)); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java index d1af619690e3..c561dd5cc947 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java @@ -21,6 +21,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Arrays.asList; import static java.util.Collections.emptyMap; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerType.KeyValueContainer; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_KEY; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY; @@ -48,7 +49,6 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import java.util.function.Supplier; @@ -56,21 +56,16 @@ import com.google.common.collect.ImmutableMap; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.RatisReplicationConfig; -import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.scm.PlacementPolicy; -import org.apache.hadoop.hdds.scm.container.ContainerReplica; -import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicyFactory; import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementMetrics; import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRackScatter; import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager.ReplicationManagerConfiguration; import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementCapacity; import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRackAware; import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRandom; -import org.apache.hadoop.hdds.scm.net.NetworkTopology; -import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; import org.apache.hadoop.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.MiniOzoneCluster; @@ -80,6 +75,8 @@ import org.apache.hadoop.ozone.client.OzoneClientFactory; import org.apache.hadoop.ozone.client.OzoneVolume; import org.apache.hadoop.ozone.client.io.OzoneInputStream; +import org.apache.hadoop.ozone.container.common.interfaces.Container; +import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; @@ -263,22 +260,6 @@ private OzoneInputStream createInputStream(OzoneClient client) throws IOExceptio return bucket.readKey(KEY); } - private void mockContainerPlacementPolicy(final MockedStatic mockedPlacementFactory, - final AtomicReference mockedDatanodeToRemove) { - mockedPlacementFactory.when(() -> ContainerPlacementPolicyFactory.getECPolicy(any(ConfigurationSource.class), - any(NodeManager.class), any(NetworkTopology.class), Mockito.anyBoolean(), - any(SCMContainerPlacementMetrics.class))).thenAnswer(i -> { - PlacementPolicy placementPolicy = (PlacementPolicy) Mockito.spy(i.callRealMethod()); - Mockito.doAnswer(args -> { - Set containerReplica = ((Set) args.getArgument(0)).stream() - .filter(r -> r.getDatanodeDetails().equals(mockedDatanodeToRemove.get())) - .collect(Collectors.toSet()); - return containerReplica; - }).when(placementPolicy).replicasToRemoveToFixOverreplication(Mockito.anySet(), Mockito.anyInt()); - return placementPolicy; - }); - } - private void mockContainerProtocolCalls(final MockedStatic mockedContainerProtocolCalls, final Map failedReadChunkCountMap) { mockedContainerProtocolCalls.when(() -> ContainerProtocolCalls.readChunk(any(), any(), any(), anyList(), any())) @@ -295,36 +276,41 @@ private void mockContainerProtocolCalls(final MockedStatic containerData = container.getContainerSet().getContainer(containerId); + if (containerData != null) { + container.getDispatcher().getHandler(KeyValueContainer).deleteContainer(containerData, true); + } + cluster.getHddsDatanode(dn).getDatanodeStateMachine().triggerHeartbeat(); + } + + @Test public void testECContainerReplication() throws Exception { OzoneConfiguration conf = createConfiguration(false); - final AtomicReference mockedDatanodeToRemove = new AtomicReference<>(); final Map failedReadChunkCountMap = new ConcurrentHashMap<>(); // Overiding Config to support 1k Chunk size conf.set("ozone.replication.allowed-configs", "(^((STANDALONE|RATIS)/(ONE|THREE))|(EC/(3-2|6-3|10-4)-" + "(512|1024|2048|4096|1)k)$)"); conf.set(OZONE_SCM_CONTAINER_PLACEMENT_EC_IMPL_KEY, SCMContainerPlacementRackScatter.class.getCanonicalName()); - try (MockedStatic mockedPlacementFactory = - Mockito.mockStatic(ContainerPlacementPolicyFactory.class, Mockito.CALLS_REAL_METHODS); - MockedStatic mockedContainerProtocolCalls = + try (MockedStatic mockedContainerProtocolCalls = Mockito.mockStatic(ContainerProtocolCalls.class, Mockito.CALLS_REAL_METHODS);) { - mockContainerPlacementPolicy(mockedPlacementFactory, mockedDatanodeToRemove); mockContainerProtocolCalls(mockedContainerProtocolCalls, failedReadChunkCountMap); - // Creating Cluster with 6 Nodes - try (MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(6).build()) { + // Creating Cluster with 5 Nodes + try (MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(5).build()) { cluster.waitForClusterToBeReady(); try (OzoneClient client = OzoneClientFactory.getRpcClient(conf)) { Set allNodes = cluster.getHddsDatanodes().stream().map(HddsDatanodeService::getDatanodeDetails).collect( Collectors.toSet()); List initialNodesWithData = new ArrayList<>(); - DatanodeDetails extraNode = null; - // Keeping 5 DNs and stopping the 6th Node here it is kept in the var extraNode + // Keeping 5 DNs and stopping the 6th Node here it is kept in the var extraNodes for (DatanodeDetails dn : allNodes) { if (initialNodesWithData.size() < 5) { initialNodesWithData.add(dn); } else { - extraNode = dn; cluster.shutdownHddsDatanode(dn); } } @@ -355,77 +341,38 @@ public void testECContainerReplication() throws Exception { try (OzoneInputStream inputStream = createInputStream(client)) { int firstReadLen = 1024 * 3; Arrays.fill(readData, (byte) 0); + //Reading first stripe. inputStream.read(readData, 0, firstReadLen); Assertions.assertEquals(0, failedReadChunkCountMap.size()); //Checking the initial state as per the latest location. assertState(cluster, ImmutableMap.of(1, replicaIndexMap.get(1), 2, replicaIndexMap.get(2), 3, replicaIndexMap.get(3), 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); - // Shutting down DN1 and waiting for underreplication - cluster.shutdownHddsDatanode(replicaIndexMap.get(1)); - waitForReplicaCount(containerID, 4, cluster); - assertState(cluster, ImmutableMap.of(2, replicaIndexMap.get(2), 3, replicaIndexMap.get(3), - 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); - - //Starting up ExtraDN. RM should run and create Replica Index 1 to ExtraDN - cluster.restartHddsDatanode(extraNode, false); - waitForReplicaCount(containerID, 5, cluster); - assertState(cluster, ImmutableMap.of(1, extraNode, 2, replicaIndexMap.get(2), - 3, replicaIndexMap.get(3), 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); - - //Stopping RM and starting DN1, this should lead to overreplication of ReplicaIndex 1 - cluster.getStorageContainerManager().getReplicationManager().stop(); - cluster.restartHddsDatanode(replicaIndexMap.get(1), true); - waitForReplicaCount(containerID, 6, cluster); - - //Mocking Overreplication processor to remove replica from DN1. Final Replica1 should be in extraNode - mockedDatanodeToRemove.set(replicaIndexMap.get(1)); - cluster.getStorageContainerManager().getReplicationManager().start(); - waitForReplicaCount(containerID, 5, cluster); - assertState(cluster, ImmutableMap.of(1, extraNode, 2, replicaIndexMap.get(2), - 3, replicaIndexMap.get(3), 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); - - //Stopping DN3 and waiting for underreplication - cluster.getStorageContainerManager().getReplicationManager().stop(); - cluster.shutdownHddsDatanode(replicaIndexMap.get(3)); - waitForReplicaCount(containerID, 4, cluster); - assertState(cluster, ImmutableMap.of(1, extraNode, 2, replicaIndexMap.get(2), - 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); - - //Starting RM, Under replication processor should create Replica 3 in DN1 - cluster.getStorageContainerManager().getReplicationManager().start(); - waitForReplicaCount(containerID, 5, cluster); - assertState(cluster, ImmutableMap.of(1, extraNode, 2, replicaIndexMap.get(2), - 3, replicaIndexMap.get(1), 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); - - //Starting DN3 leading to overreplication of replica 3 + // Stopping replication manager cluster.getStorageContainerManager().getReplicationManager().stop(); - cluster.restartHddsDatanode(replicaIndexMap.get(3), true); - waitForReplicaCount(containerID, 6, cluster); - - //Mocking Overreplication processor to remove data from DN3 leading to Replica3 to stay in DN1. - mockedDatanodeToRemove.set(replicaIndexMap.get(3)); + // Deleting the container from DN1 & DN3 + deleteContainer(cluster, replicaIndexMap.get(1), containerID); + deleteContainer(cluster, replicaIndexMap.get(3), containerID); + // Waiting for replica count of container to come down to 3. + waitForReplicaCount(containerID, 3, cluster); + // Shutting down DN1 + cluster.shutdownHddsDatanode(replicaIndexMap.get(1)); + // Starting replication manager which should process under replication & write replica 1 to DN3. cluster.getStorageContainerManager().getReplicationManager().start(); - waitForReplicaCount(containerID, 5, cluster); - assertState(cluster, ImmutableMap.of(1, extraNode, 2, replicaIndexMap.get(2), - 3, replicaIndexMap.get(1), 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); - - //Stopping Extra DN leading to underreplication of Replica 1 - cluster.getStorageContainerManager().getReplicationManager().stop(); - cluster.shutdownHddsDatanode(extraNode); waitForReplicaCount(containerID, 4, cluster); - assertState(cluster, ImmutableMap.of(2, replicaIndexMap.get(2), 3, replicaIndexMap.get(1), + // Asserting Replica 1 has been written to DN3. + assertState(cluster, ImmutableMap.of(1, replicaIndexMap.get(3), 2, replicaIndexMap.get(2), 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); - - - //RM should fix underreplication and write data to DN3 - cluster.getStorageContainerManager().getReplicationManager().start(); + // Starting DN1. + cluster.restartHddsDatanode(replicaIndexMap.get(1), false); + // Waiting for underreplication to get resolved. waitForReplicaCount(containerID, 5, cluster); + // Asserting Replica 1 & Replica 3 has been swapped b/w DN1 & DN3. assertState(cluster, ImmutableMap.of(1, replicaIndexMap.get(3), 2, replicaIndexMap.get(2), 3, replicaIndexMap.get(1), 4, replicaIndexMap.get(4), 5, replicaIndexMap.get(5))); - - // Reading the pre initialized inputStream. This leads to swap of the block1 & block3. + // Reading the Stripe 2 from the pre initialized inputStream inputStream.read(readData, firstReadLen, size - firstReadLen); + // Asserting there was a failure in the first read chunk. Assertions.assertEquals(ImmutableMap.of(1, 1, 3, 1), failedReadChunkCountMap); Assertions.assertArrayEquals(readData, originalData); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java index d9d24f4ae499..2a33ddc5677f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestHelper.java @@ -442,6 +442,6 @@ public static int countReplicas(long containerID, MiniOzoneCluster cluster) { public static void waitForReplicaCount(long containerID, int count, MiniOzoneCluster cluster) throws TimeoutException, InterruptedException { GenericTestUtils.waitFor(() -> countReplicas(containerID, cluster) == count, - 200, 300000); + 200, 30000); } } From e4c0d67687199fda068c582b16bca9ad1c671fa3 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Thu, 13 Jun 2024 17:51:37 -0700 Subject: [PATCH 17/21] HDDS-10983. Convert to mapToInt Change-Id: I7868729f0910168e6ed4610943255c79b1b00012 --- .../org/apache/hadoop/hdds/scm/storage/BlockInputStream.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java index 70154228768d..9d448c6fe363 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java @@ -260,7 +260,7 @@ private void setPipeline(Pipeline pipeline) throws IOException { if (pipeline == null) { return; } - long replicaIndexes = pipeline.getNodes().stream().map(pipeline::getReplicaIndex).distinct().count(); + long replicaIndexes = pipeline.getNodes().stream().mapToInt(pipeline::getReplicaIndex).distinct().count(); if (replicaIndexes > 1) { throw new IOException(String.format("Pipeline: %s has nodes containing different replica indexes.", From da980112079776ffd3c9bb0afa9313f4824e9db6 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Mon, 17 Jun 2024 14:31:40 -0700 Subject: [PATCH 18/21] HDDS-11013. Merge upstream master Change-Id: I43551feebf091706bd9ab2bed5d0e671cf3d4077 --- .github/workflows/intermittent-test-check.yml | 4 +- .../hadoop/hdds/scm/XceiverClientGrpc.java | 28 +++++-- .../scm/storage/BlockDataStreamOutput.java | 3 +- .../TestBlockOutputStreamCorrectness.java | 10 ++- .../ratis/ContainerCommandRequestMessage.java | 9 +- .../scm/storage/ContainerProtocolCalls.java | 84 ++++++++----------- .../apache/hadoop/ozone/ClientVersion.java | 4 +- .../TestContainerCommandRequestMessage.java | 9 +- .../ozone/container/ContainerTestHelper.java | 40 +++++---- .../container/common/impl/HddsDispatcher.java | 3 +- .../CloseContainerCommandHandler.java | 5 +- .../server/ratis/ContainerStateMachine.java | 10 +-- .../common/impl/TestHddsDispatcher.java | 6 +- .../keyvalue/TestKeyValueHandler.java | 10 +-- .../impl/TestKeyValueStreamDataChannel.java | 7 +- .../hadoop/hdds/utils/TransactionInfo.java | 2 +- .../ozone/container/common/TestEndPoint.java | 45 ++++++---- hadoop-ozone/dev-support/checks/junit.sh | 4 + .../dev-support/findbugsExcludeFile.xml | 2 +- .../hdds/scm/pipeline/TestPipelineClose.java | 13 ++- .../hadoop/ozone/TestContainerOperations.java | 56 +++++++++++++ .../rpc/TestContainerReplicationEndToEnd.java | 4 +- .../TestContainerStateMachineFailures.java | 15 ++-- .../rpc/TestDeleteWithInAdequateDN.java | 4 +- .../shell/TestOzoneContainerUpgradeShell.java | 2 + .../hadoop/ozone/shell/TestReconfigShell.java | 3 + pom.xml | 2 +- 27 files changed, 241 insertions(+), 143 deletions(-) diff --git a/.github/workflows/intermittent-test-check.yml b/.github/workflows/intermittent-test-check.yml index 4c5da2bf862a..f4020db9508a 100644 --- a/.github/workflows/intermittent-test-check.yml +++ b/.github/workflows/intermittent-test-check.yml @@ -209,12 +209,12 @@ jobs: if: ${{ !cancelled() }} - name: Archive build results uses: actions/upload-artifact@v4 - if: always() + if: ${{ failure() }} with: name: result-${{ github.run_number }}-${{ github.run_id }}-split-${{ matrix.split }} path: target/unit count-failures: - if: ${{ always() }} + if: ${{ failure() }} needs: run-test runs-on: ubuntu-20.04 steps: diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 39a69b35d5dc..4fb661cbffbb 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -52,6 +52,7 @@ import org.apache.hadoop.hdds.security.exception.SCMSecurityException; import org.apache.hadoop.hdds.tracing.GrpcClientInterceptor; import org.apache.hadoop.hdds.tracing.TracingUtil; +import org.apache.hadoop.ozone.ClientVersion; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConsts; import java.util.concurrent.TimeoutException; @@ -71,7 +72,6 @@ import org.slf4j.LoggerFactory; import static org.apache.hadoop.hdds.HddsUtils.processForDebug; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; /** * {@link XceiverClientSpi} implementation, the standalone client. @@ -278,6 +278,11 @@ public ContainerCommandResponseProto sendCommand( List datanodeList = pipeline.getNodes(); HashMap> futureHashMap = new HashMap<>(); + if (!request.hasVersion()) { + ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto.newBuilder(request); + builder.setVersion(ClientVersion.CURRENT.toProtoValue()); + request = builder.build(); + } for (DatanodeDetails dn : datanodeList) { try { futureHashMap.put(dn, sendCommandAsync(request, dn).getResponse()); @@ -338,9 +343,13 @@ private XceiverClientReply sendCommandWithTraceIDAndRetry( return TracingUtil.executeInNewSpan(spanName, () -> { - ContainerCommandRequestProto finalPayload = getContainerCommandRequestProtoBuilder(request) - .setTraceID(TracingUtil.exportCurrentSpan()).build(); - return sendCommandWithRetry(finalPayload, validators); + ContainerCommandRequestProto.Builder builder = + ContainerCommandRequestProto.newBuilder(request) + .setTraceID(TracingUtil.exportCurrentSpan()); + if (!request.hasVersion()) { + builder.setVersion(ClientVersion.CURRENT.toProtoValue()); + } + return sendCommandWithRetry(builder.build(), validators); }); } @@ -490,11 +499,14 @@ public XceiverClientReply sendCommandAsync( try (Scope ignored = GlobalTracer.get().activateSpan(span)) { - ContainerCommandRequestProto finalPayload = getContainerCommandRequestProtoBuilder(request) - .setTraceID(TracingUtil.exportCurrentSpan()) - .build(); + ContainerCommandRequestProto.Builder builder = + ContainerCommandRequestProto.newBuilder(request) + .setTraceID(TracingUtil.exportCurrentSpan()); + if (!request.hasVersion()) { + builder.setVersion(ClientVersion.CURRENT.toProtoValue()); + } XceiverClientReply asyncReply = - sendCommandAsync(finalPayload, pipeline.getFirstNode()); + sendCommandAsync(builder.build(), pipeline.getFirstNode()); if (shouldBlockAndWaitAsyncReply(request)) { asyncReply.getResponse().get(); } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java index c9174537ba70..d5423d4ec0bb 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java @@ -62,7 +62,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync; /** @@ -205,7 +204,7 @@ private DataStreamOutput setupStream(Pipeline pipeline) throws IOException { // it or remove it completely if possible String id = pipeline.getFirstNode().getUuidString(); ContainerProtos.ContainerCommandRequestProto.Builder builder = - getContainerCommandRequestProtoBuilder() + ContainerProtos.ContainerCommandRequestProto.newBuilder() .setCmdType(ContainerProtos.Type.StreamInit) .setContainerID(blockID.get().getContainerID()) .setDatanodeUuid(id).setWriteChunk(writeChunkRequest); diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockOutputStreamCorrectness.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockOutputStreamCorrectness.java index bf4830c6fcb5..df55b5bf57ae 100644 --- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockOutputStreamCorrectness.java +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockOutputStreamCorrectness.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; +import org.apache.hadoop.ozone.ClientVersion; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; @@ -228,9 +229,12 @@ public Pipeline getPipeline() { } @Override - public XceiverClientReply sendCommandAsync( - ContainerCommandRequestProto request - ) { + public XceiverClientReply sendCommandAsync(ContainerCommandRequestProto request) { + + if (!request.hasVersion()) { + request = ContainerCommandRequestProto.newBuilder(request) + .setVersion(ClientVersion.CURRENT.toProtoValue()).build(); + } final ContainerCommandResponseProto.Builder builder = ContainerCommandResponseProto.newBuilder() .setResult(Result.SUCCESS) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/ContainerCommandRequestMessage.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/ContainerCommandRequestMessage.java index 98eba583be3a..7ae6e7859046 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/ContainerCommandRequestMessage.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/ContainerCommandRequestMessage.java @@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutSmallFileRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; +import org.apache.hadoop.ozone.ClientVersion; import org.apache.hadoop.ozone.common.Checksum; import org.apache.ratis.protocol.Message; @@ -32,8 +33,6 @@ import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.apache.ratis.util.JavaUtils; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; - /** * Implementing the {@link Message} interface * for {@link ContainerCommandRequestProto}. @@ -41,10 +40,14 @@ public final class ContainerCommandRequestMessage implements Message { public static ContainerCommandRequestMessage toMessage( ContainerCommandRequestProto request, String traceId) { - final ContainerCommandRequestProto.Builder b = getContainerCommandRequestProtoBuilder(request); + final ContainerCommandRequestProto.Builder b + = ContainerCommandRequestProto.newBuilder(request); if (traceId != null) { b.setTraceID(traceId); } + if (!request.hasVersion()) { + b.setVersion(ClientVersion.CURRENT.toProtoValue()); + } ByteString data = ByteString.EMPTY; if (request.getCmdType() == Type.WriteChunk) { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index 508ebf3c61c9..659ddf2738b0 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -70,7 +70,6 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; import org.apache.hadoop.hdds.tracing.TracingUtil; -import org.apache.hadoop.ozone.ClientVersion; import org.apache.hadoop.ozone.common.Checksum; import org.apache.hadoop.ozone.common.ChecksumData; import org.apache.hadoop.security.token.Token; @@ -98,28 +97,6 @@ public final class ContainerProtocolCalls { private ContainerProtocolCalls() { } - /** - * Creates a ContainerCommandRequestProto with version set. - */ - public static ContainerCommandRequestProto.Builder getContainerCommandRequestProtoBuilder(int version) { - return getContainerCommandRequestProtoBuilder(null, version); - } - - public static ContainerCommandRequestProto.Builder getContainerCommandRequestProtoBuilder() { - return getContainerCommandRequestProtoBuilder(ClientVersion.CURRENT.toProtoValue()); - } - - public static ContainerCommandRequestProto.Builder getContainerCommandRequestProtoBuilder( - ContainerCommandRequestProto req, int version) { - return (req == null ? - ContainerCommandRequestProto.newBuilder() : ContainerCommandRequestProto.newBuilder(req)).setVersion(version); - } - - public static ContainerCommandRequestProto.Builder getContainerCommandRequestProtoBuilder( - ContainerCommandRequestProto req) { - return getContainerCommandRequestProtoBuilder(req, ClientVersion.CURRENT.toProtoValue()); - } - /** * Calls the container protocol to list blocks in container. * @@ -148,7 +125,7 @@ public static ListBlockResponseProto listBlock(XceiverClientSpi xceiverClient, xceiverClient.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder builder = - getContainerCommandRequestProtoBuilder() + ContainerCommandRequestProto.newBuilder() .setCmdType(Type.ListBlock) .setContainerID(containerID) .setDatanodeUuid(datanodeID) @@ -214,7 +191,8 @@ static T tryEachDatanode(Pipeline pipeline, public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, List validators, BlockID blockID, Token token, Map replicaIndexes) throws IOException { - ContainerCommandRequestProto.Builder builder = getContainerCommandRequestProtoBuilder() + ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto + .newBuilder() .setCmdType(Type.GetBlock) .setContainerID(blockID.getContainerID()); if (token != null) { @@ -232,14 +210,15 @@ static String toErrorMessage(BlockID blockId, DatanodeDetails d) { } public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, - BlockID datanodeBlockID, Token token, - Map replicaIndexes) throws IOException { + BlockID datanodeBlockID, + Token token, Map replicaIndexes) throws IOException { return getBlock(xceiverClient, getValidatorList(), datanodeBlockID, token, replicaIndexes); } - private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, List validators, - ContainerCommandRequestProto.Builder builder, BlockID blockID, DatanodeDetails datanode, - Map replicaIndexes) throws IOException { + private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, + List validators, + ContainerCommandRequestProto.Builder builder, BlockID blockID, + DatanodeDetails datanode, Map replicaIndexes) throws IOException { String traceId = TracingUtil.exportCurrentSpan(); if (traceId != null) { builder.setTraceID(traceId); @@ -268,15 +247,18 @@ private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, Li * @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 token) throws IOException { + public static ContainerProtos.GetCommittedBlockLengthResponseProto + getCommittedBlockLength( + XceiverClientSpi xceiverClient, BlockID blockID, + Token token) + throws IOException { ContainerProtos.GetCommittedBlockLengthRequestProto.Builder getBlockLengthRequestBuilder = ContainerProtos.GetCommittedBlockLengthRequestProto.newBuilder(). setBlockID(blockID.getDatanodeBlockIDProtobuf()); String id = xceiverClient.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder builder = - getContainerCommandRequestProtoBuilder() + ContainerCommandRequestProto.newBuilder() .setCmdType(Type.GetCommittedBlockLength) .setContainerID(blockID.getContainerID()) .setDatanodeUuid(id) @@ -322,11 +304,11 @@ public static ContainerCommandRequestProto getPutBlockRequest( .setBlockData(containerBlockData) .setEof(eof); final String id = pipeline.getFirstNode().getUuidString(); - ContainerCommandRequestProto.Builder builder = getContainerCommandRequestProtoBuilder() - .setCmdType(Type.PutBlock) - .setContainerID(containerBlockData.getBlockID().getContainerID()) - .setDatanodeUuid(id) - .setPutBlock(createBlockRequest); + ContainerCommandRequestProto.Builder builder = + ContainerCommandRequestProto.newBuilder().setCmdType(Type.PutBlock) + .setContainerID(containerBlockData.getBlockID().getContainerID()) + .setDatanodeUuid(id) + .setPutBlock(createBlockRequest); if (tokenString != null) { builder.setEncodedToken(tokenString); } @@ -344,15 +326,17 @@ 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 validators, Token token) throws IOException { + public static ContainerProtos.ReadChunkResponseProto readChunk( + XceiverClientSpi xceiverClient, ChunkInfo chunk, DatanodeBlockID blockID, + List validators, + Token token) throws IOException { ReadChunkRequestProto.Builder readChunkRequest = ReadChunkRequestProto.newBuilder() .setBlockID(blockID) .setChunkData(chunk) .setReadChunkVersion(ContainerProtos.ReadChunkVersion.V1); ContainerCommandRequestProto.Builder builder = - getContainerCommandRequestProtoBuilder().setCmdType(Type.ReadChunk) + ContainerCommandRequestProto.newBuilder().setCmdType(Type.ReadChunk) .setContainerID(blockID.getContainerID()) .setReadChunk(readChunkRequest); if (token != null) { @@ -440,7 +424,7 @@ public static XceiverClientReply writeChunkAsync( .setData(data); String id = xceiverClient.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder builder = - getContainerCommandRequestProtoBuilder() + ContainerCommandRequestProto.newBuilder() .setCmdType(Type.WriteChunk) .setContainerID(blockID.getContainerID()) .setDatanodeUuid(id) @@ -498,7 +482,7 @@ public static PutSmallFileResponseProto writeSmallFile( String id = client.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder builder = - getContainerCommandRequestProtoBuilder() + ContainerCommandRequestProto.newBuilder() .setCmdType(Type.PutSmallFile) .setContainerID(blockID.getContainerID()) .setDatanodeUuid(id) @@ -565,7 +549,7 @@ public static void createContainer(XceiverClientSpi client, String id = client.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder request = - getContainerCommandRequestProtoBuilder(); + ContainerCommandRequestProto.newBuilder(); if (encodedToken != null) { request.setEncodedToken(encodedToken); } @@ -595,7 +579,7 @@ public static void deleteContainer(XceiverClientSpi client, long containerID, String id = client.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder request = - getContainerCommandRequestProtoBuilder(); + ContainerCommandRequestProto.newBuilder(); request.setCmdType(ContainerProtos.Type.DeleteContainer); request.setContainerID(containerID); request.setDeleteContainer(deleteRequest); @@ -621,7 +605,7 @@ public static void closeContainer(XceiverClientSpi client, String id = client.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder request = - getContainerCommandRequestProtoBuilder(); + ContainerCommandRequestProto.newBuilder(); request.setCmdType(Type.CloseContainer); request.setContainerID(containerID); request.setCloseContainer(CloseContainerRequestProto.getDefaultInstance()); @@ -648,7 +632,7 @@ public static ReadContainerResponseProto readContainer( String id = client.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder request = - getContainerCommandRequestProtoBuilder(); + ContainerCommandRequestProto.newBuilder(); request.setCmdType(Type.ReadContainer); request.setContainerID(containerID); request.setReadContainer(ReadContainerRequestProto.getDefaultInstance()); @@ -686,7 +670,8 @@ public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client, .build(); String id = client.getPipeline().getClosestNode().getUuidString(); - ContainerCommandRequestProto.Builder builder = getContainerCommandRequestProtoBuilder() + ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto + .newBuilder() .setCmdType(Type.GetSmallFile) .setContainerID(blockID.getContainerID()) .setDatanodeUuid(id) @@ -796,7 +781,8 @@ public static List toValidatorList(Validator validator) { HashMap datanodeToResponseMap = new HashMap<>(); String id = xceiverClient.getPipeline().getFirstNode().getUuidString(); - ContainerCommandRequestProto.Builder builder = getContainerCommandRequestProtoBuilder() + ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto + .newBuilder() .setCmdType(Type.GetBlock) .setContainerID(datanodeBlockID.getContainerID()) .setDatanodeUuid(id) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java index 82f5afd4fedc..0e368d0ed7b8 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java @@ -28,7 +28,7 @@ /** * Versioning for protocol clients. */ -public enum ClientVersion implements ComponentVersion, Comparable { +public enum ClientVersion implements ComponentVersion { DEFAULT_VERSION(0, "Initial version"), @@ -82,6 +82,4 @@ private static ClientVersion latest() { return versions[versions.length - 2]; } - - } diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/TestContainerCommandRequestMessage.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/TestContainerCommandRequestMessage.java index b0f5bc4f4541..05fc9cb40b0b 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/TestContainerCommandRequestMessage.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/TestContainerCommandRequestMessage.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutSmallFileRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; +import org.apache.hadoop.ozone.ClientVersion; import org.apache.hadoop.ozone.common.Checksum; import org.apache.hadoop.ozone.common.ChecksumData; import org.apache.hadoop.ozone.common.OzoneChecksumException; @@ -40,8 +41,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; - /** Testing {@link ContainerCommandRequestMessage}. */ public class TestContainerCommandRequestMessage { static final Random RANDOM = new Random(); @@ -88,11 +87,12 @@ static ContainerCommandRequestProto newPutSmallFile( .setBlock(putBlockRequest) .setData(data) .build(); - return getContainerCommandRequestProtoBuilder() + return ContainerCommandRequestProto.newBuilder() .setCmdType(Type.PutSmallFile) .setContainerID(blockID.getContainerID()) .setDatanodeUuid(UUID.randomUUID().toString()) .setPutSmallFile(putSmallFileRequest) + .setVersion(ClientVersion.CURRENT.toProtoValue()) .build(); } @@ -110,11 +110,12 @@ static ContainerCommandRequestProto newWriteChunk( .setBlockID(blockID.getDatanodeBlockIDProtobuf()) .setChunkData(chunk) .setData(data); - return getContainerCommandRequestProtoBuilder() + return ContainerCommandRequestProto.newBuilder() .setCmdType(Type.WriteChunk) .setContainerID(blockID.getContainerID()) .setDatanodeUuid(UUID.randomUUID().toString()) .setWriteChunk(writeChunkRequest) + .setVersion(ClientVersion.CURRENT.toProtoValue()) .build(); } diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java index 2fa48fde8844..dce14f0d1fe8 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java @@ -53,8 +53,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.fail; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; - /** * Helpers for container tests. */ @@ -133,7 +131,7 @@ public static ContainerCommandRequestProto getWriteChunkRequest( public static ContainerCommandRequestProto getListBlockRequest( ContainerCommandRequestProto writeChunkRequest) { - return getContainerCommandRequestProtoBuilder() + return ContainerCommandRequestProto.newBuilder() .setContainerID(writeChunkRequest.getContainerID()) .setCmdType(ContainerProtos.Type.ListBlock) .setDatanodeUuid(writeChunkRequest.getDatanodeUuid()) @@ -149,7 +147,7 @@ public static ContainerCommandRequestProto getPutBlockRequest( .setSize(writeChunkRequest.getWriteChunk().getChunkData().getLen()) .setBlockID(writeChunkRequest.getWriteChunk().getBlockID()) .addChunks(writeChunkRequest.getWriteChunk().getChunkData()); - return getContainerCommandRequestProtoBuilder() + return ContainerCommandRequestProto.newBuilder() .setContainerID(writeChunkRequest.getContainerID()) .setCmdType(ContainerProtos.Type.PutBlock) .setDatanodeUuid(writeChunkRequest.getDatanodeUuid()) @@ -182,7 +180,8 @@ public static Builder newWriteChunkRequestBuilder( writeRequest.setChunkData(info.getProtoBufMessage()); writeRequest.setData(data.toByteString()); - Builder request = getContainerCommandRequestProtoBuilder(); + Builder request = + ContainerCommandRequestProto.newBuilder(); request.setCmdType(ContainerProtos.Type.WriteChunk); request.setContainerID(blockID.getContainerID()); request.setWriteChunk(writeRequest); @@ -222,7 +221,8 @@ public static ContainerCommandRequestProto getWriteSmallFileRequest( smallFileRequest.setData(data.toByteString()); smallFileRequest.setBlock(putRequest); - Builder request = getContainerCommandRequestProtoBuilder(); + Builder request = + ContainerCommandRequestProto.newBuilder(); request.setCmdType(ContainerProtos.Type.PutSmallFile); request.setContainerID(blockID.getContainerID()); request.setPutSmallFile(smallFileRequest); @@ -239,7 +239,8 @@ public static ContainerCommandRequestProto getReadSmallFileRequest( ContainerCommandRequestProto getKey = getBlockRequest(pipeline, putKey); smallFileRequest.setBlock(getKey.getGetBlock()); - Builder request = getContainerCommandRequestProtoBuilder(); + Builder request = + ContainerCommandRequestProto.newBuilder(); request.setCmdType(ContainerProtos.Type.GetSmallFile); request.setContainerID(getKey.getGetBlock().getBlockID().getContainerID()); request.setGetSmallFile(smallFileRequest); @@ -272,7 +273,8 @@ public static Builder newReadChunkRequestBuilder(Pipeline pipeline, readRequest.setChunkData(writeChunk.getChunkData()); readRequest.setReadChunkVersion(ContainerProtos.ReadChunkVersion.V1); - Builder newRequest = getContainerCommandRequestProtoBuilder(); + Builder newRequest = + ContainerCommandRequestProto.newBuilder(); newRequest.setCmdType(ContainerProtos.Type.ReadChunk); newRequest.setContainerID(readRequest.getBlockID().getContainerID()); newRequest.setReadChunk(readRequest); @@ -294,7 +296,8 @@ public static ContainerCommandRequestProto getCreateContainerRequest( private static Builder getContainerCommandRequestBuilder(long containerID, Pipeline pipeline) throws IOException { - Builder request = getContainerCommandRequestProtoBuilder(); + Builder request = + ContainerCommandRequestProto.newBuilder(); request.setCmdType(ContainerProtos.Type.CreateContainer); request.setContainerID(containerID); request.setCreateContainer( @@ -340,7 +343,8 @@ public static ContainerCommandRequestProto getUpdateContainerRequest( Pipeline pipeline = MockPipeline.createSingleNodePipeline(); - Builder request = getContainerCommandRequestProtoBuilder(); + Builder request = + ContainerCommandRequestProto.newBuilder(); request.setCmdType(ContainerProtos.Type.UpdateContainer); request.setContainerID(containerID); request.setUpdateContainer(updateRequestBuilder.build()); @@ -396,7 +400,8 @@ public static Builder newPutBlockRequestBuilder(Pipeline pipeline, blockData.setBlockCommitSequenceId(0); putRequest.setBlockData(blockData.getProtoBufMessage()); - Builder request = getContainerCommandRequestProtoBuilder(); + Builder request = + ContainerCommandRequestProto.newBuilder(); request.setCmdType(ContainerProtos.Type.PutBlock); request.setContainerID(blockData.getContainerID()); request.setPutBlock(putRequest); @@ -426,7 +431,8 @@ public static Builder newGetBlockRequestBuilder( ContainerProtos.GetBlockRequestProto.newBuilder(); getRequest.setBlockID(blockID); - Builder request = getContainerCommandRequestProtoBuilder(); + Builder request = + ContainerCommandRequestProto.newBuilder(); request.setCmdType(ContainerProtos.Type.GetBlock); request.setContainerID(blockID.getContainerID()); request.setGetBlock(getRequest); @@ -454,7 +460,7 @@ public static Builder newGetCommittedBlockLengthBuilder(Pipeline pipeline, ContainerProtos.GetCommittedBlockLengthRequestProto.newBuilder() .setBlockID(blockID); - return getContainerCommandRequestProtoBuilder() + return ContainerCommandRequestProto.newBuilder() .setCmdType(ContainerProtos.Type.GetCommittedBlockLength) .setContainerID(blockID.getContainerID()) .setDatanodeUuid(pipeline.getFirstNode().getUuidString()) @@ -470,7 +476,7 @@ public static Builder newGetCommittedBlockLengthBuilder(Pipeline pipeline, */ public static ContainerCommandRequestProto getCloseContainer( Pipeline pipeline, long containerID, Token token) throws IOException { - Builder builder = getContainerCommandRequestProtoBuilder() + Builder builder = ContainerCommandRequestProto.newBuilder() .setCmdType(ContainerProtos.Type.CloseContainer) .setContainerID(containerID) .setCloseContainer( @@ -501,7 +507,7 @@ public static ContainerCommandRequestProto getDeleteContainer( ContainerProtos.DeleteContainerRequestProto deleteRequest = ContainerProtos.DeleteContainerRequestProto.newBuilder(). setForceDelete(forceDelete).build(); - return getContainerCommandRequestProtoBuilder() + return ContainerCommandRequestProto.newBuilder() .setCmdType(ContainerProtos.Type.DeleteContainer) .setContainerID(containerID) .setDeleteContainer( @@ -559,7 +565,9 @@ public static ContainerCommandRequestProto getDummyCommandRequestProto( */ public static ContainerCommandRequestProto getDummyCommandRequestProto( ClientVersion clientVersion, ContainerProtos.Type cmdType, int replicaIndex) { - final Builder builder = getContainerCommandRequestProtoBuilder(clientVersion.toProtoValue()) + final Builder builder = + ContainerCommandRequestProto.newBuilder() + .setVersion(clientVersion.toProtoValue()) .setCmdType(cmdType) .setContainerID(DUMMY_CONTAINER_ID) .setDatanodeUuid(DATANODE_UUID); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java index f140846b79ba..e494243ccc1c 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java @@ -79,7 +79,6 @@ import static org.apache.hadoop.ozone.audit.AuditLogger.PerformanceStringBuilder; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.malformedRequest; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.unsupportedRequest; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult; /** @@ -480,7 +479,7 @@ ContainerCommandResponseProto createContainer( } ContainerCommandRequestProto.Builder requestBuilder = - getContainerCommandRequestProtoBuilder() + ContainerCommandRequestProto.newBuilder() .setCmdType(Type.CreateContainer) .setContainerID(containerRequest.getContainerID()) .setCreateContainer(createRequest.build()) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java index b3c4f745bfa4..8533f7384d41 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java @@ -48,8 +48,6 @@ import java.io.IOException; import java.util.concurrent.atomic.AtomicLong; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; - /** * Handler for close container command received from SCM. */ @@ -166,7 +164,8 @@ public void handle(SCMCommand command, OzoneContainer ozoneContainer, private ContainerCommandRequestProto getContainerCommandRequestProto( final DatanodeDetails datanodeDetails, final long containerId, final String encodedToken) { - final ContainerCommandRequestProto.Builder command = getContainerCommandRequestProtoBuilder(); + final ContainerCommandRequestProto.Builder command = + ContainerCommandRequestProto.newBuilder(); command.setCmdType(ContainerProtos.Type.CloseContainer); command.setTraceID(TracingUtil.exportCurrentSpan()); command.setContainerID(containerId); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java index ffb4d505ff8b..6351c746fe5c 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java @@ -100,8 +100,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; - /** * A {@link StateMachine} for containers, * which is responsible for handling different types of container requests. @@ -389,7 +387,7 @@ public TransactionContext startTransaction(LogEntryProto entry, RaftPeerRole rol final ContainerCommandRequestProto requestProto; if (logProto.getCmdType() == Type.WriteChunk) { // combine state machine data - requestProto = getContainerCommandRequestProtoBuilder(logProto) + requestProto = ContainerCommandRequestProto.newBuilder(logProto) .setWriteChunk(WriteChunkRequestProto.newBuilder(logProto.getWriteChunk()) .setData(stateMachineLogEntry.getStateMachineEntry().getStateMachineData())) .build(); @@ -426,7 +424,7 @@ public TransactionContext startTransaction(RaftClientRequest request) } // once the token is verified, clear it from the proto - final ContainerCommandRequestProto.Builder protoBuilder = getContainerCommandRequestProtoBuilder(proto) + final ContainerCommandRequestProto.Builder protoBuilder = ContainerCommandRequestProto.newBuilder(proto) .clearEncodedToken(); if (proto.getCmdType() == Type.WriteChunk) { final WriteChunkRequestProto write = proto.getWriteChunk(); @@ -453,7 +451,7 @@ private static ContainerCommandRequestProto getContainerCommandRequestProto( // TODO: We can avoid creating new builder and set pipeline Id if // the client is already sending the pipeline id, then we just have to // validate the pipeline Id. - return getContainerCommandRequestProtoBuilder( + return ContainerCommandRequestProto.newBuilder( ContainerCommandRequestProto.parseFrom(request)) .setPipelineID(id.getUuid().toString()).build(); } @@ -728,7 +726,7 @@ private ByteString readStateMachineData( .setChunkData(chunkInfo) .setReadChunkVersion(ContainerProtos.ReadChunkVersion.V1); ContainerCommandRequestProto dataContainerCommandProto = - getContainerCommandRequestProtoBuilder(requestProto) + ContainerCommandRequestProto.newBuilder(requestProto) .setCmdType(Type.ReadChunk).setReadChunk(readChunkRequestProto) .build(); final DispatcherContext context = DispatcherContext diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java index 222fa810ea3c..1cbd6ee4706d 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestHddsDispatcher.java @@ -87,7 +87,6 @@ import static org.apache.hadoop.hdds.fs.MockSpaceUsagePersistence.inMemory; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY; import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getContainerCommandResponse; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; import static org.apache.hadoop.ozone.container.common.ContainerTestUtils.COMMIT_STAGE; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -585,7 +584,8 @@ private ContainerCommandRequestProto getWriteChunkRequest( .setChunkData(chunk) .setData(data); - return getContainerCommandRequestProtoBuilder() + return ContainerCommandRequestProto + .newBuilder() .setContainerID(containerId) .setCmdType(ContainerProtos.Type.WriteChunk) .setDatanodeUuid(datanodeId) @@ -686,7 +686,7 @@ private ContainerCommandRequestProto getReadChunkRequest( .setBlockID(writeChunk.getBlockID()) .setChunkData(writeChunk.getChunkData()) .setReadChunkVersion(ContainerProtos.ReadChunkVersion.V1); - return getContainerCommandRequestProtoBuilder() + return ContainerCommandRequestProto.newBuilder() .setCmdType(ContainerProtos.Type.ReadChunk) .setContainerID(writeChunk.getBlockID().getContainerID()) .setTraceID(writeChunkRequest.getTraceID()) diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java index f2f09824e313..b9c8feae16ce 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java @@ -57,7 +57,6 @@ import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DATANODE_VOLUME_CHOOSING_POLICY; import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_METADATA_DIRS; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -122,7 +121,8 @@ public void setup() throws StorageContainerException { public void testHandlerCommandHandling() throws Exception { reset(handler); // Test Create Container Request handling - ContainerCommandRequestProto createContainerRequest = getContainerCommandRequestProtoBuilder() + ContainerCommandRequestProto createContainerRequest = + ContainerProtos.ContainerCommandRequestProto.newBuilder() .setCmdType(ContainerProtos.Type.CreateContainer) .setContainerID(DUMMY_CONTAINER_ID) .setDatanodeUuid(DATANODE_UUID) @@ -296,7 +296,7 @@ public void testVolumeSetInKeyValueHandler() throws Exception { private ContainerCommandRequestProto getDummyCommandRequestProto( ContainerProtos.Type cmdType) { - return getContainerCommandRequestProtoBuilder() + return ContainerCommandRequestProto.newBuilder() .setCmdType(cmdType) .setContainerID(DUMMY_CONTAINER_ID) .setDatanodeUuid(DATANODE_UUID) @@ -317,7 +317,7 @@ public void testCloseInvalidContainer(ContainerLayoutVersion layoutVersion) // Create Close container request ContainerCommandRequestProto closeContainerRequest = - getContainerCommandRequestProtoBuilder() + ContainerProtos.ContainerCommandRequestProto.newBuilder() .setCmdType(ContainerProtos.Type.CloseContainer) .setContainerID(DUMMY_CONTAINER_ID) .setDatanodeUuid(DATANODE_UUID) @@ -435,7 +435,7 @@ public void testDeleteContainer() throws IOException { private static ContainerCommandRequestProto createContainerRequest( String datanodeId, long containerID) { - return getContainerCommandRequestProtoBuilder() + return ContainerCommandRequestProto.newBuilder() .setCmdType(ContainerProtos.Type.CreateContainer) .setDatanodeUuid(datanodeId).setCreateContainer( ContainerProtos.CreateContainerRequestProto.newBuilder() diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestKeyValueStreamDataChannel.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestKeyValueStreamDataChannel.java index ee977f53b3ca..63045f76136b 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestKeyValueStreamDataChannel.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/impl/TestKeyValueStreamDataChannel.java @@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.PutBlockRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage; +import org.apache.hadoop.ozone.ClientVersion; import org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel.Buffers; import org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel.WriteMethod; import org.apache.ratis.client.api.DataStreamOutput; @@ -58,7 +59,6 @@ import static org.apache.hadoop.hdds.scm.storage.BlockDataStreamOutput.PUT_BLOCK_REQUEST_LENGTH_MAX; import static org.apache.hadoop.hdds.scm.storage.BlockDataStreamOutput.executePutBlockClose; import static org.apache.hadoop.hdds.scm.storage.BlockDataStreamOutput.getProtoLength; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; import static org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel.closeBuffers; import static org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel.readPutBlockRequest; import static org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel.writeBuffers; @@ -71,14 +71,15 @@ public class TestKeyValueStreamDataChannel { public static final Logger LOG = LoggerFactory.getLogger(TestKeyValueStreamDataChannel.class); - static final ContainerCommandRequestProto PUT_BLOCK_PROTO - = getContainerCommandRequestProtoBuilder() + private static final ContainerCommandRequestProto PUT_BLOCK_PROTO + = ContainerCommandRequestProto.newBuilder() .setCmdType(Type.PutBlock) .setPutBlock(PutBlockRequestProto.newBuilder().setBlockData( BlockData.newBuilder().setBlockID(DatanodeBlockID.newBuilder() .setContainerID(222).setLocalID(333).build()).build())) .setDatanodeUuid("datanodeId") .setContainerID(111L) + .setVersion(ClientVersion.CURRENT.toProtoValue()) .build(); static final int PUT_BLOCK_PROTO_SIZE = PUT_BLOCK_PROTO.toByteString().size(); static { diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/TransactionInfo.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/TransactionInfo.java index 75329c0c17ea..e7c4ec4ce3d6 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/TransactionInfo.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/TransactionInfo.java @@ -50,7 +50,7 @@ public static Codec getCodec() { return CODEC; } - private static TransactionInfo valueOf(String transactionInfo) { + public static TransactionInfo valueOf(String transactionInfo) { final String[] tInfo = transactionInfo.split(TRANSACTION_INFO_SPLIT_KEY); Preconditions.checkArgument(tInfo.length == 2, "Unexpected split length: %s in \"%s\"", tInfo.length, transactionInfo); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java index 6a4cebe9c7a9..2b78b73bcd56 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java @@ -405,14 +405,8 @@ private MetadataStorageReportProto getMetadataStorageReports(UUID id) { null); } - private EndpointStateMachine registerTaskHelper(InetSocketAddress scmAddress, - int rpcTimeout, boolean clearDatanodeDetails - ) throws Exception { - OzoneConfiguration conf = SCMTestUtils.getConf(tempDir); - EndpointStateMachine rpcEndPoint = - createEndpoint(conf, - scmAddress, rpcTimeout); - rpcEndPoint.setState(EndpointStateMachine.EndPointStates.REGISTER); + private RegisterEndpointTask getRegisterEndpointTask(boolean clearDatanodeDetails, OzoneConfiguration conf, + EndpointStateMachine rpcEndPoint) throws Exception { OzoneContainer ozoneContainer = mock(OzoneContainer.class); UUID datanodeID = UUID.randomUUID(); when(ozoneContainer.getNodeReport()).thenReturn(HddsTestUtils @@ -437,6 +431,16 @@ private EndpointStateMachine registerTaskHelper(InetSocketAddress scmAddress, DatanodeDetails datanodeDetails = randomDatanodeDetails(); endpointTask.setDatanodeDetails(datanodeDetails); } + return endpointTask; + } + + private EndpointStateMachine registerTaskHelper(InetSocketAddress scmAddress, + int rpcTimeout, boolean clearDatanodeDetails + ) throws Exception { + OzoneConfiguration conf = SCMTestUtils.getConf(tempDir); + EndpointStateMachine rpcEndPoint = createEndpoint(conf, scmAddress, rpcTimeout); + rpcEndPoint.setState(EndpointStateMachine.EndPointStates.REGISTER); + RegisterEndpointTask endpointTask = getRegisterEndpointTask(clearDatanodeDetails, conf, rpcEndPoint); endpointTask.call(); return rpcEndPoint; } @@ -472,14 +476,23 @@ public void testRegisterNoContainerID() throws Exception { @Test public void testRegisterRpcTimeout() throws Exception { - final long rpcTimeout = 1000; - final long tolerance = 200; - scmServerImpl.setRpcResponseDelay(1500); - long start = Time.monotonicNow(); - registerTaskHelper(serverAddress, 1000, false).close(); - long end = Time.monotonicNow(); - scmServerImpl.setRpcResponseDelay(0); - assertThat(end - start).isLessThanOrEqualTo(rpcTimeout + tolerance); + final int rpcTimeout = 1000; + final int tolerance = 200; + scmServerImpl.setRpcResponseDelay(rpcTimeout + tolerance * 2); + OzoneConfiguration conf = SCMTestUtils.getConf(tempDir); + + try (EndpointStateMachine rpcEndPoint = createEndpoint(conf, serverAddress, rpcTimeout)) { + rpcEndPoint.setState(EndpointStateMachine.EndPointStates.REGISTER); + RegisterEndpointTask endpointTask = getRegisterEndpointTask(false, conf, rpcEndPoint); + long start = Time.monotonicNow(); + endpointTask.call(); + long end = Time.monotonicNow(); + assertThat(end - start) + .isGreaterThanOrEqualTo(rpcTimeout) + .isLessThanOrEqualTo(rpcTimeout + tolerance); + } finally { + scmServerImpl.setRpcResponseDelay(0); + } } @Test diff --git a/hadoop-ozone/dev-support/checks/junit.sh b/hadoop-ozone/dev-support/checks/junit.sh index 45ec12f00b94..d4936834b268 100755 --- a/hadoop-ozone/dev-support/checks/junit.sh +++ b/hadoop-ozone/dev-support/checks/junit.sh @@ -83,6 +83,10 @@ for i in $(seq 1 ${ITERATIONS}); do FAIL_FAST=true fi + if [[ ${irc} == 0 ]]; then + rm -fr "${REPORT_DIR}" + fi + REPORT_DIR="${original_report_dir}" echo "Iteration ${i} exit code: ${irc}" | tee -a "${REPORT_FILE}" fi diff --git a/hadoop-ozone/integration-test/dev-support/findbugsExcludeFile.xml b/hadoop-ozone/integration-test/dev-support/findbugsExcludeFile.xml index 632e9fc2f479..5ca337301230 100644 --- a/hadoop-ozone/integration-test/dev-support/findbugsExcludeFile.xml +++ b/hadoop-ozone/integration-test/dev-support/findbugsExcludeFile.xml @@ -41,7 +41,7 @@ - + diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java index 858a4486757c..70c168036a56 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java @@ -45,9 +45,11 @@ import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; import org.apache.ozone.test.GenericTestUtils; import org.apache.ratis.protocol.RaftGroupId; -import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; import org.mockito.ArgumentCaptor; @@ -70,6 +72,7 @@ /** * Tests for Pipeline Closing. */ +@TestInstance(TestInstance.Lifecycle.PER_CLASS) @Timeout(300) public class TestPipelineClose { @@ -86,7 +89,7 @@ public class TestPipelineClose { * * @throws IOException */ - @BeforeEach + @BeforeAll public void init() throws Exception { conf = new OzoneConfiguration(); conf.set(OzoneConfigKeys.OZONE_SCM_CLOSE_CONTAINER_WAIT_DURATION, "2s"); @@ -102,6 +105,10 @@ public void init() throws Exception { scm = cluster.getStorageContainerManager(); containerManager = scm.getContainerManager(); pipelineManager = scm.getPipelineManager(); + } + + @BeforeEach + void createContainer() throws IOException { ContainerInfo containerInfo = containerManager .allocateContainer(RatisReplicationConfig.getInstance( ReplicationFactor.THREE), "testOwner"); @@ -116,7 +123,7 @@ public void init() throws Exception { /** * Shutdown MiniDFSCluster. */ - @AfterEach + @AfterAll public void shutdown() { if (cluster != null) { cluster.shutdown(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java index 5f8f34a2e3ce..cbd1829ef0cb 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java @@ -17,8 +17,14 @@ */ package org.apache.hadoop.ozone; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.commons.lang3.RandomUtils; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.XceiverClientManager; +import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.PlacementPolicy; @@ -27,6 +33,7 @@ import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient; import org.apache.hadoop.hdds.scm.client.ScmClient; import org.apache.hadoop.hdds.scm.ha.SCMHAUtils; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException; @@ -34,8 +41,13 @@ import java.util.List; import java.util.concurrent.TimeUnit; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name.REPLICATION; +import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB; +import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.ozone.container.ContainerTestHelper; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -56,6 +68,9 @@ public class TestContainerOperations { private static ScmClient storageClient; private static MiniOzoneCluster cluster; private static OzoneConfiguration ozoneConf; + private static StorageContainerLocationProtocolClientSideTranslatorPB + storageContainerLocationClient; + private static XceiverClientManager xceiverClientManager; @BeforeAll public static void setup() throws Exception { @@ -65,6 +80,9 @@ public static void setup() throws Exception { cluster = MiniOzoneCluster.newBuilder(ozoneConf).setNumDatanodes(3).build(); storageClient = new ContainerOperationClient(ozoneConf); cluster.waitForClusterToBeReady(); + storageContainerLocationClient = + cluster.getStorageContainerLocationClient(); + xceiverClientManager = new XceiverClientManager(ozoneConf); } @AfterAll @@ -72,6 +90,44 @@ public static void cleanup() throws Exception { if (cluster != null) { cluster.shutdown(); } + IOUtils.cleanupWithLogger(null, storageContainerLocationClient); + } + + @Test + void testContainerStateMachineIdempotency() throws Exception { + ContainerWithPipeline container = storageContainerLocationClient + .allocateContainer(HddsProtos.ReplicationType.RATIS, + HddsProtos.ReplicationFactor.ONE, OzoneConsts.OZONE); + long containerID = container.getContainerInfo().getContainerID(); + Pipeline pipeline = container.getPipeline(); + XceiverClientSpi client = xceiverClientManager.acquireClient(pipeline); + //create the container + ContainerProtocolCalls.createContainer(client, containerID, null); + // call create Container again + BlockID blockID = ContainerTestHelper.getTestBlockID(containerID); + byte[] data = + RandomStringUtils.random(RandomUtils.nextInt(0, 1024)).getBytes(UTF_8); + ContainerProtos.ContainerCommandRequestProto writeChunkRequest = + ContainerTestHelper + .getWriteChunkRequest(container.getPipeline(), blockID, + data.length); + client.sendCommand(writeChunkRequest); + + //Make the write chunk request again without requesting for overWrite + client.sendCommand(writeChunkRequest); + // Now, explicitly make a putKey request for the block. + ContainerProtos.ContainerCommandRequestProto putKeyRequest = + ContainerTestHelper + .getPutBlockRequest(pipeline, writeChunkRequest.getWriteChunk()); + client.sendCommand(putKeyRequest).getPutBlock(); + // send the putBlock again + client.sendCommand(putKeyRequest); + + // close container call + ContainerProtocolCalls.closeContainer(client, containerID, null); + ContainerProtocolCalls.closeContainer(client, containerID, null); + + xceiverClientManager.releaseClient(client, false); } /** diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerReplicationEndToEnd.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerReplicationEndToEnd.java index cd34d94d2569..78a4e78647eb 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerReplicationEndToEnd.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerReplicationEndToEnd.java @@ -67,7 +67,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertSame; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; /** * Tests delete key operation with a slow follower in the datanode @@ -195,7 +194,8 @@ public void testContainerReplication() throws Exception { Thread.sleep(2 * containerReportInterval); DatanodeDetails oldReplicaNode = pipeline.getFirstNode(); // now move the container to the closed on the datanode. - ContainerProtos.ContainerCommandRequestProto.Builder request = getContainerCommandRequestProtoBuilder(); + ContainerProtos.ContainerCommandRequestProto.Builder request = + ContainerProtos.ContainerCommandRequestProto.newBuilder(); request.setDatanodeUuid(pipeline.getFirstNode().getUuidString()); request.setCmdType(ContainerProtos.Type.CloseContainer); request.setContainerID(containerID); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java index de2b351c4885..b6eaca8e80d0 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java @@ -104,12 +104,13 @@ import org.apache.ratis.protocol.exceptions.StateMachineException; import org.apache.ratis.server.storage.FileInfo; import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage; + import org.apache.ratis.statemachine.impl.StatemachineImplTestUtil; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; + /** * Tests the containerStateMachine failure handling. */ @@ -390,7 +391,8 @@ public void testUnhealthyContainer() throws Exception { .getDatanodeStateMachine().getContainer(); HddsDispatcher dispatcher = (HddsDispatcher) ozoneContainer .getDispatcher(); - ContainerProtos.ContainerCommandRequestProto.Builder request = getContainerCommandRequestProtoBuilder(); + ContainerProtos.ContainerCommandRequestProto.Builder request = + ContainerProtos.ContainerCommandRequestProto.newBuilder(); request.setCmdType(ContainerProtos.Type.CloseContainer); request.setContainerID(containerID); request.setCloseContainer( @@ -449,7 +451,8 @@ public void testApplyTransactionFailure() throws Exception { .getContainerWithPipeline(containerID).getPipeline(); XceiverClientSpi xceiverClient = xceiverClientManager.acquireClient(pipeline); - ContainerProtos.ContainerCommandRequestProto.Builder request = getContainerCommandRequestProtoBuilder(); + ContainerProtos.ContainerCommandRequestProto.Builder request = + ContainerProtos.ContainerCommandRequestProto.newBuilder(); request.setDatanodeUuid(pipeline.getFirstNode().getUuidString()); request.setCmdType(ContainerProtos.Type.CloseContainer); request.setContainerID(containerID); @@ -533,7 +536,8 @@ void testApplyTransactionIdempotencyWithClosedContainer() .getContainerWithPipeline(containerID).getPipeline(); XceiverClientSpi xceiverClient = xceiverClientManager.acquireClient(pipeline); - ContainerProtos.ContainerCommandRequestProto.Builder request = getContainerCommandRequestProtoBuilder(); + ContainerProtos.ContainerCommandRequestProto.Builder request = + ContainerProtos.ContainerCommandRequestProto.newBuilder(); request.setDatanodeUuid(pipeline.getFirstNode().getUuidString()); request.setCmdType(ContainerProtos.Type.CloseContainer); request.setContainerID(containerID); @@ -624,7 +628,8 @@ void testWriteStateMachineDataIdempotencyWithClosedContainer() AtomicInteger failCount = new AtomicInteger(0); Runnable r1 = () -> { try { - ContainerProtos.ContainerCommandRequestProto.Builder request = getContainerCommandRequestProtoBuilder(); + ContainerProtos.ContainerCommandRequestProto.Builder request = + ContainerProtos.ContainerCommandRequestProto.newBuilder(); request.setDatanodeUuid(pipeline.getFirstNode().getUuidString()); request.setCmdType(ContainerProtos.Type.CloseContainer); request.setContainerID(containerID); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestDeleteWithInAdequateDN.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestDeleteWithInAdequateDN.java index 2d3b7881d585..d4ff85736273 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestDeleteWithInAdequateDN.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestDeleteWithInAdequateDN.java @@ -79,7 +79,6 @@ import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; /** * Tests delete key operation with inadequate datanodes. @@ -250,7 +249,8 @@ void testDeleteKeyWithInAdequateDN() throws Exception { key.close(); // now move the container to the closed on the datanode. - ContainerProtos.ContainerCommandRequestProto.Builder request = getContainerCommandRequestProtoBuilder(); + ContainerProtos.ContainerCommandRequestProto.Builder request = + ContainerProtos.ContainerCommandRequestProto.newBuilder(); request.setDatanodeUuid(pipeline.getFirstNode().getUuidString()); request.setCmdType(ContainerProtos.Type.CloseContainer); request.setContainerID(containerID); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneContainerUpgradeShell.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneContainerUpgradeShell.java index 3a9f7e322b9e..6ea6531c658c 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneContainerUpgradeShell.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneContainerUpgradeShell.java @@ -69,6 +69,7 @@ import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_PIPELINE_REPORT_INTERVAL; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ADMINISTRATORS; import static org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration.CONTAINER_SCHEMA_V3_ENABLED; import static org.assertj.core.api.Assertions.assertThat; @@ -99,6 +100,7 @@ public static void init() throws Exception { conf.set(OZONE_ADMINISTRATORS, "*"); conf.setTimeDuration(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL, 100, TimeUnit.MILLISECONDS); + conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS); conf.setTimeDuration(HDDS_HEARTBEAT_INTERVAL, 1, SECONDS); conf.setTimeDuration(HDDS_PIPELINE_REPORT_INTERVAL, 1, SECONDS); conf.setTimeDuration(HDDS_COMMAND_STATUS_REPORT_INTERVAL, 1, SECONDS); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestReconfigShell.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestReconfigShell.java index 97a43c248a14..7c7f2b77ec51 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestReconfigShell.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestReconfigShell.java @@ -23,6 +23,7 @@ import java.util.Collection; import java.util.List; import java.util.UUID; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.ReconfigurableBase; import org.apache.hadoop.hdds.cli.OzoneAdmin; @@ -42,6 +43,7 @@ import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONED; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -66,6 +68,7 @@ public class TestReconfigShell { @BeforeAll public static void setup() throws Exception { OzoneConfiguration conf = new OzoneConfiguration(); + conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS); String omServiceId = UUID.randomUUID().toString(); cluster = MiniOzoneCluster.newHABuilder(conf) .setOMServiceId(omServiceId) diff --git a/pom.xml b/pom.xml index ddc7054cd057..0ca72ee01fd5 100644 --- a/pom.xml +++ b/pom.xml @@ -250,7 +250,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs 3.3.2 3.9.0 - 3.1.1 + 3.1.2 3.1.0 3.6.0 3.4.1 From b0764e458c69218d1174af864c97a4ab14d96ec7 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Mon, 17 Jun 2024 14:41:06 -0700 Subject: [PATCH 19/21] HDDS-10983. Merge master Change-Id: Ib736cce10e071142f9f9c060cc1cbb7e10d700a7 --- .../hadoop/ozone/freon/DatanodeBlockPutter.java | 5 ++--- .../hadoop/ozone/freon/DatanodeChunkGenerator.java | 5 ++--- .../hadoop/ozone/freon/DatanodeChunkValidator.java | 13 ++++++------- .../freon/FollowerAppendLogEntryGenerator.java | 5 ++--- .../ozone/freon/LeaderAppendLogEntryGenerator.java | 6 ++---- 5 files changed, 14 insertions(+), 20 deletions(-) diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeBlockPutter.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeBlockPutter.java index 8a6aac621245..f6a5c5965018 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeBlockPutter.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeBlockPutter.java @@ -43,8 +43,6 @@ import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; - /** * Datanode test for block creation. */ @@ -148,7 +146,8 @@ private void putBlock(long stepNo) throws Exception { String id = client.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder builder = - getContainerCommandRequestProtoBuilder() + ContainerCommandRequestProto + .newBuilder() .setCmdType(Type.PutBlock) .setContainerID(blockId.getContainerID()) .setDatanodeUuid(id) diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkGenerator.java index 4d252522f8b8..6362f32d0499 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkGenerator.java @@ -52,8 +52,6 @@ import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; - /** * Data generator to use pure datanode XCeiver interface. */ @@ -236,7 +234,8 @@ private void sendWriteChunkRequest(DatanodeBlockID blockId, String id = datanodeDetails.getUuidString(); ContainerCommandRequestProto.Builder builder = - getContainerCommandRequestProtoBuilder() + ContainerCommandRequestProto + .newBuilder() .setCmdType(Type.WriteChunk) .setContainerID(blockId.getContainerID()) .setDatanodeUuid(id) diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java index d5f2955d30ad..2bbf8b6d5b24 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DatanodeChunkValidator.java @@ -39,8 +39,6 @@ import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; - /** * Data validator of chunks to use pure datanode XCeiver interface. */ @@ -181,11 +179,12 @@ private ContainerCommandRequestProto createReadChunkRequest(long stepNo) String id = xceiverClient.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder builder = - getContainerCommandRequestProtoBuilder() - .setCmdType(ContainerProtos.Type.ReadChunk) - .setContainerID(blockId.getContainerID()) - .setDatanodeUuid(id) - .setReadChunk(readChunkRequest); + ContainerCommandRequestProto + .newBuilder() + .setCmdType(ContainerProtos.Type.ReadChunk) + .setContainerID(blockId.getContainerID()) + .setDatanodeUuid(id) + .setReadChunk(readChunkRequest); return builder.build(); } diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/FollowerAppendLogEntryGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/FollowerAppendLogEntryGenerator.java index e0146852c734..d76081fd29ea 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/FollowerAppendLogEntryGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/FollowerAppendLogEntryGenerator.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumData; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.WriteChunkRequestProto; @@ -72,8 +73,6 @@ import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; - /** * Freon test to test one single datanode with a fake leader (this test). *

@@ -255,7 +254,7 @@ private AppendEntriesRequestProto createAppendLogEntry(long sequence, long blockId = chunkId / 1000; long containerId = blockId / 1000; //ozone specific - ByteString payload = getContainerCommandRequestProtoBuilder() + ByteString payload = ContainerCommandRequestProto.newBuilder() .setContainerID(containerId) .setCmdType(Type.WriteChunk) .setDatanodeUuid(serverId) diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/LeaderAppendLogEntryGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/LeaderAppendLogEntryGenerator.java index c21df661cf98..40f670260326 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/LeaderAppendLogEntryGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/LeaderAppendLogEntryGenerator.java @@ -68,8 +68,6 @@ import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.getContainerCommandRequestProtoBuilder; - /** * Test isolated LEADER datanodes. *

@@ -206,7 +204,7 @@ private XceiverClientRatis createXceiverClient(OzoneConfiguration conf) { private ContainerCommandRequestProto createContainerRequest( long containerId) { - return getContainerCommandRequestProtoBuilder() + return ContainerCommandRequestProto.newBuilder() .setContainerID(containerId) .setCmdType(Type.CreateContainer) .setDatanodeUuid(serverId) @@ -221,7 +219,7 @@ private ContainerCommandRequestProto createChunkWriteRequest(long containerId, long chunkId) { long blockId = getPrefix().hashCode() + chunkId / 1000; - return getContainerCommandRequestProtoBuilder() + return ContainerCommandRequestProto.newBuilder() .setContainerID(containerId) .setCmdType(Type.WriteChunk) .setDatanodeUuid(serverId) From 3ae6768892cdc64f707c358169682548939e0fe7 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 19 Jun 2024 11:12:37 -0700 Subject: [PATCH 20/21] HDDS-10983. Address reveiw comments Change-Id: I80025cd3e99c3cf44887e85871261467323d29b9 --- .../org/apache/hadoop/ozone/ClientVersion.java | 5 +++-- .../container/keyvalue/KeyValueHandler.java | 16 ++++++++++++---- ...estKeyValueHandlerWithUnhealthyContainer.java | 4 ++-- 3 files changed, 17 insertions(+), 8 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java index 0e368d0ed7b8..f3bd1a96b662 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/ClientVersion.java @@ -42,8 +42,9 @@ public enum ClientVersion implements ComponentVersion { "This client version has support for Object Store and File " + "System Optimized Bucket Layouts."), - ERASURE_CODING_READ_CHUNK_CORRUPTION_FIX(4, - "This client version fixes for fixing read corruption in case for EC block reads."), + EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST(4, + "This client version enforces replica index is set for fixing read corruption that could occur when " + + "replicaIndex parameter is not validated before EC block reads."), FUTURE_VERSION(-1, "Used internally when the server side is older and an" + " unknown client version has arrived from the client."); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java index ae56da92ab80..3173fc132071 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java @@ -116,7 +116,7 @@ import static org.apache.hadoop.hdds.scm.utils.ClientCommandsUtils.getReadChunkVersion; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos .ContainerDataProto.State.RECOVERING; -import static org.apache.hadoop.ozone.ClientVersion.ERASURE_CODING_READ_CHUNK_CORRUPTION_FIX; +import static org.apache.hadoop.ozone.ClientVersion.EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST; import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult; import org.apache.ratis.statemachine.StateMachine; @@ -571,6 +571,15 @@ ContainerCommandResponseProto handleEcho( return getEchoResponse(request); } + /** + * Checks if a replicaIndex needs to be checked based on the client version for a request. + * @param request ContainerCommandRequest object. + * @return true if the validation is required for the client version else false. + */ + private boolean replicaIndexCheckRequired(ContainerCommandRequestProto request) { + return request.hasVersion() && request.getVersion() >= EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST.toProtoValue(); + } + /** * Handle Get Block operation. Calls BlockManager to process the request. */ @@ -590,8 +599,7 @@ ContainerCommandResponseProto handleGetBlock( BlockID blockID = BlockID.getFromProtobuf( request.getGetBlock().getBlockID()); responseData = blockManager.getBlock(kvContainer, blockID, - request.hasVersion() && request.getVersion() >= - ERASURE_CODING_READ_CHUNK_CORRUPTION_FIX.toProtoValue()).getProtoBufMessage(); + replicaIndexCheckRequired(request)).getProtoBufMessage(); final long numBytes = responseData.getSerializedSize(); metrics.incContainerBytesStats(Type.GetBlock, numBytes); @@ -707,7 +715,7 @@ ContainerCommandResponseProto handleReadChunk( ChunkInfo chunkInfo = ChunkInfo.getFromProtoBuf(request.getReadChunk() .getChunkData()); Preconditions.checkNotNull(chunkInfo); - if (request.hasVersion() && request.getVersion() >= ERASURE_CODING_READ_CHUNK_CORRUPTION_FIX.toProtoValue()) { + if (replicaIndexCheckRequired(request)) { BlockUtils.verifyReplicaIdx(kvContainer, blockID); } BlockUtils.verifyBCSId(kvContainer, blockID); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java index b353a4d0fe83..0bff809314e1 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java @@ -119,7 +119,7 @@ public void testGetBlockWithReplicaIndexMismatch(ClientVersion clientVersion, in getDummyCommandRequestProto(clientVersion, ContainerProtos.Type.GetBlock, rid), container); assertEquals((replicaIndex > 0 && rid != replicaIndex && clientVersion.toProtoValue() >= - ClientVersion.ERASURE_CODING_READ_CHUNK_CORRUPTION_FIX.toProtoValue()) ? + ClientVersion.EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST.toProtoValue()) ? ContainerProtos.Result.CONTAINER_NOT_FOUND : UNKNOWN_BCSID, response.getResult()); } @@ -162,7 +162,7 @@ public void testReadChunkWithReplicaIndexMismatch(ClientVersion clientVersion, i handler.handleReadChunk(getDummyCommandRequestProto(clientVersion, ContainerProtos.Type.ReadChunk, rid), container, null); assertEquals((replicaIndex > 0 && rid != replicaIndex && - clientVersion.toProtoValue() >= ClientVersion.ERASURE_CODING_READ_CHUNK_CORRUPTION_FIX.toProtoValue()) ? + clientVersion.toProtoValue() >= ClientVersion.EC_REPLICA_INDEX_REQUIRED_IN_BLOCK_REQUEST.toProtoValue()) ? ContainerProtos.Result.CONTAINER_NOT_FOUND : UNKNOWN_BCSID, response.getResult()); } From 040d6d83d2c3a797f8c07b7a01ab8fc8409fa420 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran Date: Wed, 19 Jun 2024 12:54:13 -0700 Subject: [PATCH 21/21] HDDS-10983. Move replica index validation outside block manager Change-Id: If7339e4092797573a63d861092fe053b3653454e --- .../ozone/container/keyvalue/KeyValueHandler.java | 6 ++++-- .../container/keyvalue/impl/BlockManagerImpl.java | 6 +----- .../keyvalue/interfaces/BlockManager.java | 15 +-------------- 3 files changed, 6 insertions(+), 21 deletions(-) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java index 3173fc132071..2da5a05cc58f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java @@ -598,8 +598,10 @@ ContainerCommandResponseProto handleGetBlock( try { BlockID blockID = BlockID.getFromProtobuf( request.getGetBlock().getBlockID()); - responseData = blockManager.getBlock(kvContainer, blockID, - replicaIndexCheckRequired(request)).getProtoBufMessage(); + if (replicaIndexCheckRequired(request)) { + BlockUtils.verifyReplicaIdx(kvContainer, blockID); + } + responseData = blockManager.getBlock(kvContainer, blockID).getProtoBufMessage(); final long numBytes = responseData.getSerializedSize(); metrics.incContainerBytesStats(Type.GetBlock, numBytes); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java index 3a2ac7a49319..9a2cf7c5eacc 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java @@ -207,11 +207,7 @@ public static long persistPutBlock(KeyValueContainer container, } @Override - public BlockData getBlock(Container container, BlockID blockID, boolean isReplicaCheckRequired) - throws IOException { - if (isReplicaCheckRequired) { - BlockUtils.verifyReplicaIdx(container, blockID); - } + public BlockData getBlock(Container container, BlockID blockID) throws IOException { BlockUtils.verifyBCSId(container, blockID); KeyValueContainerData containerData = (KeyValueContainerData) container .getContainerData(); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java index 6ba50d7d4492..3eab64b312ff 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java @@ -55,24 +55,11 @@ long putBlock(Container container, BlockData data, boolean endOfBlock) * * @param container - Container from which block needs to be fetched. * @param blockID - BlockID of the Block. - * @param isReplicaCheckRequired - if true will match container data's replica index with provided BlockID. * @return Block Data. * @throws IOException when BcsId is unknown or mismatched */ - BlockData getBlock(Container container, BlockID blockID, boolean isReplicaCheckRequired) - throws IOException; + BlockData getBlock(Container container, BlockID blockID) throws IOException; - /** - * Gets an existing block. - * - * @param container - Container from which block needs to be fetched. - * @param blockID - BlockID of the Block. - * @return Block Data. - * @throws IOException when BcsId is unknown or mismatched - */ - default BlockData getBlock(Container container, BlockID blockID) throws IOException { - return getBlock(container, blockID, false); - } /** * Deletes an existing block.