From 0b4f5327bdb210d781f20ce99ec7d90f241b50d8 Mon Sep 17 00:00:00 2001 From: sarvekshayr Date: Sun, 26 Apr 2026 20:57:13 +0530 Subject: [PATCH 1/2] HDDS-15121. Unify SCM list container on SCMListContainerRequestProto --- .../hadoop/hdds/scm/client/ScmClient.java | 13 ++ .../ScmListContainerRequestCodec.java | 183 ++++++++++++++++++ .../StorageContainerLocationProtocol.java | 106 +--------- ...ocationProtocolClientSideTranslatorPB.java | 79 +------- ...ocationProtocolServerSideTranslatorPB.java | 49 +---- .../scm/server/SCMClientProtocolServer.java | 106 ++-------- .../server/TestSCMClientProtocolServer.java | 12 +- .../scm/cli/ContainerOperationClient.java | 51 ++--- .../scm/cli/container/ListSubcommand.java | 9 +- .../TestContainerReportSuppressOptions.java | 16 +- .../hdds/scm/TestContainerOperations.java | 4 +- hadoop-ozone/vapor/pom.xml | 4 + .../freon/ClosedContainerReplicator.java | 5 +- 13 files changed, 287 insertions(+), 350 deletions(-) create mode 100644 hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmListContainerRequestCodec.java diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java index cb4b8471a00c..126d918d3070 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DeletedBlocksTransactionSummary; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionScmResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMListContainerRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartContainerBalancerResponseProto; import org.apache.hadoop.hdds.scm.DatanodeAdminError; import org.apache.hadoop.hdds.scm.container.ContainerID; @@ -115,6 +116,15 @@ void deleteContainer(long containerId, Pipeline pipeline, boolean force) */ void deleteContainer(long containerId, boolean force) throws IOException; + /** + * Lists containers using a single SCM list container request. + * + * @param request start containerID, count and optional filters + * @return a list of containers matching the filters. + * @throws IOException + */ + ContainerListResult listContainer(SCMListContainerRequestProto request) throws IOException; + /** * Lists a range of containers and get their info. * @@ -125,6 +135,7 @@ void deleteContainer(long containerId, Pipeline pipeline, boolean force) * in "ozone.scm.container.list.max.count" and total number of containers. * @throws IOException */ + @Deprecated ContainerListResult listContainer(long startContainerID, int count) throws IOException; @@ -139,6 +150,7 @@ ContainerListResult listContainer(long startContainerID, * in "ozone.scm.container.list.max.count" and total number of containers. * @throws IOException */ + @Deprecated ContainerListResult listContainer(long startContainerID, int count, HddsProtos.LifeCycleState state, HddsProtos.ReplicationType replicationType, @@ -157,6 +169,7 @@ ContainerListResult listContainer(long startContainerID, int count, * in "ozone.scm.container.list.max.count" and total number of containers. * @throws IOException */ + @Deprecated ContainerListResult listContainer(long startContainerID, int count, HddsProtos.LifeCycleState state, HddsProtos.ReplicationType replicationType, diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmListContainerRequestCodec.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmListContainerRequestCodec.java new file mode 100644 index 000000000000..407e3b5630c1 --- /dev/null +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmListContainerRequestCodec.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.protocol; + +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.EC; + +import jakarta.annotation.Nullable; +import org.apache.hadoop.hdds.client.ECReplicationConfig; +import org.apache.hadoop.hdds.client.ReplicatedReplicationConfig; +import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMListContainerRequestProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMListContainerRequestProto.Builder; + +/** + * Codec between {@link SCMListContainerRequestProto} and {@link ListContainerQuery} for list containers. + */ +public final class ScmListContainerRequestCodec { + + private ScmListContainerRequestCodec() { + } + + /** + * Immutable view of list container request fields after {@link #fromProto}. + */ + public static final class ListContainerQuery { + private final long startContainerID; + private final int count; + @Nullable + private final HddsProtos.LifeCycleState state; + @Nullable + private final HddsProtos.ReplicationFactor factor; + @Nullable + private final HddsProtos.ReplicationType replicationType; + @Nullable + private final ReplicationConfig replicationConfig; + @Nullable + private final Boolean suppressed; + + private ListContainerQuery(long startContainerID, int count, + @Nullable HddsProtos.LifeCycleState state, + @Nullable HddsProtos.ReplicationFactor factor, + @Nullable HddsProtos.ReplicationType replicationType, + @Nullable ReplicationConfig replicationConfig, + @Nullable Boolean suppressed) { + this.startContainerID = startContainerID; + this.count = count; + this.state = state; + this.factor = factor; + this.replicationType = replicationType; + this.replicationConfig = replicationConfig; + this.suppressed = suppressed; + } + + public long getStartContainerID() { + return startContainerID; + } + + public int getCount() { + return count; + } + + @Nullable + public HddsProtos.LifeCycleState getState() { + return state; + } + + @Nullable + public HddsProtos.ReplicationFactor getFactor() { + return factor; + } + + @Nullable + public HddsProtos.ReplicationType getReplicationType() { + return replicationType; + } + + @Nullable + public ReplicationConfig getReplicationConfig() { + return replicationConfig; + } + + @Nullable + public Boolean getSuppressed() { + return suppressed; + } + } + + /** + * Builds {@link SCMListContainerRequestProto} from Java fields. + */ + @SuppressWarnings("checkstyle:ParameterNumber") + public static SCMListContainerRequestProto toProto( + long startContainerID, + int count, + @Nullable HddsProtos.LifeCycleState state, + @Nullable HddsProtos.ReplicationFactor factor, + @Nullable HddsProtos.ReplicationType replicationType, + @Nullable ReplicationConfig repConfig, + @Nullable Boolean suppressed, + @Nullable String traceId) { + Builder builder = SCMListContainerRequestProto.newBuilder() + .setCount(count) + .setStartContainerID(startContainerID); + if (traceId != null) { + builder.setTraceID(traceId); + } + if (suppressed != null) { + builder.setSuppressed(suppressed); + } + if (state != null) { + builder.setState(state); + } + if (repConfig != null) { + if (repConfig.getReplicationType() == EC) { + builder.setType(EC); + builder.setEcReplicationConfig(((ECReplicationConfig) repConfig).toProto()); + } else { + builder.setType(repConfig.getReplicationType()); + builder.setFactor(((ReplicatedReplicationConfig) repConfig) + .getReplicationFactor()); + } + } else if (replicationType != null) { + builder.setType(replicationType); + } else if (factor != null) { + builder.setFactor(factor); + } + return builder.build(); + } + + /** + * Decodes {@link SCMListContainerRequestProto} into {@link ListContainerQuery}. + */ + public static ListContainerQuery fromProto(SCMListContainerRequestProto request) { + long startContainerID = 0L; + if (request.hasStartContainerID()) { + startContainerID = request.getStartContainerID(); + } + int count = request.getCount(); + HddsProtos.LifeCycleState state = null; + HddsProtos.ReplicationFactor factor = null; + HddsProtos.ReplicationType replicationType = null; + ReplicationConfig repConfig = null; + if (request.hasState()) { + state = request.getState(); + } + if (request.hasType()) { + replicationType = request.getType(); + } + if (replicationType != null) { + if (replicationType == HddsProtos.ReplicationType.EC) { + if (request.hasEcReplicationConfig()) { + repConfig = new ECReplicationConfig(request.getEcReplicationConfig()); + } + } else { + if (request.hasFactor()) { + repConfig = ReplicationConfig + .fromProtoTypeAndFactor(request.getType(), request.getFactor()); + } + } + } else if (request.hasFactor()) { + factor = request.getFactor(); + } + Boolean suppressed = request.hasSuppressed() ? request.getSuppressed() : null; + return new ListContainerQuery(startContainerID, count, state, factor, + replicationType, repConfig, suppressed); + } +} diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java index 98f8efa9ae30..f9c1753a1fcb 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DeletedBlocksTransactionSummary; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionScmResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMListContainerRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartContainerBalancerResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type; import org.apache.hadoop.hdds.scm.DatanodeAdminError; @@ -147,110 +148,13 @@ List getExistContainerWithPipelinesInBatch( List containerIDs); /** - * Ask SCM a list of containers with a range of container names - * and the limit of count. - * Search container names between start name(exclusive), and - * use prefix name to filter the result. the max size of the - * searching range cannot exceed the value of count. + * Lists containers using a single SCM list container request. * - * @param startContainerID start container ID. - * @param count count, if count {@literal <} 0, the max size is unlimited.( - * Usually the count will be replace with a very big - * value instead of being unlimited in case the db is very big) - * - * @return a list of containers capped by max count allowed - * in "ozone.scm.container.list.max.count" and total number of containers. - * @throws IOException - */ - ContainerListResult listContainer(long startContainerID, - int count) throws IOException; - - /** - * Ask SCM a list of containers with a range of container names - * and the limit of count. - * Search container names between start name(exclusive), and - * use prefix name to filter the result. the max size of the - * searching range cannot exceed the value of count. - * - * @param startContainerID start container ID. - * @param count count, if count {@literal <} 0, the max size is unlimited.( - * Usually the count will be replace with a very big - * value instead of being unlimited in case the db is very big) - * @param state Container with this state will be returned. - * - * @return a list of containers capped by max count allowed - * in "ozone.scm.container.list.max.count" and total number of containers. - * @throws IOException - */ - ContainerListResult listContainer(long startContainerID, - int count, HddsProtos.LifeCycleState state) throws IOException; - - /** - * Ask SCM a list of containers with a range of container names - * and the limit of count. - * Search container names between start name(exclusive), and - * use prefix name to filter the result. the max size of the - * searching range cannot exceed the value of count. - * - * @param startContainerID start container ID. - * @param count count, if count {@literal <} 0, the max size is unlimited.( - * Usually the count will be replace with a very big - * value instead of being unlimited in case the db is very big) - * @param state Container with this state will be returned. - * @param factor Container factor - * @return a list of containers capped by max count allowed - * in "ozone.scm.container.list.max.count" and total number of containers. + * @param request start containerID, count and optional filters + * @return a list of containers matching the filters. * @throws IOException */ - ContainerListResult listContainer(long startContainerID, - int count, HddsProtos.LifeCycleState state, - HddsProtos.ReplicationFactor factor) throws IOException; - - /** - * Ask SCM for a list of containers with a range of container ID, state - * and replication config, and the limit of count. - * The containers are returned from startID (exclusive), and - * filtered by state and replication config. The returned list is limited to - * count entries. - * - * @param startContainerID start container ID. - * @param count count, if count {@literal <} 0, the max size is unlimited.( - * Usually the count will be replace with a very big - * value instead of being unlimited in case the db is very big) - * @param state Container with this state will be returned. - * @param replicationConfig Replication config for the containers - * @return a list of containers capped by max count allowed - * in "ozone.scm.container.list.max.count" and total number of containers. - * @throws IOException - */ - ContainerListResult listContainer(long startContainerID, - int count, HddsProtos.LifeCycleState state, - HddsProtos.ReplicationType replicationType, - ReplicationConfig replicationConfig) throws IOException; - - /** - * Ask SCM for a list of containers with a range of container ID, state - * and replication config, and the limit of count. - * The containers are returned from startID (exclusive), and - * filtered by state and replication config. The returned list is limited to - * count entries. - * - * @param startContainerID start container ID. - * @param count count, if count {@literal <} 0, the max size is unlimited.( - * Usually the count will be replace with a very big - * value instead of being unlimited in case the db is very big) - * @param state Container with this state will be returned. - * @param replicationConfig Replication config for the containers - * @param suppressed container to be suppressed/unsuppressed from report - * @return a list of containers capped by max count allowed - * in "ozone.scm.container.list.max.count" and total number of containers. - * @throws IOException - */ - ContainerListResult listContainer(long startContainerID, - int count, HddsProtos.LifeCycleState state, - HddsProtos.ReplicationType replicationType, - ReplicationConfig replicationConfig, - Boolean suppressed) throws IOException; + ContainerListResult listContainer(SCMListContainerRequestProto request) throws IOException; /** * Deletes a container in SCM. diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java index 56e2ef6408fc..d4b69dc62e55 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java @@ -17,7 +17,6 @@ package org.apache.hadoop.hdds.scm.protocolPB; -import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.EC; import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMCloseContainerResponseProto.Status.CONTAINER_ALREADY_CLOSED; import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMCloseContainerResponseProto.Status.CONTAINER_ALREADY_CLOSING; @@ -40,7 +39,6 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.annotation.InterfaceAudience; import org.apache.hadoop.hdds.client.ECReplicationConfig; -import org.apache.hadoop.hdds.client.ReplicatedReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -426,71 +424,23 @@ public List getExistContainerWithPipelinesInBatch( return cps; } - /** - * {@inheritDoc} - */ - @Override - public ContainerListResult listContainer(long startContainerID, int count) - throws IOException { - return listContainer(startContainerID, count, null, null, null); - } - - @Override - public ContainerListResult listContainer(long startContainerID, int count, - HddsProtos.LifeCycleState state) throws IOException { - return listContainer(startContainerID, count, state, null, null); - } - - @Override - public ContainerListResult listContainer(long startContainerID, int count, - HddsProtos.LifeCycleState state, - HddsProtos.ReplicationType replicationType, - ReplicationConfig replicationConfig) - throws IOException { - return listContainer(startContainerID, count, state, replicationType, replicationConfig, null); - } - @Override - public ContainerListResult listContainer(long startContainerID, int count, - HddsProtos.LifeCycleState state, - HddsProtos.ReplicationType replicationType, - ReplicationConfig replicationConfig, - Boolean suppressed) - throws IOException { + public ContainerListResult listContainer(SCMListContainerRequestProto request) throws IOException { + long startContainerID = request.hasStartContainerID() ? request.getStartContainerID() : 0L; Preconditions.checkState(startContainerID >= 0, "Container ID cannot be negative."); - Preconditions.checkState(count > 0, + Preconditions.checkState(request.getCount() > 0, "Container count must be greater than 0."); - SCMListContainerRequestProto.Builder builder = SCMListContainerRequestProto - .newBuilder(); - builder.setStartContainerID(startContainerID); - builder.setCount(count); - builder.setTraceID(TracingUtil.exportCurrentSpan()); - if (suppressed != null) { - builder.setSuppressed(suppressed); - } - if (state != null) { - builder.setState(state); - } - if (replicationConfig != null) { - if (replicationConfig.getReplicationType() == EC) { - builder.setType(EC); - builder.setEcReplicationConfig( - ((ECReplicationConfig)replicationConfig).toProto()); - } else { - builder.setType(replicationConfig.getReplicationType()); - builder.setFactor(((ReplicatedReplicationConfig)replicationConfig) - .getReplicationFactor()); - } - } else if (replicationType != null) { - builder.setType(replicationType); - } - - SCMListContainerRequestProto request = builder.build(); + SCMListContainerRequestProto withTrace = request.toBuilder() + .setTraceID(TracingUtil.exportCurrentSpan()) + .build(); + return submitListContainer(withTrace); + } + private ContainerListResult submitListContainer(SCMListContainerRequestProto request) throws IOException { SCMListContainerResponseProto response = submitRequest(Type.ListContainer, - builder1 -> builder1.setScmListContainerRequest(request)) + builder -> builder.setScmListContainerRequest(request)) .getScmListContainerResponse(); List containerList = new ArrayList<>(); for (HddsProtos.ContainerInfoProto containerInfoProto : response @@ -505,15 +455,6 @@ public ContainerListResult listContainer(long startContainerID, int count, } } - @Deprecated - @Override - public ContainerListResult listContainer(long startContainerID, int count, - HddsProtos.LifeCycleState state, HddsProtos.ReplicationFactor factor) - throws IOException { - throw new UnsupportedOperationException("Should no longer be called from " + - "the client side"); - } - /** * Ask SCM to delete a container by name. SCM will remove * the container mapping in its database. diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java index 73bf92e9cd58..b71ba5225ca3 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java @@ -40,7 +40,6 @@ import java.util.Optional; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.annotation.InterfaceAudience; -import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -862,53 +861,7 @@ public GetContainerWithPipelineResponseProto getContainerWithPipeline( public SCMListContainerResponseProto listContainer( SCMListContainerRequestProto request) throws IOException { - long startContainerID = 0; - int count = -1; - - // Arguments check. - if (request.hasStartContainerID()) { - // End container name is given. - startContainerID = request.getStartContainerID(); - } - count = request.getCount(); - HddsProtos.LifeCycleState state = null; - HddsProtos.ReplicationFactor factor = null; - HddsProtos.ReplicationType replicationType = null; - ReplicationConfig repConfig = null; - if (request.hasState()) { - state = request.getState(); - } - if (request.hasType()) { - replicationType = request.getType(); - } - if (replicationType != null) { - // This must come from an upgraded client as the older version never - // passed Type. Therefore, we must check for replicationConfig. - if (replicationType == HddsProtos.ReplicationType.EC) { - if (request.hasEcReplicationConfig()) { - repConfig = new ECReplicationConfig(request.getEcReplicationConfig()); - } - } else { - if (request.hasFactor()) { - repConfig = ReplicationConfig - .fromProtoTypeAndFactor(request.getType(), request.getFactor()); - } - } - } else if (request.hasFactor()) { - factor = request.getFactor(); - } - // Filter by suppressed: true (suppressed only), false (unsuppressed only) or null (display all). - Boolean suppressed = request.hasSuppressed() ? request.getSuppressed() : null; - - ContainerListResult containerListAndTotalCount; - if (factor != null) { - // Call from a legacy client - containerListAndTotalCount = - impl.listContainer(startContainerID, count, state, factor); - } else { - containerListAndTotalCount = - impl.listContainer(startContainerID, count, state, replicationType, repConfig, suppressed); - } + ContainerListResult containerListAndTotalCount = impl.listContainer(request); SCMListContainerResponseProto.Builder builder = SCMListContainerResponseProto.newBuilder(); for (ContainerInfo container : containerListAndTotalCount.getContainerInfoList()) { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java index 156eed688d81..4757eed6cba5 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java @@ -65,6 +65,7 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionScmResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionScmResponseProto.Builder; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMListContainerRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartContainerBalancerResponseProto; import org.apache.hadoop.hdds.protocolPB.ReconfigureProtocolPB; import org.apache.hadoop.hdds.protocolPB.ReconfigureProtocolServerSideTranslatorPB; @@ -99,6 +100,8 @@ import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException; +import org.apache.hadoop.hdds.scm.protocol.ScmListContainerRequestCodec; +import org.apache.hadoop.hdds.scm.protocol.ScmListContainerRequestCodec.ListContainerQuery; import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol; import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB; import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB; @@ -439,64 +442,23 @@ private boolean hasRequiredReplicas(ContainerInfo contInfo) { } } - /** - * Lists a range of containers and get their info. - * - * @param startContainerID start containerID. - * @param count count must be {@literal >} 0. - * - * @return a list of containers capped by max count allowed - * in "ozone.scm.container.list.max.count" and total number of containers. - * @throws IOException - */ - @Override - public ContainerListResult listContainer(long startContainerID, - int count) throws IOException { - return listContainer(startContainerID, count, null, null, null, null); - } - - /** - * Lists a range of containers and get their info. - * - * @param startContainerID start containerID. - * @param count count must be {@literal >} 0. - * @param state Container with this state will be returned. - * - * @return a list of containers capped by max count allowed - * in "ozone.scm.container.list.max.count" and total number of containers. - * @throws IOException - */ @Override - public ContainerListResult listContainer(long startContainerID, - int count, HddsProtos.LifeCycleState state) throws IOException { - return listContainer(startContainerID, count, state, null, null, null); + public ContainerListResult listContainer(SCMListContainerRequestProto request) { + return listContainerWithQuery(ScmListContainerRequestCodec.fromProto(request)); } /** - * Lists a range of containers and get their info. - * - * @param startContainerID start containerID. - * @param count count must be {@literal >} 0. - * @param state Container with this state will be returned. - * @param factor Container factor. - * @return a list of containers capped by max count allowed - * in "ozone.scm.container.list.max.count" and total number of containers. - * @throws IOException + * Applies filters from {@link ListContainerQuery}. */ - @Override - @Deprecated - public ContainerListResult listContainer(long startContainerID, - int count, HddsProtos.LifeCycleState state, - HddsProtos.ReplicationFactor factor) throws IOException { - return listContainerInternal(startContainerID, count, state, factor, null, null, null); - } + private ContainerListResult listContainerWithQuery(ListContainerQuery query) { + long startContainerID = query.getStartContainerID(); + int count = query.getCount(); + HddsProtos.LifeCycleState state = query.getState(); + HddsProtos.ReplicationFactor factor = query.getFactor(); + HddsProtos.ReplicationType replicationType = query.getReplicationType(); + ReplicationConfig repConfig = query.getReplicationConfig(); + Boolean suppressed = query.getSuppressed(); - private ContainerListResult listContainerInternal(long startContainerID, int count, - HddsProtos.LifeCycleState state, - HddsProtos.ReplicationFactor factor, - HddsProtos.ReplicationType replicationType, - ReplicationConfig repConfig, - Boolean suppressed) throws IOException { boolean auditSuccess = true; Map auditMap = buildAuditMap(startContainerID, count, state, factor, replicationType, repConfig, suppressed); @@ -585,46 +547,6 @@ private Map buildAuditMap(long startContainerID, int count, return auditMap; } - /** - * Lists a range of containers and get their info. - * - * @param startContainerID start containerID. - * @param count count must be {@literal >} 0. - * @param state Container with this state will be returned. - * @param repConfig Replication Config for the container. - * @return a list of containers capped by max count allowed - * in "ozone.scm.container.list.max.count" and total number of containers. - * @throws IOException - */ - @Override - public ContainerListResult listContainer(long startContainerID, - int count, HddsProtos.LifeCycleState state, - HddsProtos.ReplicationType replicationType, - ReplicationConfig repConfig) throws IOException { - return listContainerInternal(startContainerID, count, state, null, replicationType, repConfig, null); - } - - /** - * Lists a range of containers and get their info. - * - * @param startContainerID start containerID. - * @param count count must be {@literal >} 0. - * @param state Container with this state will be returned. - * @param repConfig Replication Config for the container. - * @param suppressed container to be suppressed/unsuppressed from report - * @return a list of containers capped by max count allowed - * in "ozone.scm.container.list.max.count" and total number of containers. - * @throws IOException - */ - @Override - public ContainerListResult listContainer(long startContainerID, - int count, HddsProtos.LifeCycleState state, - HddsProtos.ReplicationType replicationType, - ReplicationConfig repConfig, - Boolean suppressed) throws IOException { - return listContainerInternal(startContainerID, count, state, null, replicationType, repConfig, suppressed); - } - @Override public void deleteContainer(long containerID) throws IOException { Map auditMap = Maps.newHashMap(); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMClientProtocolServer.java index 7d2f399d1faf..17da88e59735 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMClientProtocolServer.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMClientProtocolServer.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub; import org.apache.hadoop.hdds.scm.ha.SCMNodeDetails; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; +import org.apache.hadoop.hdds.scm.protocol.ScmListContainerRequestCodec; import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB; import org.apache.hadoop.hdds.utils.ProtocolMessageMetrics; import org.apache.hadoop.ozone.container.common.SCMTestUtils; @@ -134,11 +135,12 @@ public void testScmListContainer() throws Exception { new SCMClientProtocolServer(new OzoneConfiguration(), mockStorageContainerManager(), mock(ReconfigurationHandler.class)); try { - assertEquals(10, scmServer.listContainer(1, 10, - null, HddsProtos.ReplicationType.RATIS, null).getContainerInfoList().size()); - // Test call from a legacy client, which uses a different method of listContainer - assertEquals(10, scmServer.listContainer(1, 10, null, - HddsProtos.ReplicationFactor.THREE).getContainerInfoList().size()); + assertEquals(10, scmServer.listContainer(ScmListContainerRequestCodec.toProto( + 1, 10, null, null, HddsProtos.ReplicationType.RATIS, null, null, null)) + .getContainerInfoList().size()); + assertEquals(10, scmServer.listContainer(ScmListContainerRequestCodec.toProto( + 1, 10, null, HddsProtos.ReplicationFactor.THREE, null, null, null, null)) + .getContainerInfoList().size()); } finally { scmServer.stop(); } diff --git a/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java b/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java index c1973891d0aa..886af07070c1 100644 --- a/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java +++ b/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DeletedBlocksTransactionSummary; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionScmResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMListContainerRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartContainerBalancerResponseProto; import org.apache.hadoop.hdds.scm.DatanodeAdminError; import org.apache.hadoop.hdds.scm.ScmConfigKeys; @@ -56,6 +57,7 @@ import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.protocol.ScmListContainerRequestCodec; import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol; import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB.ScmNodeTarget; import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; @@ -360,49 +362,48 @@ public void deleteContainer(long containerID, boolean force) ContainerWithPipeline info = getContainerWithPipeline(containerID); deleteContainer(containerID, info.getPipeline(), force); } - + @Override - public ContainerListResult listContainer(long startContainerID, - int count) throws IOException { + public ContainerListResult listContainer(SCMListContainerRequestProto request) throws IOException { + int count = request.getCount(); if (count > maxCountOfContainerList) { LOG.warn("Attempting to list {} containers. However, this exceeds" + " the cluster's current limit of {}. The results will be capped at the" + " maximum allowed count.", count, maxCountOfContainerList); - count = maxCountOfContainerList; + request = request.toBuilder() + .setCount(maxCountOfContainerList) + .build(); } - return storageContainerLocationClient.listContainer( - startContainerID, count); + return storageContainerLocationClient.listContainer(request); + } + + @Deprecated + @Override + public ContainerListResult listContainer(long startContainerID, + int count) throws IOException { + return listContainer(ScmListContainerRequestCodec.toProto( + startContainerID, count, null, null, null, null, null, null)); } + @Deprecated @Override public ContainerListResult listContainer(long startContainerID, int count, HddsProtos.LifeCycleState state, HddsProtos.ReplicationType repType, ReplicationConfig replicationConfig) throws IOException { - if (count > maxCountOfContainerList) { - LOG.warn("Attempting to list {} containers. However, this exceeds" + - " the cluster's current limit of {}. The results will be capped at the" + - " maximum allowed count.", count, maxCountOfContainerList); - count = maxCountOfContainerList; - } - return storageContainerLocationClient.listContainer( - startContainerID, count, state, repType, replicationConfig); + return listContainer(ScmListContainerRequestCodec.toProto( + startContainerID, count, state, null, repType, replicationConfig, null, null)); } + @Deprecated @Override public ContainerListResult listContainer(long startContainerID, int count, HddsProtos.LifeCycleState state, - HddsProtos.ReplicationType repType, - ReplicationConfig replicationConfig, - Boolean suppressed) throws IOException { - if (count > maxCountOfContainerList) { - LOG.warn("Attempting to list {} containers. However, this exceeds" + - " the cluster's current limit of {}. The results will be capped at the" + - " maximum allowed count.", count, maxCountOfContainerList); - count = maxCountOfContainerList; - } - return storageContainerLocationClient.listContainer( - startContainerID, count, state, repType, replicationConfig, suppressed); + HddsProtos.ReplicationType repType, + ReplicationConfig replicationConfig, + Boolean suppressed) throws IOException { + return listContainer(ScmListContainerRequestCodec.toProto( + startContainerID, count, state, null, repType, replicationConfig, suppressed, null)); } @Override diff --git a/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java b/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java index 0cc4b8bdee1c..61c833bd39c1 100644 --- a/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java +++ b/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hdds.scm.client.ScmClient; import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.ContainerListResult; +import org.apache.hadoop.hdds.scm.protocol.ScmListContainerRequestCodec; import org.apache.hadoop.hdds.server.JsonUtils; import picocli.CommandLine.Command; import picocli.CommandLine.Help.Visibility; @@ -133,8 +134,8 @@ public void execute(ScmClient scmClient) throws IOException { count = maxCountAllowed; } - ContainerListResult containerListResult = - scmClient.listContainer(startId, count, state, type, repConfig, suppressed); + ContainerListResult containerListResult = scmClient.listContainer(ScmListContainerRequestCodec.toProto( + startId, count, state, null, type, repConfig, suppressed, null)); writeContainers(sequenceWriter, containerListResult.getContainerInfoList()); @@ -173,8 +174,8 @@ private void listAllContainers(ScmClient scmClient, SequenceWriter writer, int fetchedCount; do { - ContainerListResult result = - scmClient.listContainer(currentStartId, batchSize, state, type, repConfig, suppressed); + ContainerListResult result = scmClient.listContainer(ScmListContainerRequestCodec.toProto( + currentStartId, batchSize, state, null, type, repConfig, suppressed, null)); fetchedCount = result.getContainerInfoList().size(); writeContainers(writer, result.getContainerInfoList()); diff --git a/hadoop-ozone/cli-admin/src/test/java/org/apache/hadoop/hdds/scm/cli/container/TestContainerReportSuppressOptions.java b/hadoop-ozone/cli-admin/src/test/java/org/apache/hadoop/hdds/scm/cli/container/TestContainerReportSuppressOptions.java index 2e4ef55b8ec2..5ac63e579759 100644 --- a/hadoop-ozone/cli-admin/src/test/java/org/apache/hadoop/hdds/scm/cli/container/TestContainerReportSuppressOptions.java +++ b/hadoop-ozone/cli-admin/src/test/java/org/apache/hadoop/hdds/scm/cli/container/TestContainerReportSuppressOptions.java @@ -23,9 +23,8 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyList; -import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -41,6 +40,7 @@ import org.apache.hadoop.hdds.client.RatisReplicationConfig; import org.apache.hadoop.hdds.protocol.DatanodeID; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SCMListContainerRequestProto; import org.apache.hadoop.hdds.scm.client.ScmClient; import org.apache.hadoop.hdds.scm.container.ContainerHealthState; import org.apache.hadoop.hdds.scm.container.ContainerInfo; @@ -82,9 +82,9 @@ public void setup() throws IOException { when(scmClient.getReplicationManagerReport()).thenAnswer(inv -> createMockReport()); // Mock listContainer - when(scmClient.listContainer(anyLong(), anyInt(), eq(null), eq(null), eq(null), eq(true))) + when(scmClient.listContainer(argThat((SCMListContainerRequestProto r) -> isSuppressedFilter(r)))) .thenAnswer(inv -> listSuppressedContainers()); - when(scmClient.listContainer(anyLong(), anyInt(), eq(null), eq(null), eq(null), eq(false))) + when(scmClient.listContainer(argThat((SCMListContainerRequestProto r) -> isNonSuppressedFilter(r)))) .thenAnswer(inv -> listNonSuppressedContainers()); // Mock suppress/unsuppress @@ -307,4 +307,12 @@ private ContainerListResult listNonSuppressedContainers() { } return new ContainerListResult(nonSuppressed, nonSuppressed.size()); } + + private static boolean isSuppressedFilter(SCMListContainerRequestProto r) { + return r != null && r.hasSuppressed() && r.getSuppressed(); + } + + private static boolean isNonSuppressedFilter(SCMListContainerRequestProto r) { + return r != null && r.hasSuppressed() && !r.getSuppressed(); + } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestContainerOperations.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestContainerOperations.java index 8f46d73e17b9..f3f9e1256358 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestContainerOperations.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestContainerOperations.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException; +import org.apache.hadoop.hdds.scm.protocol.ScmListContainerRequestCodec; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; import org.apache.hadoop.hdds.utils.IOUtils; @@ -146,7 +147,8 @@ public void testListContainerExceedMaxAllowedCountOperations() throws Exception .ONE, OzoneConsts.OZONE); } - int count = storageClient.listContainer(0, CONTAINER_LIST_LIMIT + 1) + int count = storageClient.listContainer(ScmListContainerRequestCodec.toProto( + 0, CONTAINER_LIST_LIMIT + 1, null, null, null, null, null, null)) .getContainerInfoList() .size(); assertEquals(CONTAINER_LIST_LIMIT, count); diff --git a/hadoop-ozone/vapor/pom.xml b/hadoop-ozone/vapor/pom.xml index 9d3cf31af4bb..9b0014950e4e 100644 --- a/hadoop-ozone/vapor/pom.xml +++ b/hadoop-ozone/vapor/pom.xml @@ -98,6 +98,10 @@ org.apache.ozone hdds-container-service + + org.apache.ozone + hdds-interface-admin + org.apache.ozone hdds-interface-client diff --git a/hadoop-ozone/vapor/src/main/java/org/apache/hadoop/ozone/freon/ClosedContainerReplicator.java b/hadoop-ozone/vapor/src/main/java/org/apache/hadoop/ozone/freon/ClosedContainerReplicator.java index 6e1be57d4201..449815b7b525 100644 --- a/hadoop-ozone/vapor/src/main/java/org/apache/hadoop/ozone/freon/ClosedContainerReplicator.java +++ b/hadoop-ozone/vapor/src/main/java/org/apache/hadoop/ozone/freon/ClosedContainerReplicator.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient; import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.protocol.ScmListContainerRequestCodec; import org.apache.hadoop.hdds.utils.HddsServerUtil; import org.apache.hadoop.hdfs.server.datanode.StorageLocation; import org.apache.hadoop.ozone.container.checksum.ContainerChecksumTreeManager; @@ -127,7 +128,9 @@ public Void replicate() throws Exception { new ContainerOperationClient(conf); final List containerInfos = - containerOperationClient.listContainer(0L, 1_000_000).getContainerInfoList(); + containerOperationClient.listContainer(ScmListContainerRequestCodec.toProto( + 0L, 1_000_000, null, null, null, null, null, null)) + .getContainerInfoList(); //logic same as the download+import on the destination datanode initializeReplicationSupervisor(conf, containerInfos.size() * 2); From d6332ee7264a2d45d0e8fdecca13f4ccb507decb Mon Sep 17 00:00:00 2001 From: sarvekshayr Date: Thu, 30 Apr 2026 11:36:04 +0530 Subject: [PATCH 2/2] Code cleanup --- .../hdds/scm/protocol/ScmListContainerRequestCodec.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmListContainerRequestCodec.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmListContainerRequestCodec.java index 407e3b5630c1..70539c19ab3d 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmListContainerRequestCodec.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocol/ScmListContainerRequestCodec.java @@ -127,11 +127,12 @@ public static SCMListContainerRequestProto toProto( builder.setState(state); } if (repConfig != null) { - if (repConfig.getReplicationType() == EC) { + HddsProtos.ReplicationType repType = repConfig.getReplicationType(); + if (repType == EC) { builder.setType(EC); builder.setEcReplicationConfig(((ECReplicationConfig) repConfig).toProto()); } else { - builder.setType(repConfig.getReplicationType()); + builder.setType(repType); builder.setFactor(((ReplicatedReplicationConfig) repConfig) .getReplicationFactor()); } @@ -163,14 +164,14 @@ public static ListContainerQuery fromProto(SCMListContainerRequestProto request) replicationType = request.getType(); } if (replicationType != null) { - if (replicationType == HddsProtos.ReplicationType.EC) { + if (replicationType == EC) { if (request.hasEcReplicationConfig()) { repConfig = new ECReplicationConfig(request.getEcReplicationConfig()); } } else { if (request.hasFactor()) { repConfig = ReplicationConfig - .fromProtoTypeAndFactor(request.getType(), request.getFactor()); + .fromProtoTypeAndFactor(replicationType, request.getFactor()); } } } else if (request.hasFactor()) {