diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java index d7076df3ba0..2db21d5ae2d 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java @@ -103,6 +103,7 @@ public class BlockOutputStream extends OutputStream { = new AtomicReference<>(); private final BlockData.Builder containerBlockData; + private final ContainerProtos.StorageTypeProto storageType; private volatile XceiverClientFactory xceiverClientFactory; private XceiverClientSpi xceiverClient; private OzoneClientConfig config; @@ -177,7 +178,8 @@ public BlockOutputStream( OzoneClientConfig config, Token token, ContainerClientMetrics clientMetrics, StreamBufferArgs streamBufferArgs, - Supplier blockOutputStreamResourceProvider + Supplier blockOutputStreamResourceProvider, + ContainerProtos.StorageTypeProto storageType ) throws IOException { this.xceiverClientFactory = xceiverClientManager; this.config = config; @@ -195,6 +197,10 @@ public BlockOutputStream( if (replicationIndex > 0) { blkIDBuilder.setReplicaIndex(replicationIndex); } + if (storageType != null) { + blkIDBuilder.setStorageType(storageType); + } + this.storageType = storageType; this.containerBlockData = BlockData.newBuilder().setBlockID( blkIDBuilder.build()).addMetadata(keyValue); this.pipeline = pipeline; @@ -964,7 +970,8 @@ private CompletableFuture writeChunkToContainer( } asyncReply = writeChunkAsync(xceiverClient, chunkInfo, - blockID.get(), data, tokenString, replicationIndex, blockData, close); + blockID.get(), data, tokenString, replicationIndex, blockData, close, + storageType); CompletableFuture respFuture = asyncReply.getResponse(); validateFuture = respFuture.thenApplyAsync(e -> { diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java index d798b3a9385..5a3f25cc72d 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java @@ -82,10 +82,12 @@ public ECBlockOutputStream( OzoneClientConfig config, Token token, ContainerClientMetrics clientMetrics, StreamBufferArgs streamBufferArgs, - Supplier executorServiceSupplier + Supplier executorServiceSupplier, + ContainerProtos.StorageTypeProto storageType ) throws IOException { super(blockID, -1, xceiverClientManager, - pipeline, bufferPool, config, token, clientMetrics, streamBufferArgs, executorServiceSupplier); + pipeline, bufferPool, config, token, clientMetrics, streamBufferArgs, executorServiceSupplier, + storageType); // In EC stream, there will be only one node in pipeline. this.datanodeDetails = pipeline.getClosestNode(); } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/RatisBlockOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/RatisBlockOutputStream.java index 684707908fb..8713d02d9ba 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/RatisBlockOutputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/RatisBlockOutputStream.java @@ -78,10 +78,12 @@ public RatisBlockOutputStream( OzoneClientConfig config, Token token, ContainerClientMetrics clientMetrics, StreamBufferArgs streamBufferArgs, - Supplier blockOutputStreamResourceProvider + Supplier blockOutputStreamResourceProvider, + org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.StorageTypeProto storageType ) throws IOException { super(blockID, blockSize, xceiverClientManager, pipeline, - bufferPool, config, token, clientMetrics, streamBufferArgs, blockOutputStreamResourceProvider); + bufferPool, config, token, clientMetrics, streamBufferArgs, blockOutputStreamResourceProvider, + storageType); this.commitWatcher = new CommitWatcher(bufferPool, getXceiverClient()); } 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 440b5b3d4d5..c65e4566010 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 @@ -179,7 +179,8 @@ private BlockOutputStream createBlockOutputStream(BufferPool bufferPool) null, ContainerClientMetrics.acquire(), streamBufferArgs, - () -> newFixedThreadPool(10)); + () -> newFixedThreadPool(10), + null); } private ECBlockOutputStream createECBlockOutputStream(OzoneClientConfig clientConfig, @@ -193,7 +194,7 @@ private ECBlockOutputStream createECBlockOutputStream(OzoneClientConfig clientCo StreamBufferArgs.getDefaultStreamBufferArgs(repConfig, clientConfig); return new ECBlockOutputStream(blockID, xcm, pipeline, BufferPool.empty(), clientConfig, null, - clientMetrics, streamBufferArgs, () -> newFixedThreadPool(2)); + clientMetrics, streamBufferArgs, () -> newFixedThreadPool(2), null); } /** diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/hdds/protocol/StorageType.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/StorageType.java similarity index 100% rename from hadoop-ozone/common/src/main/java/org/apache/hadoop/hdds/protocol/StorageType.java rename to hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/StorageType.java diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/PipelineRequestInformation.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/PipelineRequestInformation.java index bef11eb7365..4ba99082caa 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/PipelineRequestInformation.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/PipelineRequestInformation.java @@ -17,17 +17,21 @@ package org.apache.hadoop.hdds.scm; +import org.apache.hadoop.hdds.protocol.StorageType; + /** * The information of the request of pipeline. */ public final class PipelineRequestInformation { private final long size; + private final StorageType storageType; /** * Builder for PipelineRequestInformation. */ public static class Builder { private long size; + private StorageType storageType; public static Builder getBuilder() { return new Builder(); @@ -43,16 +47,26 @@ public Builder setSize(long sz) { return this; } + public Builder setStorageType(StorageType st) { + this.storageType = st; + return this; + } + public PipelineRequestInformation build() { - return new PipelineRequestInformation(size); + return new PipelineRequestInformation(size, storageType); } } - private PipelineRequestInformation(long size) { + private PipelineRequestInformation(long size, StorageType storageType) { this.size = size; + this.storageType = storageType; } public long getSize() { return size; } + + public StorageType getStorageType() { + return storageType; + } } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java index 23400b1a06b..49c767725b0 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java @@ -445,6 +445,11 @@ public final class ScmConfigKeys { public static final String OZONE_SCM_PIPELINE_SCRUB_INTERVAL_DEFAULT = "150s"; + public static final String OZONE_SCM_PIPELINE_CREATION_STORAGE_TYPE_AWARE = + "ozone.scm.pipeline.creation.storage-type-aware.enabled"; + public static final boolean + OZONE_SCM_PIPELINE_CREATION_STORAGE_TYPE_AWARE_DEFAULT = false; + // Allow SCM to auto create factor ONE ratis pipeline. public static final String OZONE_SCM_PIPELINE_AUTO_CREATE_FACTOR_ONE = "ozone.scm.pipeline.creation.auto.factor.one"; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java index c09ba8a2f1d..c431905d564 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerInfo.java @@ -28,6 +28,7 @@ import org.apache.commons.lang3.builder.HashCodeBuilder; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.utils.db.Codec; @@ -87,6 +88,7 @@ public final class ContainerInfo implements Comparable { private long sequenceId; // Health state of the container (determined by ReplicationManager) private ContainerHealthState healthState; + private final StorageType storageType; private ContainerInfo(Builder b) { containerID = ContainerID.valueOf(b.containerID); @@ -102,6 +104,7 @@ private ContainerInfo(Builder b) { replicationConfig = b.replicationConfig; clock = b.clock; healthState = b.healthState != null ? b.healthState : ContainerHealthState.HEALTHY; + storageType = b.storageType; } public static Codec getCodec() { @@ -126,6 +129,9 @@ public static ContainerInfo fromProtobuf(HddsProtos.ContainerInfoProto info) { if (info.hasPipelineID()) { builder.setPipelineID(PipelineID.getFromProtobuf(info.getPipelineID())); } + if (info.hasStorageType()) { + builder.setStorageType(StorageType.valueOf(info.getStorageType())); + } return builder.build(); } @@ -288,9 +294,17 @@ public HddsProtos.ContainerInfoProto getProtobuf() { builder.setPipelineID(getPipelineID().getProtobuf()); } + if (storageType != null) { + builder.setStorageType(storageType.toProto()); + } + return builder.build(); } + public StorageType getStorageType() { + return storageType; + } + public String getOwner() { return owner; } @@ -390,6 +404,7 @@ public static class Builder { private PipelineID pipelineID; private ReplicationConfig replicationConfig; private ContainerHealthState healthState; + private StorageType storageType; public Builder setPipelineID(PipelineID pipelineId) { this.pipelineID = pipelineId; @@ -447,6 +462,11 @@ public Builder setHealthState(ContainerHealthState healthState) { return this; } + public Builder setStorageType(StorageType storageType) { + this.storageType = storageType; + return this; + } + /** * Also resets {@code stateEnterTime}, so make sure to set clock first. */ diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockLocationInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockLocationInfo.java index e84a05b3552..b7ff98d0313 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockLocationInfo.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockLocationInfo.java @@ -19,6 +19,7 @@ import java.util.Objects; import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; import org.apache.hadoop.security.token.Token; @@ -43,6 +44,8 @@ public class BlockLocationInfo { // The block is under construction. Apply to hsynced file last block. private boolean underConstruction; + private StorageType storageType; + protected BlockLocationInfo(Builder builder) { this.blockID = builder.blockID; this.pipeline = builder.pipeline; @@ -51,6 +54,7 @@ protected BlockLocationInfo(Builder builder) { this.token = builder.token; this.partNumber = builder.partNumber; this.createVersion = builder.createVersion; + this.storageType = builder.storageType; } public void setCreateVersion(long version) { @@ -121,6 +125,14 @@ public boolean isUnderConstruction() { return this.underConstruction; } + public StorageType getStorageType() { + return storageType; + } + + public void setStorageType(StorageType storageType) { + this.storageType = storageType; + } + /** * Builder of BlockLocationInfo. */ @@ -132,6 +144,7 @@ public static class Builder { private Pipeline pipeline; private int partNumber; private long createVersion; + private StorageType storageType; public Builder setBlockID(BlockID blockId) { this.blockID = blockId; @@ -168,6 +181,11 @@ public Builder setCreateVersion(long version) { return this; } + public Builder setStorageType(StorageType storageType) { + this.storageType = storageType; + return this; + } + public BlockLocationInfo build() { return new BlockLocationInfo(this); } 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 15879fb4764..d25b8410989 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 @@ -436,17 +436,21 @@ static long getLen(ReadChunkResponseProto response) { public static XceiverClientReply writeChunkAsync( XceiverClientSpi xceiverClient, ChunkInfo chunk, BlockID blockID, ByteString data, String tokenString, - int replicationIndex, BlockData blockData, boolean close) + int replicationIndex, BlockData blockData, boolean close, + ContainerProtos.StorageTypeProto storageType) throws IOException, ExecutionException, InterruptedException { + DatanodeBlockID.Builder blkIDBuilder = DatanodeBlockID.newBuilder() + .setContainerID(blockID.getContainerID()) + .setLocalID(blockID.getLocalID()) + .setBlockCommitSequenceId(blockID.getBlockCommitSequenceId()) + .setReplicaIndex(replicationIndex); + if (storageType != null) { + blkIDBuilder.setStorageType(storageType); + } WriteChunkRequestProto.Builder writeChunkRequest = WriteChunkRequestProto.newBuilder() - .setBlockID(DatanodeBlockID.newBuilder() - .setContainerID(blockID.getContainerID()) - .setLocalID(blockID.getLocalID()) - .setBlockCommitSequenceId(blockID.getBlockCommitSequenceId()) - .setReplicaIndex(replicationIndex) - .build()) + .setBlockID(blkIDBuilder.build()) .setChunkData(chunk) .setData(data); if (blockData != null) { @@ -548,6 +552,15 @@ public static void createRecoveringContainer(XceiverClientSpi client, ContainerProtos.ContainerDataProto.State.RECOVERING, replicaIndex); } + @InterfaceStability.Evolving + public static void createRecoveringContainer(XceiverClientSpi client, + long containerID, String encodedToken, int replicaIndex, + ContainerProtos.StorageTypeProto storageType) throws IOException { + createContainer(client, containerID, encodedToken, + ContainerProtos.ContainerDataProto.State.RECOVERING, replicaIndex, + storageType); + } + /** * createContainer call that creates a container on the datanode. * @param client - client @@ -556,7 +569,7 @@ public static void createRecoveringContainer(XceiverClientSpi client, */ public static void createContainer(XceiverClientSpi client, long containerID, String encodedToken) throws IOException { - createContainer(client, containerID, encodedToken, null, 0); + createContainer(client, containerID, encodedToken, null, 0, null); } /** @@ -571,6 +584,24 @@ public static void createContainer(XceiverClientSpi client, long containerID, String encodedToken, ContainerProtos.ContainerDataProto.State state, int replicaIndex) throws IOException { + createContainer(client, containerID, encodedToken, state, replicaIndex, + null); + } + + /** + * createContainer call that creates a container on the datanode. + * @param client - client + * @param containerID - ID of container + * @param encodedToken - encodedToken if security is enabled + * @param state - state of the container + * @param replicaIndex - index position of the container replica + * @param storageType - storage type for volume selection on the datanode + */ + public static void createContainer(XceiverClientSpi client, + long containerID, String encodedToken, + ContainerProtos.ContainerDataProto.State state, int replicaIndex, + ContainerProtos.StorageTypeProto storageType) + throws IOException { ContainerProtos.CreateContainerRequestProto.Builder createRequest = ContainerProtos.CreateContainerRequestProto.newBuilder(); createRequest @@ -581,6 +612,9 @@ public static void createContainer(XceiverClientSpi client, if (replicaIndex > 0) { createRequest.setReplicaIndex(replicaIndex); } + if (storageType != null) { + createRequest.setStorageType(storageType); + } String id = client.getPipeline().getFirstNode().getUuidString(); ContainerCommandRequestProto.Builder request = diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java index 3bd8388f950..c0f94b4d553 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java @@ -289,6 +289,7 @@ public final class OzoneConsts { public static final String MAX_NUM_OF_BUCKETS = "maxNumOfBuckets"; public static final String HAS_SNAPSHOT = "hasSnapshot"; public static final String STORAGE_TYPE = "storageType"; + public static final String STORAGE_POLICY = "storagePolicy"; public static final String RESOURCE_TYPE = "resourceType"; public static final String IS_VERSION_ENABLED = "isVersionEnabled"; public static final String CREATION_TIME = "creationTime"; diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index d63dedb1541..2d097237b20 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -1693,6 +1693,19 @@ If enabled, SCM will auto create RATIS factor ONE pipeline. + + ozone.scm.pipeline.creation.storage-type-aware.enabled + false + OZONE, SCM, PIPELINE + + If enabled, the background pipeline creator will proactively create + storage-type-constrained pipelines for each StorageType (SSD, DISK, + ARCHIVE) in addition to untyped pipelines. This ensures that + pipelines suitable for HOT/WARM/COLD storage policies are available + when containers need to be allocated. Only enable on clusters that + use storage tiering with mixed StorageType datanodes. + + hdds.scm.safemode.threshold.pct 0.99 @@ -4195,6 +4208,28 @@ + + ozone.storage.policy.enabled + false + OZONE, MANAGEMENT + + When enabled, Ozone Manager will resolve and enforce storage policies + (HOT, WARM, COLD, ALL_SSD) on buckets and keys. When disabled, all + storage policy metadata is ignored and default placement is used. + + + + + ozone.default.storage.policy + WARM + OZONE, MANAGEMENT + + Default storage policy used by Ozone Manager when a client does not + specify a storage policy. Supported values are HOT, WARM, COLD, + ALL_SSD. + + + ozone.client.max.ec.stripe.write.retries 10 diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerInfo.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerInfo.java index f38eceb52ad..35154922a54 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerInfo.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerInfo.java @@ -32,6 +32,7 @@ import org.apache.commons.lang3.builder.HashCodeBuilder; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.RatisReplicationConfig; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.ozone.test.TestClock; @@ -131,6 +132,29 @@ void restoreState() { assertThrows(IllegalStateException.class, subject::revertState); } + @Test + void getProtobufWithStorageType() { + ContainerInfo container = newBuilderForTest() + .setReplicationConfig(RatisReplicationConfig.getInstance(THREE)) + .setStorageType(StorageType.SSD) + .build(); + + HddsProtos.ContainerInfoProto proto = container.getProtobuf(); + ContainerInfo recovered = ContainerInfo.fromProtobuf(proto); + assertEquals(StorageType.SSD, recovered.getStorageType()); + } + + @Test + void getProtobufWithNullStorageType() { + ContainerInfo container = newBuilderForTest() + .setReplicationConfig(RatisReplicationConfig.getInstance(THREE)) + .build(); + + HddsProtos.ContainerInfoProto proto = container.getProtobuf(); + ContainerInfo recovered = ContainerInfo.fromProtobuf(proto); + assertEquals(null, recovered.getStorageType()); + } + public static ContainerInfo.Builder newBuilderForTest() { return new ContainerInfo.Builder() .setContainerID(1234) 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 12fed16b364..f6385b26cbd 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 @@ -495,15 +495,36 @@ ContainerCommandResponseProto createContainer( ContainerProtos.ContainerType.KeyValueContainer; createRequest.setContainerType(containerType); + ContainerProtos.StorageTypeProto storageType = null; if (containerRequest.hasWriteChunk()) { - createRequest.setReplicaIndex( - containerRequest.getWriteChunk().getBlockID().getReplicaIndex()); + ContainerProtos.DatanodeBlockID bid = + containerRequest.getWriteChunk().getBlockID(); + createRequest.setReplicaIndex(bid.getReplicaIndex()); + if (bid.hasStorageType()) { + storageType = bid.getStorageType(); + } } if (containerRequest.hasPutBlock()) { - createRequest.setReplicaIndex( - containerRequest.getPutBlock().getBlockData().getBlockID() - .getReplicaIndex()); + ContainerProtos.DatanodeBlockID bid = + containerRequest.getPutBlock().getBlockData().getBlockID(); + createRequest.setReplicaIndex(bid.getReplicaIndex()); + if (bid.hasStorageType()) { + storageType = bid.getStorageType(); + } + } + + if (containerRequest.hasPutSmallFile()) { + ContainerProtos.DatanodeBlockID bid = + containerRequest.getPutSmallFile() + .getBlock().getBlockData().getBlockID(); + if (bid.hasStorageType()) { + storageType = bid.getStorageType(); + } + } + + if (storageType != null) { + createRequest.setStorageType(storageType); } ContainerCommandRequestProto.Builder requestBuilder = diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECContainerOperationClient.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECContainerOperationClient.java index c967501591e..627bb582af3 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECContainerOperationClient.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECContainerOperationClient.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto.State; import org.apache.hadoop.hdds.scm.XceiverClientManager; @@ -159,12 +160,23 @@ public void deleteContainerInState(long containerID, DatanodeDetails dn, public void createRecoveringContainer(long containerID, DatanodeDetails dn, ECReplicationConfig repConfig, String encodedToken, int replicaIndex) throws IOException { + createRecoveringContainer(containerID, dn, repConfig, encodedToken, + replicaIndex, null); + } + + public void createRecoveringContainer(long containerID, DatanodeDetails dn, + ECReplicationConfig repConfig, String encodedToken, int replicaIndex, + StorageType storageType) throws IOException { XceiverClientSpi xceiverClient = this.xceiverClientManager.acquireClient( singleNodePipeline(dn, repConfig)); try { + ContainerProtos.StorageTypeProto storageTypeProto = + storageType != null + ? ContainerProtos.StorageTypeProto.valueOf(storageType.name()) + : null; ContainerProtocolCalls .createRecoveringContainer(xceiverClient, containerID, encodedToken, - replicaIndex); + replicaIndex, storageTypeProto); } finally { this.xceiverClientManager.releaseClient(xceiverClient, false); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCommandInfo.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCommandInfo.java index 7f86fe8894a..bc739f34a8e 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCommandInfo.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCommandInfo.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hdds.StringUtils; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.ozone.protocol.commands.ReconstructECContainersCommand; import org.apache.hadoop.ozone.protocol.commands.ReconstructECContainersCommand.DatanodeDetailsAndReplicaIndex; @@ -42,6 +43,7 @@ public class ECReconstructionCommandInfo { private final ByteString missingContainerIndexes; private final long deadlineMsSinceEpoch; private final long term; + private final StorageType storageType; public ECReconstructionCommandInfo(ReconstructECContainersCommand cmd) { this.containerID = cmd.getContainerID(); @@ -49,6 +51,7 @@ public ECReconstructionCommandInfo(ReconstructECContainersCommand cmd) { this.missingContainerIndexes = cmd.getMissingContainerIndexes(); this.deadlineMsSinceEpoch = cmd.getDeadline(); this.term = cmd.getTerm(); + this.storageType = cmd.getStorageType(); sourceNodeMap = cmd.getSources().stream() .collect(toMap( @@ -97,4 +100,8 @@ public long getTerm() { return term; } + public StorageType getStorageType() { + return storageType; + } + } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java index 9b869f4a81f..ab0fc173653 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.scm.ContainerClientMetrics; import org.apache.hadoop.hdds.scm.OzoneClientConfig; @@ -145,7 +146,8 @@ public ECReconstructionCoordinator( public void reconstructECContainerGroup(long containerID, ECReplicationConfig repConfig, SortedMap sourceNodeMap, - SortedMap targetNodeMap) throws IOException { + SortedMap targetNodeMap, + StorageType storageType) throws IOException { Pipeline pipeline = rebuildInputPipeline(repConfig, sourceNodeMap); @@ -168,7 +170,7 @@ public void reconstructECContainerGroup(long containerID, containerID, dn, index); containerOperationClient .createRecoveringContainer(containerID, dn, repConfig, - containerToken, index); + containerToken, index, storageType); recoveringContainersCreatedDNs.add(dn); } @@ -232,7 +234,8 @@ private ECBlockOutputStream getECBlockOutputStream( containerOperationClient.singleNodePipeline(datanodeDetails, repConfig, replicaIndex), BufferPool.empty(), ozoneClientConfig, - blockLocationInfo.getToken(), clientMetrics, streamBufferArgs, ecReconstructWriteExecutor); + blockLocationInfo.getToken(), clientMetrics, streamBufferArgs, ecReconstructWriteExecutor, + null); } @VisibleForTesting diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinatorTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinatorTask.java index e9535c6afe8..36c92edd187 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinatorTask.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinatorTask.java @@ -80,7 +80,8 @@ public void runTask() { reconstructionCommandInfo.getContainerID(), reconstructionCommandInfo.getEcReplicationConfig(), reconstructionCommandInfo.getSourceNodeMap(), - reconstructionCommandInfo.getTargetNodeMap()); + reconstructionCommandInfo.getTargetNodeMap(), + reconstructionCommandInfo.getStorageType()); long elapsed = Time.monotonicNow() - start; setStatus(Status.DONE); LOG.info("{} in {} ms", this, elapsed); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java index b8214882f12..782da59a69f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java @@ -53,6 +53,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; import org.apache.commons.io.FileUtils; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileUtil; @@ -148,6 +149,23 @@ public void setCheckChunksFilePath(boolean bCheckChunksDirFilePath) { @Override public void create(VolumeSet volumeSet, VolumeChoosingPolicy volumeChoosingPolicy, String clusterId) throws StorageContainerException { + create(volumeSet, volumeChoosingPolicy, clusterId, null); + } + + /** + * Creates a container, filtering volumes by the requested StorageType + * before choosing a volume. If no volumes match the requested type, + * falls back to all available volumes. + * + * @param volumeSet the set of available volumes + * @param volumeChoosingPolicy policy for choosing among candidate volumes + * @param clusterId the cluster ID + * @param storageType the requested storage type, or null for no filtering + */ + public void create(VolumeSet volumeSet, VolumeChoosingPolicy + volumeChoosingPolicy, String clusterId, + org.apache.hadoop.hdds.protocol.StorageType storageType) + throws StorageContainerException { Objects.requireNonNull(volumeChoosingPolicy, "VolumeChoosingPolicy == null"); Objects.requireNonNull(volumeSet, "volumeSet == null"); Objects.requireNonNull(clusterId, "clusterId == null"); @@ -159,6 +177,20 @@ public void create(VolumeSet volumeSet, VolumeChoosingPolicy try { List volumes = StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()); + if (storageType != null) { + org.apache.hadoop.fs.StorageType fsStorageType = + org.apache.hadoop.fs.StorageType.valueOf(storageType.name()); + List filtered = volumes.stream() + .filter(v -> v.getStorageType() == fsStorageType) + .collect(Collectors.toList()); + if (!filtered.isEmpty()) { + volumes = filtered; + } else { + LOG.warn("No volumes found with storage type {}, falling back to" + + " all volumes for container {}", storageType, + containerData.getContainerID()); + } + } while (true) { HddsVolume containerVolume; String hddsVolumeDir; 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 ef598f3c0cb..3bea204cc2a 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 @@ -100,6 +100,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; @@ -475,12 +476,20 @@ ContainerCommandResponseProto handleCreateContainer( KeyValueContainer newContainer = new KeyValueContainer( newContainerData, conf); + // Extract storageType for volume selection on heterogeneous nodes. + StorageType requestedStorageType = null; + if (request.getCreateContainer().hasStorageType()) { + requestedStorageType = StorageType.valueOf( + request.getCreateContainer().getStorageType().name()); + } + boolean created = false; Lock containerIdLock = containerCreationLocks.get(containerID); containerIdLock.lock(); try { if (containerSet.getContainer(containerID) == null) { - newContainer.create(volumeSet, volumeChoosingPolicy, clusterId); + newContainer.create(volumeSet, volumeChoosingPolicy, clusterId, + requestedStorageType); if (RECOVERING == newContainer.getContainerState()) { created = containerSet.addContainerByOverwriteMissingContainer(newContainer); } else { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java index 7b42006b229..ff441809dad 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerImporter.java @@ -25,7 +25,9 @@ import java.nio.file.Paths; import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Set; +import java.util.stream.Collectors; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; @@ -147,11 +149,30 @@ private static void deleteFileQuietely(Path tarFilePath) { } HddsVolume chooseNextVolume(long spaceToReserve) throws IOException { - // Choose volume that can hold both container in tmp and dest directory - LOG.debug("Choosing volume to reserve space : {}", spaceToReserve); - return volumeChoosingPolicy.chooseVolume( - StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()), - spaceToReserve); + return chooseNextVolume(spaceToReserve, null); + } + + HddsVolume chooseNextVolume(long spaceToReserve, + org.apache.hadoop.hdds.protocol.StorageType storageType) + throws IOException { + LOG.debug("Choosing volume to reserve space : {}, storageType: {}", + spaceToReserve, storageType); + List volumes = + StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList()); + if (storageType != null) { + org.apache.hadoop.fs.StorageType fsStorageType = + org.apache.hadoop.fs.StorageType.valueOf(storageType.name()); + List filtered = volumes.stream() + .filter(v -> v.getStorageType() == fsStorageType) + .collect(Collectors.toList()); + if (!filtered.isEmpty()) { + volumes = filtered; + } else { + LOG.warn("No volumes found with storage type {}, falling back to" + + " all volumes for replication", storageType); + } + } + return volumeChoosingPolicy.chooseVolume(volumes, spaceToReserve); } public static Path getUntarDirectory(HddsVolume hddsVolume) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java index 55874511ba7..e8f26c208bb 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ContainerUploader.java @@ -21,6 +21,7 @@ import java.io.OutputStream; import java.util.concurrent.CompletableFuture; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; /** * Client-side interface for sending a container to a target datanode. @@ -29,4 +30,10 @@ public interface ContainerUploader { OutputStream startUpload(long containerId, DatanodeDetails target, CompletableFuture callback, CopyContainerCompression compression) throws IOException; + + default OutputStream startUpload(long containerId, DatanodeDetails target, + CompletableFuture callback, CopyContainerCompression compression, + StorageType storageType) throws IOException { + return startUpload(containerId, target, callback, compression); + } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java index 2457b592b14..a68276e4eff 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/DownloadAndImportReplicator.java @@ -74,7 +74,8 @@ public void replicate(ReplicationTask task) { try { targetVolume = containerImporter.chooseNextVolume( - containerImporter.getDefaultReplicationSpace()); + containerImporter.getDefaultReplicationSpace(), + task.getStorageType()); // Wait for the download. This thread pool is limiting the parallel // downloads, so it's ok to block here and wait for the full download. diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java index 64adcb6c616..f342596e62e 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcContainerUploader.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.SendContainerRequest; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.SendContainerResponse; import org.apache.hadoop.hdds.security.SecurityConfig; @@ -59,8 +60,16 @@ public GrpcContainerUploader( @Override public OutputStream startUpload(long containerId, DatanodeDetails target, - CompletableFuture callback, CopyContainerCompression compression) throws IOException { - + CompletableFuture callback, CopyContainerCompression compression) + throws IOException { + return startUpload(containerId, target, callback, compression, null); + } + + @Override + public OutputStream startUpload(long containerId, DatanodeDetails target, + CompletableFuture callback, CopyContainerCompression compression, + StorageType storageType) throws IOException { + // Get container size from local datanode instead of using passed replicateSize Long containerSize = null; Container container = containerController.getContainer(containerId); @@ -69,7 +78,7 @@ public OutputStream startUpload(long containerId, DatanodeDetails target, containerId, target, container.getContainerData().getBytesUsed()); containerSize = container.getContainerData().getBytesUsed(); } - + GrpcReplicationClient client = createReplicationClient(target, compression); try { // gRPC runtime always provides implementation of CallStreamObserver @@ -82,7 +91,8 @@ public OutputStream startUpload(long containerId, DatanodeDetails target, (CallStreamObserver) client.upload( responseObserver), responseObserver); return new SendContainerOutputStream(requestStream, containerId, - GrpcReplicationService.BUFFER_SIZE, compression, containerSize) { + GrpcReplicationService.BUFFER_SIZE, compression, containerSize, + storageType) { @Override public void close() throws IOException { try { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java index 759aff722ba..2a03ffdbd47 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/PushReplicator.java @@ -61,7 +61,8 @@ public void replicate(ReplicationTask task) { CountingOutputStream output = null; try { output = new CountingOutputStream( - uploader.startUpload(containerID, target, fut, compression)); + uploader.startUpload(containerID, target, fut, compression, + task.getStorageType())); source.copyData(containerID, output, compression); fut.get(); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationTask.java index a32e9b41ab1..85155c1efc4 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationTask.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/ReplicationTask.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.Objects; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand; /** @@ -129,6 +130,10 @@ DatanodeDetails getTarget() { return cmd.getTargetDatanode(); } + public StorageType getStorageType() { + return cmd.getStorageType(); + } + @Override public void runTask() { replicator.replicate(this); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java index 3bb7e463d9d..a3e15193b91 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerOutputStream.java @@ -17,6 +17,8 @@ package org.apache.hadoop.ozone.container.replication; +import org.apache.hadoop.hdds.protocol.StorageType; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.SendContainerRequest; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.io.grpc.stub.CallStreamObserver; @@ -28,14 +30,23 @@ class SendContainerOutputStream extends GrpcOutputStream { private final CopyContainerCompression compression; private final Long size; + private final StorageType storageType; SendContainerOutputStream( CallStreamObserver streamObserver, long containerId, int bufferSize, CopyContainerCompression compression, Long size) { + this(streamObserver, containerId, bufferSize, compression, size, null); + } + + SendContainerOutputStream( + CallStreamObserver streamObserver, + long containerId, int bufferSize, CopyContainerCompression compression, + Long size, StorageType storageType) { super(streamObserver, containerId, bufferSize); this.compression = compression; this.size = size; + this.storageType = storageType; } @Override @@ -45,10 +56,16 @@ protected void sendPart(boolean eof, int length, ByteString data) { .setData(data) .setOffset(getWrittenBytes()) .setCompression(compression.toProto()); - - // Include container size in the first request - if (getWrittenBytes() == 0 && size != null) { - requestBuilder.setSize(size); + + // Include container size and storageType in the first request + if (getWrittenBytes() == 0) { + if (size != null) { + requestBuilder.setSize(size); + } + if (storageType != null) { + requestBuilder.setStorageType( + ContainerProtos.StorageTypeProto.valueOf(storageType.name())); + } } getStreamObserver().onNext(requestBuilder.build()); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java index 0824341127c..309c0fab86e 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SendContainerRequestHandler.java @@ -23,6 +23,7 @@ import java.io.OutputStream; import java.nio.file.Files; import java.nio.file.Path; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.SendContainerRequest; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.SendContainerResponse; @@ -55,6 +56,7 @@ class SendContainerRequestHandler private CopyContainerCompression compression; private final ZeroCopyMessageMarshaller marshaller; private long spaceToReserve = 0; + private StorageType storageType; SendContainerRequestHandler( ContainerImporter importer, @@ -85,12 +87,16 @@ public void onNext(SendContainerRequest req) { if (containerId == -1) { containerId = req.getContainerID(); - + // Use container size if available, otherwise fall back to default spaceToReserve = importer.getSpaceToReserve( req.hasSize() ? req.getSize() : null); - volume = importer.chooseNextVolume(spaceToReserve); + if (req.hasStorageType()) { + storageType = StorageType.valueOf( + req.getStorageType().name()); + } + volume = importer.chooseNextVolume(spaceToReserve, storageType); Path dir = ContainerImporter.getUntarDirectory(volume); Files.createDirectories(dir); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReconstructECContainersCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReconstructECContainersCommand.java index 59f606608d0..b957775c4bf 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReconstructECContainersCommand.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReconstructECContainersCommand.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.HddsIdFactory; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReconstructECContainersCommandProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReconstructECContainersCommandProto.Builder; @@ -40,25 +41,36 @@ public class ReconstructECContainersCommand private final List targetDatanodes; private final ByteString missingContainerIndexes; private final ECReplicationConfig ecReplicationConfig; + private final StorageType storageType; public ReconstructECContainersCommand(long containerID, List sources, List targetDatanodes, ByteString missingContainerIndexes, ECReplicationConfig ecReplicationConfig) { this(containerID, sources, targetDatanodes, missingContainerIndexes, - ecReplicationConfig, HddsIdFactory.getLongId()); + ecReplicationConfig, HddsIdFactory.getLongId(), null); } public ReconstructECContainersCommand(long containerID, List sourceDatanodes, List targetDatanodes, ByteString missingContainerIndexes, ECReplicationConfig ecReplicationConfig, long id) { + this(containerID, sourceDatanodes, targetDatanodes, + missingContainerIndexes, ecReplicationConfig, id, null); + } + + public ReconstructECContainersCommand(long containerID, + List sourceDatanodes, + List targetDatanodes, ByteString missingContainerIndexes, + ECReplicationConfig ecReplicationConfig, long id, + StorageType storageType) { super(id); this.containerID = containerID; this.sources = sourceDatanodes; this.targetDatanodes = targetDatanodes; this.missingContainerIndexes = missingContainerIndexes; this.ecReplicationConfig = ecReplicationConfig; + this.storageType = storageType; if (targetDatanodes.size() != missingContainerIndexes.size()) { throw new IllegalArgumentException("Number of target datanodes and " + "container indexes should be same"); @@ -83,6 +95,9 @@ public ReconstructECContainersCommandProto getProto() { } builder.setMissingContainerIndexes(missingContainerIndexes); builder.setEcReplicationConfig(ecReplicationConfig.toProto()); + if (storageType != null) { + builder.setStorageType(storageType.toProto()); + } return builder.build(); } @@ -98,11 +113,13 @@ public static ReconstructECContainersCommand getFromProtobuf( protoMessage.getTargetsList().stream() .map(DatanodeDetails::getFromProtoBuf).collect(Collectors.toList()); + StorageType st = protoMessage.hasStorageType() + ? StorageType.valueOf(protoMessage.getStorageType()) : null; return new ReconstructECContainersCommand(protoMessage.getContainerID(), srcDatanodeDetails, targetDatanodeDetails, protoMessage.getMissingContainerIndexes(), new ECReplicationConfig(protoMessage.getEcReplicationConfig()), - protoMessage.getCmdId()); + protoMessage.getCmdId(), st); } public long getContainerID() { @@ -125,6 +142,10 @@ public ECReplicationConfig getEcReplicationConfig() { return ecReplicationConfig; } + public StorageType getStorageType() { + return storageType; + } + @Override public String toString() { StringBuilder sb = new StringBuilder(); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java index 8574909b386..6959086053b 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ReplicateContainerCommand.java @@ -23,6 +23,7 @@ import java.util.Objects; import java.util.stream.Collectors; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReplicateContainerCommandProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReplicateContainerCommandProto.Builder; @@ -42,6 +43,7 @@ public final class ReplicateContainerCommand private int replicaIndex = 0; private ReplicationCommandPriority priority = ReplicationCommandPriority.NORMAL; + private StorageType storageType; public static ReplicateContainerCommand fromSources(long containerID, List sourceDatanodes) { @@ -82,6 +84,14 @@ public void setPriority(ReplicationCommandPriority priority) { this.priority = priority; } + public void setStorageType(StorageType storageType) { + this.storageType = storageType; + } + + public StorageType getStorageType() { + return storageType; + } + @Override public Type getType() { return SCMCommandProto.Type.replicateContainerCommand; @@ -105,6 +115,9 @@ public ReplicateContainerCommandProto getProto() { builder.setTarget(targetDatanode.getProtoBufMessage()); } builder.setPriority(priority); + if (storageType != null) { + builder.setStorageType(storageType.toProto()); + } return builder.build(); } @@ -131,6 +144,9 @@ public static ReplicateContainerCommand getFromProtobuf( if (protoMessage.hasPriority()) { cmd.setPriority(protoMessage.getPriority()); } + if (protoMessage.hasStorageType()) { + cmd.setStorageType(StorageType.valueOf(protoMessage.getStorageType())); + } return cmd; } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java index 9e66aaeb067..b0f6e282b5c 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java @@ -200,6 +200,202 @@ private void testCreateContainer() throws StorageContainerException { "DB does not exist"); } + @ContainerTestVersionInfo.ContainerTest + public void testCreateContainerWithStorageTypeFiltering( + ContainerTestVersionInfo versionInfo) throws Exception { + init(versionInfo); + + // Create two volumes: one SSD, one DISK + File ssdDir = new File(folder, "ssd"); + File diskDir = new File(folder, "disk"); + assertTrue(ssdDir.mkdirs()); + assertTrue(diskDir.mkdirs()); + + HddsVolume ssdVolume = new HddsVolume.Builder(ssdDir.toString()) + .conf(CONF) + .datanodeUuid(datanodeId.toString()) + .storageType(org.apache.hadoop.fs.StorageType.SSD) + .build(); + HddsVolume diskVolume = new HddsVolume.Builder(diskDir.toString()) + .conf(CONF) + .datanodeUuid(datanodeId.toString()) + .storageType(org.apache.hadoop.fs.StorageType.DISK) + .build(); + + StorageVolumeUtil.checkVolume(ssdVolume, scmId, scmId, CONF, null, null); + StorageVolumeUtil.checkVolume(diskVolume, scmId, scmId, CONF, null, null); + + List mixedVolumes = new ArrayList<>(); + mixedVolumes.add(ssdVolume); + mixedVolumes.add(diskVolume); + + VolumeSet mixedVolumeSet = mock(MutableVolumeSet.class); + when(mixedVolumeSet.getVolumesList()) + .thenAnswer(i -> mixedVolumes.stream() + .map(v -> (StorageVolume) v) + .collect(Collectors.toList())); + + // volumeChoosingPolicy returns the first volume from the filtered list + RoundRobinVolumeChoosingPolicy policy = + mock(RoundRobinVolumeChoosingPolicy.class); + when(policy.chooseVolume(anyList(), anyLong())).thenAnswer( + invocation -> { + List volumes = invocation.getArgument(0); + return volumes.get(0); + }); + + // Request SSD storage type - should only see ssdVolume + KeyValueContainerData ssdContainerData = new KeyValueContainerData(100L, + layout, + (long) StorageUnit.GB.toBytes(5), UUID.randomUUID().toString(), + datanodeId.toString()); + KeyValueContainer ssdContainer = + new KeyValueContainer(ssdContainerData, CONF); + ssdContainer.create(mixedVolumeSet, policy, scmId, + org.apache.hadoop.hdds.protocol.StorageType.SSD); + + assertEquals(ssdVolume, ssdContainerData.getVolume()); + + // Request DISK storage type - should only see diskVolume + KeyValueContainerData diskContainerData = new KeyValueContainerData(101L, + layout, + (long) StorageUnit.GB.toBytes(5), UUID.randomUUID().toString(), + datanodeId.toString()); + KeyValueContainer diskContainer = + new KeyValueContainer(diskContainerData, CONF); + diskContainer.create(mixedVolumeSet, policy, scmId, + org.apache.hadoop.hdds.protocol.StorageType.DISK); + + assertEquals(diskVolume, diskContainerData.getVolume()); + } + + @ContainerTestVersionInfo.ContainerTest + public void testCreateContainerWithStorageTypeFallback( + ContainerTestVersionInfo versionInfo) throws Exception { + init(versionInfo); + + // Create only DISK volumes - no SSD available + File diskDir = new File(folder, "diskonly"); + assertTrue(diskDir.mkdirs()); + + HddsVolume diskVolume = new HddsVolume.Builder(diskDir.toString()) + .conf(CONF) + .datanodeUuid(datanodeId.toString()) + .storageType(org.apache.hadoop.fs.StorageType.DISK) + .build(); + StorageVolumeUtil.checkVolume(diskVolume, scmId, scmId, CONF, null, null); + + List diskOnlyVolumes = new ArrayList<>(); + diskOnlyVolumes.add(diskVolume); + + VolumeSet diskOnlyVolumeSet = mock(MutableVolumeSet.class); + when(diskOnlyVolumeSet.getVolumesList()) + .thenAnswer(i -> diskOnlyVolumes.stream() + .map(v -> (StorageVolume) v) + .collect(Collectors.toList())); + + RoundRobinVolumeChoosingPolicy policy = + mock(RoundRobinVolumeChoosingPolicy.class); + when(policy.chooseVolume(anyList(), anyLong())).thenAnswer( + invocation -> { + List volumes = invocation.getArgument(0); + return volumes.get(0); + }); + + // Request SSD but only DISK is available - should fall back to DISK + KeyValueContainerData fallbackData = new KeyValueContainerData(102L, + layout, + (long) StorageUnit.GB.toBytes(5), UUID.randomUUID().toString(), + datanodeId.toString()); + KeyValueContainer fallbackContainer = + new KeyValueContainer(fallbackData, CONF); + fallbackContainer.create(diskOnlyVolumeSet, policy, scmId, + org.apache.hadoop.hdds.protocol.StorageType.SSD); + + // Should succeed and use the DISK volume as fallback + assertEquals(diskVolume, fallbackData.getVolume()); + } + + @ContainerTestVersionInfo.ContainerTest + public void testCreateContainerWithNullStorageType( + ContainerTestVersionInfo versionInfo) throws Exception { + init(versionInfo); + // Null storageType should behave identically to the original create() + keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId, null); + keyValueContainerData = keyValueContainer.getContainerData(); + + assertNotNull(keyValueContainerData.getMetadataPath()); + assertNotNull(keyValueContainerData.getChunksPath()); + assertTrue(keyValueContainer.getContainerFile().exists()); + assertTrue(keyValueContainer.getContainerDBFile().exists()); + } + + @ContainerTestVersionInfo.ContainerTest + public void testCreateContainerFilteringPassesOnlyMatchingVolumes( + ContainerTestVersionInfo versionInfo) throws Exception { + init(versionInfo); + + // Create 2 SSD + 1 DISK volumes + File ssd1Dir = new File(folder, "ssd1"); + File ssd2Dir = new File(folder, "ssd2"); + File diskDir = new File(folder, "disk2"); + assertTrue(ssd1Dir.mkdirs()); + assertTrue(ssd2Dir.mkdirs()); + assertTrue(diskDir.mkdirs()); + + HddsVolume ssd1 = new HddsVolume.Builder(ssd1Dir.toString()) + .conf(CONF).datanodeUuid(datanodeId.toString()) + .storageType(org.apache.hadoop.fs.StorageType.SSD).build(); + HddsVolume ssd2 = new HddsVolume.Builder(ssd2Dir.toString()) + .conf(CONF).datanodeUuid(datanodeId.toString()) + .storageType(org.apache.hadoop.fs.StorageType.SSD).build(); + HddsVolume disk = new HddsVolume.Builder(diskDir.toString()) + .conf(CONF).datanodeUuid(datanodeId.toString()) + .storageType(org.apache.hadoop.fs.StorageType.DISK).build(); + + StorageVolumeUtil.checkVolume(ssd1, scmId, scmId, CONF, null, null); + StorageVolumeUtil.checkVolume(ssd2, scmId, scmId, CONF, null, null); + StorageVolumeUtil.checkVolume(disk, scmId, scmId, CONF, null, null); + + List allVolumes = new ArrayList<>(); + allVolumes.add(ssd1); + allVolumes.add(ssd2); + allVolumes.add(disk); + + VolumeSet vs = mock(MutableVolumeSet.class); + when(vs.getVolumesList()) + .thenAnswer(i -> allVolumes.stream() + .map(v -> (StorageVolume) v) + .collect(Collectors.toList())); + + // Capture which volumes the policy actually sees + List> capturedVolumeLists = new ArrayList<>(); + RoundRobinVolumeChoosingPolicy policy = + mock(RoundRobinVolumeChoosingPolicy.class); + when(policy.chooseVolume(anyList(), anyLong())).thenAnswer( + invocation -> { + List volumes = invocation.getArgument(0); + capturedVolumeLists.add(new ArrayList<>(volumes)); + return volumes.get(0); + }); + + KeyValueContainerData data = new KeyValueContainerData(200L, + layout, + (long) StorageUnit.GB.toBytes(5), UUID.randomUUID().toString(), + datanodeId.toString()); + KeyValueContainer container = new KeyValueContainer(data, CONF); + container.create(vs, policy, scmId, + org.apache.hadoop.hdds.protocol.StorageType.SSD); + + // Policy should have received only the 2 SSD volumes, not the DISK one + assertEquals(1, capturedVolumeLists.size()); + List receivedVolumes = capturedVolumeLists.get(0); + assertEquals(2, receivedVolumes.size()); + assertTrue(receivedVolumes.contains(ssd1)); + assertTrue(receivedVolumes.contains(ssd2)); + assertFalse(receivedVolumes.contains(disk)); + } + /** * Tests repair of containers affected by the bug reported in HDDS-6235. */ 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 0385564ebf7..e64a166256f 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 @@ -923,6 +923,190 @@ private static ContainerCommandRequestProto createContainerRequest( .build(); } + @Test + public void testCreateContainerWithStorageType() throws Exception { + final String clusterId = UUID.randomUUID().toString(); + final String datanodeId = UUID.randomUUID().toString(); + conf = new OzoneConfiguration(); + + // Create SSD and DISK volumes + Path ssdPath = tempDir.resolve("ssd"); + Path diskPath = tempDir.resolve("disk"); + Files.createDirectories(ssdPath); + Files.createDirectories(diskPath); + + final ContainerSet containerSet = spy(newContainerSet()); + final MutableVolumeSet volumeSet = mock(MutableVolumeSet.class); + + HddsVolume ssdVolume = new HddsVolume.Builder(ssdPath.toString()) + .conf(conf).clusterID(clusterId).datanodeUuid(datanodeId) + .storageType(org.apache.hadoop.fs.StorageType.SSD) + .volumeSet(volumeSet).build(); + ssdVolume.format(clusterId); + ssdVolume.createWorkingDir(clusterId, null); + ssdVolume.createTmpDirs(clusterId); + + HddsVolume diskVolume = new HddsVolume.Builder(diskPath.toString()) + .conf(conf).clusterID(clusterId).datanodeUuid(datanodeId) + .storageType(org.apache.hadoop.fs.StorageType.DISK) + .volumeSet(volumeSet).build(); + diskVolume.format(clusterId); + diskVolume.createWorkingDir(clusterId, null); + diskVolume.createTmpDirs(clusterId); + + when(volumeSet.getVolumesList()) + .thenReturn(java.util.Arrays.asList(ssdVolume, diskVolume)); + + final ContainerMetrics metrics = ContainerMetrics.create(conf); + try { + final AtomicInteger icrReceived = new AtomicInteger(0); + final KeyValueHandler kvHandler = new KeyValueHandler(conf, + datanodeId, containerSet, volumeSet, metrics, + c -> icrReceived.incrementAndGet(), + new ContainerChecksumTreeManager(conf)); + kvHandler.setClusterID(clusterId); + + // Create container with SSD storageType + ContainerCommandRequestProto ssdRequest = + ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.CreateContainer) + .setDatanodeUuid(datanodeId) + .setCreateContainer( + ContainerProtos.CreateContainerRequestProto.newBuilder() + .setContainerType(ContainerType.KeyValueContainer) + .setStorageType(ContainerProtos.StorageTypeProto.SSD) + .build()) + .setContainerID(1L) + .setPipelineID(UUID.randomUUID().toString()) + .build(); + + ContainerCommandResponseProto response = + kvHandler.handleCreateContainer(ssdRequest, null); + assertEquals(ContainerProtos.Result.SUCCESS, response.getResult()); + + // Verify container was placed on SSD volume + Container createdContainer = containerSet.getContainer(1L); + assertNotNull(createdContainer); + assertEquals(ssdVolume, + createdContainer.getContainerData().getVolume()); + + // Create container with DISK storageType + ContainerCommandRequestProto diskRequest = + ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.CreateContainer) + .setDatanodeUuid(datanodeId) + .setCreateContainer( + ContainerProtos.CreateContainerRequestProto.newBuilder() + .setContainerType(ContainerType.KeyValueContainer) + .setStorageType(ContainerProtos.StorageTypeProto.DISK) + .build()) + .setContainerID(2L) + .setPipelineID(UUID.randomUUID().toString()) + .build(); + + response = kvHandler.handleCreateContainer(diskRequest, null); + assertEquals(ContainerProtos.Result.SUCCESS, response.getResult()); + + Container diskContainer = containerSet.getContainer(2L); + assertNotNull(diskContainer); + assertEquals(diskVolume, + diskContainer.getContainerData().getVolume()); + } finally { + ssdVolume.getVolumeInfoStats().unregister(); + ssdVolume.getVolumeIOStats().unregister(); + diskVolume.getVolumeInfoStats().unregister(); + diskVolume.getVolumeIOStats().unregister(); + ContainerMetrics.remove(); + } + } + + @Test + public void testStorageTypeProtoSerialization() { + // Verify storageType field round-trips in the proto correctly + ContainerProtos.CreateContainerRequestProto withSsd = + ContainerProtos.CreateContainerRequestProto.newBuilder() + .setContainerType(ContainerType.KeyValueContainer) + .setStorageType(ContainerProtos.StorageTypeProto.SSD) + .build(); + assertTrue(withSsd.hasStorageType()); + assertEquals(ContainerProtos.StorageTypeProto.SSD, + withSsd.getStorageType()); + + ContainerProtos.CreateContainerRequestProto withDisk = + ContainerProtos.CreateContainerRequestProto.newBuilder() + .setContainerType(ContainerType.KeyValueContainer) + .setStorageType(ContainerProtos.StorageTypeProto.DISK) + .build(); + assertEquals(ContainerProtos.StorageTypeProto.DISK, + withDisk.getStorageType()); + + // Without storageType set - backward compatibility + ContainerProtos.CreateContainerRequestProto noStorageType = + ContainerProtos.CreateContainerRequestProto.newBuilder() + .setContainerType(ContainerType.KeyValueContainer) + .build(); + assertFalse(noStorageType.hasStorageType()); + + // Verify round-trip through serialization + byte[] bytes = withSsd.toByteArray(); + try { + ContainerProtos.CreateContainerRequestProto deserialized = + ContainerProtos.CreateContainerRequestProto.parseFrom(bytes); + assertTrue(deserialized.hasStorageType()); + assertEquals(ContainerProtos.StorageTypeProto.SSD, + deserialized.getStorageType()); + } catch (Exception e) { + fail("Proto round-trip failed: " + e.getMessage()); + } + } + + @Test + public void testCreateContainerWithoutStorageType() throws Exception { + final String clusterId = UUID.randomUUID().toString(); + final String datanodeId = UUID.randomUUID().toString(); + conf = new OzoneConfiguration(); + + final ContainerSet containerSet = spy(newContainerSet()); + final MutableVolumeSet volumeSet = mock(MutableVolumeSet.class); + + HddsVolume hddsVolume = new HddsVolume.Builder(tempDir.toString()) + .conf(conf).clusterID(clusterId).datanodeUuid(datanodeId) + .volumeSet(volumeSet).build(); + hddsVolume.format(clusterId); + hddsVolume.createWorkingDir(clusterId, null); + hddsVolume.createTmpDirs(clusterId); + + when(volumeSet.getVolumesList()) + .thenReturn(Collections.singletonList(hddsVolume)); + + final ContainerMetrics metrics = ContainerMetrics.create(conf); + try { + final AtomicInteger icrReceived = new AtomicInteger(0); + final KeyValueHandler kvHandler = new KeyValueHandler(conf, + datanodeId, containerSet, volumeSet, metrics, + c -> icrReceived.incrementAndGet(), + new ContainerChecksumTreeManager(conf)); + kvHandler.setClusterID(clusterId); + + // Create container without storageType (original behavior) + ContainerCommandRequestProto request = createContainerRequest( + datanodeId, 1L); + + ContainerCommandResponseProto response = + kvHandler.handleCreateContainer(request, null); + assertEquals(ContainerProtos.Result.SUCCESS, response.getResult()); + + Container createdContainer = containerSet.getContainer(1L); + assertNotNull(createdContainer); + assertEquals(hddsVolume, + createdContainer.getContainerData().getVolume()); + } finally { + hddsVolume.getVolumeInfoStats().unregister(); + hddsVolume.getVolumeIOStats().unregister(); + ContainerMetrics.remove(); + } + } + private KeyValueHandler createKeyValueHandler(Path path) throws IOException { final ContainerSet containerSet = newContainerSet(); final MutableVolumeSet volumeSet = mock(MutableVolumeSet.class); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerImporter.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerImporter.java index e594dad3e58..ad3a420a6c6 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerImporter.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestContainerImporter.java @@ -22,6 +22,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.atLeastOnce; @@ -49,6 +50,7 @@ import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream; import org.apache.commons.io.IOUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; @@ -234,6 +236,65 @@ public void testImportContainerResetsLastScanTime() throws Exception { assertEquals(Optional.empty(), containerData.lastDataScanTime()); } + @Test + void testChooseNextVolumeWithStorageType() throws IOException { + // Set up two volumes: SSD and DISK + File ssdDir = new File(tempDir, "ssd"); + File diskDir = new File(tempDir, "disk"); + assertTrue(ssdDir.mkdirs()); + assertTrue(diskDir.mkdirs()); + + OzoneConfiguration typedConf = new OzoneConfiguration(); + typedConf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, + "[SSD]" + ssdDir.getAbsolutePath() + "," + + "[DISK]" + diskDir.getAbsolutePath()); + VolumeChoosingPolicy policy = VolumeChoosingPolicyFactory.getPolicy( + typedConf); + MutableVolumeSet typedVolumeSet = new MutableVolumeSet("test", typedConf, + null, StorageVolume.VolumeType.DATA_VOLUME, null); + ContainerImporter typedImporter = new ContainerImporter(typedConf, + containerSet, controllerMock, typedVolumeSet, policy); + + // Choose SSD volume + HddsVolume ssdVolume = typedImporter.chooseNextVolume(1, StorageType.SSD); + assertEquals(org.apache.hadoop.fs.StorageType.SSD, + ssdVolume.getStorageType()); + + // Choose DISK volume + HddsVolume diskVolume = typedImporter.chooseNextVolume(1, StorageType.DISK); + assertEquals(org.apache.hadoop.fs.StorageType.DISK, + diskVolume.getStorageType()); + } + + @Test + void testChooseNextVolumeFallback() throws IOException { + // Set up only DISK volumes + File diskDir = new File(tempDir, "disk-only"); + assertTrue(diskDir.mkdirs()); + + OzoneConfiguration diskOnlyConf = new OzoneConfiguration(); + diskOnlyConf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, + "[DISK]" + diskDir.getAbsolutePath()); + VolumeChoosingPolicy policy = VolumeChoosingPolicyFactory.getPolicy( + diskOnlyConf); + MutableVolumeSet diskOnlyVolumeSet = new MutableVolumeSet("test", + diskOnlyConf, null, StorageVolume.VolumeType.DATA_VOLUME, null); + ContainerImporter diskOnlyImporter = new ContainerImporter(diskOnlyConf, + containerSet, controllerMock, diskOnlyVolumeSet, policy); + + // Request SSD but only DISK available — should fall back + HddsVolume volume = diskOnlyImporter.chooseNextVolume(1, StorageType.SSD); + assertEquals(org.apache.hadoop.fs.StorageType.DISK, + volume.getStorageType()); + } + + @Test + void testChooseNextVolumeNullStorageType() throws IOException { + // null storageType should return any volume without filtering + HddsVolume volume = containerImporter.chooseNextVolume(1, null); + assertThat(volume).isNotNull(); + } + private File containerTarFile(long id, ContainerData data) throws IOException { File yamlFile = new File(tempDir, "container.yaml"); ContainerDataYaml.createContainerFile(data, yamlFile); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java index c690b50d642..7ada873c9a0 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestDownloadAndImportReplicator.java @@ -22,7 +22,10 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.io.File; @@ -33,6 +36,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy; @@ -41,6 +45,7 @@ import org.apache.hadoop.ozone.container.common.volume.StorageVolume; import org.apache.hadoop.ozone.container.common.volume.VolumeChoosingPolicyFactory; import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController; +import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand; import org.apache.ozone.test.GenericTestUtils; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -116,4 +121,41 @@ public void testCommitSpaceReleasedOnReplicationFailure() throws Exception { // Verify commit space is released assertEquals(initialCommittedBytes, volume.getCommittedBytes()); } + + @Test + public void testPullReplicatorPassesStorageType() throws Exception { + long containerId = 2; + // Create a spy importer to verify method calls + OzoneConfiguration spyConf = new OzoneConfiguration(); + spyConf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, + tempDir.getAbsolutePath()); + VolumeChoosingPolicy volumeChoosingPolicy = + VolumeChoosingPolicyFactory.getPolicy(spyConf); + ContainerSet containerSet = newContainerSet(0); + ContainerImporter importerSpy = spy(new ContainerImporter(spyConf, + containerSet, mock(ContainerController.class), volumeSet, + volumeChoosingPolicy)); + + DownloadAndImportReplicator pullReplicator = + new DownloadAndImportReplicator(spyConf, containerSet, importerSpy, + downloader); + + // Mock downloader to return null (download failure); + // chooseNextVolume is called before download + when(downloader.getContainerDataFromReplicas(anyLong(), any(), any(), + any())) + .thenReturn(null); + + ReplicateContainerCommand cmd = + ReplicateContainerCommand.fromSources(containerId, + Collections.singletonList(mock(DatanodeDetails.class))); + cmd.setStorageType(StorageType.ARCHIVE); + ReplicationTask task = new ReplicationTask(cmd, pullReplicator); + + pullReplicator.replicate(task); + + // Verify chooseNextVolume was called with ARCHIVE + verify(importerSpy).chooseNextVolume(anyLong(), + eq(StorageType.ARCHIVE)); + } } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java index 8b831fa0646..e14a01b3d69 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestGrpcReplicationService.java @@ -150,9 +150,10 @@ public void init() throws Exception { }).when(importer).importContainer(anyLong(), any(), any(), any()); doReturn(true).when(importer).isAllowedContainerImport(eq( CONTAINER_ID)); - when(importer.chooseNextVolume(anyLong())).thenReturn(new HddsVolume.Builder( - Files.createDirectory(tempDir.resolve("ImporterDir")).toString()).conf( - conf).build()); + when(importer.chooseNextVolume(anyLong(), any())).thenReturn( + new HddsVolume.Builder( + Files.createDirectory(tempDir.resolve("ImporterDir")).toString()) + .conf(conf).build()); replicationServer = new ReplicationServer(containerController, replicationConfig, secConf, diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java index a4463410cea..44b37dffa45 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestPushReplicator.java @@ -25,6 +25,7 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.io.IOException; @@ -35,7 +36,9 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.ozone.container.replication.AbstractReplicationTask.Status; +import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand; import org.apache.ozone.test.SpyOutputStream; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -122,6 +125,45 @@ void packFailsWithException() throws IOException { output.assertClosedExactlyOnce(); } + @Test + void pushReplicatorPassesStorageType() throws IOException { + // GIVEN + long containerID = randomContainerID(); + DatanodeDetails target = MockDatanodeDetails.randomDatanodeDetails(); + SpyOutputStream output = new SpyOutputStream(NULL_OUTPUT_STREAM); + + ContainerReplicationSource source = mock(ContainerReplicationSource.class); + ContainerUploader uploader = mock(ContainerUploader.class); + ArgumentCaptor> futureCaptor = + ArgumentCaptor.forClass(CompletableFuture.class); + + when(uploader.startUpload(eq(containerID), eq(target), + futureCaptor.capture(), any(CopyContainerCompression.class), + eq(StorageType.SSD))) + .thenReturn(output); + + doAnswer(invocation -> { + futureCaptor.getValue().complete(null); + return null; + }).when(source).copyData(eq(containerID), any(), any()); + + PushReplicator replicator = new PushReplicator(conf, source, uploader); + + ReplicateContainerCommand cmd = + ReplicateContainerCommand.toTarget(containerID, target); + cmd.setStorageType(StorageType.SSD); + ReplicationTask task = new ReplicationTask(cmd, replicator); + + // WHEN + replicator.replicate(task); + + // THEN + verify(uploader).startUpload(eq(containerID), eq(target), + any(CompletableFuture.class), any(CopyContainerCompression.class), + eq(StorageType.SSD)); + assertEquals(Status.DONE, task.getStatus()); + } + private static long randomContainerID() { return ThreadLocalRandom.current().nextLong(); } @@ -140,7 +182,8 @@ private ContainerReplicator createSubject( when( uploader.startUpload(eq(containerID), eq(target), - futureArgument.capture(), compressionArgument.capture() + futureArgument.capture(), compressionArgument.capture(), + any() )) .thenReturn(outputStream); diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java index 9ceb0a99e9f..b139d99e89f 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestReplicationSupervisor.java @@ -1026,7 +1026,8 @@ private class FakeECReconstructionCoordinator extends ECReconstructionCoordinato @Override public void reconstructECContainerGroup(long containerID, ECReplicationConfig repConfig, SortedMap sourceNodeMap, - SortedMap targetNodeMap) { + SortedMap targetNodeMap, + org.apache.hadoop.hdds.protocol.StorageType storageType) { assertEquals(1, supervisor.getTotalInFlightReplications()); KeyValueContainerData kvcd = new KeyValueContainerData( diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerOutputStream.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerOutputStream.java index 716bf4d3aeb..23c67b24e43 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerOutputStream.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerOutputStream.java @@ -22,8 +22,11 @@ import static org.mockito.Mockito.verify; import java.io.OutputStream; +import org.apache.hadoop.hdds.protocol.StorageType; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.SendContainerRequest; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -64,6 +67,49 @@ void usesCompression(CopyContainerCompression compression) throws Exception { verify(getObserver()).onCompleted(); } + @Test + void usesStorageType() throws Exception { + OutputStream subject = new SendContainerOutputStream( + getObserver(), getContainerId(), getBufferSize(), NO_COMPRESSION, + null, StorageType.SSD); + + byte[] bytes = getRandomBytes(16); + subject.write(bytes, 0, bytes.length); + subject.close(); + + SendContainerRequest req = SendContainerRequest.newBuilder() + .setContainerID(getContainerId()) + .setOffset(0) + .setData(ByteString.copyFrom(bytes)) + .setCompression(NO_COMPRESSION.toProto()) + .setStorageType(ContainerProtos.StorageTypeProto.SSD) + .build(); + + verify(getObserver()).onNext(req); + verify(getObserver()).onCompleted(); + } + + @Test + void noStorageTypeWhenNull() throws Exception { + OutputStream subject = new SendContainerOutputStream( + getObserver(), getContainerId(), getBufferSize(), NO_COMPRESSION, + null, null); + + byte[] bytes = getRandomBytes(16); + subject.write(bytes, 0, bytes.length); + subject.close(); + + SendContainerRequest req = SendContainerRequest.newBuilder() + .setContainerID(getContainerId()) + .setOffset(0) + .setData(ByteString.copyFrom(bytes)) + .setCompression(NO_COMPRESSION.toProto()) + .build(); + + verify(getObserver()).onNext(req); + verify(getObserver()).onCompleted(); + } + @Override protected ByteString verifyPart(SendContainerRequest response, int expectedOffset, int size) { diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java index 4fb801532f0..5d2d74b1b6e 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/replication/TestSendContainerRequestHandler.java @@ -24,10 +24,12 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.io.File; @@ -35,6 +37,7 @@ import java.util.stream.Stream; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.StorageUnit; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; @@ -236,6 +239,39 @@ public void testOverAllocatedReservesMoreSpace() { assertEquals(2 * containerMaxSize, defaultReservation); } + @Test + public void testHandlerPassesStorageTypeToImporter() throws Exception { + long containerId = 10; + + ContainerProtos.SendContainerRequest request = + ContainerProtos.SendContainerRequest.newBuilder() + .setContainerID(containerId) + .setData(ByteString.copyFromUtf8("test")) + .setOffset(0) + .setCompression(NO_COMPRESSION.toProto()) + .setStorageType(ContainerProtos.StorageTypeProto.SSD) + .build(); + + sendContainerRequestHandler.onNext(request); + + // Verify importer.chooseNextVolume was called with StorageType.SSD + verify(importer).chooseNextVolume(anyLong(), eq(StorageType.SSD)); + } + + @Test + public void testHandlerPassesNullStorageTypeWhenNotSet() throws Exception { + long containerId = 11; + + ContainerProtos.SendContainerRequest request = createRequest( + containerId, ByteString.copyFromUtf8("test"), 0, null); + + sendContainerRequestHandler.onNext(request); + + // Verify importer.chooseNextVolume was called with null storageType + verify(importer).chooseNextVolume(anyLong(), + eq((StorageType) null)); + } + private ContainerProtos.SendContainerRequest createRequest( long containerId, ByteString data, int offset, Long size) { ContainerProtos.SendContainerRequest.Builder builder = diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/protocol/commands/TestReconstructionECContainersCommands.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/protocol/commands/TestReconstructionECContainersCommands.java index 2cd19a556a9..c24940e8c1a 100644 --- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/protocol/commands/TestReconstructionECContainersCommands.java +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/protocol/commands/TestReconstructionECContainersCommands.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; import org.junit.jupiter.api.Test; @@ -113,4 +114,56 @@ private List getDNDetails(int numDns) { return dns; } + @Test + public void testStorageTypeRoundTrip() { + byte[] missingIndexes = {1, 2}; + final ByteString missingContainerIndexes = + UnsafeByteOperations.unsafeWrap(missingIndexes); + ECReplicationConfig ecReplicationConfig = new ECReplicationConfig(3, 2); + final List dnDetails = getDNDetails(5); + + List + sources = dnDetails.stream().map( + a -> new ReconstructECContainersCommand + .DatanodeDetailsAndReplicaIndex(a, dnDetails.indexOf(a))) + .collect(Collectors.toList()); + List targets = getDNDetails(2); + ReconstructECContainersCommand cmd = + new ReconstructECContainersCommand(1L, sources, targets, + missingContainerIndexes, ecReplicationConfig, 42L, + StorageType.SSD); + + assertEquals(StorageType.SSD, cmd.getStorageType()); + + StorageContainerDatanodeProtocolProtos + .ReconstructECContainersCommandProto proto = cmd.getProto(); + ReconstructECContainersCommand fromProto = + ReconstructECContainersCommand.getFromProtobuf(proto); + + assertEquals(StorageType.SSD, fromProto.getStorageType()); + assertEquals(cmd.getContainerID(), fromProto.getContainerID()); + } + + @Test + public void testStorageTypeNullRoundTrip() { + byte[] missingIndexes = {1}; + final ByteString missingContainerIndexes = + UnsafeByteOperations.unsafeWrap(missingIndexes); + ECReplicationConfig ecReplicationConfig = new ECReplicationConfig(3, 2); + List targets = getDNDetails(1); + + ReconstructECContainersCommand cmd = + new ReconstructECContainersCommand(2L, Collections.emptyList(), + targets, missingContainerIndexes, ecReplicationConfig); + + assertEquals(null, cmd.getStorageType()); + + StorageContainerDatanodeProtocolProtos + .ReconstructECContainersCommandProto proto = cmd.getProto(); + ReconstructECContainersCommand fromProto = + ReconstructECContainersCommand.getFromProtobuf(proto); + + assertEquals(null, fromProto.getStorageType()); + } + } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/protocol/commands/TestReplicateContainerCommand.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/protocol/commands/TestReplicateContainerCommand.java new file mode 100644 index 00000000000..bc7f6989f70 --- /dev/null +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/protocol/commands/TestReplicateContainerCommand.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.ozone.protocol.commands; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReplicateContainerCommandProto; +import org.junit.jupiter.api.Test; + +/** + * Test for {@link ReplicateContainerCommand}. + */ +public class TestReplicateContainerCommand { + + @Test + public void testStorageTypeRoundTrip() { + DatanodeDetails target = MockDatanodeDetails.randomDatanodeDetails(); + ReplicateContainerCommand cmd = + ReplicateContainerCommand.toTarget(1L, target); + cmd.setStorageType(StorageType.ARCHIVE); + + assertEquals(StorageType.ARCHIVE, cmd.getStorageType()); + + ReplicateContainerCommandProto proto = cmd.getProto(); + ReplicateContainerCommand fromProto = + ReplicateContainerCommand.getFromProtobuf(proto); + + assertEquals(StorageType.ARCHIVE, fromProto.getStorageType()); + assertEquals(cmd.getContainerID(), fromProto.getContainerID()); + } + + @Test + public void testStorageTypeNullRoundTrip() { + ReplicateContainerCommand cmd = + ReplicateContainerCommand.forTest(2L); + + assertNull(cmd.getStorageType()); + + ReplicateContainerCommandProto proto = cmd.getProto(); + ReplicateContainerCommand fromProto = + ReplicateContainerCommand.getFromProtobuf(proto); + + assertNull(fromProto.getStorageType()); + } +} diff --git a/hadoop-hdds/docs/content/design/configless.md b/hadoop-hdds/docs/content/design/configless.md index bea8b39c9e2..aa119a596ff 100644 --- a/hadoop-hdds/docs/content/design/configless.md +++ b/hadoop-hdds/docs/content/design/configless.md @@ -28,4 +28,4 @@ author: Márton Elek, Anu Engineer # Link - * https://issues.apache.org/jira/secure/attachment/12966992/configless.pdf \ No newline at end of file + * https://issues.apache.org/jira/secure/attachment/12966992/configless.pdf diff --git a/hadoop-hdds/docs/content/design/storage-policy.md b/hadoop-hdds/docs/content/design/storage-policy.md new file mode 100644 index 00000000000..1d092fc474d --- /dev/null +++ b/hadoop-hdds/docs/content/design/storage-policy.md @@ -0,0 +1,389 @@ +--- +title: Storage Class +summary: New abstraction to configure replication methods. +date: 2020-06-08 +jira: HDDS-3755 +status: draft +author: Marton Ele +--- + + +# Ozone Storage Tiering — MVP Requirements + +Scoped subset of `01-user-requirements.md`. Four delivery stages: **MVP-1** (bucket policy + SCM placement), **MVP-2** (prefix policy + Mover), **MVP-3** (write-path hardening), **Future** (full feature set). + +--- + +## Assumptions + +- Bucket layout: **OBJECT_STORE** with **EC replication** (EC 3+2) only +- **FSO/Ratis buckets**: Storage tiering is **disabled** — setting a storage policy on an FSO or Ratis bucket is rejected. Writes to FSO/Ratis buckets continue using default placement (no StorageType passed to SCM). This avoids complexity with Ratis pipeline tiering and FSO directory inheritance in the MVP. +- Storage types: **DISK** (default) and **SSD/NVMe** only — no ARCHIVE +- Storage policies: **HOT** (SSD) and **WARM** (DISK, default) +- Policy management: **CLI only** — no S3 `x-amz-storage-class` integration +- Migration: **External Mover tool** (daily cron) — no internal OM TieringSatisfier +- Target cluster: 32 nodes — 16 SSD-only (12 SSDs each), 16 DISK-only (6 DISKs each) +- **Node topology note**: Heterogeneous nodes (both SSD + DISK on every DN) are preferred over dedicated nodes. With heterogeneous nodes, every DN qualifies for both SSD and DISK pipelines, so `BackgroundPipelineCreator` creates the same number of pipelines for each type from the full node pool. With dedicated nodes (SSD-only vs DISK-only), the node pool is split — fewer pipelines per type and on-demand creation (MVP-3) becomes more important. + +--- + +## Implementation Overview + +### MVP-1: Bucket Policy + SCM Placement (~830 lines) + +1. **Policy model** — HOT/WARM enum in proto and Java +2. **Bucket policy** — store policy on bucket, accept via create/update +3. **Resolve at write time** — OM reads bucket policy → target StorageType +4. **Tell SCM** — OM passes StorageType to SCM's block allocation +5. **SCM filters pipelines** — pick pipeline where all nodes have the right type +6. **SCM creates typed pipelines** — proactively create SSD-only and DISK-only pipelines +7. **DN picks right volume** — create container on matching volume type +8. **Fallback** — no SSD pipeline → use DISK, log warning +9. **CLI** — `ozone sh bucket create/update --storage-policy HOT|WARM` + +### MVP-2: Prefix Policy + Mover (~835 lines) + +10. **Prefix policy** — store policy on key prefixes +11. **Extended resolution** — prefix → bucket → default +12. **Mover tool** — scan keys, rewrite mismatched ones to correct tier +13. **CLI** — `ozone sh prefix setStoragePolicy`, `ozone admin mover` + +### MVP-3: Write-Path Hardening (~140 lines) + +14. **On-demand pipeline creation** — create matching pipeline on-the-fly before falling back + +--- + +## Scope Summary + +| Requirement | Service | MVP-1 | MVP-2 | MVP-3 | Future | Lines | +|-------------|---------|-------|-------|-------|--------|-------| +| StoragePolicyProto enum, bucket/key fields, AllocateBlockRequest | Proto | **Yes** | | | | ~35 | +| Bucket policy model + handlers, resolution, pass StorageType to SCM | OM | **Yes** | | | | ~255 | +| Pipeline filtering + creation, PlacementPolicy filter, fallback | SCM | **Yes** | | | | ~240 | +| VolumeChoosingPolicy + KeyValueHandler StorageType threading | DN | **Yes** | | | | ~25 | +| bucket --storage-policy commands | CLI | **Yes** | | | | ~100 | +| Prefix proto fields | Proto | | **Yes** | | | ~20 | +| Prefix policy model + handlers, extended resolution | OM | | **Yes** | | | ~315 | +| prefix set/get/removeStoragePolicy commands | CLI | | **Yes** | | | ~50 | +| Mover tool (bulk migration + dry-run) | Client | | **Yes** | | | ~450 | +| CreateContainerCommand proto field | Proto | | | **Yes** | | ~10 | +| On-demand pipeline creation, error handling | SCM | | | **Yes** | | ~130 | +| FSO dirs, key-level policy, TieringSatisfier, fallback config | OM | | | | **Yes** | ~950+ | +| Full pipeline tiering, EC reconstruction, ContainerBalancer | SCM | | | | **Yes** | ~1500+ | +| Admin observability tools | CLI | | | | **Yes** | ~300+ | +| x-amz-storage-class integration | S3GW | | | | **Yes** | ~200+ | + +--- + +## Detailed Scope Summary + +| Requirement | Service | MVP-1 | MVP-2 | MVP-3 | Future | Lines | +|-------------|---------|-------|-------|-------|--------|-------| +| Proto: StoragePolicyProto enum + fields on Bucket, Key | Proto | **Yes** | | | | ~30 | +| OM: OzoneStoragePolicy enum + OmBucketInfo fields | OM | **Yes** | | | | ~150 | +| OM: Bucket create/update handlers for storagePolicy | OM | **Yes** | | | | ~80 | +| OM: Policy resolution (bucket → default) | OM | **Yes** | | | | ~15 | +| CLI: bucket --storage-policy | CLI | **Yes** | | | | ~100 | +| Proto: StorageType on AllocateBlockRequest | Proto | **Yes** | | | | ~5 | +| OM: Pass resolved StorageType to SCM allocateBlock | OM | **Yes** | | | | ~10 | +| SCM: BlockManagerImpl accepts StorageType, passes to provider | SCM | **Yes** | | | | ~10 | +| SCM: WritableContainerProvider filters pipelines by node StorageType | SCM | **Yes** | | | | ~30 | +| SCM: BackgroundPipelineCreator creates per-StorageType pipelines | SCM | **Yes** | | | | ~40 | +| SCM: PlacementPolicy StorageType filter in chooseDatanodes | SCM | **Yes** | | | | ~50 | +| SCM: Pipeline providers accept StorageType in node selection | SCM | **Yes** | | | | ~60 | +| SCM: Pipeline storageType attribute for tracking | SCM | **Yes** | | | | ~20 | +| SCM: Fallback to DISK when SSD unavailable + warn logging | SCM | **Yes** | | | | ~30 | +| DN: VolumeChoosingPolicy filters volumes by StorageType | DN | **Yes** | | | | ~20 | +| DN: KeyValueHandler threads StorageType to volume selection | DN | **Yes** | | | | ~5 | +| Proto: fields on Prefix | Proto | | **Yes** | | | ~20 | +| OM: OmPrefixInfo storagePolicy field | OM | | **Yes** | | | ~50 | +| OM: Prefix set/get/remove storagePolicy request handlers | OM | | **Yes** | | | ~250 | +| OM: Policy resolution extended (prefix → bucket → default) | OM | | **Yes** | | | ~15 | +| CLI: prefix set/get/removeStoragePolicy | CLI | | **Yes** | | | ~50 | +| Client: Mover tool (bulk migration + dry-run) | Client | | **Yes** | | | ~450 | +| SCM: WritableContainerProvider on-demand pipeline creation | SCM | | | **Yes** | | ~80 | +| Proto: StorageType on CreateContainerCommand | Proto | | | **Yes** | | ~10 | +| SCM/DN: Error handling, logging | SCM/DN | | | **Yes** | | ~50 | +| OM: FSO directory policy (tree-walk inheritance) | OM | | | | **Yes** | ~300+ | +| OM/Client: Key-level explicit policy | OM/Client | | | | **Yes** | ~100+ | +| S3GW: x-amz-storage-class integration | S3GW | | | | **Yes** | ~200+ | +| OM: TieringSatisfier (background service) | OM | | | | **Yes** | ~500+ | +| OM: Per-bucket fallback config | OM | | | | **Yes** | ~50+ | +| SCM: Full pipeline tiering (separate pools, lifecycle) | SCM | | | | **Yes** | ~1000+ | +| SCM: EC reconstruction StorageType awareness | SCM | | | | **Yes** | ~200+ | +| SCM: ContainerBalancer per-StorageType | SCM | | | | **Yes** | ~300+ | +| CLI: Admin observability tools | CLI/OM | | | | **Yes** | ~300+ | + +--- + +# MVP-1: Bucket Policy + SCM Placement + +Minimum viable deliverable. Bucket-level storage policy with SCM honoring StorageType at allocation time. New writes land on the correct tier. If SSD capacity is exhausted, falls back to DISK with a warning log. + +~830 lines excluding tests. Low-medium risk. + +--- + +## 1.1: Storage Policy Model + +### Named Storage Policies +- **HOT** — SSD/NVMe +- **WARM** — DISK (default) + +### Policy on Buckets +Set a default storage policy on a bucket via CLI. All keys in the bucket are placed on the corresponding storage tier. + +### Dynamic Resolution +Resolved at runtime, not stamped on keys: +1. Bucket default +2. Server default (WARM) + +Changing a bucket policy immediately changes the effective policy for all keys in it. Existing data remains on the old tier until migrated (MVP-2 Mover). + +### Unset / Remove Policy +Remove a policy from a bucket → falls through to server default (WARM). + +--- + +## 1.2: SCM Changes (Placement + Pipeline Creation) + +Both new writes and the future Mover (MVP-2) call SCM's `allocateBlock()` to get blocks. Without SCM honoring StorageType, data always lands on whatever tier SCM happens to pick. MVP-1 includes SCM changes to make placement work. + +### What Exists Today + +SCM **already knows** which DataNodes have which storage types — DataNodes report per-volume storage types in every heartbeat via `StorageReportProto.storageType` (field 6). This data is stored in `DatanodeInfo.storageReports` and accessible via `SCMNodeStorageStatMap`. It's just never used for placement decisions. + +### Why Proactive Pipeline Creation Is Required + +On a cluster with 32 nodes — 16 SSD-only, 16 DISK-only — using EC 3+2 (5 nodes per pipeline), the probability that a randomly formed pipeline has all 5 members from SSD nodes is only **~2.2%** ((16/32)×(15/31)×(14/30)×(13/29)×(12/28)). With ~30 open pipelines, on average zero or one would qualify for HOT placement. Without proactive pipeline creation, writes cannot be placed on SSD. + +With proactive creation, SCM intentionally forms SSD pipelines from SSD-capable nodes and DISK pipelines from DISK-capable nodes. Based on current cluster data (14 SSD-only nodes → 44 pipelines with EC 3+2), 16 SSD nodes would yield ~50 SSD pipelines and 16 DISK nodes ~50 DISK pipelines. + +**Heterogeneous vs dedicated nodes**: If every DN has both SSD and DISK volumes (heterogeneous), all nodes qualify for both pipeline types. SCM creates the same number of pipelines for each type from the full node pool — no shortage. The 2.2% problem only applies to dedicated nodes (SSD-only vs DISK-only) where the node pool is split. With heterogeneous nodes, on-demand pipeline creation (MVP-3) becomes largely unnecessary. + +### What MVP-1 Adds (~280 lines) + +| Layer | Change | Lines | +|-------|--------|-------| +| **Proto** | Add `optional StorageTypeProto storageType` to `AllocateBlockRequest` | ~5 | +| **OM** | Pass resolved StorageType to SCM's `allocateBlock()` | ~10 | +| **SCM BlockManagerImpl** | Accept StorageType, pass to WritableContainerProvider | ~10 | +| **WritableContainerProvider** (Ratis + EC) | Filter open pipelines: keep only those where ALL member nodes have volumes of the requested StorageType | ~30 | +| **SCM BackgroundPipelineCreator** | Create per-StorageType pipelines (SSD pipelines from SSD nodes, DISK pipelines from DISK nodes) | ~40 | +| **SCM PlacementPolicy** | Add StorageType filter to `chooseDatanodes()` / `hasEnoughSpace()` — only select nodes with the requested type | ~50 | +| **SCM Pipeline providers** (Ratis + EC) | Accept StorageType filter in node selection when creating new pipelines | ~60 | +| **SCM Pipeline** | Add optional `storageType` attribute for tracking which type a pipeline serves | ~20 | +| **SCM Fallback** | SSD unavailable → fall back to DISK, log warning for monitoring | ~30 | +| **DN VolumeChoosingPolicy** | When StorageType is specified, filter candidate volumes by type before choosing | ~20 | +| **DN KeyValueHandler** | Thread StorageType from `CreateContainerCommand` to volume selection | ~5 | +| **Total** | | **~280** | + +### Configuration + +Proactive per-StorageType pipeline creation is controlled by a config flag: +``` +ozone.scm.pipeline.creation.storage-type-aware.enabled = true (default: false) +``` +When disabled, `BackgroundPipelineCreator` behaves as today — no StorageType filtering. Useful for clusters that don't use tiering, during rollback, or on clusters with heterogeneous nodes (all DNs have both SSD and DISK) where random pipeline formation already produces pipelines that qualify for both types. + +### How It Works + +``` +At startup / periodically (if storage-type-aware pipeline creation enabled): + BackgroundPipelineCreator scans cluster storage types + → Creates SSD pipelines using only SSD-capable nodes + → Creates DISK pipelines using only DISK-capable nodes + +On write: + OM resolves policy (bucket → default) → StorageType = SSD + ↓ + SCM allocateBlock(size, replicationConfig, storageType=SSD, ...) + ↓ + Filter open pipelines: keep only those where ALL member nodes have SSD volumes + ├─ Found → allocate block on that pipeline + └─ Not found → FALL BACK to any open pipeline (DISK) + → Log warning for monitoring +``` + +--- + +## 1.3: CLI + +### Bucket Policy +``` +ozone sh bucket create --storage-policy HOT|WARM +ozone sh bucket update --storage-policy HOT|WARM +ozone sh bucket info # shows storage policy +``` + +--- + +## 1.4: Backward Compatibility + +### Existing Data Unaffected +Keys without a policy resolve to WARM (DISK). No behavior change for clusters that don't set policies. + +### Proto Backward Compat +All new fields `optional` with `UNSET = 0`. Old clients ignore new fields. + +### Upgrade Path +Cold restart with new JARs. No DB migration — proto optional fields with `UNSET = 0` default handle old data. Layout versioning (`ozone admin om finalize`) can gate feature activation if needed. + +--- + +## 1.5: How New Writes Work in MVP-1 + +- OM resolves effective policy at write time (bucket → default) +- OM passes resolved StorageType to SCM's `allocateBlock()` +- SCM filters open pipelines by node StorageType — `BackgroundPipelineCreator` ensures per-StorageType pipelines exist +- If a matching pipeline exists, the block lands on the correct tier +- If no matching pipeline exists (e.g., all SSD nodes full), SCM falls back to any open pipeline and logs a warning + +MVP-1 is designed so that MVP-2 and MVP-3 are purely additive — bucket policy, proto fields, SCM placement, and CLI all remain unchanged. + +--- + +# MVP-2: Prefix Policy + Mover + +Builds on MVP-1. Adds prefix-level storage policy for finer-grained control within a bucket, and the Mover tool for bulk migration of existing data after policy changes. + +~835 additional lines excluding tests. Low risk — OM metadata + external client tool. + +--- + +## 2.1: Prefix Storage Policy + +### Policy on Prefixes (Object Store) +Set a storage policy on a key prefix. Longest-prefix-match wins. + +### Extended Resolution +Resolution chain extended to: +1. Prefix policy (longest match) +2. Bucket default +3. Server default (WARM) + +Changing a prefix policy immediately changes the effective policy for all keys matching it. The Mover migrates existing mismatched data. + +**Implementation note**: The resolution uses the same in-memory `PrefixManager` RadixTree that serves ACL inheritance. `resolveEffectiveStoragePolicy()` calls `prefixManager.getLongestPrefixPath()` independently — it does not depend on ACLs being enabled. The RadixTree is loaded at OM startup and maintained in memory, so this is a pure in-memory tree traversal with zero RocksDB I/O. Negligible performance impact. + +### Unset / Remove Policy +Remove a policy from a prefix → falls through to bucket default → server default. + +--- + +## 2.2: Mover Tool + +### Bulk Migration +External CLI tool (`ozone admin mover`): +- Scans a bucket or prefix path +- For each key: resolves effective policy, compares to actual storage tier +- Moves mismatched keys via `rewriteKey()` +- Run daily via cron, or on-demand after a policy change + +### Dry-Run Mode +`--dry-run` reports mismatches without moving data. Serves as the policy compliance check. + +### Progress Reporting +Keys scanned, moved, skipped, failed, bytes migrated. + +### Safety +Generation-based optimistic concurrency via `rewriteKey()`. Concurrent modification → move fails safely, retried on next run. + +--- + +## 2.3: CLI + +### Prefix Policy +``` +ozone sh prefix setStoragePolicy --policy HOT|WARM +ozone sh prefix getStoragePolicy +ozone sh prefix removeStoragePolicy +``` + +### Mover +``` +ozone admin mover --path o3://om/vol/bucket/ # entire bucket +ozone admin mover --path o3://om/vol/bucket/prefix/ # specific prefix +ozone admin mover --path o3://om/vol/bucket/ --dry-run # report only +ozone admin mover --path o3://om/vol/bucket/ --threads 8 # parallel +``` + +--- + +# MVP-3: Write-Path Hardening + +Builds on MVP-2. Adds on-demand pipeline creation when no matching pipeline exists (instead of falling back to DISK), and additional error handling. After MVP-3, writes almost never fall back to DISK unless the target storage type is genuinely full. + +~140 additional lines excluding tests. Medium risk — extends SCM write path. + +--- + +## 3.1: On-Demand Pipeline Creation + +When no open pipeline matches the requested StorageType (e.g., all SSD pipelines are full/closed), `WritableContainerProvider` creates a new pipeline on-the-fly using only nodes that have the requested type, before falling back to DISK. + +``` +Filter open pipelines for SSD + ├─ Found → allocate block + └─ Not found → create new pipeline using only SSD-capable nodes + ├─ Created → allocate block + └─ Can't create (not enough SSD nodes) → FALL BACK to DISK (already in MVP-1) +``` + +--- + +## 3.2: Changes Required + +| Layer | Change | Lines | +|-------|--------|-------| +| **WritableContainerProvider** (Ratis + EC) | On-demand pipeline creation for missing types | ~80 | +| **Proto** | Add `StorageType` to `CreateContainerCommand` | ~10 | +| **Error handling, logging** | | ~50 | +| **Total** | | **~140** | + +--- + +# Future (Deferred) + +Full feature set from `01-user-requirements.md` and `03-detailed-technical-plan.md`. Adds: + +- **FSO/Ratis bucket support** — enable storage tiering for FSO layout and Ratis replication (MVP only supports OBS+EC) +- **FSO directory policy** — storage policy on directories with tree-walk inheritance (like HDFS) +- **Key-level explicit policy** — client sets policy per key at write time +- **S3 Gateway integration** — `x-amz-storage-class` header maps to storage policy +- **TieringSatisfier** — internal OM background service for continuous compliance (complements Mover) +- **Full pipeline tiering** — separate pipeline pools per storage type with lifecycle management +- **EC reconstruction awareness** — reconstruction targets filtered by StorageType +- **ContainerBalancer per-StorageType** — balance SSD→SSD and DISK→DISK separately +- **Per-bucket fallback config** — `allowFallbackStoragePolicy` per bucket instead of global +- **Admin observability** — `ozone admin storagepolicies list|check|usageinfo` +- **COLD/ARCHIVE tier** — exercise the COLD policy with ARCHIVE storage type + +See `01-user-requirements.md` for full requirements and `03-detailed-technical-plan.md` for implementation details. + +--- + +# Effort Summary + +| Stage | Scope | Lines (excl. tests) | Risk | Deliverable | +|-------|-------|-------|------|-------------| +| **MVP-1** | Bucket policy + SCM placement + pipeline creation | ~830 | Low-medium | Bucket-level policy via CLI, per-StorageType pipelines, writes land on correct tier, fallback to DISK with warning log | +| **MVP-2** | Prefix policy + Mover | ~835 | Low | Prefix-level policy, Mover migrates misplaced data after policy changes | +| **MVP-3** | Write-path hardening | ~140 | Medium | On-demand pipeline creation — writes almost never fall back unless tier is genuinely full | +| **MVP-1 + MVP-2 + MVP-3** | Combined | **~1,805** | Medium | Full pilot-ready solution | +| **Future** | Full feature set | ~3,000+ | High | Production-grade, all layouts, S3, auto-satisfier | + +MVP-1 is independently useful — bucket-level policy with correct write placement. MVP-2 adds prefix granularity and the Mover for bulk migration. MVP-3 adds resilience. All three together at ~1,805 lines. diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocol.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocol.java index a34420b3de0..86a407bb6b9 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocol.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocol.java @@ -23,6 +23,7 @@ import java.util.concurrent.TimeoutException; import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; import org.apache.hadoop.hdds.scm.AddSCMRequest; @@ -87,7 +88,7 @@ default List allocateBlock(long size, int numBlocks, ReplicationConfig replicationConfig, String owner, ExcludeList excludeList) throws IOException { return allocateBlock(size, numBlocks, replicationConfig, owner, - excludeList, null); + excludeList, null, StorageType.DEFAULT); } /** @@ -107,9 +108,17 @@ default List allocateBlock(long size, int numBlocks, * @return allocated block accessing info (key, pipeline). * @throws IOException */ + default List allocateBlock(long size, int numBlocks, + ReplicationConfig replicationConfig, String owner, + ExcludeList excludeList, String clientMachine) throws IOException { + return allocateBlock(size, numBlocks, replicationConfig, owner, + excludeList, clientMachine, StorageType.DEFAULT); + } + List allocateBlock(long size, int numBlocks, ReplicationConfig replicationConfig, String owner, - ExcludeList excludeList, String clientMachine) throws IOException; + ExcludeList excludeList, String clientMachine, + StorageType storageType) throws IOException; /** * Delete blocks for a set of object keys. diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java index ae4d58fd18f..5fb1eaac0af 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateBlockResponse; import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateScmBlockRequestProto; @@ -173,7 +174,8 @@ public List allocateBlock( long size, int num, ReplicationConfig replicationConfig, String owner, ExcludeList excludeList, - String clientMachine + String clientMachine, + StorageType storageType ) throws IOException { Preconditions.checkArgument(size > 0, "block size must be greater than 0"); @@ -189,6 +191,10 @@ public List allocateBlock( requestBuilder.setClient(clientMachine); } + if (storageType != null) { + requestBuilder.setStorageType(storageType.toProto()); + } + switch (replicationConfig.getReplicationType()) { case STAND_ALONE: requestBuilder.setFactor( diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 05c94624c99..fc32df09c1c 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -149,7 +149,7 @@ message DatanodeBlockID { required int64 localID = 2; optional uint64 blockCommitSequenceId = 3 [default = 0]; optional int32 replicaIndex = 4; - + optional StorageTypeProto storageType = 5; } message KeyValue { @@ -264,6 +264,17 @@ enum ContainerType { KeyValueContainer = 1; } +/** + * Storage type for volume selection on datanodes. + * Values mirror StorageTypeProto in hdds.proto. + */ +enum StorageTypeProto { + DISK = 1; + SSD = 2; + ARCHIVE = 3; + RAM_DISK = 4; +} + // Container Messages. message CreateContainerRequestProto { @@ -271,6 +282,7 @@ message CreateContainerRequestProto { optional ContainerType containerType = 3 [default = KeyValueContainer]; optional int32 replicaIndex = 4; optional ContainerDataProto.State state = 5; + optional StorageTypeProto storageType = 6; } message CreateContainerResponseProto { @@ -545,6 +557,7 @@ message SendContainerRequest { optional int64 checksum = 4; optional CopyContainerCompressProto compression = 5; optional int64 size = 6; + optional StorageTypeProto storageType = 7; } message SendContainerResponse { diff --git a/hadoop-hdds/interface-client/src/main/proto/hdds.proto b/hadoop-hdds/interface-client/src/main/proto/hdds.proto index eb819b80a3e..40b3546d8d0 100644 --- a/hadoop-hdds/interface-client/src/main/proto/hdds.proto +++ b/hadoop-hdds/interface-client/src/main/proto/hdds.proto @@ -271,6 +271,7 @@ message ContainerInfoProto { optional ReplicationFactor replicationFactor = 10; required ReplicationType replicationType = 11; optional ECReplicationConfig ecReplicationConfig = 12; + optional StorageTypeProto storageType = 13; } message ContainerWithPipeline { diff --git a/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto b/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto index b5f6c9e80ad..73191eac7c7 100644 --- a/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto +++ b/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto @@ -428,6 +428,7 @@ message ReplicateContainerCommandProto { optional int32 replicaIndex = 4; optional DatanodeDetailsProto target = 5; optional ReplicationCommandPriority priority = 6 [default = NORMAL]; + optional StorageTypeProto storageType = 7; } /** @@ -441,6 +442,7 @@ message ReconstructECContainersCommandProto { required bytes missingContainerIndexes = 4; required ECReplicationConfig ecReplicationConfig = 5; required int64 cmdId = 6; + optional StorageTypeProto storageType = 7; } message DatanodeDetailsAndReplicaIndexProto { diff --git a/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto b/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto index 4c794fe7dc1..a26e8acaeec 100644 --- a/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto +++ b/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto @@ -161,6 +161,7 @@ message AllocateScmBlockRequestProto { optional string client = 9; + optional hadoop.hdds.StorageTypeProto storageType = 10; } /** diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java index 54a1648f7c1..6ae0b9a00e8 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.concurrent.TimeoutException; import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock; import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; import org.apache.hadoop.ozone.common.BlockGroup; @@ -42,8 +43,8 @@ public interface BlockManager extends Closeable { * @throws IOException */ AllocatedBlock allocateBlock(long size, ReplicationConfig replicationConfig, - String owner, - ExcludeList excludeList) throws IOException, TimeoutException; + String owner, ExcludeList excludeList, + StorageType storageType) throws IOException, TimeoutException; /** * Deletes a list of blocks in an atomic operation. Internally, SCM diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java index 6b0136abf66..008083c8629 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.StorageUnit; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.scm.ScmConfig; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.ContainerInfo; @@ -145,11 +146,14 @@ public void stop() throws IOException { @Override public AllocatedBlock allocateBlock(final long size, ReplicationConfig replicationConfig, - String owner, ExcludeList excludeList) + String owner, ExcludeList excludeList, + StorageType storageType) throws IOException { if (LOG.isTraceEnabled()) { LOG.trace("Size : {} , replicationConfig: {}", size, replicationConfig); } + LOG.debug("Allocating block: size={}, replication={}, storageType={}", + size, replicationConfig, storageType); if (scm.getScmContext().isInSafeMode()) { throw new SCMException("SafeModePrecheck failed for allocateBlock", SCMException.ResultCodes.SAFE_MODE_EXCEPTION); @@ -160,8 +164,21 @@ public AllocatedBlock allocateBlock(final long size, INVALID_BLOCK_SIZE); } - ContainerInfo containerInfo = writableContainerFactory.getContainer( - size, replicationConfig, owner, excludeList); + ContainerInfo containerInfo; + try { + containerInfo = writableContainerFactory.getContainer( + size, replicationConfig, owner, excludeList, storageType); + } catch (IOException e) { + StorageType fallback = getFallbackStorageType(storageType); + if (fallback != null) { + LOG.warn("No pipeline with StorageType={} found for owner={}," + + " falling back to {}", storageType, owner, fallback, e); + containerInfo = writableContainerFactory.getContainer( + size, replicationConfig, owner, excludeList, fallback); + } else { + throw e; + } + } if (containerInfo != null) { return newBlock(containerInfo); @@ -256,6 +273,13 @@ public DeletedBlockLog getDeletedBlockLog() { return this.deletedBlockLog; } + private static StorageType getFallbackStorageType(StorageType primaryType) { + if (primaryType == StorageType.SSD) { + return StorageType.DISK; + } + return null; + } + /** * Close the resources for BlockManager. * diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManager.java index 370c219ac60..7dd390074eb 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManager.java @@ -24,6 +24,7 @@ import java.util.Map; import java.util.Set; import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; @@ -199,6 +200,17 @@ ContainerInfo getMatchingContainer(long size, String owner, Pipeline pipeline, Set excludedContainerIDS); + /** + * Returns ContainerInfo which matches the requirements, recording the + * storageType on newly allocated containers. + */ + @Nullable + default ContainerInfo getMatchingContainer(long size, String owner, + Pipeline pipeline, Set excludedContainerIDS, + StorageType storageType) { + return getMatchingContainer(size, owner, pipeline, excludedContainerIDS); + } + /** * Once after report processor handler completes, call this to notify * container manager to increment metrics. diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java index dc701a0be66..b4eb36de9ea 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java @@ -34,6 +34,7 @@ import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerInfoProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent; @@ -227,7 +228,13 @@ public ContainerInfo allocateContainer( private ContainerInfo createContainer(Pipeline pipeline, String owner) throws IOException { - final ContainerInfo containerInfo = allocateContainer(pipeline, owner); + return createContainer(pipeline, owner, null); + } + + private ContainerInfo createContainer(Pipeline pipeline, String owner, + StorageType storageType) throws IOException { + final ContainerInfo containerInfo = + allocateContainer(pipeline, owner, storageType); if (LOG.isTraceEnabled()) { LOG.trace("New container allocated: {}", containerInfo); } @@ -235,7 +242,8 @@ private ContainerInfo createContainer(Pipeline pipeline, String owner) } private ContainerInfo allocateContainer(final Pipeline pipeline, - final String owner) + final String owner, + final StorageType storageType) throws IOException { if (!pipelineManager.hasEnoughSpace(pipeline, maxContainerSize)) { LOG.debug("Cannot allocate a new container because pipeline {} does not have the required space {}.", @@ -268,6 +276,10 @@ private ContainerInfo allocateContainer(final Pipeline pipeline, ReplicationConfig.getLegacyFactor(pipeline.getReplicationConfig())); } + if (storageType != null) { + containerInfoBuilder.setStorageType(storageType.toProto()); + } + containerStateManager.addContainer(containerInfoBuilder.build()); scmContainerManagerMetrics.incNumSuccessfulCreateContainers(); return containerStateManager.getContainer(containerID); @@ -350,13 +362,22 @@ public void updateDeleteTransactionId( @Override public ContainerInfo getMatchingContainer(final long size, final String owner, final Pipeline pipeline, final Set excludedContainerIDs) { + return getMatchingContainer(size, owner, pipeline, excludedContainerIDs, + null); + } + + @Override + public ContainerInfo getMatchingContainer(final long size, final String owner, + final Pipeline pipeline, final Set excludedContainerIDs, + final StorageType storageType) { NavigableSet containerIDs; ContainerInfo containerInfo; try { synchronized (pipeline.getId()) { containerIDs = getContainersForOwner(pipeline, owner); if (containerIDs.size() < pipelineManager.openContainerLimit(pipeline.getNodes())) { - ContainerInfo allocated = allocateContainer(pipeline, owner); + ContainerInfo allocated = + createContainer(pipeline, owner, storageType); if (allocated != null) { // New container was created, refresh IDs so it becomes eligible. containerIDs = getContainersForOwner(pipeline, owner); @@ -366,7 +387,7 @@ public ContainerInfo getMatchingContainer(final long size, final String owner, containerInfo = containerStateManager.getMatchingContainer( size, owner, pipeline.getId(), containerIDs); if (containerInfo == null) { - containerInfo = allocateContainer(pipeline, owner); + containerInfo = createContainer(pipeline, owner, storageType); } return containerInfo; } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECUnderReplicationHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECUnderReplicationHandler.java index 0bf886569c9..910b6300326 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECUnderReplicationHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECUnderReplicationHandler.java @@ -33,6 +33,7 @@ import java.util.Set; import java.util.stream.Collectors; import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.hdds.HddsIdFactory; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.StorageUnit; @@ -367,7 +368,8 @@ private int processMissingIndexes( new ReconstructECContainersCommand(container.getContainerID(), sourceDatanodesWithIndex, selectedDatanodes, integers2ByteString(missingIndexes), - repConfig); + repConfig, HddsIdFactory.getLongId(), + container.getStorageType()); // This can throw a CommandTargetOverloadedException, but there is no // point in retrying here. The sources we picked already have the // overloaded nodes excluded, so we should not get an overloaded @@ -611,6 +613,7 @@ private void createReplicateCommand( // For EC containers, we need to track the replica index which is // to be replicated, so add it to the command. replicateCommand.setReplicaIndex(replica.getReplicaIndex()); + replicateCommand.setStorageType(container.getStorageType()); replicationManager.sendDatanodeCommand(replicateCommand, container, target); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisUnderReplicationHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisUnderReplicationHandler.java index 57eb29033e4..15d014e3c1a 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisUnderReplicationHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/RatisUnderReplicationHandler.java @@ -475,6 +475,7 @@ private int sendReplicationCommands( ReplicateContainerCommand command = ReplicateContainerCommand.fromSources( containerInfo.getContainerID(), sources); + command.setStorageType(containerInfo.getStorageType()); replicationManager.sendDatanodeCommand(command, containerInfo, target); commandsSent++; } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java index 83d3825b66c..7c72fd7be63 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java @@ -531,6 +531,7 @@ public void sendThrottledReplicationCommand(ContainerInfo containerInfo, ReplicateContainerCommand cmd = ReplicateContainerCommand.toTarget(containerID, target); cmd.setReplicaIndex(replicaIndex); + cmd.setStorageType(containerInfo.getStorageType()); sendDatanodeCommand(cmd, containerInfo, source); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java index 0aefbedbd43..4420ec5a1fc 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java @@ -25,11 +25,14 @@ import static org.apache.hadoop.hdds.scm.ha.SCMService.Event.PRE_CHECK_COMPLETED; import static org.apache.hadoop.hdds.scm.ha.SCMService.Event.UNHEALTHY_TO_HEALTHY_NODE_HANDLER_TRIGGERED; +import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.io.IOException; import java.time.Clock; +import java.util.AbstractMap; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Lock; @@ -40,6 +43,7 @@ import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.scm.ScmConfigKeys; @@ -88,6 +92,7 @@ public class BackgroundPipelineCreator implements SCMService { private final AtomicBoolean running = new AtomicBoolean(false); private final long intervalInMillis; private final Clock clock; + private final boolean storageTypeAwareCreation; BackgroundPipelineCreator(PipelineManager pipelineManager, ConfigurationSource conf, SCMContext scmContext, Clock clock) { @@ -110,6 +115,10 @@ public class BackgroundPipelineCreator implements SCMService { ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS); + this.storageTypeAwareCreation = conf.getBoolean( + ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_STORAGE_TYPE_AWARE, + ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_STORAGE_TYPE_AWARE_DEFAULT); + threadName = scmContext.threadNamePrefix() + THREAD_NAME; } @@ -203,7 +212,8 @@ private boolean skipCreation(ReplicationConfig replicationConfig, return true; } - private void createPipelines() throws RuntimeException { + @VisibleForTesting + void createPipelines() throws RuntimeException { // TODO: #CLUTIL Different replication factor may need to be supported HddsProtos.ReplicationType type = HddsProtos.ReplicationType.valueOf( conf.get(OzoneConfigKeys.OZONE_REPLICATION_TYPE, @@ -212,8 +222,7 @@ private void createPipelines() throws RuntimeException { ScmConfigKeys.OZONE_SCM_PIPELINE_AUTO_CREATE_FACTOR_ONE, ScmConfigKeys.OZONE_SCM_PIPELINE_AUTO_CREATE_FACTOR_ONE_DEFAULT); - List list = - new ArrayList<>(); + List replicationConfigs = new ArrayList<>(); for (HddsProtos.ReplicationFactor factor : HddsProtos.ReplicationFactor .values()) { if (factor == ReplicationFactor.ZERO) { @@ -233,10 +242,20 @@ private void createPipelines() throws RuntimeException { // Skip this iteration for creating pipeline continue; } - list.add(replicationConfig); + replicationConfigs.add(replicationConfig); + } + + if (storageTypeAwareCreation) { + createTypedPipelines(replicationConfigs); + } else { + createUntypedPipelines(replicationConfigs); } - LoopingIterator it = new LoopingIterator(list); + LOG.debug("BackgroundPipelineCreator createPipelines finished."); + } + + private void createUntypedPipelines(List configs) { + LoopingIterator it = new LoopingIterator(configs); while (it.hasNext()) { ReplicationConfig replicationConfig = (ReplicationConfig) it.next(); @@ -251,8 +270,46 @@ private void createPipelines() throws RuntimeException { it.remove(); } } + } - LOG.debug("BackgroundPipelineCreator createPipelines finished."); + private void createTypedPipelines(List configs) { + // Build (ReplicationConfig, StorageType) pairs: for each config, + // one null entry (untyped) plus one per concrete StorageType. + StorageType[] storageTypes = { + StorageType.SSD, StorageType.DISK, StorageType.ARCHIVE + }; + List> pairs = new ArrayList<>(); + for (ReplicationConfig config : configs) { + pairs.add(new AbstractMap.SimpleEntry<>(config, null)); + for (StorageType st : storageTypes) { + pairs.add(new AbstractMap.SimpleEntry<>(config, st)); + } + } + + LoopingIterator it = new LoopingIterator(pairs); + while (it.hasNext()) { + @SuppressWarnings("unchecked") + Map.Entry entry = + (Map.Entry) it.next(); + + try { + Pipeline pipeline; + if (entry.getValue() == null) { + pipeline = pipelineManager.createPipeline(entry.getKey()); + } else { + pipeline = pipelineManager.createPipeline( + entry.getKey(), entry.getValue()); + } + LOG.info("Created new pipeline {} with StorageType {}", + pipeline, entry.getValue()); + } catch (IOException ioe) { + it.remove(); + } catch (Throwable t) { + LOG.error("Error while creating pipelines for StorageType " + + entry.getValue(), t); + it.remove(); + } + } } @Override diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java index 6a448d6c88d..52c55471669 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java @@ -25,6 +25,7 @@ import java.util.Set; import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.container.ContainerReplica; import org.apache.hadoop.hdds.utils.db.CodecException; @@ -39,6 +40,11 @@ public interface PipelineManager extends Closeable, PipelineManagerMXBean { Pipeline createPipeline(ReplicationConfig replicationConfig) throws IOException; + default Pipeline createPipeline(ReplicationConfig replicationConfig, + StorageType storageType) throws IOException { + return createPipeline(replicationConfig); + } + Pipeline createPipeline(ReplicationConfig replicationConfig, List excludedNodes, List favoredNodes) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java index 9c529e22e7e..f473335c572 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java @@ -30,6 +30,7 @@ import java.util.Map; import java.util.NavigableSet; import java.util.Set; +import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -41,6 +42,7 @@ import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; @@ -56,6 +58,7 @@ import org.apache.hadoop.hdds.scm.ha.SCMServiceManager; import org.apache.hadoop.hdds.scm.node.DatanodeInfo; import org.apache.hadoop.hdds.scm.node.NodeManager; +import org.apache.hadoop.hdds.scm.node.NodeStatus; import org.apache.hadoop.hdds.scm.server.upgrade.FinalizationManager; import org.apache.hadoop.hdds.server.events.EventPublisher; import org.apache.hadoop.hdds.utils.db.CodecException; @@ -268,6 +271,33 @@ public Pipeline createPipeline(ReplicationConfig replicationConfig, } } + @Override + public Pipeline createPipeline(ReplicationConfig replicationConfig, + StorageType storageType) throws IOException { + if (storageType == null) { + return createPipeline(replicationConfig); + } + // Compute excluded nodes: all healthy nodes that do NOT have the + // requested StorageType. + List allHealthy = + nodeManager.getNodes(NodeStatus.inServiceHealthy()); + Set qualifiedNodeIds = + PipelineStorageTypeFilter.getNodesWithStorageType( + nodeManager, storageType); + + if (qualifiedNodeIds.isEmpty()) { + throw new IOException("No healthy nodes with StorageType " + + storageType + " available for pipeline creation"); + } + + List excludedNodes = allHealthy.stream() + .filter(dn -> !qualifiedNodeIds.contains(dn.getUuid())) + .collect(Collectors.toList()); + + return createPipeline(replicationConfig, excludedNodes, + Collections.emptyList()); + } + private void checkIfPipelineCreationIsAllowed( ReplicationConfig replicationConfig) throws IOException { if (!isPipelineCreationAllowed() && !factorOne(replicationConfig)) { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStorageTypeFilter.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStorageTypeFilter.java new file mode 100644 index 00000000000..92a5949c17e --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStorageTypeFilter.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdds.scm.pipeline; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto; +import org.apache.hadoop.hdds.scm.node.DatanodeInfo; +import org.apache.hadoop.hdds.scm.node.NodeManager; +import org.apache.hadoop.hdds.scm.node.NodeStatus; + +/** + * Utility for filtering pipelines by StorageType. Builds a set of + * qualifying node UUIDs (nodes that have at least one volume of the + * requested StorageType), then filters pipelines to those whose every + * member is in that set. + */ +final class PipelineStorageTypeFilter { + + private PipelineStorageTypeFilter() { + } + + static Set getNodesWithStorageType(NodeManager nodeManager, + StorageType storageType) { + Set result = new HashSet<>(); + for (DatanodeDetails dn : + nodeManager.getNodes(NodeStatus.inServiceHealthy())) { + DatanodeInfo info = nodeManager.getDatanodeInfo(dn); + if (info == null) { + continue; + } + for (StorageReportProto report : info.getStorageReports()) { + if (StorageType.valueOf(report.getStorageType()) == storageType) { + result.add(dn.getUuid()); + break; + } + } + } + return result; + } + + static List filter(List pipelines, + NodeManager nodeManager, StorageType storageType) { + if (storageType == null) { + return pipelines; + } + Set qualifiedNodes = + getNodesWithStorageType(nodeManager, storageType); + return pipelines.stream() + .filter(p -> p.getNodes().stream() + .allMatch(dn -> qualifiedNodes.contains(dn.getUuid()))) + .collect(Collectors.toList()); + } +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java index b816bc4de7f..ba41495bc3d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerFactory.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; import org.apache.hadoop.hdds.scm.pipeline.WritableECContainerProvider.WritableECContainerProviderConfig; @@ -45,7 +46,8 @@ public WritableContainerFactory(StorageContainerManager scm) { this.ratisProvider = new WritableRatisContainerProvider( scm.getPipelineManager(), - scm.getContainerManager(), scm.getPipelineChoosePolicy()); + scm.getContainerManager(), scm.getPipelineChoosePolicy(), + scm.getScmNodeManager()); this.standaloneProvider = ratisProvider; WritableECContainerProviderConfig ecProviderConfig = @@ -79,6 +81,25 @@ public ContainerInfo getContainer(final long size, } } + public ContainerInfo getContainer(final long size, + ReplicationConfig repConfig, String owner, ExcludeList excludeList, + StorageType storageType) throws IOException { + switch (repConfig.getReplicationType()) { + case STAND_ALONE: + return standaloneProvider.getContainer(size, repConfig, owner, + excludeList, storageType); + case RATIS: + return ratisProvider.getContainer(size, repConfig, owner, + excludeList, storageType); + case EC: + return ecProvider.getContainer(size, (ECReplicationConfig) repConfig, + owner, excludeList, storageType); + default: + throw new IOException(repConfig.getReplicationType() + + " is an invalid replication type"); + } + } + private long getConfiguredContainerSize(ConfigurationSource conf) { return (long) conf.getStorageSize(OZONE_SCM_CONTAINER_SIZE, OZONE_SCM_CONTAINER_SIZE_DEFAULT, BYTES); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerProvider.java index 8eb3d1233f2..b5c976ab66e 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerProvider.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableContainerProvider.java @@ -19,6 +19,7 @@ import java.io.IOException; import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; @@ -53,4 +54,10 @@ ContainerInfo getContainer(long size, T repConfig, String owner, ExcludeList excludeList) throws IOException; + default ContainerInfo getContainer(long size, T repConfig, + String owner, ExcludeList excludeList, StorageType storageType) + throws IOException { + return getContainer(size, repConfig, owner, excludeList); + } + } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableECContainerProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableECContainerProvider.java index d7c4b7705f4..2b1e0c24842 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableECContainerProvider.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableECContainerProvider.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hdds.conf.PostConstruct; import org.apache.hadoop.hdds.conf.ReconfigurableConfig; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.scm.PipelineChoosePolicy; import org.apache.hadoop.hdds.scm.PipelineRequestInformation; import org.apache.hadoop.hdds.scm.container.ContainerID; @@ -93,6 +94,20 @@ public WritableECContainerProvider(WritableECContainerProviderConfig config, public ContainerInfo getContainer(final long size, ECReplicationConfig repConfig, String owner, ExcludeList excludeList) throws IOException { + return getContainerInternal(size, repConfig, owner, excludeList, null); + } + + @Override + public ContainerInfo getContainer(final long size, + ECReplicationConfig repConfig, String owner, ExcludeList excludeList, + StorageType storageType) throws IOException { + return getContainerInternal(size, repConfig, owner, excludeList, + storageType); + } + + private ContainerInfo getContainerInternal(final long size, + ECReplicationConfig repConfig, String owner, ExcludeList excludeList, + StorageType storageType) throws IOException { int maximumPipelines = getMaximumPipelines(repConfig); int openPipelineCount; synchronized (this) { @@ -100,7 +115,8 @@ public ContainerInfo getContainer(final long size, Pipeline.PipelineState.OPEN); if (openPipelineCount < maximumPipelines) { try { - return allocateContainer(repConfig, size, owner, excludeList); + return allocateContainer(repConfig, size, owner, excludeList, + storageType); } catch (IOException e) { LOG.warn("Unable to allocate a container with {} existing ones; " + "requested size={}, replication={}, owner={}, {}", @@ -115,12 +131,15 @@ public ContainerInfo getContainer(final long size, } List existingPipelines = pipelineManager.getPipelines( repConfig, Pipeline.PipelineState.OPEN); + existingPipelines = PipelineStorageTypeFilter.filter( + existingPipelines, nodeManager, storageType); final int pipelineCount = existingPipelines.size(); LOG.debug("Checking existing pipelines: {}", existingPipelines); PipelineRequestInformation pri = PipelineRequestInformation.Builder.getBuilder() .setSize(size) + .setStorageType(storageType) .build(); while (!existingPipelines.isEmpty()) { int pipelineIndex = @@ -169,7 +188,8 @@ public ContainerInfo getContainer(final long size, } if (openPipelineCount < maximumPipelines) { synchronized (this) { - return allocateContainer(repConfig, size, owner, excludeList); + return allocateContainer(repConfig, size, owner, excludeList, + storageType); } } throw new IOException("Pipeline limit (" + maximumPipelines @@ -193,8 +213,8 @@ private int getMaximumPipelines(ECReplicationConfig repConfig) { } private ContainerInfo allocateContainer(ReplicationConfig repConfig, - long size, String owner, ExcludeList excludeList) - throws IOException { + long size, String owner, ExcludeList excludeList, + StorageType storageType) throws IOException { List excludedNodes = Collections.emptyList(); if (!excludeList.getDatanodes().isEmpty()) { @@ -206,7 +226,8 @@ private ContainerInfo allocateContainer(ReplicationConfig repConfig, // the returned ContainerInfo should not be null (due to not enough space in the Datanodes specifically) because // this is a new pipeline and pipeline creation checks for sufficient space in the Datanodes ContainerInfo container = - containerManager.getMatchingContainer(size, owner, newPipeline); + containerManager.getMatchingContainer(size, owner, newPipeline, + Collections.emptySet(), storageType); if (container == null) { // defensive null handling throw new IOException("Could not allocate a new container"); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java index a61b3289235..11bfc1d4445 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java @@ -22,12 +22,14 @@ import java.util.List; import java.util.stream.Collectors; import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.scm.PipelineChoosePolicy; import org.apache.hadoop.hdds.scm.PipelineRequestInformation; import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.ContainerManager; import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; import org.apache.hadoop.hdds.scm.exceptions.SCMException; +import org.apache.hadoop.hdds.scm.node.NodeManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,20 +45,44 @@ public class WritableRatisContainerProvider private final PipelineManager pipelineManager; private final PipelineChoosePolicy pipelineChoosePolicy; private final ContainerManager containerManager; + private final NodeManager nodeManager; public WritableRatisContainerProvider( PipelineManager pipelineManager, ContainerManager containerManager, - PipelineChoosePolicy pipelineChoosePolicy) { + PipelineChoosePolicy pipelineChoosePolicy, + NodeManager nodeManager) { this.pipelineManager = pipelineManager; this.containerManager = containerManager; this.pipelineChoosePolicy = pipelineChoosePolicy; + this.nodeManager = nodeManager; + } + + public WritableRatisContainerProvider( + PipelineManager pipelineManager, + ContainerManager containerManager, + PipelineChoosePolicy pipelineChoosePolicy) { + this(pipelineManager, containerManager, pipelineChoosePolicy, null); } @Override public ContainerInfo getContainer(final long size, ReplicationConfig repConfig, String owner, ExcludeList excludeList) throws IOException { + return getContainerInternal(size, repConfig, owner, excludeList, null); + } + + @Override + public ContainerInfo getContainer(final long size, + ReplicationConfig repConfig, String owner, ExcludeList excludeList, + StorageType storageType) throws IOException { + return getContainerInternal(size, repConfig, owner, excludeList, + storageType); + } + + private ContainerInfo getContainerInternal(final long size, + ReplicationConfig repConfig, String owner, ExcludeList excludeList, + StorageType storageType) throws IOException { /* Here is the high level logic. @@ -77,10 +103,13 @@ public ContainerInfo getContainer(final long size, //in downstream managers. PipelineRequestInformation req = - PipelineRequestInformation.Builder.getBuilder().setSize(size).build(); + PipelineRequestInformation.Builder.getBuilder() + .setSize(size) + .setStorageType(storageType) + .build(); ContainerInfo containerInfo = - getContainer(repConfig, owner, excludeList, req); + getContainer(repConfig, owner, excludeList, req, storageType); if (containerInfo != null) { return containerInfo; } @@ -126,7 +155,8 @@ public ContainerInfo getContainer(final long size, // If Exception occurred or successful creation of pipeline do one // final try to fetch pipelines. - containerInfo = getContainer(repConfig, owner, excludeList, req); + containerInfo = getContainer(repConfig, owner, excludeList, req, + storageType); if (containerInfo != null) { return containerInfo; } @@ -143,7 +173,8 @@ public ContainerInfo getContainer(final long size, @Nullable private ContainerInfo getContainer(ReplicationConfig repConfig, String owner, - ExcludeList excludeList, PipelineRequestInformation req) { + ExcludeList excludeList, PipelineRequestInformation req, + StorageType storageType) { // Acquire pipeline manager lock, to avoid any updates to pipeline // while allocate container happens. This is to avoid scenario like // mentioned in HDDS-5655. @@ -151,7 +182,12 @@ private ContainerInfo getContainer(ReplicationConfig repConfig, String owner, try { List availablePipelines = findPipelinesByState(repConfig, excludeList, Pipeline.PipelineState.OPEN); - return selectContainer(availablePipelines, req, owner, excludeList); + if (nodeManager != null) { + availablePipelines = PipelineStorageTypeFilter.filter( + availablePipelines, nodeManager, storageType); + } + return selectContainer(availablePipelines, req, owner, excludeList, + storageType); } finally { pipelineManager.releaseReadLock(); } @@ -174,15 +210,16 @@ private List findPipelinesByState( private @Nullable ContainerInfo selectContainer( List availablePipelines, PipelineRequestInformation req, - String owner, ExcludeList excludeList) { + String owner, ExcludeList excludeList, StorageType storageType) { while (!availablePipelines.isEmpty()) { Pipeline pipeline = pipelineChoosePolicy.choosePipeline( availablePipelines, req); // look for OPEN containers that match the criteria. - final ContainerInfo containerInfo = containerManager.getMatchingContainer( - req.getSize(), owner, pipeline, excludeList.getContainerIds()); + final ContainerInfo containerInfo = + containerManager.getMatchingContainer(req.getSize(), owner, + pipeline, excludeList.getContainerIds(), storageType); if (containerInfo != null) { return containerInfo; diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocolServerSideTranslatorPB.java index f9fa80bf42d..f30035ff027 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocolServerSideTranslatorPB.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmBlockLocationProtocolServerSideTranslatorPB.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos; import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateBlockResponse; @@ -193,6 +194,10 @@ private Status exceptionToResponseStatus(IOException ex) { public AllocateScmBlockResponseProto allocateScmBlock( AllocateScmBlockRequestProto request, int clientVersion) throws IOException { + StorageType storageType = request.hasStorageType() + ? StorageType.valueOf(request.getStorageType()) + : StorageType.DEFAULT; + List allocatedBlocks = impl.allocateBlock(request.getSize(), request.getNumBlocks(), @@ -202,7 +207,8 @@ public AllocateScmBlockResponseProto allocateScmBlock( request.getEcReplicationConfig()), request.getOwner(), ExcludeList.getFromProtoBuf(request.getExcludeList()), - request.getClient()); + request.getClient(), + storageType); AllocateScmBlockResponseProto.Builder builder = AllocateScmBlockResponseProto.newBuilder(); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java index 1fc7c2d96b2..c64e64460a4 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeID; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos; import org.apache.hadoop.hdds.scm.AddSCMRequest; import org.apache.hadoop.hdds.scm.ScmInfo; @@ -189,7 +190,8 @@ public List allocateBlock( long size, int num, ReplicationConfig replicationConfig, String owner, ExcludeList excludeList, - String clientMachine + String clientMachine, + StorageType storageType ) throws IOException { long startNanos = Time.monotonicNowNanos(); Map auditMap = Maps.newHashMap(); @@ -198,6 +200,7 @@ public List allocateBlock( auditMap.put("replication", replicationConfig.toString()); auditMap.put("owner", owner); auditMap.put("client", clientMachine); + auditMap.put("storageType", String.valueOf(storageType)); List blocks = new ArrayList<>(num); if (LOG.isDebugEnabled()) { @@ -207,7 +210,8 @@ public List allocateBlock( try { for (int i = 0; i < num; i++) { AllocatedBlock block = scm.getScmBlockManager() - .allocateBlock(size, replicationConfig, owner, excludeList); + .allocateBlock(size, replicationConfig, owner, excludeList, + storageType); if (block != null) { // Sort the datanodes if client machine is specified final Node client = getClientNode(clientMachine); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java index 45c947cb00a..f5e2af10147 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hdds.client.RatisReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto; @@ -197,7 +198,8 @@ public void testAllocateBlock() throws Exception { pipelineManager.createPipeline(replicationConfig); HddsTestUtils.openAllRatisPipelines(pipelineManager); AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE, - replicationConfig, OzoneConsts.OZONE, new ExcludeList()); + replicationConfig, OzoneConsts.OZONE, new ExcludeList(), + StorageType.DEFAULT); assertNotNull(block); } @@ -216,7 +218,7 @@ public void testAllocateBlockWithExclusion() throws Exception { .get(0).getId()); AllocatedBlock block = blockManager .allocateBlock(DEFAULT_BLOCK_SIZE, replicationConfig, OzoneConsts.OZONE, - excludeList); + excludeList, StorageType.DEFAULT); assertNotNull(block); for (PipelineID id : excludeList.getPipelineIds()) { assertNotEquals(block.getPipeline().getId(), id); @@ -227,7 +229,7 @@ public void testAllocateBlockWithExclusion() throws Exception { } block = blockManager .allocateBlock(DEFAULT_BLOCK_SIZE, replicationConfig, OzoneConsts.OZONE, - excludeList); + excludeList, StorageType.DEFAULT); assertNotNull(block); assertThat(excludeList.getPipelineIds()).contains(block.getPipeline().getId()); } @@ -249,7 +251,7 @@ void testAllocateBlockInParallel() throws Exception { future.complete(blockManager .allocateBlock(DEFAULT_BLOCK_SIZE, replicationConfig, OzoneConsts.OZONE, - new ExcludeList())); + new ExcludeList(), StorageType.DEFAULT)); } catch (IOException e) { future.completeExceptionally(e); } @@ -287,7 +289,7 @@ void testBlockDistribution() throws Exception { AllocatedBlock block = blockManager .allocateBlock(DEFAULT_BLOCK_SIZE, replicationConfig, OzoneConsts.OZONE, - new ExcludeList()); + new ExcludeList(), StorageType.DEFAULT); long containerId = block.getBlockID().getContainerID(); if (!allocatedBlockMap.containsKey(containerId)) { blockList = new ArrayList<>(); @@ -343,7 +345,7 @@ void testBlockDistributionWithMultipleDisks() throws Exception { AllocatedBlock block = blockManager .allocateBlock(DEFAULT_BLOCK_SIZE, replicationConfig, OzoneConsts.OZONE, - new ExcludeList()); + new ExcludeList(), StorageType.DEFAULT); long containerId = block.getBlockID().getContainerID(); if (!allocatedBlockMap.containsKey(containerId)) { blockList = new ArrayList<>(); @@ -403,7 +405,7 @@ void testBlockDistributionWithMultipleRaftLogDisks() throws Exception { AllocatedBlock block = blockManager .allocateBlock(DEFAULT_BLOCK_SIZE, replicationConfig, OzoneConsts.OZONE, - new ExcludeList()); + new ExcludeList(), StorageType.DEFAULT); long containerId = block.getBlockID().getContainerID(); if (!allocatedBlockMap.containsKey(containerId)) { blockList = new ArrayList<>(); @@ -439,7 +441,8 @@ public void testAllocateOversizedBlock() { long size = 6 * GB; Throwable t = assertThrows(IOException.class, () -> blockManager.allocateBlock(size, - replicationConfig, OzoneConsts.OZONE, new ExcludeList())); + replicationConfig, OzoneConsts.OZONE, new ExcludeList(), + StorageType.DEFAULT)); assertEquals("Unsupported block size: " + size, t.getMessage()); } @@ -450,7 +453,8 @@ public void testAllocateBlockFailureInSafeMode() { // Test1: In safe mode expect an SCMException. Throwable t = assertThrows(IOException.class, () -> blockManager.allocateBlock(DEFAULT_BLOCK_SIZE, - replicationConfig, OzoneConsts.OZONE, new ExcludeList())); + replicationConfig, OzoneConsts.OZONE, new ExcludeList(), + StorageType.DEFAULT)); assertEquals("SafeModePrecheck failed for allocateBlock", t.getMessage()); } @@ -459,7 +463,8 @@ public void testAllocateBlockFailureInSafeMode() { public void testAllocateBlockSucInSafeMode() throws Exception { // Test2: Exit safe mode and then try allocateBock again. assertNotNull(blockManager.allocateBlock(DEFAULT_BLOCK_SIZE, - replicationConfig, OzoneConsts.OZONE, new ExcludeList())); + replicationConfig, OzoneConsts.OZONE, new ExcludeList(), + StorageType.DEFAULT)); } @Test @@ -472,14 +477,14 @@ public void testMultipleBlockAllocation() AllocatedBlock allocatedBlock = blockManager .allocateBlock(DEFAULT_BLOCK_SIZE, replicationConfig, OzoneConsts.OZONE, - new ExcludeList()); + new ExcludeList(), StorageType.DEFAULT); // block should be allocated in different pipelines GenericTestUtils.waitFor(() -> { try { AllocatedBlock block = blockManager .allocateBlock(DEFAULT_BLOCK_SIZE, replicationConfig, OzoneConsts.OZONE, - new ExcludeList()); + new ExcludeList(), StorageType.DEFAULT); return !block.getPipeline().getId() .equals(allocatedBlock.getPipeline().getId()); } catch (IOException e) { @@ -525,7 +530,7 @@ public void testMultipleBlockAllocationWithClosedContainer() blockManager .allocateBlock(DEFAULT_BLOCK_SIZE, replicationConfig, OzoneConsts.OZONE, - new ExcludeList()); + new ExcludeList(), StorageType.DEFAULT); } catch (IOException e) { } return verifyNumberOfContainersInPipelines( @@ -550,7 +555,7 @@ public void testMultipleBlockAllocationWithClosedContainer() blockManager .allocateBlock(DEFAULT_BLOCK_SIZE, replicationConfig, OzoneConsts.OZONE, - new ExcludeList()); + new ExcludeList(), StorageType.DEFAULT); } catch (IOException e) { } return verifyNumberOfContainersInPipelines( @@ -567,7 +572,35 @@ public void testBlockAllocationWithNoAvailablePipelines() assertEquals(0, pipelineManager.getPipelines(replicationConfig).size()); assertNotNull(blockManager .allocateBlock(DEFAULT_BLOCK_SIZE, replicationConfig, OzoneConsts.OZONE, - new ExcludeList())); + new ExcludeList(), StorageType.DEFAULT)); + } + + /** + * Integration test: verifies the full BlockManagerImpl → WritableContainerFactory + * → WritableRatisContainerProvider → PipelineStorageTypeFilter chain. + * MockNodeManager reports only DISK volumes. Requesting SSD triggers the + * fallback path (SSD → DISK) and succeeds because DISK pipelines are + * available. + */ + @Test + public void testAllocateBlockFallsBackOnNonMatchingStorageType() + throws Exception { + pipelineManager.createPipeline(replicationConfig); + HddsTestUtils.openAllRatisPipelines(pipelineManager); + + // Verify DISK allocation works (baseline — all nodes report DISK) + AllocatedBlock diskBlock = blockManager.allocateBlock( + DEFAULT_BLOCK_SIZE, replicationConfig, OzoneConsts.OZONE, + new ExcludeList(), StorageType.DISK); + assertNotNull(diskBlock); + + // SSD allocation should fall back to DISK and succeed: MockNodeManager + // nodes only have DISK volumes, so the PipelineStorageTypeFilter + // removes all pipelines for SSD, but the fallback retries with DISK. + AllocatedBlock ssdBlock = blockManager.allocateBlock( + DEFAULT_BLOCK_SIZE, replicationConfig, OzoneConsts.OZONE, + new ExcludeList(), StorageType.SSD); + assertNotNull(ssdBlock); } private class DatanodeCommandHandler implements diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManagerImpl.java new file mode 100644 index 00000000000..e1d14f8379d --- /dev/null +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManagerImpl.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdds.scm.block; + +import static org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator.LOCAL_ID; +import static org.apache.hadoop.ozone.OzoneConsts.MB; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.CALLS_REAL_METHODS; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.lang.reflect.Field; +import org.apache.hadoop.hdds.client.RatisReplicationConfig; +import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.protocol.StorageType; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; +import org.apache.hadoop.hdds.scm.container.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock; +import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; +import org.apache.hadoop.hdds.scm.ha.SCMContext; +import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.pipeline.PipelineID; +import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; +import org.apache.hadoop.hdds.scm.pipeline.WritableContainerFactory; +import org.apache.hadoop.hdds.scm.server.StorageContainerManager; +import org.apache.ozone.test.GenericTestUtils; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; + +/** + * Unit tests for BlockManagerImpl storage-type fallback logic. + */ +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.LENIENT) +class TestBlockManagerImpl { + + private static final long DEFAULT_BLOCK_SIZE = 128 * MB; + private static final long CONTAINER_SIZE = DEFAULT_BLOCK_SIZE * 4; + private static final ReplicationConfig REPLICATION_CONFIG = + RatisReplicationConfig.getInstance(ReplicationFactor.THREE); + private static final String OWNER = "testOwner"; + + @Mock + private WritableContainerFactory mockFactory; + @Mock + private PipelineManager mockPipelineManager; + @Mock + private SequenceIdGenerator mockSequenceIdGen; + @Mock + private StorageContainerManager mockScm; + @Mock + private SCMContext mockScmContext; + @Mock + private ContainerInfo mockContainerInfo; + @Mock + private Pipeline mockPipeline; + + private BlockManagerImpl blockManager; + + @BeforeEach + void setUp() throws Exception { + when(mockScm.getScmContext()).thenReturn(mockScmContext); + when(mockScmContext.isInSafeMode()).thenReturn(false); + + PipelineID pipelineID = PipelineID.randomId(); + when(mockContainerInfo.getPipelineID()).thenReturn(pipelineID); + when(mockContainerInfo.getContainerID()).thenReturn(1L); + when(mockPipelineManager.getPipeline(pipelineID)) + .thenReturn(mockPipeline); + when(mockPipeline.getId()).thenReturn(pipelineID); + when(mockSequenceIdGen.getNextId(LOCAL_ID)).thenReturn(1L); + + blockManager = mock(BlockManagerImpl.class, CALLS_REAL_METHODS); + setField(blockManager, "scm", mockScm); + setField(blockManager, "writableContainerFactory", mockFactory); + setField(blockManager, "pipelineManager", mockPipelineManager); + setField(blockManager, "sequenceIdGen", mockSequenceIdGen); + setField(blockManager, "containerSize", CONTAINER_SIZE); + } + + @Test + void testAllocateBlockFallsBackOnStorageTypeFailure() throws Exception { + when(mockFactory.getContainer( + anyLong(), any(), anyString(), any(), eq(StorageType.SSD))) + .thenThrow(new IOException("No SSD pipeline available")); + when(mockFactory.getContainer( + anyLong(), any(), anyString(), any(), eq(StorageType.DISK))) + .thenReturn(mockContainerInfo); + + GenericTestUtils.LogCapturer logCapturer = + GenericTestUtils.LogCapturer.captureLogs(BlockManagerImpl.class); + + AllocatedBlock block = blockManager.allocateBlock( + DEFAULT_BLOCK_SIZE, REPLICATION_CONFIG, OWNER, + new ExcludeList(), StorageType.SSD); + + assertNotNull(block); + verify(mockFactory).getContainer( + anyLong(), any(), anyString(), any(), eq(StorageType.SSD)); + verify(mockFactory).getContainer( + anyLong(), any(), anyString(), any(), eq(StorageType.DISK)); + assertThat(logCapturer.getOutput()).contains("falling back to DISK"); + } + + @Test + void testAllocateBlockNoFallbackForDisk() throws Exception { + when(mockFactory.getContainer( + anyLong(), any(), anyString(), any(), eq(StorageType.DISK))) + .thenThrow(new IOException("No DISK pipeline available")); + + assertThrows(IOException.class, + () -> blockManager.allocateBlock(DEFAULT_BLOCK_SIZE, + REPLICATION_CONFIG, OWNER, new ExcludeList(), StorageType.DISK)); + + verify(mockFactory).getContainer( + anyLong(), any(), anyString(), any(), eq(StorageType.DISK)); + verify(mockFactory, never()).getContainer( + anyLong(), any(), anyString(), any(), eq(StorageType.SSD)); + } + + @Test + void testAllocateBlockPrimarySucceedsNoFallback() throws Exception { + when(mockFactory.getContainer( + anyLong(), any(), anyString(), any(), eq(StorageType.SSD))) + .thenReturn(mockContainerInfo); + + AllocatedBlock block = blockManager.allocateBlock( + DEFAULT_BLOCK_SIZE, REPLICATION_CONFIG, OWNER, + new ExcludeList(), StorageType.SSD); + + assertNotNull(block); + verify(mockFactory).getContainer( + anyLong(), any(), anyString(), any(), eq(StorageType.SSD)); + verify(mockFactory, never()).getContainer( + anyLong(), any(), anyString(), any(), eq(StorageType.DISK)); + } + + private static void setField(Object target, String fieldName, Object value) + throws Exception { + Class clazz = target.getClass(); + while (clazz != null) { + try { + Field field = clazz.getDeclaredField(fieldName); + field.setAccessible(true); + field.set(target, value); + return; + } catch (NoSuchFieldException e) { + clazz = clazz.getSuperclass(); + } + } + throw new NoSuchFieldException(fieldName); + } +} diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestECUnderReplicationHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestECUnderReplicationHandler.java index 5d2af561196..ba9bd3e45cf 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestECUnderReplicationHandler.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestECUnderReplicationHandler.java @@ -73,6 +73,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto; import org.apache.hadoop.hdds.scm.ContainerPlacementStatus; @@ -103,6 +104,7 @@ import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.ArgumentCaptor; /** * Tests the ECUnderReplicationHandling functionality. @@ -1150,6 +1152,71 @@ public void testMaintenanceIndexCopiedWhenContainerUnRecoverable() assertEquals(maintReplica.getDatanodeDetails(), target); } + @Test + public void testReconstructionCommandCarriesStorageType() throws IOException { + // Create a container with storageType SSD + ContainerInfo containerWithStorageType = new ContainerInfo.Builder() + .setContainerID(container.getContainerID()) + .setPipelineID(container.getPipelineID()) + .setState(HddsProtos.LifeCycleState.CLOSED) + .setOwner("scm") + .setReplicationConfig(repConfig) + .setStorageType(StorageType.SSD) + .build(); + + Set availableReplicas = createReplicas(4); + ECUnderReplicationHandler ecURH = + new ECUnderReplicationHandler(policy, conf, replicationManager); + UnderReplicatedHealthResult result = + mock(UnderReplicatedHealthResult.class); + when(result.getContainerInfo()).thenReturn(containerWithStorageType); + + ecURH.processAndSendCommands(availableReplicas, emptyList(), result, + remainingMaintenanceRedundancy); + assertEquals(1, commandsSent.size()); + SCMCommand cmd = commandsSent.iterator().next().getValue(); + assertThat(cmd).isInstanceOf(ReconstructECContainersCommand.class); + assertEquals(StorageType.SSD, + ((ReconstructECContainersCommand) cmd).getStorageType()); + } + + @Test + public void testReplicateCommandCarriesStorageType() throws IOException { + // Create a container with storageType SSD + ContainerInfo containerWithStorageType = new ContainerInfo.Builder() + .setContainerID(container.getContainerID()) + .setPipelineID(container.getPipelineID()) + .setState(HddsProtos.LifeCycleState.CLOSED) + .setOwner("scm") + .setReplicationConfig(repConfig) + .setStorageType(StorageType.SSD) + .build(); + + // Decommissioning index triggers a replicate command + Set availableReplicas = ReplicationTestUtil + .createReplicas(Pair.of(DECOMMISSIONING, 1), Pair.of(IN_SERVICE, 2), + Pair.of(IN_SERVICE, 3), Pair.of(IN_SERVICE, 4), + Pair.of(IN_SERVICE, 5)); + + ECUnderReplicationHandler ecURH = + new ECUnderReplicationHandler(policy, conf, replicationManager); + UnderReplicatedHealthResult result = + mock(UnderReplicatedHealthResult.class); + when(result.getContainerInfo()).thenReturn(containerWithStorageType); + + ecURH.processAndSendCommands(availableReplicas, emptyList(), result, + remainingMaintenanceRedundancy); + + // Verify that the ContainerInfo passed to sendThrottledReplicationCommand + // has storageType SSD + ArgumentCaptor containerCaptor = + ArgumentCaptor.forClass(ContainerInfo.class); + verify(replicationManager).sendThrottledReplicationCommand( + containerCaptor.capture(), anyList(), any(DatanodeDetails.class), + anyInt()); + assertEquals(StorageType.SSD, containerCaptor.getValue().getStorageType()); + } + public Set>> testUnderReplicationWithMissingIndexes( List missingIndexes, Set availableReplicas, diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestRatisUnderReplicationHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestRatisUnderReplicationHandler.java index f10fff8695b..783b8ab286d 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestRatisUnderReplicationHandler.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestRatisUnderReplicationHandler.java @@ -33,6 +33,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyList; import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; @@ -54,6 +55,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State; @@ -722,6 +724,45 @@ public void testQuasiClosedReplicasAreSourcesWhenOnlyTheyAreAvailable() getUnderReplicatedHealthResult(), 2, 2); } + @Test + public void testReplicationCommandCarriesStorageType() + throws IOException { + // Create a container with storageType ARCHIVE + ContainerInfo containerWithStorageType = new ContainerInfo.Builder() + .setContainerID(container.getContainerID()) + .setPipelineID(container.getPipelineID()) + .setState(HddsProtos.LifeCycleState.CLOSED) + .setOwner("scm") + .setReplicationConfig(RATIS_REPLICATION_CONFIG) + .setStorageType(StorageType.ARCHIVE) + .build(); + + Set replicas + = createReplicas(containerWithStorageType.containerID(), State.CLOSED, 0); + + RatisUnderReplicationHandler handler = + new RatisUnderReplicationHandler(policy, conf, replicationManager); + + UnderReplicatedHealthResult healthResult = + mock(UnderReplicatedHealthResult.class); + when(healthResult.getContainerInfo()).thenReturn(containerWithStorageType); + + handler.processAndSendCommands(replicas, Collections.emptyList(), + healthResult, 2); + assertEquals(2, commandsSent.size()); + + // Verify that the ContainerInfo passed to sendThrottledReplicationCommand + // has storageType ARCHIVE + ArgumentCaptor containerCaptor = + ArgumentCaptor.forClass(ContainerInfo.class); + verify(replicationManager, times(2)).sendThrottledReplicationCommand( + containerCaptor.capture(), anyList(), any(DatanodeDetails.class), + anyInt()); + for (ContainerInfo captured : containerCaptor.getAllValues()) { + assertEquals(StorageType.ARCHIVE, captured.getStorageType()); + } + } + /** * Tests whether the specified expectNumCommands number of commands are * created by the handler. diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java index 1d7d619efb0..910dde5951f 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java @@ -72,6 +72,7 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeID; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto; @@ -84,6 +85,7 @@ import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; import org.apache.hadoop.hdds.scm.container.ContainerReplica; import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport; +import org.apache.hadoop.hdds.scm.container.TestContainerInfo; import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementStatusDefault; import org.apache.hadoop.hdds.scm.events.SCMEvents; import org.apache.hadoop.hdds.scm.exceptions.SCMException; @@ -1759,6 +1761,32 @@ public void testReconfigureContainerSampleLimit() { "Second report should have 50 samples after reconfiguration"); } + @Test + public void testSendThrottledReplicationCommandSetsStorageType() + throws Exception { + ContainerInfo containerWithStorageType = + TestContainerInfo.newBuilderForTest() + .setContainerID(1) + .setState(HddsProtos.LifeCycleState.CLOSED) + .setReplicationConfig(RatisReplicationConfig.getInstance(THREE)) + .setStorageType(StorageType.SSD) + .build(); + + DatanodeDetails source = MockDatanodeDetails.randomDatanodeDetails(); + DatanodeDetails target = MockDatanodeDetails.randomDatanodeDetails(); + mockReplicationCommandCounts(any -> 0, any -> 0); + + replicationManager.sendThrottledReplicationCommand( + containerWithStorageType, + Collections.singletonList(source), target, 0); + + assertEquals(1, commandsSent.size()); + SCMCommand cmd = commandsSent.iterator().next().getValue(); + assertThat(cmd).isInstanceOf(ReplicateContainerCommand.class); + assertEquals(StorageType.SSD, + ((ReplicateContainerCommand) cmd).getStorageType()); + } + @SafeVarargs private final Set addReplicas(ContainerInfo container, ContainerReplicaProto.State replicaState, diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestBackgroundPipelineCreatorStorageType.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestBackgroundPipelineCreatorStorageType.java new file mode 100644 index 00000000000..8c3e150c0f9 --- /dev/null +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestBackgroundPipelineCreatorStorageType.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdds.scm.pipeline; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.time.Instant; +import java.time.ZoneOffset; +import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.StorageType; +import org.apache.hadoop.hdds.scm.ScmConfigKeys; +import org.apache.hadoop.hdds.scm.ha.SCMContext; +import org.apache.ozone.test.TestClock; +import org.junit.jupiter.api.Test; + +/** + * Tests for storage-type-aware pipeline creation in + * BackgroundPipelineCreator. + */ +public class TestBackgroundPipelineCreatorStorageType { + + @Test + public void testStorageTypeAwareDisabled() throws IOException { + OzoneConfiguration conf = new OzoneConfiguration(); + conf.setBoolean( + ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_STORAGE_TYPE_AWARE, + false); + + PipelineManager pipelineManager = mock(PipelineManager.class); + when(pipelineManager.createPipeline(any(ReplicationConfig.class))) + .thenThrow(new IOException("exhausted")); + + SCMContext scmContext = SCMContext.emptyContext(); + + TestClock clock = new TestClock(Instant.now(), ZoneOffset.UTC); + BackgroundPipelineCreator creator = + new BackgroundPipelineCreator(pipelineManager, conf, scmContext, + clock); + + creator.createPipelines(); + + // Untyped createPipeline(ReplicationConfig) should have been called. + verify(pipelineManager, atLeastOnce()) + .createPipeline(any(ReplicationConfig.class)); + // Typed createPipeline(ReplicationConfig, StorageType) should NOT + // have been called. + verify(pipelineManager, never()) + .createPipeline(any(ReplicationConfig.class), + any(StorageType.class)); + } + + @Test + public void testStorageTypeAwareEnabled() throws IOException { + OzoneConfiguration conf = new OzoneConfiguration(); + conf.setBoolean( + ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_STORAGE_TYPE_AWARE, + true); + + PipelineManager pipelineManager = mock(PipelineManager.class); + when(pipelineManager.createPipeline(any(ReplicationConfig.class))) + .thenThrow(new IOException("exhausted")); + when(pipelineManager.createPipeline(any(ReplicationConfig.class), + any(StorageType.class))) + .thenThrow(new IOException("exhausted")); + + SCMContext scmContext = SCMContext.emptyContext(); + + TestClock clock = new TestClock(Instant.now(), ZoneOffset.UTC); + BackgroundPipelineCreator creator = + new BackgroundPipelineCreator(pipelineManager, conf, scmContext, + clock); + + creator.createPipelines(); + + // When storage-type-aware is enabled, the typed method should be called + // for SSD, DISK, and ARCHIVE. + verify(pipelineManager, atLeastOnce()) + .createPipeline(any(ReplicationConfig.class), + any(StorageType.class)); + } +} diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java index e7fc6f14f9b..557ebb3a2ca 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java @@ -71,6 +71,7 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeID; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; @@ -410,12 +411,14 @@ public void testPipelineReport() throws Exception { public void testPipelineCreationFailedMetric() throws Exception { PipelineManagerImpl pipelineManager = createPipelineManager(true); - // No pipeline at start + // Capture baseline counters (may be non-zero from prior tests + // sharing the same JMX metrics registry) MetricsRecordBuilder metrics = getMetrics( SCMPipelineMetrics.class.getSimpleName()); - long numPipelineAllocated = getLongCounter("NumPipelineAllocated", + long baselineAllocated = getLongCounter("NumPipelineAllocated", + metrics); + long baselineFailed = getLongCounter("NumPipelineCreationFailed", metrics); - assertEquals(0, numPipelineAllocated); // 3 DNs are unhealthy. // Create 5 pipelines (Use up 15 Datanodes) @@ -429,12 +432,12 @@ public void testPipelineCreationFailedMetric() throws Exception { metrics = getMetrics( SCMPipelineMetrics.class.getSimpleName()); - numPipelineAllocated = getLongCounter("NumPipelineAllocated", metrics); - assertEquals(maxPipelineCount, numPipelineAllocated); + long numPipelineAllocated = getLongCounter("NumPipelineAllocated", metrics); + assertEquals(maxPipelineCount, numPipelineAllocated - baselineAllocated); long numPipelineCreateFailed = getLongCounter( "NumPipelineCreationFailed", metrics); - assertEquals(0, numPipelineCreateFailed); + assertEquals(0, numPipelineCreateFailed - baselineFailed); //This should fail... SCMException e = @@ -446,11 +449,11 @@ public void testPipelineCreationFailedMetric() throws Exception { metrics = getMetrics( SCMPipelineMetrics.class.getSimpleName()); numPipelineAllocated = getLongCounter("NumPipelineAllocated", metrics); - assertEquals(maxPipelineCount, numPipelineAllocated); + assertEquals(maxPipelineCount, numPipelineAllocated - baselineAllocated); numPipelineCreateFailed = getLongCounter( "NumPipelineCreationFailed", metrics); - assertEquals(1, numPipelineCreateFailed); + assertEquals(1, numPipelineCreateFailed - baselineFailed); // clean up pipelineManager.close(); @@ -881,7 +884,7 @@ public void testWaitForAllocatedPipeline() throws IOException { pipelineManager.addContainerToPipeline( allocatedPipeline.getId(), container.containerID()); doReturn(container).when(containerManager).getMatchingContainer(anyLong(), - anyString(), eq(allocatedPipeline), any()); + anyString(), eq(allocatedPipeline), any(), any()); assertTrue(pipelineManager.getPipelines(repConfig, OPEN) @@ -920,6 +923,34 @@ public void testWaitForAllocatedPipeline() throws IOException { pipelineManager.close(); } + @Test + public void testCreatePipelineWithStorageType() throws Exception { + PipelineManagerImpl pipelineManager = createPipelineManager(true); + + // MockNodeManager creates storage reports with DISK type by default. + // DISK-typed pipeline should succeed. + Pipeline diskPipeline = pipelineManager.createPipeline( + RatisReplicationConfig.getInstance(ReplicationFactor.THREE), + StorageType.DISK); + assertNotNull(diskPipeline); + assertEquals(3, diskPipeline.getNodes().size()); + + // SSD-typed pipeline should fail since no nodes have SSD storage. + assertThrows(IOException.class, + () -> pipelineManager.createPipeline( + RatisReplicationConfig.getInstance(ReplicationFactor.THREE), + StorageType.SSD)); + + // null StorageType should fall through to untyped creation. + Pipeline untypedPipeline = pipelineManager.createPipeline( + RatisReplicationConfig.getInstance(ReplicationFactor.THREE), + (StorageType) null); + assertNotNull(untypedPipeline); + assertEquals(3, untypedPipeline.getNodes().size()); + + pipelineManager.close(); + } + public void testCreatePipelineForRead() throws IOException { PipelineManager pipelineManager = createPipelineManager(true); List dns = nodeManager diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineStorageTypeFilter.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineStorageTypeFilter.java new file mode 100644 index 00000000000..eca565c0d75 --- /dev/null +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineStorageTypeFilter.java @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdds.scm.pipeline; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.StorageTypeProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto; +import org.apache.hadoop.hdds.scm.node.DatanodeInfo; +import org.apache.hadoop.hdds.scm.node.NodeManager; +import org.apache.hadoop.hdds.scm.node.NodeStatus; +import org.junit.jupiter.api.Test; + +/** + * Tests for PipelineStorageTypeFilter. + */ +class TestPipelineStorageTypeFilter { + + @Test + void testNullStorageTypeReturnsAllPipelines() { + NodeManager nodeManager = mock(NodeManager.class); + List pipelines = Arrays.asList( + MockPipeline.createRatisPipeline(), + MockPipeline.createRatisPipeline()); + List result = PipelineStorageTypeFilter.filter( + pipelines, nodeManager, null); + assertEquals(2, result.size()); + } + + @Test + void testFilterRetainsPipelinesWithMatchingNodes() { + NodeManager nodeManager = mock(NodeManager.class); + + // Create nodes with SSD storage + DatanodeDetails ssdNode1 = MockDatanodeDetails.randomDatanodeDetails(); + DatanodeDetails ssdNode2 = MockDatanodeDetails.randomDatanodeDetails(); + DatanodeDetails ssdNode3 = MockDatanodeDetails.randomDatanodeDetails(); + + // Create nodes with DISK storage only + DatanodeDetails diskNode1 = MockDatanodeDetails.randomDatanodeDetails(); + DatanodeDetails diskNode2 = MockDatanodeDetails.randomDatanodeDetails(); + DatanodeDetails diskNode3 = MockDatanodeDetails.randomDatanodeDetails(); + + List allNodes = Arrays.asList( + ssdNode1, ssdNode2, ssdNode3, diskNode1, diskNode2, diskNode3); + when(nodeManager.getNodes(NodeStatus.inServiceHealthy())) + .thenReturn(allNodes); + + // Configure SSD nodes to report SSD storage + for (DatanodeDetails ssdNode : Arrays.asList(ssdNode1, ssdNode2, + ssdNode3)) { + DatanodeInfo ssdInfo = mockDatanodeInfo(ssdNode, + StorageTypeProto.SSD); + when(nodeManager.getDatanodeInfo(ssdNode)).thenReturn(ssdInfo); + } + + // Configure DISK nodes to report DISK storage + for (DatanodeDetails diskNode : Arrays.asList(diskNode1, diskNode2, + diskNode3)) { + DatanodeInfo diskInfo = mockDatanodeInfo(diskNode, + StorageTypeProto.DISK); + when(nodeManager.getDatanodeInfo(diskNode)).thenReturn(diskInfo); + } + + // Pipeline with all SSD nodes + Pipeline ssdPipeline = createPipelineWithNodes( + Arrays.asList(ssdNode1, ssdNode2, ssdNode3)); + + // Pipeline with all DISK nodes + Pipeline diskPipeline = createPipelineWithNodes( + Arrays.asList(diskNode1, diskNode2, diskNode3)); + + // Mixed pipeline + Pipeline mixedPipeline = createPipelineWithNodes( + Arrays.asList(ssdNode1, diskNode1, ssdNode2)); + + List pipelines = new ArrayList<>( + Arrays.asList(ssdPipeline, diskPipeline, mixedPipeline)); + + // Filter for SSD — only the all-SSD pipeline should remain + List ssdResult = PipelineStorageTypeFilter.filter( + pipelines, nodeManager, StorageType.SSD); + assertEquals(1, ssdResult.size()); + assertEquals(ssdPipeline.getId(), ssdResult.get(0).getId()); + + // Filter for DISK — only the all-DISK pipeline should remain + List diskResult = PipelineStorageTypeFilter.filter( + pipelines, nodeManager, StorageType.DISK); + assertEquals(1, diskResult.size()); + assertEquals(diskPipeline.getId(), diskResult.get(0).getId()); + } + + @Test + void testFilterReturnsEmptyWhenNoMatch() { + NodeManager nodeManager = mock(NodeManager.class); + + DatanodeDetails diskNode1 = MockDatanodeDetails.randomDatanodeDetails(); + DatanodeDetails diskNode2 = MockDatanodeDetails.randomDatanodeDetails(); + DatanodeDetails diskNode3 = MockDatanodeDetails.randomDatanodeDetails(); + + when(nodeManager.getNodes(NodeStatus.inServiceHealthy())) + .thenReturn(Arrays.asList(diskNode1, diskNode2, diskNode3)); + + for (DatanodeDetails dn : Arrays.asList(diskNode1, diskNode2, + diskNode3)) { + DatanodeInfo info = mockDatanodeInfo(dn, StorageTypeProto.DISK); + when(nodeManager.getDatanodeInfo(dn)).thenReturn(info); + } + + Pipeline pipeline = createPipelineWithNodes( + Arrays.asList(diskNode1, diskNode2, diskNode3)); + List pipelines = new ArrayList<>( + Collections.singletonList(pipeline)); + + // Filter for SSD — no pipeline should match + List result = PipelineStorageTypeFilter.filter( + pipelines, nodeManager, StorageType.SSD); + assertTrue(result.isEmpty()); + } + + @Test + void testGetNodesWithStorageType() { + NodeManager nodeManager = mock(NodeManager.class); + + DatanodeDetails ssdNode = MockDatanodeDetails.randomDatanodeDetails(); + DatanodeDetails diskNode = MockDatanodeDetails.randomDatanodeDetails(); + DatanodeDetails bothNode = MockDatanodeDetails.randomDatanodeDetails(); + + when(nodeManager.getNodes(NodeStatus.inServiceHealthy())) + .thenReturn(Arrays.asList(ssdNode, diskNode, bothNode)); + + // Create mock DatanodeInfo objects first, then stub nodeManager + DatanodeInfo ssdInfo = mockDatanodeInfo(ssdNode, StorageTypeProto.SSD); + DatanodeInfo diskInfo = mockDatanodeInfo(diskNode, StorageTypeProto.DISK); + DatanodeInfo bothInfo = mock(DatanodeInfo.class); + when(bothInfo.getStorageReports()).thenReturn(Arrays.asList( + createStorageReport(StorageTypeProto.SSD), + createStorageReport(StorageTypeProto.DISK))); + + when(nodeManager.getDatanodeInfo(ssdNode)).thenReturn(ssdInfo); + when(nodeManager.getDatanodeInfo(diskNode)).thenReturn(diskInfo); + when(nodeManager.getDatanodeInfo(bothNode)).thenReturn(bothInfo); + + Set ssdNodes = PipelineStorageTypeFilter + .getNodesWithStorageType(nodeManager, StorageType.SSD); + assertEquals(2, ssdNodes.size()); + assertTrue(ssdNodes.contains(ssdNode.getUuid())); + assertTrue(ssdNodes.contains(bothNode.getUuid())); + + Set diskNodes = PipelineStorageTypeFilter + .getNodesWithStorageType(nodeManager, StorageType.DISK); + assertEquals(2, diskNodes.size()); + assertTrue(diskNodes.contains(diskNode.getUuid())); + assertTrue(diskNodes.contains(bothNode.getUuid())); + } + + @Test + void testNodeWithNullDatanodeInfoIsSkipped() { + NodeManager nodeManager = mock(NodeManager.class); + + DatanodeDetails node = MockDatanodeDetails.randomDatanodeDetails(); + when(nodeManager.getNodes(NodeStatus.inServiceHealthy())) + .thenReturn(Collections.singletonList(node)); + when(nodeManager.getDatanodeInfo(node)).thenReturn(null); + + Set result = PipelineStorageTypeFilter + .getNodesWithStorageType(nodeManager, StorageType.SSD); + assertTrue(result.isEmpty()); + } + + private static DatanodeInfo mockDatanodeInfo(DatanodeDetails dn, + StorageTypeProto storageType) { + DatanodeInfo info = mock(DatanodeInfo.class); + when(info.getStorageReports()).thenReturn( + Collections.singletonList(createStorageReport(storageType))); + return info; + } + + private static StorageReportProto createStorageReport( + StorageTypeProto storageType) { + return StorageReportProto.newBuilder() + .setStorageUuid("uuid-" + UUID.randomUUID()) + .setStorageLocation("/data") + .setCapacity(100L * 1024 * 1024 * 1024) + .setScmUsed(10L * 1024 * 1024 * 1024) + .setRemaining(90L * 1024 * 1024 * 1024) + .setStorageType(storageType) + .build(); + } + + private static Pipeline createPipelineWithNodes( + List nodes) { + return Pipeline.newBuilder() + .setState(Pipeline.PipelineState.OPEN) + .setId(PipelineID.randomId()) + .setReplicationConfig( + org.apache.hadoop.hdds.client.RatisReplicationConfig.getInstance( + org.apache.hadoop.hdds.protocol.proto.HddsProtos + .ReplicationFactor.THREE)) + .setNodes(nodes) + .build(); + } +} diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableECContainerProvider.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableECContainerProvider.java index b3c34b44e4c..2384f3899aa 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableECContainerProvider.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableECContainerProvider.java @@ -35,6 +35,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import java.io.File; import java.io.IOException; @@ -55,7 +56,10 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.StorageTypeProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto; import org.apache.hadoop.hdds.scm.PipelineChoosePolicy; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.ContainerID; @@ -70,6 +74,9 @@ import org.apache.hadoop.hdds.scm.net.NetworkTopologyImpl; import org.apache.hadoop.hdds.scm.net.NodeSchema; import org.apache.hadoop.hdds.scm.net.NodeSchemaManager; +import org.apache.hadoop.hdds.scm.node.DatanodeInfo; +import org.apache.hadoop.hdds.scm.node.NodeManager; +import org.apache.hadoop.hdds.scm.node.NodeStatus; import org.apache.hadoop.hdds.scm.pipeline.WritableECContainerProvider.WritableECContainerProviderConfig; import org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.CapacityPipelineChoosePolicy; import org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.HealthyPipelineChoosePolicy; @@ -78,6 +85,7 @@ import org.apache.hadoop.hdds.utils.db.DBStoreBuilder; import org.apache.hadoop.hdds.utils.db.RocksDatabaseException; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; @@ -144,7 +152,7 @@ void setup(@TempDir File testDir) throws IOException { containers.put(container.containerID(), container); return container; }).when(containerManager).getMatchingContainer(anyLong(), - anyString(), any(Pipeline.class)); + anyString(), any(Pipeline.class), any(), any()); doAnswer(call -> containers.get((ContainerID)call.getArguments()[0])) @@ -576,4 +584,236 @@ private long getMaxContainerSize() { ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, BYTES); } + @ParameterizedTest + @MethodSource("policies") + void testNullStorageTypeSkipsFilter(PipelineChoosePolicy policy) + throws IOException { + provider = createSubject(policy); + // Null storageType should behave identically to the 4-param method — + // both succeed without error + ContainerInfo container4 = provider.getContainer( + 1, repConfig, OWNER, new ExcludeList()); + assertNotNull(container4); + + ContainerInfo container5 = provider.getContainer( + 1, repConfig, OWNER, new ExcludeList(), null); + assertNotNull(container5); + } + + @ParameterizedTest + @MethodSource("policies") + void testStorageTypeDiskMatchesDefaultNodes(PipelineChoosePolicy policy) + throws IOException { + provider = createSubject(policy); + // MockNodeManager reports DISK by default (null StorageTypeProto + // defaults to DISK), so filtering for DISK should succeed + ContainerInfo container = provider.getContainer( + 1, repConfig, OWNER, new ExcludeList(), StorageType.DISK); + assertNotNull(container); + } + + /** + * Tests that StorageType filtering actually rejects pipelines whose nodes + * don't have the requested type. Uses fully mocked components to control + * exactly which storage types each node reports, and prevents new pipeline + * creation so the test can't pass via the fallback allocation path. + */ + @Test + void testStorageTypeFilterRejectsNonMatchingPipelines() + throws IOException { + // Create 5 EC nodes (3+2) + List nodes = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + nodes.add(MockDatanodeDetails.randomDatanodeDetails()); + } + Map replicaIndexes = new HashMap<>(); + for (int i = 0; i < nodes.size(); i++) { + replicaIndexes.put(nodes.get(i), i + 1); + } + + Pipeline pipeline = Pipeline.newBuilder() + .setState(Pipeline.PipelineState.OPEN) + .setId(PipelineID.randomId()) + .setReplicationConfig(repConfig) + .setNodes(nodes) + .setReplicaIndexes(replicaIndexes) + .build(); + + ContainerInfo containerInfo = createContainer(pipeline, repConfig, 1L); + + // Mock PipelineManager: one open pipeline, createPipeline always fails + PipelineManager pm = mock(PipelineManager.class); + when(pm.getPipelineCount(repConfig, Pipeline.PipelineState.OPEN)) + .thenReturn(100); // at max, skip initial allocateContainer + when(pm.getPipelines(repConfig, Pipeline.PipelineState.OPEN)) + .thenReturn(new ArrayList<>(Collections.singletonList(pipeline))); + when(pm.getContainersInPipeline(pipeline.getId())) + .thenReturn(new java.util.TreeSet<>( + Collections.singleton(containerInfo.containerID()))); + + // Mock ContainerManager + ContainerManager cm = mock(ContainerManager.class); + when(cm.getContainer(containerInfo.containerID())) + .thenReturn(containerInfo); + when(cm.getMatchingContainer(anyLong(), anyString(), + any(Pipeline.class), any(), any())) + .thenReturn(containerInfo); + + // Mock NodeManager: all nodes report only DISK + NodeManager nm = mock(NodeManager.class); + when(nm.getNodes(NodeStatus.inServiceHealthy())).thenReturn(nodes); + when(nm.getNodeCount(NodeStatus.inServiceHealthy())).thenReturn( + nodes.size()); + for (DatanodeDetails dn : nodes) { + DatanodeInfo info = mock(DatanodeInfo.class); + when(info.getStorageReports()).thenReturn( + Collections.singletonList(createDiskStorageReport())); + when(nm.getDatanodeInfo(dn)).thenReturn(info); + } + + WritableECContainerProvider ecProvider = new WritableECContainerProvider( + providerConf, getMaxContainerSize(), nm, pm, cm, + new RandomPipelineChoosePolicy()); + + // null storageType: filter is no-op, should return the container + ContainerInfo result = ecProvider.getContainer( + 1, repConfig, OWNER, new ExcludeList(), null); + assertNotNull(result); + assertEquals(containerInfo.containerID(), result.containerID()); + + // DISK storageType: all nodes have DISK, pipeline should pass filter + result = ecProvider.getContainer( + 1, repConfig, OWNER, new ExcludeList(), StorageType.DISK); + assertNotNull(result); + assertEquals(containerInfo.containerID(), result.containerID()); + + // SSD storageType: no nodes have SSD, filter removes the pipeline. + // Since openPipelineCount >= maximumPipelines and nodeCount <= + // maximumPipelines, the fallback allocateContainer is also blocked. + assertThrows(IOException.class, + () -> ecProvider.getContainer( + 1, repConfig, OWNER, new ExcludeList(), StorageType.SSD)); + } + + /** + * Tests that filtering works correctly with a mix of SSD and DISK + * pipelines — the SSD pipeline is returned when SSD is requested, and + * the DISK pipeline is returned when DISK is requested. + */ + @Test + void testStorageTypeFilterSelectsCorrectPipeline() + throws IOException { + // Create two sets of nodes: SSD nodes and DISK nodes + List ssdNodes = new ArrayList<>(); + List diskNodes = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + ssdNodes.add(MockDatanodeDetails.randomDatanodeDetails()); + diskNodes.add(MockDatanodeDetails.randomDatanodeDetails()); + } + + Map ssdIndexes = new HashMap<>(); + Map diskIndexes = new HashMap<>(); + for (int i = 0; i < 5; i++) { + ssdIndexes.put(ssdNodes.get(i), i + 1); + diskIndexes.put(diskNodes.get(i), i + 1); + } + + Pipeline ssdPipeline = Pipeline.newBuilder() + .setState(Pipeline.PipelineState.OPEN) + .setId(PipelineID.randomId()) + .setReplicationConfig(repConfig) + .setNodes(ssdNodes) + .setReplicaIndexes(ssdIndexes) + .build(); + + Pipeline diskPipeline = Pipeline.newBuilder() + .setState(Pipeline.PipelineState.OPEN) + .setId(PipelineID.randomId()) + .setReplicationConfig(repConfig) + .setNodes(diskNodes) + .setReplicaIndexes(diskIndexes) + .build(); + + ContainerInfo ssdContainer = createContainer(ssdPipeline, repConfig, 1L); + ContainerInfo diskContainer = createContainer(diskPipeline, repConfig, 2L); + + // Mock PipelineManager with both pipelines + PipelineManager pm = mock(PipelineManager.class); + when(pm.getPipelineCount(repConfig, Pipeline.PipelineState.OPEN)) + .thenReturn(100); + when(pm.getPipelines(repConfig, Pipeline.PipelineState.OPEN)) + .thenReturn(new ArrayList<>( + java.util.Arrays.asList(ssdPipeline, diskPipeline))); + when(pm.getContainersInPipeline(ssdPipeline.getId())) + .thenReturn(new java.util.TreeSet<>( + Collections.singleton(ssdContainer.containerID()))); + when(pm.getContainersInPipeline(diskPipeline.getId())) + .thenReturn(new java.util.TreeSet<>( + Collections.singleton(diskContainer.containerID()))); + + // Mock ContainerManager + ContainerManager cm = mock(ContainerManager.class); + when(cm.getContainer(ssdContainer.containerID())) + .thenReturn(ssdContainer); + when(cm.getContainer(diskContainer.containerID())) + .thenReturn(diskContainer); + + // Mock NodeManager + List allNodes = new ArrayList<>(); + allNodes.addAll(ssdNodes); + allNodes.addAll(diskNodes); + NodeManager nm = mock(NodeManager.class); + when(nm.getNodes(NodeStatus.inServiceHealthy())).thenReturn(allNodes); + when(nm.getNodeCount(NodeStatus.inServiceHealthy())) + .thenReturn(allNodes.size()); + for (DatanodeDetails dn : ssdNodes) { + DatanodeInfo info = mock(DatanodeInfo.class); + when(info.getStorageReports()).thenReturn( + Collections.singletonList(createSsdStorageReport())); + when(nm.getDatanodeInfo(dn)).thenReturn(info); + } + for (DatanodeDetails dn : diskNodes) { + DatanodeInfo info = mock(DatanodeInfo.class); + when(info.getStorageReports()).thenReturn( + Collections.singletonList(createDiskStorageReport())); + when(nm.getDatanodeInfo(dn)).thenReturn(info); + } + + WritableECContainerProvider ecProvider = new WritableECContainerProvider( + providerConf, getMaxContainerSize(), nm, pm, cm, + new RandomPipelineChoosePolicy()); + + // Request SSD: should get the SSD pipeline's container + ContainerInfo result = ecProvider.getContainer( + 1, repConfig, OWNER, new ExcludeList(), StorageType.SSD); + assertEquals(ssdPipeline.getId(), result.getPipelineID()); + + // Request DISK: should get the DISK pipeline's container + result = ecProvider.getContainer( + 1, repConfig, OWNER, new ExcludeList(), StorageType.DISK); + assertEquals(diskPipeline.getId(), result.getPipelineID()); + } + + private static StorageReportProto createDiskStorageReport() { + return StorageReportProto.newBuilder() + .setStorageUuid("uuid-" + java.util.UUID.randomUUID()) + .setStorageLocation("/data") + .setCapacity(100L * 1024 * 1024 * 1024) + .setScmUsed(10L * 1024 * 1024 * 1024) + .setRemaining(90L * 1024 * 1024 * 1024) + .setStorageType(StorageTypeProto.DISK) + .build(); + } + + private static StorageReportProto createSsdStorageReport() { + return StorageReportProto.newBuilder() + .setStorageUuid("uuid-" + java.util.UUID.randomUUID()) + .setStorageLocation("/ssd") + .setCapacity(100L * 1024 * 1024 * 1024) + .setScmUsed(10L * 1024 * 1024 * 1024) + .setRemaining(90L * 1024 * 1024 * 1024) + .setStorageType(StorageTypeProto.SSD) + .build(); + } + } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableRatisContainerProvider.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableRatisContainerProvider.java index a1ba81d0a70..ea6aa35952e 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableRatisContainerProvider.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestWritableRatisContainerProvider.java @@ -24,6 +24,9 @@ import static org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState.OPEN; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -31,17 +34,26 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.hdds.client.RatisReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.StorageTypeProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto; import org.apache.hadoop.hdds.scm.PipelineChoosePolicy; import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.ContainerManager; import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; import org.apache.hadoop.hdds.scm.exceptions.SCMException; +import org.apache.hadoop.hdds.scm.node.DatanodeInfo; +import org.apache.hadoop.hdds.scm.node.NodeManager; +import org.apache.hadoop.hdds.scm.node.NodeStatus; import org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.RandomPipelineChoosePolicy; import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.Test; @@ -133,7 +145,8 @@ private ContainerInfo pipelineHasContainer(Pipeline pipeline) { .setPipelineID(pipeline.getId()) .build(); - when(containerManager.getMatchingContainer(CONTAINER_SIZE, OWNER, pipeline, emptySet())) + when(containerManager.getMatchingContainer( + eq((long) CONTAINER_SIZE), eq(OWNER), eq(pipeline), eq(emptySet()), any())) .thenReturn(container); return container; @@ -161,6 +174,12 @@ private WritableRatisContainerProvider createSubject() { pipelineManager, containerManager, policy); } + private WritableRatisContainerProvider createSubject( + NodeManager nodeManager) { + return new WritableRatisContainerProvider( + pipelineManager, containerManager, policy, nodeManager); + } + private void verifyPipelineCreated() throws IOException { verify(pipelineManager, times(2)) .getPipelines(REPLICATION_CONFIG, OPEN, emptySet(), emptySet()); @@ -175,4 +194,84 @@ private void verifyPipelineNotCreated() throws IOException { .createPipeline(REPLICATION_CONFIG); } + @Test + void returnsContainerWithNullStorageType() throws Exception { + Pipeline pipeline = MockPipeline.createPipeline(3); + ContainerInfo existingContainer = pipelineHasContainer(pipeline); + existingPipelines(pipeline); + + ContainerInfo container = createSubject().getContainer( + CONTAINER_SIZE, REPLICATION_CONFIG, OWNER, NO_EXCLUSION, null); + assertSame(existingContainer, container); + } + + @Test + void returnsContainerWithMatchingStorageType() throws Exception { + Pipeline pipeline = MockPipeline.createPipeline(3); + ContainerInfo existingContainer = pipelineHasContainer(pipeline); + existingPipelines(pipeline); + + // Set up NodeManager to report SSD for all pipeline nodes + NodeManager nodeManager = mock(NodeManager.class); + List nodes = pipeline.getNodes(); + when(nodeManager.getNodes(NodeStatus.inServiceHealthy())) + .thenReturn(nodes); + for (DatanodeDetails dn : nodes) { + DatanodeInfo info = mock(DatanodeInfo.class); + when(info.getStorageReports()).thenReturn( + Collections.singletonList(StorageReportProto.newBuilder() + .setStorageUuid("uuid-" + UUID.randomUUID()) + .setStorageLocation("/data") + .setCapacity(100L * 1024 * 1024 * 1024) + .setScmUsed(10L * 1024 * 1024 * 1024) + .setRemaining(90L * 1024 * 1024 * 1024) + .setStorageType(StorageTypeProto.SSD) + .build())); + when(nodeManager.getDatanodeInfo(dn)).thenReturn(info); + } + + ContainerInfo container = createSubject(nodeManager).getContainer( + CONTAINER_SIZE, REPLICATION_CONFIG, OWNER, NO_EXCLUSION, + StorageType.SSD); + assertSame(existingContainer, container); + } + + @Test + void filtersOutPipelinesWithNonMatchingStorageType() throws Exception { + Pipeline pipeline = MockPipeline.createPipeline(3); + pipelineHasContainer(pipeline); + existingPipelines(pipeline); + + // Set up NodeManager to report DISK for all nodes + NodeManager nodeManager = mock(NodeManager.class); + List nodes = pipeline.getNodes(); + when(nodeManager.getNodes(NodeStatus.inServiceHealthy())) + .thenReturn(nodes); + for (DatanodeDetails dn : nodes) { + DatanodeInfo info = mock(DatanodeInfo.class); + when(info.getStorageReports()).thenReturn( + Collections.singletonList(StorageReportProto.newBuilder() + .setStorageUuid("uuid-" + UUID.randomUUID()) + .setStorageLocation("/data") + .setCapacity(100L * 1024 * 1024 * 1024) + .setScmUsed(10L * 1024 * 1024 * 1024) + .setRemaining(90L * 1024 * 1024 * 1024) + .setStorageType(StorageTypeProto.DISK) + .build())); + when(nodeManager.getDatanodeInfo(dn)).thenReturn(info); + } + + // A new pipeline will also be created with DISK nodes, but the filter + // on existing pipelines should remove them. + // Pipeline creation will also fail since we're in a mock environment. + when(pipelineManager.createPipeline(REPLICATION_CONFIG)) + .thenThrow(new SCMException( + SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE)); + + assertThrows(IOException.class, + () -> createSubject(nodeManager).getContainer( + CONTAINER_SIZE, REPLICATION_CONFIG, OWNER, NO_EXCLUSION, + StorageType.SSD)); + } + } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMBlockProtocolServer.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMBlockProtocolServer.java index 895baef27d6..6b71f7595be 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMBlockProtocolServer.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMBlockProtocolServer.java @@ -44,8 +44,12 @@ import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.StorageTypeProto; import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos; +import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateScmBlockRequestProto; +import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateScmBlockResponseProto; import org.apache.hadoop.hdds.scm.HddsTestUtils; import org.apache.hadoop.hdds.scm.block.BlockManager; import org.apache.hadoop.hdds.scm.block.DeletedBlockLog; @@ -80,6 +84,7 @@ public class TestSCMBlockProtocolServer { private StorageContainerManager scm; private NodeManager nodeManager; private ScmBlockLocationProtocolServerSideTranslatorPB service; + private BlockManagerStub blockManagerStub; private static final int NODE_COUNT = 10; private static final Map EDGE_NODES = ImmutableMap.of( @@ -90,16 +95,23 @@ public class TestSCMBlockProtocolServer { private static class BlockManagerStub implements BlockManager { private final List datanodes; + private volatile StorageType lastStorageType; BlockManagerStub(List datanodes) { assertNotNull(datanodes, "Datanodes cannot be null"); this.datanodes = datanodes; } + StorageType getLastStorageType() { + return lastStorageType; + } + @Override public AllocatedBlock allocateBlock(long size, ReplicationConfig replicationConfig, String owner, - ExcludeList excludeList) throws IOException, TimeoutException { + ExcludeList excludeList, StorageType storageType) + throws IOException, TimeoutException { + this.lastStorageType = storageType; List nodes = new ArrayList<>(datanodes); Collections.shuffle(nodes); Pipeline pipeline; @@ -174,10 +186,11 @@ void setUp(@TempDir File dir) throws Exception { config.set(StaticMapping.KEY_HADOOP_CONFIGURED_NODE_MAPPING, String.join(",", nodeMapping)); + blockManagerStub = new BlockManagerStub(datanodes); SCMConfigurator configurator = new SCMConfigurator(); configurator.setSCMHAManager(SCMHAManagerStub.getInstance(true)); configurator.setScmContext(SCMContext.emptyContext()); - configurator.setScmBlockManager(new BlockManagerStub(datanodes)); + configurator.setScmBlockManager(blockManagerStub); scm = HddsTestUtils.getScm(config, configurator); scm.start(); scm.exitSafeMode(); @@ -338,6 +351,81 @@ void testAllocateBlockWithClientMachine() throws IOException { } } + @Test + void testAllocateBlockPassesStorageType() throws IOException { + final ReplicationConfig replicationConfig = RatisReplicationConfig + .getInstance(ReplicationFactor.THREE); + final long blockSize = 128 * MB; + + server.allocateBlock(blockSize, 1, replicationConfig, "o", + new ExcludeList(), "", StorageType.SSD); + assertEquals(StorageType.SSD, blockManagerStub.getLastStorageType()); + + server.allocateBlock(blockSize, 1, replicationConfig, "o", + new ExcludeList(), "", StorageType.ARCHIVE); + assertEquals(StorageType.ARCHIVE, blockManagerStub.getLastStorageType()); + } + + @Test + void testAllocateBlockDefaultStorageType() throws IOException { + final ReplicationConfig replicationConfig = RatisReplicationConfig + .getInstance(ReplicationFactor.THREE); + final long blockSize = 128 * MB; + + // 6-param overload should default to DISK + server.allocateBlock(blockSize, 1, replicationConfig, "o", + new ExcludeList(), ""); + assertEquals(StorageType.DEFAULT, blockManagerStub.getLastStorageType()); + } + + @Test + void testStorageTypeProtoRoundTrip() throws IOException { + final ReplicationConfig replicationConfig = RatisReplicationConfig + .getInstance(ReplicationFactor.THREE); + final long blockSize = 128 * MB; + + // Build a proto request with storageType = SSD + AllocateScmBlockRequestProto request = AllocateScmBlockRequestProto + .newBuilder() + .setSize(blockSize) + .setNumBlocks(1) + .setType(replicationConfig.getReplicationType()) + .setFactor(ReplicationFactor.THREE) + .setOwner("o") + .setExcludeList(new ExcludeList().getProtoBuf()) + .setStorageType(StorageTypeProto.SSD) + .build(); + + AllocateScmBlockResponseProto response = + service.allocateScmBlock(request, ClientVersion.CURRENT_VERSION); + assertNotNull(response); + assertEquals(1, response.getBlocksCount()); + assertEquals(StorageType.SSD, blockManagerStub.getLastStorageType()); + } + + @Test + void testStorageTypeProtoDefaultWhenUnset() throws IOException { + final ReplicationConfig replicationConfig = RatisReplicationConfig + .getInstance(ReplicationFactor.THREE); + final long blockSize = 128 * MB; + + // Build a proto request WITHOUT storageType + AllocateScmBlockRequestProto request = AllocateScmBlockRequestProto + .newBuilder() + .setSize(blockSize) + .setNumBlocks(1) + .setType(replicationConfig.getReplicationType()) + .setFactor(ReplicationFactor.THREE) + .setOwner("o") + .setExcludeList(new ExcludeList().getProtoBuf()) + .build(); + + AllocateScmBlockResponseProto response = + service.allocateScmBlock(request, ClientVersion.CURRENT_VERSION); + assertNotNull(response); + assertEquals(StorageType.DEFAULT, blockManagerStub.getLastStorageType()); + } + private List getNetworkNames() { return nodeManager.getAllNodes().stream() .map(NodeImpl::getNetworkName) diff --git a/hadoop-ozone/cli-shell/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java b/hadoop-ozone/cli-shell/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java index e74fb0b1b43..b89a04ad0fc 100644 --- a/hadoop-ozone/cli-shell/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java +++ b/hadoop-ozone/cli-shell/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.hadoop.hdds.client.DefaultReplicationConfig; import org.apache.hadoop.hdds.client.OzoneQuota; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.client.BucketArgs; @@ -68,6 +69,10 @@ public class CreateBucketHandler extends BucketHandler { @CommandLine.Mixin private SetSpaceQuotaOptions quotaOptions; + @Option(names = {"--storage-policy"}, + description = "Storage policy (HOT, WARM)") + private String storagePolicy; + /** * Executes create bucket. */ @@ -116,6 +121,11 @@ public void execute(OzoneClient client, OzoneAddress address) bb.setQuotaInNamespace(OzoneQuota.parseNameSpaceQuota( quotaOptions.getQuotaInNamespace()).getQuotaInNamespace()); } + + if (storagePolicy != null) { + bb.setStoragePolicy(OzoneStoragePolicy.fromString(storagePolicy)); + } + String volumeName = address.getVolumeName(); String bucketName = address.getBucketName(); diff --git a/hadoop-ozone/cli-shell/src/main/java/org/apache/hadoop/ozone/shell/bucket/UpdateBucketHandler.java b/hadoop-ozone/cli-shell/src/main/java/org/apache/hadoop/ozone/shell/bucket/UpdateBucketHandler.java index 58f7190628f..df663603114 100644 --- a/hadoop-ozone/cli-shell/src/main/java/org/apache/hadoop/ozone/shell/bucket/UpdateBucketHandler.java +++ b/hadoop-ozone/cli-shell/src/main/java/org/apache/hadoop/ozone/shell/bucket/UpdateBucketHandler.java @@ -18,6 +18,7 @@ package org.apache.hadoop.ozone.shell.bucket; import java.io.IOException; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.shell.OzoneAddress; @@ -35,6 +36,10 @@ public class UpdateBucketHandler extends BucketHandler { description = "Owner of the bucket to set") private String ownerName; + @Option(names = {"--storage-policy"}, + description = "Storage policy (HOT, WARM)") + private String storagePolicy; + @Override protected void execute(OzoneClient client, OzoneAddress address) throws IOException { @@ -52,6 +57,10 @@ protected void execute(OzoneClient client, OzoneAddress address) } } + if (storagePolicy != null && !storagePolicy.isEmpty()) { + bucket.setStoragePolicy(OzoneStoragePolicy.fromString(storagePolicy)); + } + OzoneBucket updatedBucket = client.getObjectStore().getVolume(volumeName) .getBucket(bucketName); printObjectAsJson(updatedBucket); diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java index f4173963dd6..4e7814a00e5 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java @@ -25,6 +25,7 @@ import java.util.Map; import net.jcip.annotations.Immutable; import org.apache.hadoop.hdds.client.DefaultReplicationConfig; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.ozone.OzoneAcl; import org.apache.hadoop.ozone.OzoneConsts; @@ -69,6 +70,8 @@ public final class BucketArgs { private final String owner; + private final OzoneStoragePolicy storagePolicy; + /** * Bucket Layout. */ @@ -86,6 +89,7 @@ private BucketArgs(Builder b) { quotaInNamespace = b.quotaInNamespace; bucketLayout = b.bucketLayout; owner = b.owner; + storagePolicy = b.storagePolicy; defaultReplicationConfig = b.defaultReplicationConfig; } @@ -185,6 +189,10 @@ public String getOwner() { return owner; } + public OzoneStoragePolicy getStoragePolicy() { + return storagePolicy; + } + /** * Builder for OmBucketInfo. */ @@ -200,6 +208,7 @@ public static class Builder { private long quotaInNamespace; private BucketLayout bucketLayout; private String owner; + private OzoneStoragePolicy storagePolicy; private DefaultReplicationConfig defaultReplicationConfig; public Builder() { @@ -268,6 +277,11 @@ public BucketArgs.Builder setOwner(String ownerName) { return this; } + public BucketArgs.Builder setStoragePolicy(OzoneStoragePolicy policy) { + storagePolicy = policy; + return this; + } + public BucketArgs.Builder setDefaultReplicationConfig( DefaultReplicationConfig defaultReplConfig) { defaultReplicationConfig = defaultReplConfig; diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java index 56642742422..e6db954cbbc 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hdds.client.ReplicationFactor; import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.scm.client.HddsClientUtils; import org.apache.hadoop.ozone.OmUtils; @@ -152,6 +153,8 @@ public class OzoneBucket extends WithMetadata { */ private String owner; + private OzoneStoragePolicy storagePolicy; + protected OzoneBucket(Builder builder) { super(builder); this.proxy = builder.proxy; @@ -191,6 +194,7 @@ protected OzoneBucket(Builder builder) { this.bucketLayout = builder.bucketLayout; } this.owner = builder.owner; + this.storagePolicy = builder.storagePolicy; } /** @@ -300,6 +304,10 @@ public String getOwner() { return owner; } + public OzoneStoragePolicy getStoragePolicy() { + return storagePolicy; + } + public int getListCacheSize() { return listCacheSize; } @@ -348,6 +356,17 @@ public void setStorageType(StorageType newStorageType) throws IOException { storageType = newStorageType; } + /** + * Sets/Changes the storage policy of the bucket. + * @param newStoragePolicy Storage policy to be set + * @throws IOException + */ + public void setStoragePolicy(OzoneStoragePolicy newStoragePolicy) + throws IOException { + proxy.setBucketStoragePolicy(volumeName, name, newStoragePolicy); + storagePolicy = newStoragePolicy; + } + /** * Enable/Disable versioning of the bucket. * @param newVersioning @@ -1127,6 +1146,7 @@ public static class Builder extends WithMetadata.Builder { private long quotaInNamespace; private BucketLayout bucketLayout; private String owner; + private OzoneStoragePolicy storagePolicy; protected Builder() { } @@ -1223,6 +1243,11 @@ public Builder setOwner(String owner) { return this; } + public Builder setStoragePolicy(OzoneStoragePolicy storagePolicy) { + this.storagePolicy = storagePolicy; + return this; + } + public OzoneBucket build() { return new OzoneBucket(this); } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java index 5fc0e92fb72..039e75161c5 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java @@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Syncable; 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.scm.ContainerClientMetrics; import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.StreamBufferArgs; @@ -70,6 +71,7 @@ public class BlockOutputStreamEntry extends OutputStream { private final ContainerClientMetrics clientMetrics; private final StreamBufferArgs streamBufferArgs; private final Supplier executorServiceSupplier; + private final ContainerProtos.StorageTypeProto storageType; /** * An indicator that this BlockOutputStream is created to handoff writes from another faulty BlockOutputStream. @@ -97,6 +99,7 @@ public class BlockOutputStreamEntry extends OutputStream { this.streamBufferArgs = b.streamBufferArgs; this.executorServiceSupplier = b.executorServiceSupplier; this.isHandlingRetry = b.forRetry; + this.storageType = b.storageType; } @Override @@ -155,7 +158,7 @@ void waitForAllPendingFlushes() throws IOException { void createOutputStream() throws IOException { outputStream = new RatisBlockOutputStream(blockID, length, xceiverClientManager, pipeline, bufferPool, config, token, clientMetrics, streamBufferArgs, - executorServiceSupplier); + executorServiceSupplier, storageType); } ContainerClientMetrics getClientMetrics() { @@ -398,6 +401,10 @@ BufferPool getBufferPool() { return this.bufferPool; } + ContainerProtos.StorageTypeProto getStorageType() { + return this.storageType; + } + /** * Builder class for ChunkGroupOutputStreamEntry. * */ @@ -415,6 +422,7 @@ public static class Builder { private StreamBufferArgs streamBufferArgs; private Supplier executorServiceSupplier; private boolean forRetry; + private ContainerProtos.StorageTypeProto storageType; public Pipeline getPipeline() { return pipeline; @@ -485,6 +493,11 @@ public Builder setForRetry(boolean forRetry) { return this; } + public Builder setStorageType(ContainerProtos.StorageTypeProto storageType) { + this.storageType = storageType; + return this; + } + public BlockOutputStreamEntry build() { return new BlockOutputStreamEntry(this); } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java index f3b98626f09..1ed47c3741e 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java @@ -168,6 +168,10 @@ BlockOutputStreamEntry createStreamEntry(OmKeyLocationInfo subKeyInfo, boolean f .setStreamBufferArgs(streamBufferArgs) .setExecutorServiceSupplier(executorServiceSupplier) .setForRetry(forRetry) + .setStorageType(subKeyInfo.getStorageType() != null + ? org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos + .StorageTypeProto.valueOf(subKeyInfo.getStorageType().name()) + : null) .build(); } 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 d4b583c6a92..505951fb418 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 @@ -84,7 +84,7 @@ void checkStream() throws IOException { new ECBlockOutputStream(getBlockID(), getXceiverClientManager(), createSingleECBlockPipeline(getPipeline(), nodes.get(i), i + 1), getBufferPool(), getConf(), getToken(), getClientMetrics(), getStreamBufferArgs(), - getExecutorServiceSupplier()); + getExecutorServiceSupplier(), getStorageType()); } blockOutputStreams = streams; } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntryPool.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntryPool.java index ef08ef9e9d0..98e778ea111 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntryPool.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntryPool.java @@ -49,7 +49,11 @@ ECBlockOutputStreamEntry createStreamEntry(OmKeyLocationInfo subKeyInfo, boolean .setToken(subKeyInfo.getToken()) .setClientMetrics(getClientMetrics()) .setStreamBufferArgs(getStreamBufferArgs()) - .setExecutorServiceSupplier(getExecutorServiceSupplier()); + .setExecutorServiceSupplier(getExecutorServiceSupplier()) + .setStorageType(subKeyInfo.getStorageType() != null + ? org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos + .StorageTypeProto.valueOf(subKeyInfo.getStorageType().name()) + : null); return b.build(); } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java index e3a57589634..9e71012eb17 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hdds.client.ReplicationFactor; import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.io.Text; import org.apache.hadoop.ozone.OzoneAcl; @@ -276,6 +277,16 @@ void setBucketStorageType(String volumeName, String bucketName, StorageType storageType) throws IOException; + /** + * Sets the Storage Policy of a Bucket. + * @param volumeName Name of the Volume + * @param bucketName Name of the Bucket + * @param storagePolicy OzoneStoragePolicy to be set + * @throws IOException + */ + void setBucketStoragePolicy(String volumeName, String bucketName, + OzoneStoragePolicy storagePolicy) throws IOException; + /** * Deletes a bucket if it is empty. * @param volumeName Name of the Volume diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java index 3947e4b6818..44b456c5182 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java @@ -76,6 +76,7 @@ import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.ContainerClientMetrics; @@ -657,6 +658,10 @@ public void createBucket( .setBucketLayout(bucketLayout) .setOwner(owner); + if (bucketArgs.getStoragePolicy() != null) { + builder.setStoragePolicy(bucketArgs.getStoragePolicy()); + } + if (bucketArgs.getAcls() != null) { builder.acls().addAll(bucketArgs.getAcls()); } @@ -1198,6 +1203,19 @@ public void setBucketStorageType( ozoneManagerClient.setBucketProperty(builder.build()); } + @Override + public void setBucketStoragePolicy(String volumeName, String bucketName, + OzoneStoragePolicy storagePolicy) throws IOException { + verifyVolumeName(volumeName); + verifyBucketName(bucketName); + Objects.requireNonNull(storagePolicy, "storagePolicy == null"); + OmBucketArgs.Builder builder = OmBucketArgs.newBuilder(); + builder.setVolumeName(volumeName) + .setBucketName(bucketName) + .setStoragePolicy(storagePolicy); + ozoneManagerClient.setBucketProperty(builder.build()); + } + @Override public void setBucketQuota(String volumeName, String bucketName, long quotaInNamespace, long quotaInBytes) throws IOException { @@ -1303,6 +1321,7 @@ public OzoneBucket getBucketDetails( .setQuotaInNamespace(bucketInfo.getQuotaInNamespace()) .setBucketLayout(bucketInfo.getBucketLayout()) .setOwner(bucketInfo.getOwner()) + .setStoragePolicy(bucketInfo.getStoragePolicy()) .setDefaultReplicationConfig(bucketInfo.getDefaultReplicationConfig()) .build(); } @@ -1334,6 +1353,7 @@ public List listBuckets(String volumeName, String bucketPrefix, .setQuotaInNamespace(bucket.getQuotaInNamespace()) .setBucketLayout(bucket.getBucketLayout()) .setOwner(bucket.getOwner()) + .setStoragePolicy(bucket.getStoragePolicy()) .setDefaultReplicationConfig( bucket.getDefaultReplicationConfig()) .build()) diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/hdds/protocol/OzoneStoragePolicy.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/hdds/protocol/OzoneStoragePolicy.java new file mode 100644 index 00000000000..2bb3b2c981a --- /dev/null +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/hdds/protocol/OzoneStoragePolicy.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdds.protocol; + +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.StoragePolicyProto; + +/** + * Enum representing named storage policies that map semantic intent + * (HOT, WARM, COLD) to physical {@link StorageType} values. + * + *
    + *
  • HOT – primary SSD, fallback DISK
  • + *
  • WARM – primary DISK, no fallback
  • + *
  • COLD – primary ARCHIVE, no fallback
  • + *
+ */ +public enum OzoneStoragePolicy { + + HOT(StorageType.SSD, StorageType.DISK), + WARM(StorageType.DISK, null), + COLD(StorageType.ARCHIVE, null); + + private final StorageType primaryType; + private final StorageType fallbackType; + + OzoneStoragePolicy(StorageType primaryType, StorageType fallbackType) { + this.primaryType = primaryType; + this.fallbackType = fallbackType; + } + + public StorageType getPrimaryStorageType() { + return primaryType; + } + + /** + * Returns the fallback storage type, or {@code null} if no fallback + * is available (write fails when primary is unavailable). + */ + public StorageType getFallbackStorageType() { + return fallbackType; + } + + public static OzoneStoragePolicy getDefault() { + return WARM; + } + + public StoragePolicyProto toProto() { + switch (this) { + case HOT: + return StoragePolicyProto.HOT; + case WARM: + return StoragePolicyProto.WARM; + case COLD: + return StoragePolicyProto.COLD; + default: + throw new IllegalStateException( + "BUG: OzoneStoragePolicy not found, policy=" + this); + } + } + + /** + * Converts a protobuf {@link StoragePolicyProto} to the corresponding + * {@link OzoneStoragePolicy}. Returns {@code null} for + * {@link StoragePolicyProto#STORAGE_POLICY_UNSET}, which means + * "not set / inherit from parent". + */ + public static OzoneStoragePolicy fromProto(StoragePolicyProto proto) { + if (proto == null) { + return null; + } + switch (proto) { + case HOT: + return HOT; + case WARM: + return WARM; + case COLD: + return COLD; + case STORAGE_POLICY_UNSET: + return null; + default: + throw new IllegalStateException( + "BUG: StoragePolicyProto not found, proto=" + proto); + } + } + + /** + * Case-insensitive parse from string. Intended for CLI / config parsing. + * + * @param policy the policy name (e.g. "hot", "WARM") + * @return the matching {@link OzoneStoragePolicy} + * @throws IllegalArgumentException if the string does not match any policy + */ + public static OzoneStoragePolicy fromString(String policy) { + return valueOf(policy.toUpperCase()); + } +} diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java index b831af6d9c0..a2fc4cb7b60 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java @@ -20,6 +20,7 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.hdds.client.ReplicationFactor; import org.apache.hadoop.hdds.client.ReplicationType; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; import org.apache.hadoop.ozone.om.helpers.BucketLayout; import org.apache.ratis.util.TimeDuration; @@ -373,6 +374,18 @@ public final class OMConfigKeys { public static final String OZONE_BUCKET_LAYOUT_OBJECT_STORE = BucketLayout.OBJECT_STORE.name(); + // Storage policy tiering feature toggle. + public static final String OZONE_STORAGE_POLICY_ENABLED = + "ozone.storage.policy.enabled"; + public static final boolean OZONE_STORAGE_POLICY_ENABLED_DEFAULT = false; + + // Default storage policy used by Ozone Manager when a client does not + // specify a storage policy. + public static final String OZONE_DEFAULT_STORAGE_POLICY = + "ozone.default.storage.policy"; + public static final String OZONE_DEFAULT_STORAGE_POLICY_DEFAULT = + OzoneStoragePolicy.WARM.name(); + /** * Configuration properties for Directory Deleting Service. */ diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java index 6491a2ec146..09dc88b9725 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java @@ -21,6 +21,7 @@ import java.util.Map; import java.util.Objects; import org.apache.hadoop.hdds.client.DefaultReplicationConfig; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.audit.Auditable; @@ -63,6 +64,8 @@ public final class OmBucketArgs extends WithMetadata implements Auditable { */ private final String ownerName; + private final OzoneStoragePolicy storagePolicy; + private OmBucketArgs(Builder b) { super(b); this.volumeName = b.volumeName; @@ -76,6 +79,7 @@ private OmBucketArgs(Builder b) { this.quotaInNamespaceSet = b.quotaInNamespaceSet; this.quotaInNamespace = quotaInNamespaceSet ? b.quotaInNamespace : OzoneConsts.QUOTA_RESET; this.bekInfo = b.bekInfo; + this.storagePolicy = b.storagePolicy; } /** @@ -160,6 +164,10 @@ public String getOwnerName() { return ownerName; } + public OzoneStoragePolicy getStoragePolicy() { + return storagePolicy; + } + /** * Returns new builder class that builds a OmBucketArgs. * @return Builder @@ -183,6 +191,9 @@ public Map toAuditMap() { if (this.ownerName != null) { auditMap.put(OzoneConsts.OWNER, this.ownerName); } + if (this.storagePolicy != null) { + auditMap.put(OzoneConsts.STORAGE_POLICY, this.storagePolicy.name()); + } if (this.quotaInBytesSet && quotaInBytes > 0 || (this.quotaInBytes != OzoneConsts.QUOTA_RESET)) { auditMap.put(OzoneConsts.QUOTA_IN_BYTES, @@ -222,6 +233,7 @@ public static class Builder extends WithMetadata.Builder { private BucketEncryptionKeyInfo bekInfo; private DefaultReplicationConfig defaultReplicationConfig; private String ownerName; + private OzoneStoragePolicy storagePolicy; /** * Constructs a builder. @@ -288,6 +300,11 @@ public Builder setOwnerName(String owner) { return this; } + public Builder setStoragePolicy(OzoneStoragePolicy policy) { + this.storagePolicy = policy; + return this; + } + /** * Constructs the OmBucketArgs. * @return instance of OmBucketArgs. @@ -326,6 +343,9 @@ public BucketArgs getProtobuf() { if (ownerName != null) { builder.setOwnerName(ownerName); } + if (storagePolicy != null) { + builder.setStoragePolicy(storagePolicy.toProto()); + } if (bekInfo != null) { builder.setBekInfo(OMPBHelper.convert(bekInfo)); @@ -353,6 +373,10 @@ public static Builder builderFromProtobuf(BucketArgs bucketArgs) { if (bucketArgs.hasOwnerName()) { builder.setOwnerName(bucketArgs.getOwnerName()); } + if (bucketArgs.hasStoragePolicy()) { + builder.setStoragePolicy( + OzoneStoragePolicy.fromProto(bucketArgs.getStoragePolicy())); + } if (bucketArgs.hasDefaultReplicationConfig()) { builder.setDefaultReplicationConfig( diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java index bce6adb636a..df1ae3793c4 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java @@ -24,6 +24,7 @@ import java.util.Objects; import java.util.stream.Collectors; import org.apache.hadoop.hdds.client.DefaultReplicationConfig; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.utils.db.Codec; import org.apache.hadoop.hdds.utils.db.CopyObject; @@ -33,6 +34,7 @@ import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.audit.Auditable; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketInfo; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.StoragePolicyProto; import org.apache.hadoop.ozone.protocolPB.OMPBHelper; /** @@ -107,6 +109,8 @@ public final class OmBucketInfo extends WithObjectID implements Auditable, CopyO private final String owner; + private final OzoneStoragePolicy storagePolicy; + private OmBucketInfo(Builder b) { super(b); this.volumeName = b.volumeName; @@ -128,6 +132,7 @@ private OmBucketInfo(Builder b) { this.bucketLayout = b.bucketLayout; this.owner = b.owner; this.defaultReplicationConfig = b.defaultReplicationConfig; + this.storagePolicy = b.storagePolicy; } public static Codec getCodec() { @@ -303,6 +308,10 @@ public String getOwner() { return owner; } + public OzoneStoragePolicy getStoragePolicy() { + return storagePolicy; + } + /** * Returns new builder class that builds a OmBucketInfo. * @@ -341,6 +350,8 @@ public Map toAuditMap() { auditMap.put(OzoneConsts.SNAPSHOT_USED_BYTES, String.valueOf(this.snapshotUsedBytes)); auditMap.put(OzoneConsts.SNAPSHOT_USED_NAMESPACE, String.valueOf(this.snapshotUsedNamespace)); auditMap.put(OzoneConsts.OWNER, this.owner); + auditMap.put(OzoneConsts.STORAGE_POLICY, + (this.storagePolicy != null) ? this.storagePolicy.name() : null); auditMap.put(OzoneConsts.REPLICATION_TYPE, (this.defaultReplicationConfig != null) ? String.valueOf(this.defaultReplicationConfig.getType()) : null); @@ -378,7 +389,8 @@ public Builder toBuilder() { .setSnapshotUsedNamespace(snapshotUsedNamespace) .setBucketLayout(bucketLayout) .setOwner(owner) - .setDefaultReplicationConfig(defaultReplicationConfig); + .setDefaultReplicationConfig(defaultReplicationConfig) + .setStoragePolicy(storagePolicy); } /** @@ -401,6 +413,7 @@ public static class Builder extends WithObjectID.Builder { private long quotaInNamespace = OzoneConsts.QUOTA_RESET; private BucketLayout bucketLayout = BucketLayout.DEFAULT; private String owner; + private OzoneStoragePolicy storagePolicy; private DefaultReplicationConfig defaultReplicationConfig; private long snapshotUsedBytes; private long snapshotUsedNamespace; @@ -544,6 +557,11 @@ public Builder setOwner(String ownerName) { return this; } + public Builder setStoragePolicy(OzoneStoragePolicy policy) { + this.storagePolicy = policy; + return this; + } + public Builder setDefaultReplicationConfig( DefaultReplicationConfig defaultReplConfig) { this.defaultReplicationConfig = defaultReplConfig; @@ -604,6 +622,9 @@ public BucketInfo getProtobuf() { if (owner != null) { bib.setOwner(owner); } + if (storagePolicy != null) { + bib.setStoragePolicy(storagePolicy.toProto()); + } return bib.build(); } @@ -673,6 +694,11 @@ public static Builder builderFromProtobuf(BucketInfo bucketInfo, if (bucketInfo.hasOwner()) { obib.setOwner(bucketInfo.getOwner()); } + if (bucketInfo.hasStoragePolicy() + && bucketInfo.getStoragePolicy() != StoragePolicyProto.STORAGE_POLICY_UNSET) { + obib.setStoragePolicy( + OzoneStoragePolicy.fromProto(bucketInfo.getStoragePolicy())); + } return obib; } @@ -745,6 +771,7 @@ public boolean equals(Object o) { Objects.equals(getMetadata(), that.getMetadata()) && Objects.equals(bekInfo, that.bekInfo) && Objects.equals(owner, that.owner) && + Objects.equals(storagePolicy, that.storagePolicy) && Objects.equals(defaultReplicationConfig, that.defaultReplicationConfig); } diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java index d3fea73b211..0e615c17904 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java @@ -18,6 +18,7 @@ package org.apache.hadoop.ozone.om.helpers; import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier; @@ -82,6 +83,12 @@ public Builder setCreateVersion(long version) { return this; } + @Override + public Builder setStorageType(StorageType storageType) { + super.setStorageType(storageType); + return this; + } + @Override public OmKeyLocationInfo build() { return new OmKeyLocationInfo(this); @@ -122,6 +129,9 @@ public KeyLocation getProtobuf(boolean ignorePipeline, int clientVersion) { builder.setPipeline(pipeline.getProtobufMessage(clientVersion)); } } + if (getStorageType() != null) { + builder.setStorageType(getStorageType().toProto()); + } return builder.build(); } @@ -142,6 +152,9 @@ public static OmKeyLocationInfo getFromProtobuf(KeyLocation keyLocation) { OMPBHelper.tokenFromProto(keyLocation.getToken()); builder.setToken(token); } + if (keyLocation.hasStorageType()) { + builder.setStorageType(StorageType.valueOf(keyLocation.getStorageType())); + } return builder.build(); } diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/hdds/protocol/TestOzoneStoragePolicy.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/hdds/protocol/TestOzoneStoragePolicy.java new file mode 100644 index 00000000000..09c3b2259a3 --- /dev/null +++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/hdds/protocol/TestOzoneStoragePolicy.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdds.protocol; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.StoragePolicyProto; +import org.junit.jupiter.api.Test; + +/** + * Tests for {@link OzoneStoragePolicy}. + */ +public class TestOzoneStoragePolicy { + + @Test + public void defaultPolicyIsWarm() { + assertEquals(OzoneStoragePolicy.WARM, OzoneStoragePolicy.getDefault()); + } + + @Test + public void primaryStorageTypes() { + assertEquals(StorageType.SSD, + OzoneStoragePolicy.HOT.getPrimaryStorageType()); + assertEquals(StorageType.DISK, + OzoneStoragePolicy.WARM.getPrimaryStorageType()); + assertEquals(StorageType.ARCHIVE, + OzoneStoragePolicy.COLD.getPrimaryStorageType()); + } + + @Test + public void fallbackStorageTypes() { + assertEquals(StorageType.DISK, + OzoneStoragePolicy.HOT.getFallbackStorageType()); + assertNull(OzoneStoragePolicy.WARM.getFallbackStorageType()); + assertNull(OzoneStoragePolicy.COLD.getFallbackStorageType()); + } + + @Test + public void toProtoRoundTrip() { + for (OzoneStoragePolicy policy : OzoneStoragePolicy.values()) { + StoragePolicyProto proto = policy.toProto(); + OzoneStoragePolicy recovered = OzoneStoragePolicy.fromProto(proto); + assertEquals(policy, recovered); + } + } + + @Test + public void fromProtoUnsetReturnsNull() { + assertNull(OzoneStoragePolicy.fromProto( + StoragePolicyProto.STORAGE_POLICY_UNSET)); + } + + @Test + public void fromProtoNullReturnsNull() { + assertNull(OzoneStoragePolicy.fromProto(null)); + } + + @Test + public void fromStringCaseInsensitive() { + assertEquals(OzoneStoragePolicy.HOT, + OzoneStoragePolicy.fromString("HOT")); + assertEquals(OzoneStoragePolicy.HOT, + OzoneStoragePolicy.fromString("hot")); + assertEquals(OzoneStoragePolicy.WARM, + OzoneStoragePolicy.fromString("Warm")); + assertEquals(OzoneStoragePolicy.COLD, + OzoneStoragePolicy.fromString("cold")); + } + + @Test + public void fromStringInvalidThrows() { + assertThrows(IllegalArgumentException.class, + () -> OzoneStoragePolicy.fromString("INVALID")); + } +} diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketArgs.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketArgs.java index 147255b3b57..141a1310060 100644 --- a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketArgs.java +++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketArgs.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.client.DefaultReplicationConfig; import org.apache.hadoop.hdds.client.ECReplicationConfig; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; import org.junit.jupiter.api.Test; /** @@ -90,4 +91,35 @@ public void testDefaultReplicationConfigIsSetCorrectly() { assertEquals(EC, argsFromProto.getDefaultReplicationConfig().getType()); } + + @Test + public void testStoragePolicyRoundTrip() { + OmBucketArgs bucketArgs = OmBucketArgs.newBuilder() + .setBucketName("bucket") + .setVolumeName("volume") + .setStoragePolicy(OzoneStoragePolicy.HOT) + .build(); + + assertEquals(OzoneStoragePolicy.HOT, bucketArgs.getStoragePolicy()); + + OmBucketArgs argsFromProto = OmBucketArgs.getFromProtobuf( + bucketArgs.getProtobuf()); + + assertEquals(OzoneStoragePolicy.HOT, argsFromProto.getStoragePolicy()); + } + + @Test + public void testStoragePolicyNullByDefault() { + OmBucketArgs bucketArgs = OmBucketArgs.newBuilder() + .setBucketName("bucket") + .setVolumeName("volume") + .build(); + + assertNull(bucketArgs.getStoragePolicy()); + + OmBucketArgs argsFromProto = OmBucketArgs.getFromProtobuf( + bucketArgs.getProtobuf()); + + assertNull(argsFromProto.getStoragePolicy()); + } } diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java index 857103a20c0..e578f49c48b 100644 --- a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java +++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationType; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.ozone.OzoneAcl; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; @@ -147,4 +148,41 @@ public void getProtobufMessageEC() { recovered.getDefaultReplicationConfig().getReplicationConfig(); assertEquals(new ECReplicationConfig(3, 2), config); } + + @Test + public void protobufConversionWithStoragePolicy() { + OmBucketInfo bucket = OmBucketInfo.newBuilder() + .setBucketName("bucket") + .setVolumeName("vol1") + .setCreationTime(1L) + .setIsVersionEnabled(false) + .setStorageType(StorageType.ARCHIVE) + .setStoragePolicy(OzoneStoragePolicy.COLD) + .build(); + + assertEquals(OzoneStoragePolicy.COLD, bucket.getStoragePolicy()); + + OmBucketInfo recovered = + OmBucketInfo.getFromProtobuf(bucket.getProtobuf()); + assertEquals(bucket, recovered); + assertEquals(OzoneStoragePolicy.COLD, recovered.getStoragePolicy()); + } + + @Test + public void protobufConversionWithoutStoragePolicy() { + OmBucketInfo bucket = OmBucketInfo.newBuilder() + .setBucketName("bucket") + .setVolumeName("vol1") + .setCreationTime(1L) + .setIsVersionEnabled(false) + .setStorageType(StorageType.ARCHIVE) + .build(); + + assertNull(bucket.getStoragePolicy()); + + OmBucketInfo recovered = + OmBucketInfo.getFromProtobuf(bucket.getProtobuf()); + assertEquals(bucket, recovered); + assertNull(recovered.getStoragePolicy()); + } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineMetrics.java index f0ebb60078c..e572819812d 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineMetrics.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineMetrics.java @@ -29,6 +29,7 @@ import java.util.Optional; import java.util.concurrent.TimeoutException; import org.apache.hadoop.hdds.client.RatisReplicationConfig; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock; import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; @@ -93,7 +94,7 @@ public void testNumBlocksAllocated() throws IOException, TimeoutException { cluster.getStorageContainerManager().getScmBlockManager() .allocateBlock(5, RatisReplicationConfig.getInstance(ReplicationFactor.ONE), - "Test", new ExcludeList()); + "Test", new ExcludeList(), StorageType.DEFAULT); MetricsRecordBuilder metrics = getMetrics(SCMPipelineMetrics.class.getSimpleName()); Pipeline pipeline = block.getPipeline(); 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 7ce4f9319db..c8961c81950 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 @@ -374,7 +374,7 @@ public void testOrphanBlock() throws Exception { // Attempt to reconstruct the container. coordinator.reconstructECContainerGroup(orphanContainerID, (ECReplicationConfig) repConfig, - sourceNodeMap, targetNodeMap); + sourceNodeMap, targetNodeMap, null); } // Check the block listing for the recovered containers 4 or 5 and they @@ -759,7 +759,7 @@ private void testECReconstructionCoordinator(List missingIndexes, coordinator.reconstructECContainerGroup(conID, (ECReplicationConfig) containerPipeline.getReplicationConfig(), - sourceNodeMap, targetNodeMap); + sourceNodeMap, targetNodeMap, null); // Assert the original container metadata with the new recovered one Iterator> iterator = @@ -918,7 +918,7 @@ public void testECReconstructionCoordinatorShouldCleanupContainersOnFailure() null, ECReconstructionMetrics.create(), "")) { coordinator.reconstructECContainerGroup(conID, (ECReplicationConfig) containerPipeline.getReplicationConfig(), - sourceNodeMap, targetNodeMap); + sourceNodeMap, targetNodeMap, null); } }); final DatanodeDetails targetDNToCheckContainerCLeaned = goodTargetNode; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java index dcdb3828ae6..cbe13f76c6c 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java @@ -227,7 +227,9 @@ public static void setUp() throws Exception { any(ReplicationConfig.class), anyString(), any(ExcludeList.class), - anyString())).thenThrow( + anyString(), + any(org.apache.hadoop.hdds.protocol.StorageType.class))) + .thenThrow( new SCMException("SafeModePrecheck failed for allocateBlock", ResultCodes.SAFE_MODE_EXCEPTION)); createVolume(VOLUME_NAME); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java index 4e69848b307..158742c446a 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmContainerLocationCache.java @@ -719,7 +719,8 @@ private void mockScmAllocationOnDn1(long containerID, any(ReplicationConfig.class), anyString(), any(ExcludeList.class), - anyString())) + anyString(), + any(org.apache.hadoop.hdds.protocol.StorageType.class))) .thenReturn(Collections.singletonList(block)); } @@ -735,7 +736,8 @@ private void mockScmAllocationEcPipeline(long containerID, long localId) any(ECReplicationConfig.class), anyString(), any(ExcludeList.class), - anyString())) + anyString(), + any(org.apache.hadoop.hdds.protocol.StorageType.class))) .thenReturn(Collections.singletonList(block)); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestStoragePolicyPlacement.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestStoragePolicyPlacement.java new file mode 100644 index 00000000000..030700ab43c --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestStoragePolicyPlacement.java @@ -0,0 +1,506 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.ozone.om; + +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_COMMAND_STATUS_REPORT_INTERVAL; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_PIPELINE_REPORT_INTERVAL; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DEADNODE_INTERVAL; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_STORAGE_TYPE_AWARE; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; +import org.apache.hadoop.hdds.protocol.StorageType; +import org.apache.hadoop.hdds.scm.container.ContainerID; +import org.apache.hadoop.hdds.scm.container.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.ContainerManager; +import org.apache.hadoop.hdds.scm.container.ContainerReplica; +import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager.ReplicationManagerConfiguration; +import org.apache.hadoop.ozone.HddsDatanodeService; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.OzoneTestUtils; +import org.apache.hadoop.ozone.UniformDatanodesFactory; +import org.apache.hadoop.ozone.client.BucketArgs; +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.OzoneVolume; +import org.apache.hadoop.ozone.client.io.OzoneOutputStream; +import org.apache.hadoop.ozone.container.common.impl.ContainerData; +import org.apache.hadoop.ozone.container.common.impl.ContainerSet; +import org.apache.hadoop.ozone.container.common.interfaces.Container; +import org.apache.hadoop.ozone.container.common.volume.HddsVolume; +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.ozone.test.GenericTestUtils; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Integration test verifying that storage policy (HOT/WARM/COLD) is correctly + * threaded from OM to DN, so that container replicas land on the right volume + * type (SSD/DISK/ARCHIVE). + */ +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public class TestStoragePolicyPlacement { + + private static final Logger LOG = + LoggerFactory.getLogger(TestStoragePolicyPlacement.class); + + private static final int NUM_DATANODES = 5; + + private MiniOzoneCluster cluster; + private OzoneClient client; + + /** + * A DatanodeFactory that delegates to {@link UniformDatanodesFactory} for + * all standard setup (ports, metadata, ratis dirs, etc.) but replaces the + * data dirs with storage-type-annotated paths. + */ + private static MiniOzoneCluster.DatanodeFactory typedVolumeFactory( + String... storageTypes) { + UniformDatanodesFactory base = UniformDatanodesFactory.newBuilder() + .setNumDataVolumes(storageTypes.length) + .build(); + return conf -> { + OzoneConfiguration dnConf = base.apply(conf); + // UniformDatanodesFactory already created the data dirs. + // Replace them with storage-type-annotated dirs. + String existingDirs = dnConf.get(HDDS_DATANODE_DIR_KEY); + String[] dirs = existingDirs.split(","); + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < dirs.length && i < storageTypes.length; i++) { + if (sb.length() > 0) { + sb.append(','); + } + sb.append('[').append(storageTypes[i]).append(']').append(dirs[i]); + } + dnConf.set(HDDS_DATANODE_DIR_KEY, sb.toString()); + return dnConf; + }; + } + + @BeforeAll + void setup() throws Exception { + OzoneConfiguration conf = new OzoneConfiguration(); + conf.setBoolean(OZONE_SCM_PIPELINE_CREATION_STORAGE_TYPE_AWARE, true); + + cluster = MiniOzoneCluster.newBuilder(conf) + .setNumDatanodes(NUM_DATANODES) + .setDatanodeFactory(typedVolumeFactory("SSD", "DISK")) + .build(); + cluster.waitForClusterToBeReady(); + + // Wait for pipelines to form (the background creator may take a few + // seconds to discover SSD-capable nodes and create pipelines). + waitForPipelines(30, TimeUnit.SECONDS); + + client = cluster.newClient(); + } + + @AfterAll + void teardown() { + if (client != null) { + try { + client.close(); + } catch (IOException e) { + LOG.warn("Error closing client", e); + } + } + if (cluster != null) { + cluster.shutdown(); + } + } + + /** + * HOT-policy bucket writes should place container replicas on SSD volumes. + */ + @Test + void testHotPolicyWritesLandOnSSD() throws Exception { + String volumeName = "vol-" + UUID.randomUUID(); + String bucketName = "bkt-hot"; + String keyName = "key-hot"; + + ObjectStore store = client.getObjectStore(); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName, + BucketArgs.newBuilder() + .setStoragePolicy(OzoneStoragePolicy.HOT) + .build()); + OzoneBucket bucket = volume.getBucket(bucketName); + + // Write a small key + byte[] data = "hello-hot-policy".getBytes(StandardCharsets.UTF_8); + try (OzoneOutputStream out = bucket.createKey(keyName, data.length)) { + out.write(data); + } + + // Look up the key to find container locations + OmKeyArgs keyArgs = new OmKeyArgs.Builder() + .setVolumeName(volumeName) + .setBucketName(bucketName) + .setKeyName(keyName) + .build(); + OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs); + assertNotNull(keyInfo); + List locations = + keyInfo.getLatestVersionLocations().getLocationList(); + assertFalse(locations.isEmpty(), "Key should have at least one block"); + + // For each block, check that the container replica is on an SSD volume + for (OmKeyLocationInfo loc : locations) { + long containerId = loc.getContainerID(); + LOG.info("Checking container {} for SSD placement", containerId); + assertContainerOnVolumeType(containerId, StorageType.SSD); + } + } + + /** + * When no SSD volumes exist, HOT-policy writes should fall back to DISK. + */ + @Test + void testHotPolicyFallsBackToDiskWhenNoSSD() throws Exception { + // Use a separate mini cluster with DISK-only volumes + OzoneConfiguration diskOnlyConf = new OzoneConfiguration(); + diskOnlyConf.setBoolean( + OZONE_SCM_PIPELINE_CREATION_STORAGE_TYPE_AWARE, true); + + MiniOzoneCluster diskCluster = MiniOzoneCluster.newBuilder(diskOnlyConf) + .setNumDatanodes(3) + .setDatanodeFactory(typedVolumeFactory("DISK")) + .build(); + try { + diskCluster.waitForClusterToBeReady(); + + try (OzoneClient diskClient = diskCluster.newClient()) { + String volumeName = "vol-" + UUID.randomUUID(); + String bucketName = "bkt-fallback"; + String keyName = "key-fallback"; + + ObjectStore store = diskClient.getObjectStore(); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName, + BucketArgs.newBuilder() + .setStoragePolicy(OzoneStoragePolicy.HOT) + .build()); + OzoneBucket bucket = volume.getBucket(bucketName); + + // Write should succeed even though no SSD volumes exist + byte[] data = "hello-fallback".getBytes(StandardCharsets.UTF_8); + try (OzoneOutputStream out = bucket.createKey(keyName, data.length)) { + out.write(data); + } + + // Verify key is readable + byte[] readBuf = new byte[data.length]; + try (java.io.InputStream in = bucket.readKey(keyName)) { + int bytesRead = in.read(readBuf); + assertEquals(data.length, bytesRead); + } + + // Verify replicas are on DISK volumes + OmKeyArgs keyArgs = new OmKeyArgs.Builder() + .setVolumeName(volumeName) + .setBucketName(bucketName) + .setKeyName(keyName) + .build(); + OmKeyInfo keyInfo = diskCluster.getOzoneManager().lookupKey(keyArgs); + assertNotNull(keyInfo); + List locations = + keyInfo.getLatestVersionLocations().getLocationList(); + assertFalse(locations.isEmpty()); + + for (OmKeyLocationInfo loc : locations) { + long containerId = loc.getContainerID(); + assertContainerOnVolumeType( + diskCluster, containerId, StorageType.DISK); + } + } + } finally { + diskCluster.shutdown(); + } + } + + /** + * End-to-end test: when a container created on SSD (HOT policy) is + * under-replicated and replication manager creates a new replica, + * that new replica should also land on an SSD volume. + */ + @Test + void testReplicatedContainerLandsOnCorrectVolumeType() throws Exception { + OzoneConfiguration repConf = new OzoneConfiguration(); + repConf.setBoolean(OZONE_SCM_PIPELINE_CREATION_STORAGE_TYPE_AWARE, true); + // Aggressive intervals for fast detection (matching + // TestReplicationManagerIntegration pattern) + repConf.setTimeDuration(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL, + 100, TimeUnit.MILLISECONDS); + repConf.setTimeDuration(HDDS_HEARTBEAT_INTERVAL, + 1, TimeUnit.SECONDS); + repConf.setTimeDuration(HDDS_PIPELINE_REPORT_INTERVAL, + 1, TimeUnit.SECONDS); + repConf.setTimeDuration(HDDS_COMMAND_STATUS_REPORT_INTERVAL, + 1, TimeUnit.SECONDS); + repConf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, + 1, TimeUnit.SECONDS); + repConf.setTimeDuration(HDDS_NODE_REPORT_INTERVAL, + 1, TimeUnit.SECONDS); + repConf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, + 3, TimeUnit.SECONDS); + repConf.setTimeDuration(OZONE_SCM_DEADNODE_INTERVAL, + 6, TimeUnit.SECONDS); + repConf.setTimeDuration(HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT, + 0, TimeUnit.SECONDS); + // Fast replication manager via typed config object + ReplicationManagerConfiguration rmConf = + repConf.getObject(ReplicationManagerConfiguration.class); + rmConf.setInterval(Duration.ofSeconds(1)); + rmConf.setUnderReplicatedInterval(Duration.ofMillis(500)); + repConf.setFromObject(rmConf); + + // 4 datanodes: Ratis THREE needs 3 active + 1 for replication target + MiniOzoneCluster repCluster = MiniOzoneCluster.newBuilder(repConf) + .setNumDatanodes(4) + .setDatanodeFactory(typedVolumeFactory("SSD", "DISK")) + .build(); + try { + repCluster.waitForClusterToBeReady(); + waitForPipelines(repCluster, 30, TimeUnit.SECONDS); + + try (OzoneClient repClient = repCluster.newClient()) { + String volumeName = "vol-" + UUID.randomUUID(); + String bucketName = "bkt-rep"; + String keyName = "key-rep"; + + ObjectStore store = repClient.getObjectStore(); + store.createVolume(volumeName); + OzoneVolume volume = store.getVolume(volumeName); + volume.createBucket(bucketName, + BucketArgs.newBuilder() + .setStoragePolicy(OzoneStoragePolicy.HOT) + .build()); + OzoneBucket bucket = volume.getBucket(bucketName); + + // Write a key — containers created on SSD + byte[] data = "hello-replication-test".getBytes(StandardCharsets.UTF_8); + try (OzoneOutputStream out = bucket.createKey(keyName, data.length)) { + out.write(data); + } + + // Look up the key to find container locations + OmKeyArgs keyArgs = new OmKeyArgs.Builder() + .setVolumeName(volumeName) + .setBucketName(bucketName) + .setKeyName(keyName) + .build(); + OmKeyInfo keyInfo = + repCluster.getOzoneManager().lookupKey(keyArgs); + assertNotNull(keyInfo); + List locations = + keyInfo.getLatestVersionLocations().getLocationList(); + assertFalse(locations.isEmpty(), + "Key should have at least one block"); + + long containerIdVal = locations.get(0).getContainerID(); + LOG.info("Container {} written with HOT policy", containerIdVal); + + // Verify initial replicas are on SSD + assertContainerOnVolumeType( + repCluster, containerIdVal, StorageType.SSD); + + closeAndTriggerReplication(repCluster, containerIdVal); + + // Verify ALL current replicas (including new ones) are on SSD + assertContainerOnVolumeType( + repCluster, containerIdVal, StorageType.SSD); + } + } finally { + repCluster.shutdown(); + } + } + + /** + * Close a container through the proper pipeline-close flow, then stop + * one datanode to trigger under-replication and wait for the replication + * manager to create a new replica. + */ + private void closeAndTriggerReplication( + MiniOzoneCluster repCluster, long containerIdVal) throws Exception { + ContainerManager cm = repCluster.getStorageContainerManager() + .getContainerManager(); + ContainerID containerId = ContainerID.valueOf(containerIdVal); + ContainerInfo containerInfo = cm.getContainer(containerId); + + // Close via pipeline close — triggers proper DN close + report flow + OzoneTestUtils.closeContainer( + repCluster.getStorageContainerManager(), containerInfo); + + Set replicas = cm.getContainerReplicas(containerId); + assertEquals(3, replicas.size(), "Ratis THREE should have 3 replicas"); + DatanodeDetails dnToStop = + replicas.iterator().next().getDatanodeDetails(); + LOG.info("Shutting down datanode {} to trigger under-replication", + dnToStop.getUuidString()); + repCluster.shutdownHddsDatanode(dnToStop); + + GenericTestUtils.waitFor(() -> { + try { + long healthyCount = cm.getContainerReplicas(containerId).stream() + .filter(r -> !r.getDatanodeDetails().equals(dnToStop)) + .count(); + return healthyCount >= 3; + } catch (Exception e) { + return false; + } + }, 2000, 120000); + + assertTrue(cm.getContainerReplicas(containerId).size() >= 3, + "Expected at least 3 replicas"); + } + + /** + * Assert that a container's replicas on the datanodes in {@code this.cluster} + * are placed on volumes of the expected type. + */ + private void assertContainerOnVolumeType( + long containerId, StorageType expectedType) throws Exception { + assertContainerOnVolumeType(cluster, containerId, expectedType); + } + + /** + * Assert that a container's replicas on the datanodes in the given cluster + * are placed on volumes of the expected type. + */ + private void assertContainerOnVolumeType( + MiniOzoneCluster miniCluster, long containerId, + StorageType expectedType) throws Exception { + ContainerManager cm = + miniCluster.getStorageContainerManager().getContainerManager(); + Set replicas = + cm.getContainerReplicas(ContainerID.valueOf(containerId)); + assertFalse(replicas.isEmpty(), + "Container " + containerId + " should have replicas"); + + for (ContainerReplica replica : replicas) { + DatanodeDetails dn = replica.getDatanodeDetails(); + HddsDatanodeService dnService = findDatanode(miniCluster, dn); + if (dnService == null) { + LOG.info("Skipping stopped datanode {}", dn.getUuidString()); + continue; + } + + ContainerSet containerSet = dnService.getDatanodeStateMachine() + .getContainer().getContainerSet(); + Container container = containerSet.getContainer(containerId); + if (container == null) { + continue; + } + + ContainerData containerData = container.getContainerData(); + HddsVolume vol = containerData.getVolume(); + assertNotNull(vol, "Container volume is null"); + + LOG.info("Container {} on DN {} is on volume {} (type={})", + containerId, dn.getUuidString(), + vol.getStorageDir(), vol.getStorageType()); + assertEquals(expectedType.name(), vol.getStorageType().name(), + "Container " + containerId + " on DN " + dn.getUuidString() + + " expected volume type " + expectedType + + " but found " + vol.getStorageType()); + } + } + + private HddsDatanodeService findDatanode( + MiniOzoneCluster miniCluster, DatanodeDetails target) { + for (HddsDatanodeService dn : miniCluster.getHddsDatanodes()) { + if (dn.getDatanodeDetails().getUuid().equals(target.getUuid())) { + return dn; + } + } + return null; + } + + /** + * Wait until at least one open pipeline exists in the given cluster. + */ + private static void waitForPipelines( + MiniOzoneCluster miniCluster, + long timeout, TimeUnit unit) throws Exception { + long deadline = System.currentTimeMillis() + unit.toMillis(timeout); + while (System.currentTimeMillis() < deadline) { + boolean found = miniCluster.getStorageContainerManager() + .getPipelineManager() + .getPipelines() + .stream() + .anyMatch(p -> p.isOpen() && p.getNodes().size() >= 3); + if (found) { + LOG.info("Found open pipeline(s) -- proceeding with tests"); + return; + } + Thread.sleep(1000); + } + LOG.warn("Timed out waiting for pipelines -- " + + "tests may still pass if fallback works"); + } + + /** + * Wait until at least one open pipeline exists. + */ + private void waitForPipelines( + long timeout, TimeUnit unit) throws Exception { + long deadline = System.currentTimeMillis() + unit.toMillis(timeout); + while (System.currentTimeMillis() < deadline) { + boolean found = cluster.getStorageContainerManager() + .getPipelineManager() + .getPipelines() + .stream() + .anyMatch(p -> p.isOpen() && p.getNodes().size() >= 3); + if (found) { + LOG.info("Found open pipeline(s) -- proceeding with tests"); + return; + } + Thread.sleep(1000); + } + LOG.warn("Timed out waiting for pipelines -- " + + "tests may still pass if fallback works"); + } +} diff --git a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto index bdb3cc3cee3..269bb4f3b63 100644 --- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto +++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto @@ -774,6 +774,7 @@ message BucketInfo { optional hadoop.hdds.DefaultReplicationConfig defaultReplicationConfig = 20; optional uint64 snapshotUsedBytes = 21; optional uint64 snapshotUsedNamespace = 22; + optional StoragePolicyProto storagePolicy = 23; } enum BucketLayoutProto { @@ -782,6 +783,13 @@ enum BucketLayoutProto { OBJECT_STORE = 3; } +enum StoragePolicyProto { + STORAGE_POLICY_UNSET = 0; + HOT = 1; + WARM = 2; + COLD = 3; +} + /** * Cipher suite. */ @@ -847,6 +855,7 @@ message BucketArgs { optional string ownerName = 10; optional hadoop.hdds.DefaultReplicationConfig defaultReplicationConfig = 11; optional BucketEncryptionInfoProto bekInfo = 12; + optional StoragePolicyProto storagePolicy = 13; } message PrefixInfo { @@ -1100,6 +1109,7 @@ message KeyLocation { optional hadoop.hdds.Pipeline pipeline = 7; optional int32 partNumber = 9 [default = -1]; + optional hadoop.hdds.StorageTypeProto storageType = 10; } message KeyLocationList { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java index 7a9d66f86df..90adb19a74f 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java @@ -53,6 +53,8 @@ import static org.apache.hadoop.ozone.OzoneConsts.TRANSACTION_INFO_KEY; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT_DEFAULT; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_DEFAULT_STORAGE_POLICY; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_DEFAULT_STORAGE_POLICY_DEFAULT; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_DIR_DELETING_SERVICE_INTERVAL; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_KEY_DELETING_LIMIT_PER_TASK; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY; @@ -172,6 +174,7 @@ import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.ReconfigurationHandler; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; import org.apache.hadoop.hdds.protocol.SecretKeyProtocol; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.ReconfigureProtocolProtos.ReconfigureProtocolService; @@ -4780,6 +4783,13 @@ public BucketLayout getOMDefaultBucketLayout() { return this.defaultBucketLayout; } + public OzoneStoragePolicy getDefaultStoragePolicy() { + String policyString = configuration.getTrimmed( + OZONE_DEFAULT_STORAGE_POLICY, + OZONE_DEFAULT_STORAGE_POLICY_DEFAULT); + return OzoneStoragePolicy.fromString(policyString); + } + /** * Create volume which is required for S3Gateway operations. */ diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java index a88e5fb7333..a3b4e3be745 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java @@ -25,6 +25,7 @@ import java.util.Objects; import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension; import org.apache.hadoop.hdds.client.DefaultReplicationConfig; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.utils.db.cache.CacheKey; import org.apache.hadoop.hdds.utils.db.cache.CacheValue; @@ -172,6 +173,12 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, Execut bucketName, volumeName); } + //Check StoragePolicy to update + OzoneStoragePolicy storagePolicy = omBucketArgs.getStoragePolicy(); + if (storagePolicy != null) { + bucketInfoBuilder.setStoragePolicy(storagePolicy); + } + //Check Versioning to update Boolean versioning = omBucketArgs.getIsVersionEnabled(); if (versioning != null) { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java index 9788cfbafe1..fcaaf34ff72 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileCreateRequest.java @@ -32,6 +32,7 @@ import java.util.Objects; import java.util.stream.Collectors; import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; import org.apache.hadoop.hdds.utils.UniqueId; @@ -121,6 +122,9 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException { bucketInfo.getDefaultReplicationConfig(), ozoneManager); + final StorageType storageType = resolveEffectiveStoragePolicy( + bucketInfo, ozoneManager).getPrimaryStorageType(); + // TODO: Here we are allocating block with out any check for // bucket/key/volume or not and also with out any authorization checks. // We also allocate block even if requestedSize is 0 because unlike @@ -138,7 +142,8 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException { ozoneManager.getOMServiceId(), ozoneManager.getMetrics(), keyArgs.getSortDatanodes(), - userInfo); + userInfo, + storageType); KeyArgs.Builder newKeyArgs = keyArgs.toBuilder() .setModificationTime(Time.now()).setType(type).setFactor(factor) diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java index b692cf9d55e..7b31f41c392 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Objects; import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; import org.apache.hadoop.hdds.utils.db.cache.CacheKey; import org.apache.hadoop.hdds.utils.db.cache.CacheValue; @@ -107,6 +108,10 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException { UserInfo userInfo = getUserIfNotExists(ozoneManager); ReplicationConfig repConfig = ReplicationConfig.fromProto(keyArgs.getType(), keyArgs.getFactor(), keyArgs.getEcReplicationConfig()); + final OmBucketInfo bucketInfo = ozoneManager + .getBucketInfo(keyArgs.getVolumeName(), keyArgs.getBucketName()); + final StorageType storageType = resolveEffectiveStoragePolicy( + bucketInfo, ozoneManager).getPrimaryStorageType(); // To allocate atleast one block passing requested size and scmBlockSize // as same value. When allocating block requested size is same as // scmBlockSize. @@ -117,7 +122,8 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException { ozoneManager.getPreallocateBlocksMax(), ozoneManager.isGrpcBlockTokenEnabled(), ozoneManager.getOMServiceId(), ozoneManager.getMetrics(), - keyArgs.getSortDatanodes(), userInfo); + keyArgs.getSortDatanodes(), userInfo, + storageType); // Set modification time and normalize key if required. KeyArgs.Builder newKeyArgs = diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java index d34320ecb8d..ada368235a4 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyCreateRequest.java @@ -32,6 +32,7 @@ import java.util.Objects; import java.util.stream.Collectors; import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; import org.apache.hadoop.hdds.utils.UniqueId; @@ -141,6 +142,9 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException { bucketInfo.getDefaultReplicationConfig(), ozoneManager); + final StorageType storageType = resolveEffectiveStoragePolicy( + bucketInfo, ozoneManager).getPrimaryStorageType(); + // TODO: Here we are allocating block with out any check for // bucket/key/volume or not and also with out any authorization checks. // As for a client for the first time this can be executed on any OM, @@ -163,7 +167,8 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException { ozoneManager.getOMServiceId(), ozoneManager.getMetrics(), keyArgs.getSortDatanodes(), - userInfo)); + userInfo, + storageType)); effectiveDataSize = requestedSize; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java index 7df2619e9e4..5a8b1d29a2d 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java @@ -55,6 +55,8 @@ import org.apache.hadoop.hdds.client.ContainerBlockID; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock; import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; @@ -190,7 +192,8 @@ protected List< OmKeyLocationInfo > allocateBlock(ScmClient scmClient, ReplicationConfig replicationConfig, ExcludeList excludeList, long requestedSize, long scmBlockSize, int preallocateBlocksMax, boolean grpcBlockTokenEnabled, String serviceID, OMMetrics omMetrics, - boolean shouldSortDatanodes, UserInfo userInfo) + boolean shouldSortDatanodes, UserInfo userInfo, + StorageType storageType) throws IOException { int dataGroupSize = replicationConfig instanceof ECReplicationConfig ? ((ECReplicationConfig) replicationConfig).getData() : 1; @@ -204,11 +207,13 @@ protected List< OmKeyLocationInfo > allocateBlock(ScmClient scmClient, List locationInfos = new ArrayList<>(numBlocks); String remoteUser = getRemoteUser().getShortUserName(); + LOG.debug("Allocating block with storageType={} for replication={}", + storageType, replicationConfig); List allocatedBlocks; try { allocatedBlocks = scmClient.getBlockClient() .allocateBlock(scmBlockSize, numBlocks, replicationConfig, serviceID, - excludeList, clientMachine); + excludeList, clientMachine, storageType); } catch (SCMException ex) { omMetrics.incNumBlockAllocateCallFails(); if (ex.getResult() @@ -224,7 +229,8 @@ protected List< OmKeyLocationInfo > allocateBlock(ScmClient scmClient, .setBlockID(blockID) .setLength(scmBlockSize) .setOffset(0) - .setPipeline(allocatedBlock.getPipeline()); + .setPipeline(allocatedBlock.getPipeline()) + .setStorageType(storageType); if (grpcBlockTokenEnabled) { builder.setToken(secretManager.generateToken(remoteUser, blockID, EnumSet.of(READ, WRITE), scmBlockSize)); @@ -234,6 +240,14 @@ protected List< OmKeyLocationInfo > allocateBlock(ScmClient scmClient, return locationInfos; } + protected static OzoneStoragePolicy resolveEffectiveStoragePolicy( + @Nullable OmBucketInfo bucketInfo, OzoneManager ozoneManager) { + if (bucketInfo != null && bucketInfo.getStoragePolicy() != null) { + return bucketInfo.getStoragePolicy(); + } + return ozoneManager.getDefaultStoragePolicy(); + } + /* Optimize ugi lookup for RPC operations to avoid a trip through * UGI.getCurrentUser which is synch'ed. */ diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ScmBlockLocationTestingClient.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ScmBlockLocationTestingClient.java index 823a6405257..32aeb473afd 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ScmBlockLocationTestingClient.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ScmBlockLocationTestingClient.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.scm.AddSCMRequest; import org.apache.hadoop.hdds.scm.ScmInfo; @@ -119,7 +120,8 @@ public ScmBlockLocationTestingClient(String clusterID, String scmId, @Override public List allocateBlock(long size, int num, ReplicationConfig config, - String owner, ExcludeList excludeList, String clientMachine) + String owner, ExcludeList excludeList, String clientMachine, + StorageType storageType) throws IOException { DatanodeDetails datanodeDetails = randomDatanodeDetails(); Pipeline pipeline = createPipeline(datanodeDetails); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMFileCreateRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMFileCreateRequest.java index 3004f511480..e3bdab432d2 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMFileCreateRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMFileCreateRequest.java @@ -730,7 +730,8 @@ public void testZeroSizedFileShouldCallAllocateBlock() throws Exception { verify(scmBlockLocationProtocol, atLeastOnce()) .allocateBlock(anyLong(), anyInt(), any(ReplicationConfig.class), anyString(), - any(ExcludeList.class), anyString()); + any(ExcludeList.class), anyString(), + any(org.apache.hadoop.hdds.protocol.StorageType.class)); // Verify key locations are present in the response assertTrue(modifiedOmRequest.hasCreateFileRequest()); @@ -802,7 +803,8 @@ public void testFileWithoutDataSizeShouldAllocateBlock() throws Exception { verify(scmBlockLocationProtocol, atLeastOnce()) .allocateBlock(anyLong(), anyInt(), any(ReplicationConfig.class), anyString(), - any(ExcludeList.class), anyString()); + any(ExcludeList.class), anyString(), + any(org.apache.hadoop.hdds.protocol.StorageType.class)); // Verify key locations are present in the response assertTrue(modifiedOmRequest.hasCreateFileRequest()); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequest.java index 1666f4cb38e..32f66152775 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequest.java @@ -1120,12 +1120,14 @@ public void testEmptyKeyKeyDoesNotCallScmAllocateBlock() throws Exception { verify(scmBlockLocationProtocol, never()) .allocateBlock(anyLong(), anyInt(), any(ReplicationConfig.class), anyString(), - any(ExcludeList.class), anyString()); + any(ExcludeList.class), anyString(), + any(org.apache.hadoop.hdds.protocol.StorageType.class)); verify(scmBlockLocationProtocol, never()) .allocateBlock(anyLong(), anyInt(), any(ReplicationConfig.class), anyString(), - any(ExcludeList.class), anyString()); + any(ExcludeList.class), anyString(), + any(org.apache.hadoop.hdds.protocol.StorageType.class)); assertTrue(modifiedOmRequest.hasCreateKeyRequest()); CreateKeyRequest responseCreateKeyRequest = @@ -1197,12 +1199,14 @@ public void testKeyWithoutDataSizeCallsScmAllocateBlock() throws Exception { verify(scmBlockLocationProtocol, never()) .allocateBlock(anyLong(), anyInt(), any(ReplicationConfig.class), anyString(), - any(ExcludeList.class), anyString()); + any(ExcludeList.class), anyString(), + any(org.apache.hadoop.hdds.protocol.StorageType.class)); verify(scmBlockLocationProtocol, never()) .allocateBlock(anyLong(), anyInt(), any(ReplicationConfig.class), anyString(), - any(ExcludeList.class), anyString()); + any(ExcludeList.class), anyString(), + any(org.apache.hadoop.hdds.protocol.StorageType.class)); assertTrue(modifiedOmRequest.hasCreateKeyRequest()); CreateKeyRequest responseCreateKeyRequest = diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java index b84294370c5..928a9d8b43b 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.scm.container.ContainerInfo; @@ -164,6 +165,7 @@ public void setup() throws Exception { .thenReturn(true); when(ozoneManager.getBucketInfo(anyString(), anyString())).thenReturn( new OmBucketInfo.Builder().setVolumeName("").setBucketName("").build()); + when(ozoneManager.getDefaultStoragePolicy()).thenReturn(OzoneStoragePolicy.WARM); doNothing().when(auditLogger).logWrite(any(AuditMessage.class)); AuditMessage mockAuditMessage = mock(AuditMessage.class); @@ -221,7 +223,9 @@ public void setup() throws Exception { when(scmBlockLocationProtocol.allocateBlock(anyLong(), anyInt(), any(ReplicationConfig.class), anyString(), any(ExcludeList.class), - anyString())).thenAnswer(invocation -> { + anyString(), + any(org.apache.hadoop.hdds.protocol.StorageType.class))) + .thenAnswer(invocation -> { int num = invocation.getArgument(1); List allocatedBlocks = new ArrayList<>(num); for (int i = 0; i < num; i++) { diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ClientProtocolStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ClientProtocolStub.java index 739babce1d0..b83e461ac61 100644 --- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ClientProtocolStub.java +++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ClientProtocolStub.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hdds.client.ReplicationFactor; import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.OzoneStoragePolicy; import org.apache.hadoop.hdds.protocol.StorageType; import org.apache.hadoop.io.Text; import org.apache.hadoop.ozone.OzoneAcl; @@ -185,6 +186,12 @@ public void setBucketStorageType(String volumeName, String bucketName, } + @Override + public void setBucketStoragePolicy(String volumeName, String bucketName, + OzoneStoragePolicy storagePolicy) throws IOException { + + } + @Override public void deleteBucket(String volumeName, String bucketName) throws IOException {