This is an automated email from the ASF dual-hosted git repository.
sumitagrawal pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git
The following commit(s) were added to refs/heads/master by this push:
new 911a583e72 HDDS-8188. Support max allowed length in response of ozone
admin container list (#7181)
911a583e72 is described below
commit 911a583e72b8c01437f90fda652a6334a7813b52
Author: Sarveksha Yeshavantha Raju
<[email protected]>
AuthorDate: Thu Oct 10 12:39:00 2024 +0530
HDDS-8188. Support max allowed length in response of ozone admin container
list (#7181)
---
.../org/apache/hadoop/hdds/scm/ScmConfigKeys.java | 5 +
.../apache/hadoop/hdds/scm/client/ScmClient.java | 11 +-
.../hdds/scm/container/ContainerListResult.java | 57 +++++++
.../protocol/StorageContainerLocationProtocol.java | 22 ++-
.../common/src/main/resources/ozone-default.xml | 7 +
...inerLocationProtocolClientSideTranslatorPB.java | 18 +-
.../src/main/proto/ScmAdminProtocol.proto | 1 +
.../hdds/scm/container/ContainerManagerImpl.java | 6 +
...inerLocationProtocolServerSideTranslatorPB.java | 13 +-
.../hdds/scm/server/SCMClientProtocolServer.java | 181 ++++++++++-----------
.../scm/server/TestSCMClientProtocolServer.java | 54 ++++++
.../hdds/scm/cli/ContainerOperationClient.java | 21 ++-
.../hdds/scm/cli/container/ContainerCommands.java | 8 +
.../hdds/scm/cli/container/ListSubcommand.java | 61 ++++++-
.../src/main/smoketest/admincli/container.robot | 12 ++
.../hadoop/ozone/TestContainerOperations.java | 19 +++
.../hadoop/ozone/shell/TestOzoneShellHA.java | 34 +++-
.../ozone/freon/ClosedContainerReplicator.java | 2 +-
18 files changed, 399 insertions(+), 133 deletions(-)
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 db789783c7..c4b42acec4 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
@@ -131,6 +131,11 @@ public final class ScmConfigKeys {
"hdds.ratis.snapshot.threshold";
public static final long HDDS_RATIS_SNAPSHOT_THRESHOLD_DEFAULT = 100000;
+ public static final String OZONE_SCM_CONTAINER_LIST_MAX_COUNT =
+ "ozone.scm.container.list.max.count";
+
+ public static final int OZONE_SCM_CONTAINER_LIST_MAX_COUNT_DEFAULT = 4096;
+
// TODO : this is copied from OzoneConsts, may need to move to a better place
public static final String OZONE_SCM_CHUNK_SIZE_KEY = "ozone.scm.chunk.size";
// 4 MB by default
diff --git
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
index c41e516d7a..8662cac809 100644
---
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
+++
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
@@ -27,6 +27,7 @@ import
org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolPro
import
org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto;
import org.apache.hadoop.hdds.scm.DatanodeAdminError;
import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerListResult;
import org.apache.hadoop.hdds.scm.container.ContainerReplicaInfo;
import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
import
org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
@@ -122,10 +123,11 @@ public interface ScmClient extends Closeable {
* @param startContainerID start containerID.
* @param count count must be {@literal >} 0.
*
- * @return a list of pipeline.
+ * @return a list of containers capped by max count allowed
+ * in "ozone.scm.container.list.max.count" and total number of containers.
* @throws IOException
*/
- List<ContainerInfo> listContainer(long startContainerID,
+ ContainerListResult listContainer(long startContainerID,
int count) throws IOException;
/**
@@ -135,10 +137,11 @@ public interface ScmClient extends Closeable {
* @param count count must be {@literal >} 0.
* @param state Container of this state will be returned.
* @param replicationConfig container replication Config.
- * @return a list of pipeline.
+ * @return a list of containers capped by max count allowed
+ * in "ozone.scm.container.list.max.count" and total number of containers.
* @throws IOException
*/
- List<ContainerInfo> listContainer(long startContainerID, int count,
+ ContainerListResult listContainer(long startContainerID, int count,
HddsProtos.LifeCycleState state,
HddsProtos.ReplicationType replicationType,
ReplicationConfig replicationConfig)
diff --git
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerListResult.java
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerListResult.java
new file mode 100644
index 0000000000..9e8d5738db
--- /dev/null
+++
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerListResult.java
@@ -0,0 +1,57 @@
+/*
+ * 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.container;
+
+import java.util.List;
+
+/**
+ * Wrapper class for the result of listing containers with their total count.
+ */
+public class ContainerListResult {
+ private final List<ContainerInfo> containerInfoList;
+ private final long totalCount;
+
+ /**
+ * Constructs a new ContainerListResult.
+ *
+ * @param containerInfoList the list of containers
+ * @param totalCount the total number of containers
+ */
+ public ContainerListResult(List<ContainerInfo> containerInfoList, long
totalCount) {
+ this.containerInfoList = containerInfoList;
+ this.totalCount = totalCount;
+ }
+
+ /**
+ * Gets the list of containers.
+ *
+ * @return the list of containers
+ */
+ public List<ContainerInfo> getContainerInfoList() {
+ return containerInfoList;
+ }
+
+ /**
+ * Gets the total count of containers.
+ *
+ * @return the total count of containers
+ */
+ public long getTotalCount() {
+ return totalCount;
+ }
+}
diff --git
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
index f280e9bb6e..3c3786b38d 100644
---
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
+++
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
@@ -29,6 +29,7 @@ import
org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolPro
import org.apache.hadoop.hdds.scm.DatanodeAdminError;
import org.apache.hadoop.hdds.scm.ScmConfig;
import org.apache.hadoop.hdds.scm.ScmInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerListResult;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
@@ -146,10 +147,11 @@ public interface StorageContainerLocationProtocol extends
Closeable {
* 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 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
*/
- List<ContainerInfo> listContainer(long startContainerID,
+ ContainerListResult listContainer(long startContainerID,
int count) throws IOException;
/**
@@ -165,10 +167,11 @@ public interface StorageContainerLocationProtocol extends
Closeable {
* 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 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
*/
- List<ContainerInfo> listContainer(long startContainerID,
+ ContainerListResult listContainer(long startContainerID,
int count, HddsProtos.LifeCycleState state) throws IOException;
/**
@@ -184,14 +187,14 @@ public interface StorageContainerLocationProtocol extends
Closeable {
* 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 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
*/
- List<ContainerInfo> listContainer(long startContainerID,
+ 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.
@@ -205,10 +208,11 @@ public interface StorageContainerLocationProtocol extends
Closeable {
* 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 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
*/
- List<ContainerInfo> listContainer(long startContainerID,
+ ContainerListResult listContainer(long startContainerID,
int count, HddsProtos.LifeCycleState state,
HddsProtos.ReplicationType replicationType,
ReplicationConfig replicationConfig) throws IOException;
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml
b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index a8e1f7017e..9944051f3e 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -160,6 +160,13 @@
this not set. Ideally, this should be mapped to a fast disk like an SSD.
</description>
</property>
+ <property>
+ <name>ozone.scm.container.list.max.count</name>
+ <value>4096</value>
+ <tag>OZONE, SCM, CONTAINER</tag>
+ <description>The max number of containers info could be included in
+ response of ListContainer request.</description>
+ </property>
<property>
<name>hdds.datanode.dir</name>
<value/>
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 a5fdfea0f6..d76ce3b72c 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
@@ -108,6 +108,7 @@ import
org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolPro
import
org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoRequestProto;
import org.apache.hadoop.hdds.scm.DatanodeAdminError;
import org.apache.hadoop.hdds.scm.ScmInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerListResult;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
@@ -389,19 +390,19 @@ public final class
StorageContainerLocationProtocolClientSideTranslatorPB
* {@inheritDoc}
*/
@Override
- public List<ContainerInfo> listContainer(long startContainerID, int count)
+ public ContainerListResult listContainer(long startContainerID, int count)
throws IOException {
return listContainer(startContainerID, count, null, null, null);
}
@Override
- public List<ContainerInfo> listContainer(long startContainerID, int count,
+ public ContainerListResult listContainer(long startContainerID, int count,
HddsProtos.LifeCycleState state) throws IOException {
return listContainer(startContainerID, count, state, null, null);
}
@Override
- public List<ContainerInfo> listContainer(long startContainerID, int count,
+ public ContainerListResult listContainer(long startContainerID, int count,
HddsProtos.LifeCycleState state,
HddsProtos.ReplicationType replicationType,
ReplicationConfig replicationConfig)
@@ -443,12 +444,17 @@ public final class
StorageContainerLocationProtocolClientSideTranslatorPB
.getContainersList()) {
containerList.add(ContainerInfo.fromProtobuf(containerInfoProto));
}
- return containerList;
+
+ if (response.hasContainerCount()) {
+ return new ContainerListResult(containerList,
response.getContainerCount());
+ } else {
+ return new ContainerListResult(containerList, -1);
+ }
}
@Deprecated
@Override
- public List<ContainerInfo> listContainer(long startContainerID, int count,
+ public ContainerListResult listContainer(long startContainerID, int count,
HddsProtos.LifeCycleState state, HddsProtos.ReplicationFactor factor)
throws IOException {
throw new UnsupportedOperationException("Should no longer be called from "
+
@@ -1209,7 +1215,7 @@ public final class
StorageContainerLocationProtocolClientSideTranslatorPB
public List<ContainerInfo> getListOfContainers(
long startContainerID, int count, HddsProtos.LifeCycleState state)
throws IOException {
- return listContainer(startContainerID, count, state);
+ return listContainer(startContainerID, count,
state).getContainerInfoList();
}
@Override
diff --git a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto
b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto
index 039914369b..ee2df89e81 100644
--- a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto
+++ b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto
@@ -296,6 +296,7 @@ message SCMListContainerRequestProto {
message SCMListContainerResponseProto {
repeated ContainerInfoProto containers = 1;
+ optional int64 containerCount = 2;
}
message SCMDeleteContainerRequestProto {
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 00aee0f62c..d61f9ee366 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
@@ -86,6 +86,8 @@ public class ContainerManagerImpl implements ContainerManager
{
@SuppressWarnings("java:S2245") // no need for secure random
private final Random random = new Random();
+ private int maxCountOfContainerList;
+
/**
*
*/
@@ -115,6 +117,10 @@ public class ContainerManagerImpl implements
ContainerManager {
.getInt(ScmConfigKeys.OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT,
ScmConfigKeys.OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT_DEFAULT);
+ this.maxCountOfContainerList = conf
+ .getInt(ScmConfigKeys.OZONE_SCM_CONTAINER_LIST_MAX_COUNT,
+ ScmConfigKeys.OZONE_SCM_CONTAINER_LIST_MAX_COUNT_DEFAULT);
+
this.scmContainerManagerMetrics = SCMContainerManagerMetrics.create();
}
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 6d94716e95..c4d2bfdc64 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
@@ -115,6 +115,7 @@ import
org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolPro
import
org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ResetDeletedBlockRetryCountResponseProto;
import org.apache.hadoop.hdds.scm.DatanodeAdminError;
import org.apache.hadoop.hdds.scm.ScmInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerListResult;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import
org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
@@ -857,21 +858,21 @@ public final class
StorageContainerLocationProtocolServerSideTranslatorPB
} else if (request.hasFactor()) {
factor = request.getFactor();
}
- List<ContainerInfo> containerList;
+ ContainerListResult containerListAndTotalCount;
if (factor != null) {
// Call from a legacy client
- containerList =
+ containerListAndTotalCount =
impl.listContainer(startContainerID, count, state, factor);
} else {
- containerList =
- impl.listContainer(startContainerID, count, state, replicationType,
- repConfig);
+ containerListAndTotalCount =
+ impl.listContainer(startContainerID, count, state, replicationType,
repConfig);
}
SCMListContainerResponseProto.Builder builder =
SCMListContainerResponseProto.newBuilder();
- for (ContainerInfo container : containerList) {
+ for (ContainerInfo container :
containerListAndTotalCount.getContainerInfoList()) {
builder.addContainers(container.getProtobuf());
}
+ builder.setContainerCount(containerListAndTotalCount.getTotalCount());
return builder.build();
}
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 33ea27923b..6fdc81ac9a 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
@@ -48,6 +48,7 @@ import org.apache.hadoop.hdds.ratis.RatisHelper;
import org.apache.hadoop.hdds.scm.DatanodeAdminError;
import org.apache.hadoop.hdds.scm.FetchMetrics;
import org.apache.hadoop.hdds.scm.ScmInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerListResult;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
@@ -107,6 +108,7 @@ import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
@@ -417,11 +419,12 @@ public class SCMClientProtocolServer implements
* @param startContainerID start containerID.
* @param count count must be {@literal >} 0.
*
- * @return a list of pipeline.
+ * @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 List<ContainerInfo> listContainer(long startContainerID,
+ public ContainerListResult listContainer(long startContainerID,
int count) throws IOException {
return listContainer(startContainerID, count, null, null, null);
}
@@ -433,11 +436,12 @@ public class SCMClientProtocolServer implements
* @param count count must be {@literal >} 0.
* @param state Container with this state will be returned.
*
- * @return a list of pipeline.
+ * @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 List<ContainerInfo> listContainer(long startContainerID,
+ public ContainerListResult listContainer(long startContainerID,
int count, HddsProtos.LifeCycleState state) throws IOException {
return listContainer(startContainerID, count, state, null, null);
}
@@ -449,53 +453,36 @@ public class SCMClientProtocolServer implements
* @param count count must be {@literal >} 0.
* @param state Container with this state will be returned.
* @param factor Container factor.
- * @return a list of pipeline.
+ * @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
@Deprecated
- public List<ContainerInfo> listContainer(long startContainerID,
+ public ContainerListResult listContainer(long startContainerID,
int count, HddsProtos.LifeCycleState state,
HddsProtos.ReplicationFactor factor) throws IOException {
+ return listContainerInternal(startContainerID, count, state, factor, null,
null);
+ }
+
+ private ContainerListResult listContainerInternal(long startContainerID, int
count,
+ HddsProtos.LifeCycleState state,
+ HddsProtos.ReplicationFactor factor,
+ HddsProtos.ReplicationType replicationType,
+ ReplicationConfig repConfig) throws IOException {
boolean auditSuccess = true;
- Map<String, String> auditMap = Maps.newHashMap();
- auditMap.put("startContainerID", String.valueOf(startContainerID));
- auditMap.put("count", String.valueOf(count));
- if (state != null) {
- auditMap.put("state", state.name());
- }
- if (factor != null) {
- auditMap.put("factor", factor.name());
- }
+ Map<String, String> auditMap = buildAuditMap(startContainerID, count,
state, factor, replicationType, repConfig);
+
try {
- final ContainerID containerId = ContainerID.valueOf(startContainerID);
- if (state != null) {
- if (factor != null) {
- return scm.getContainerManager().getContainers(state).stream()
- .filter(info -> info.containerID().getId() >= startContainerID)
- //Filtering EC replication type as EC will not have factor.
- .filter(info -> info
- .getReplicationType() != HddsProtos.ReplicationType.EC)
- .filter(info -> (info.getReplicationFactor() == factor))
- .sorted().limit(count).collect(Collectors.toList());
- } else {
- return scm.getContainerManager().getContainers(state).stream()
- .filter(info -> info.containerID().getId() >= startContainerID)
- .sorted().limit(count).collect(Collectors.toList());
- }
- } else {
- if (factor != null) {
- return scm.getContainerManager().getContainers().stream()
- .filter(info -> info.containerID().getId() >= startContainerID)
- //Filtering EC replication type as EC will not have factor.
- .filter(info -> info
- .getReplicationType() != HddsProtos.ReplicationType.EC)
- .filter(info -> info.getReplicationFactor() == factor)
- .sorted().limit(count).collect(Collectors.toList());
- } else {
- return scm.getContainerManager().getContainers(containerId, count);
- }
- }
+ Stream<ContainerInfo> containerStream =
+ buildContainerStream(factor, replicationType, repConfig,
getBaseContainerStream(state));
+ List<ContainerInfo> containerInfos =
+ containerStream.filter(info -> info.containerID().getId() >=
startContainerID)
+ .sorted().collect(Collectors.toList());
+ List<ContainerInfo> limitedContainers =
+ containerInfos.stream().limit(count).collect(Collectors.toList());
+ long totalCount = (long) containerInfos.size();
+ return new ContainerListResult(limitedContainers, totalCount);
} catch (Exception ex) {
auditSuccess = false;
AUDIT.logReadFailure(
@@ -509,74 +496,74 @@ public class SCMClientProtocolServer implements
}
}
- /**
- * 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 pipeline.
- * @throws IOException
- */
- @Override
- public List<ContainerInfo> listContainer(long startContainerID,
- int count, HddsProtos.LifeCycleState state,
+ private Stream<ContainerInfo>
buildContainerStream(HddsProtos.ReplicationFactor factor,
HddsProtos.ReplicationType replicationType,
- ReplicationConfig repConfig) throws IOException {
- boolean auditSuccess = true;
- Map<String, String> auditMap = Maps.newHashMap();
+ ReplicationConfig repConfig,
+ Stream<ContainerInfo> containerStream) {
+ if (factor != null) {
+ containerStream = containerStream.filter(info ->
info.getReplicationType() != HddsProtos.ReplicationType.EC)
+ .filter(info -> info.getReplicationFactor() == factor);
+ } else if (repConfig != null) {
+ // If we have repConfig filter by it, as it includes repType too.
+ // Otherwise, we may have a filter just for repType, eg all EC containers
+ // without filtering on their replication scheme
+ containerStream = containerStream
+ .filter(info -> info.getReplicationConfig().equals(repConfig));
+ } else if (replicationType != null) {
+ containerStream = containerStream.filter(info ->
info.getReplicationType() == replicationType);
+ }
+ return containerStream;
+ }
+
+ private Stream<ContainerInfo>
getBaseContainerStream(HddsProtos.LifeCycleState state) {
+ if (state != null) {
+ return scm.getContainerManager().getContainers(state).stream();
+ } else {
+ return scm.getContainerManager().getContainers().stream();
+ }
+ }
+
+ private Map<String, String> buildAuditMap(long startContainerID, int count,
+ HddsProtos.LifeCycleState state,
+ HddsProtos.ReplicationFactor factor,
+ HddsProtos.ReplicationType replicationType,
+ ReplicationConfig repConfig) {
+ Map<String, String> auditMap = new HashMap<>();
auditMap.put("startContainerID", String.valueOf(startContainerID));
auditMap.put("count", String.valueOf(count));
if (state != null) {
auditMap.put("state", state.name());
}
+ if (factor != null) {
+ auditMap.put("factor", factor.name());
+ }
if (replicationType != null) {
auditMap.put("replicationType", replicationType.toString());
}
if (repConfig != null) {
auditMap.put("replicationConfig", repConfig.toString());
}
- try {
- final ContainerID containerId = ContainerID.valueOf(startContainerID);
- if (state == null && replicationType == null && repConfig == null) {
- // Not filters, so just return everything
- return scm.getContainerManager().getContainers(containerId, count);
- }
- List<ContainerInfo> containerList;
- if (state != null) {
- containerList = scm.getContainerManager().getContainers(state);
- } else {
- containerList = scm.getContainerManager().getContainers();
- }
+ return auditMap;
+ }
- Stream<ContainerInfo> containerStream = containerList.stream()
- .filter(info -> info.containerID().getId() >= startContainerID);
- // If we have repConfig filter by it, as it includes repType too.
- // Otherwise, we may have a filter just for repType, eg all EC containers
- // without filtering on their replication scheme
- if (repConfig != null) {
- containerStream = containerStream
- .filter(info -> info.getReplicationConfig().equals(repConfig));
- } else if (replicationType != null) {
- containerStream = containerStream
- .filter(info -> info.getReplicationType() == replicationType);
- }
- return containerStream.sorted()
- .limit(count)
- .collect(Collectors.toList());
- } catch (Exception ex) {
- auditSuccess = false;
- AUDIT.logReadFailure(
- buildAuditMessageForFailure(SCMAction.LIST_CONTAINER, auditMap, ex));
- throw ex;
- } finally {
- if (auditSuccess) {
- AUDIT.logReadSuccess(
- buildAuditMessageForSuccess(SCMAction.LIST_CONTAINER, 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);
}
@Override
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 7c06b79a2f..8e21eef930 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
@@ -17,12 +17,19 @@
*/
package org.apache.hadoop.hdds.scm.server;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.ReconfigurationHandler;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import
org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionScmRequestProto;
import
org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionScmResponseProto;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManagerImpl;
import org.apache.hadoop.hdds.scm.HddsTestUtils;
import org.apache.hadoop.hdds.scm.ha.SCMContext;
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.StorageContainerLocationProtocolServerSideTranslatorPB;
import org.apache.hadoop.hdds.utils.ProtocolMessageMetrics;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
@@ -35,9 +42,13 @@ import org.junit.jupiter.api.io.TempDir;
import java.io.File;
import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
import static
org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_READONLY_ADMINISTRATORS;
import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -112,4 +123,47 @@ public class TestSCMClientProtocolServer {
UserGroupInformation.reset();
}
}
+
+ /**
+ * Tests listContainer of scm.
+ */
+ @Test
+ public void testScmListContainer() throws Exception {
+ SCMClientProtocolServer scmServer =
+ new SCMClientProtocolServer(new OzoneConfiguration(),
+ mockStorageContainerManager(), mock(ReconfigurationHandler.class));
+
+ 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());
+ }
+
+ private StorageContainerManager mockStorageContainerManager() {
+ List<ContainerInfo> infos = new ArrayList<>();
+ for (int i = 0; i < 10; i++) {
+ infos.add(newContainerInfoForTest());
+ }
+ ContainerManagerImpl containerManager = mock(ContainerManagerImpl.class);
+ when(containerManager.getContainers()).thenReturn(infos);
+ StorageContainerManager storageContainerManager =
mock(StorageContainerManager.class);
+
when(storageContainerManager.getContainerManager()).thenReturn(containerManager);
+
+ SCMNodeDetails scmNodeDetails = mock(SCMNodeDetails.class);
+ when(scmNodeDetails.getClientProtocolServerAddress()).thenReturn(new
InetSocketAddress("localhost", 9876));
+
when(scmNodeDetails.getClientProtocolServerAddressKey()).thenReturn("test");
+
when(storageContainerManager.getScmNodeDetails()).thenReturn(scmNodeDetails);
+ return storageContainerManager;
+ }
+
+ private ContainerInfo newContainerInfoForTest() {
+ return new ContainerInfo.Builder()
+ .setContainerID(1)
+ .setPipelineID(PipelineID.randomId())
+ .setReplicationConfig(
+ RatisReplicationConfig
+ .getInstance(HddsProtos.ReplicationFactor.THREE))
+ .build();
+ }
}
diff --git
a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java
b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java
index 220abd1fcd..884d7f939a 100644
---
a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java
+++
b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.client.ClientTrustManager;
+import org.apache.hadoop.hdds.scm.container.ContainerListResult;
import
org.apache.hadoop.hdds.security.x509.certificate.client.CACertificateProvider;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import org.apache.hadoop.hdds.scm.container.ContainerID;
@@ -83,6 +84,7 @@ public class ContainerOperationClient implements ScmClient {
private final boolean containerTokenEnabled;
private final OzoneConfiguration configuration;
private XceiverClientManager xceiverClientManager;
+ private int maxCountOfContainerList;
public synchronized XceiverClientManager getXceiverClientManager()
throws IOException {
@@ -110,6 +112,9 @@ public class ContainerOperationClient implements ScmClient {
}
containerTokenEnabled = conf.getBoolean(HDDS_CONTAINER_TOKEN_ENABLED,
HDDS_CONTAINER_TOKEN_ENABLED_DEFAULT);
+ maxCountOfContainerList = conf
+ .getInt(ScmConfigKeys.OZONE_SCM_CONTAINER_LIST_MAX_COUNT,
+ ScmConfigKeys.OZONE_SCM_CONTAINER_LIST_MAX_COUNT_DEFAULT);
}
private XceiverClientManager newXCeiverClientManager(ConfigurationSource
conf)
@@ -339,17 +344,29 @@ public class ContainerOperationClient implements
ScmClient {
}
@Override
- public List<ContainerInfo> listContainer(long startContainerID,
+ public ContainerListResult listContainer(long startContainerID,
int count) 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);
}
@Override
- public List<ContainerInfo> listContainer(long startContainerID,
+ 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);
}
diff --git
a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommands.java
b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommands.java
index 54c69273f0..15dd873491 100644
---
a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommands.java
+++
b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommands.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hdds.cli.SubcommandWithParent;
import org.kohsuke.MetaInfServices;
import picocli.CommandLine.Command;
+import picocli.CommandLine.ParentCommand;
import picocli.CommandLine.Model.CommandSpec;
import picocli.CommandLine.Spec;
@@ -51,6 +52,9 @@ public class ContainerCommands implements Callable<Void>,
SubcommandWithParent {
@Spec
private CommandSpec spec;
+ @ParentCommand
+ private OzoneAdmin parent;
+
@Override
public Void call() throws Exception {
GenericCli.missingSubcommand(spec);
@@ -61,4 +65,8 @@ public class ContainerCommands implements Callable<Void>,
SubcommandWithParent {
public Class<?> getParentType() {
return OzoneAdmin.class;
}
+
+ public OzoneAdmin getParent() {
+ return parent;
+ }
}
diff --git
a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java
b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java
index ecc43d0408..88ccef702b 100644
---
a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java
+++
b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.hdds.scm.cli.container;
import java.io.IOException;
-import java.util.List;
import com.google.common.base.Strings;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
@@ -26,7 +25,9 @@ import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.container.ContainerListResult;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
@@ -37,6 +38,7 @@ import com.fasterxml.jackson.databind.ObjectWriter;
import com.fasterxml.jackson.databind.SerializationFeature;
import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule;
import picocli.CommandLine.Command;
+import picocli.CommandLine.ParentCommand;
import picocli.CommandLine.Help.Visibility;
import picocli.CommandLine.Option;
@@ -55,10 +57,15 @@ public class ListSubcommand extends ScmSubcommand {
private long startId;
@Option(names = {"-c", "--count"},
- description = "Maximum number of containers to list",
+ description = "Maximum number of containers to list.",
defaultValue = "20", showDefaultValue = Visibility.ALWAYS)
private int count;
+ @Option(names = {"-a", "--all"},
+ description = "List all containers.",
+ defaultValue = "false")
+ private boolean all;
+
@Option(names = {"--state"},
description = "Container state(OPEN, CLOSING, QUASI_CLOSED, CLOSED, " +
"DELETING, DELETED)")
@@ -75,6 +82,9 @@ public class ListSubcommand extends ScmSubcommand {
private static final ObjectWriter WRITER;
+ @ParentCommand
+ private ContainerCommands parent;
+
static {
ObjectMapper mapper = new ObjectMapper()
.registerModule(new JavaTimeModule())
@@ -105,12 +115,49 @@ public class ListSubcommand extends ScmSubcommand {
ReplicationType.fromProto(type),
replication, new OzoneConfiguration());
}
- List<ContainerInfo> containerList =
- scmClient.listContainer(startId, count, state, type, repConfig);
- // Output data list
- for (ContainerInfo container : containerList) {
- outputContainerInfo(container);
+ int maxCountAllowed = parent.getParent().getOzoneConf()
+ .getInt(ScmConfigKeys.OZONE_SCM_CONTAINER_LIST_MAX_COUNT,
+ ScmConfigKeys.OZONE_SCM_CONTAINER_LIST_MAX_COUNT_DEFAULT);
+
+ ContainerListResult containerListAndTotalCount;
+
+ if (!all) {
+ if (count > maxCountAllowed) {
+ System.err.printf("Attempting to list the first %d records of
containers." +
+ " However it exceeds the cluster's current limit of %d. The
results will be capped at the" +
+ " maximum allowed count.%n", count,
ScmConfigKeys.OZONE_SCM_CONTAINER_LIST_MAX_COUNT_DEFAULT);
+ count = maxCountAllowed;
+ }
+ containerListAndTotalCount = scmClient.listContainer(startId, count,
state, type, repConfig);
+ for (ContainerInfo container :
containerListAndTotalCount.getContainerInfoList()) {
+ outputContainerInfo(container);
+ }
+
+ if (containerListAndTotalCount.getTotalCount() > count) {
+ System.err.printf("Displaying %d out of %d containers. " +
+ "Container list has more containers.%n",
+ count, containerListAndTotalCount.getTotalCount());
+ }
+ } else {
+ // Batch size is either count passed through cli or maxCountAllowed
+ int batchSize = (count > 0) ? count : maxCountAllowed;
+ long currentStartId = startId;
+ int fetchedCount;
+
+ do {
+ // Fetch containers in batches of 'batchSize'
+ containerListAndTotalCount = scmClient.listContainer(currentStartId,
batchSize, state, type, repConfig);
+ fetchedCount =
containerListAndTotalCount.getContainerInfoList().size();
+
+ for (ContainerInfo container :
containerListAndTotalCount.getContainerInfoList()) {
+ outputContainerInfo(container);
+ }
+
+ if (fetchedCount > 0) {
+ currentStartId =
containerListAndTotalCount.getContainerInfoList().get(fetchedCount -
1).getContainerID() + 1;
+ }
+ } while (fetchedCount > 0);
}
}
}
diff --git a/hadoop-ozone/dist/src/main/smoketest/admincli/container.robot
b/hadoop-ozone/dist/src/main/smoketest/admincli/container.robot
index c50daa724d..83c0731ff7 100644
--- a/hadoop-ozone/dist/src/main/smoketest/admincli/container.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/admincli/container.robot
@@ -84,6 +84,18 @@ Report containers as JSON
Should contain ${output} stats
Should contain ${output} samples
+List all containers
+ ${output} = Execute ozone admin container list --all
+ Should contain ${output} OPEN
+
+List all containers according to count (batchSize)
+ ${output} = Execute ozone admin container list --all
--count 10
+ Should contain ${output} OPEN
+
+List all containers from a particular container ID
+ ${output} = Execute ozone admin container list --all
--start 1
+ Should contain ${output} OPEN
+
Close container
${container} = Execute ozone admin container list --state
OPEN | jq -r 'select(.replicationConfig.replicationFactor == "THREE") |
.containerID' | head -1
Execute ozone admin container close
"${container}"
diff --git
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java
index cbd1829ef0..798e8a1599 100644
---
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java
+++
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java
@@ -77,6 +77,7 @@ public class TestContainerOperations {
ozoneConf = new OzoneConfiguration();
ozoneConf.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
SCMContainerPlacementCapacity.class, PlacementPolicy.class);
+ ozoneConf.set(ScmConfigKeys.OZONE_SCM_CONTAINER_LIST_MAX_COUNT, "1");
cluster =
MiniOzoneCluster.newBuilder(ozoneConf).setNumDatanodes(3).build();
storageClient = new ContainerOperationClient(ozoneConf);
cluster.waitForClusterToBeReady();
@@ -144,6 +145,24 @@ public class TestContainerOperations {
.getContainerID());
}
+ /**
+ * Test to try to list number of containers over the max number Ozone allows.
+ * @throws Exception
+ */
+ @Test
+ public void testListContainerExceedMaxAllowedCountOperations() throws
Exception {
+ // create 2 containers in cluster where the limit of max count for
+ // listing container is set to 1
+ for (int i = 0; i < 2; i++) {
+ storageClient.createContainer(HddsProtos
+ .ReplicationType.STAND_ALONE, HddsProtos.ReplicationFactor
+ .ONE, OzoneConsts.OZONE);
+ }
+
+ assertEquals(1, storageClient.listContainer(0, 2)
+ .getContainerInfoList().size());
+ }
+
/**
* A simple test to get Pipeline with {@link ContainerOperationClient}.
* @throws Exception
diff --git
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java
index 9cccd56d4d..1a661ecdd7 100644
---
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java
+++
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneShellHA.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.crypto.key.KeyProvider;
import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
import org.apache.hadoop.crypto.key.kms.server.MiniKMS;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.utils.IOUtils;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileChecksum;
@@ -154,6 +155,8 @@ public class TestOzoneShellHA {
private static String omServiceId;
private static int numOfOMs;
+ private static OzoneConfiguration ozoneConfiguration;
+
/**
* Create a MiniOzoneCluster for testing with using distributed Ozone
* handler type.
@@ -198,6 +201,8 @@ public class TestOzoneShellHA {
getKeyProviderURI(miniKMS));
conf.setInt(OMConfigKeys.OZONE_DIR_DELETING_SERVICE_INTERVAL, 10);
conf.setBoolean(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS, true);
+ conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_LIST_MAX_COUNT, 1);
+ ozoneConfiguration = conf;
MiniOzoneHAClusterImpl.Builder builder =
MiniOzoneCluster.newHABuilder(conf);
builder.setOMServiceId(omServiceId)
.setNumOfOzoneManagers(numOfOMs)
@@ -233,7 +238,7 @@ public class TestOzoneShellHA {
@BeforeEach
public void setup() throws UnsupportedEncodingException {
ozoneShell = new OzoneShell();
- ozoneAdminShell = new OzoneAdmin();
+ ozoneAdminShell = new OzoneAdmin(ozoneConfiguration);
System.setOut(new PrintStream(out, false, DEFAULT_ENCODING));
System.setErr(new PrintStream(err, false, DEFAULT_ENCODING));
}
@@ -957,6 +962,33 @@ public class TestOzoneShellHA {
return res;
}
+ @Test
+ public void testOzoneAdminCmdListAllContainer()
+ throws UnsupportedEncodingException {
+ String[] args = new String[] {"container", "create", "--scm",
+ "localhost:" +
cluster.getStorageContainerManager().getClientRpcPort()};
+ for (int i = 0; i < 2; i++) {
+ execute(ozoneAdminShell, args);
+ }
+
+ String[] args1 = new String[] {"container", "list", "-c", "10", "--scm",
+ "localhost:" +
cluster.getStorageContainerManager().getClientRpcPort()};
+ execute(ozoneAdminShell, args1);
+ //results will be capped at the maximum allowed count
+ assertEquals(1, getNumOfContainers());
+
+ String[] args2 = new String[] {"container", "list", "-a", "--scm",
+ "localhost:" +
cluster.getStorageContainerManager().getClientRpcPort()};
+ execute(ozoneAdminShell, args2);
+ //Lists all containers
+ assertNotEquals(1, getNumOfContainers());
+ }
+
+ private int getNumOfContainers()
+ throws UnsupportedEncodingException {
+ return out.toString(DEFAULT_ENCODING).split("\"containerID\" :").length -
1;
+ }
+
/**
* Helper function to retrieve Ozone client configuration for trash testing.
* @param hostPrefix Scheme + Authority. e.g. ofs://om-service-test1
diff --git
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/ClosedContainerReplicator.java
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/ClosedContainerReplicator.java
index d471c13462..393c7e599c 100644
---
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/ClosedContainerReplicator.java
+++
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/ClosedContainerReplicator.java
@@ -101,7 +101,7 @@ public class ClosedContainerReplicator extends
BaseFreonGenerator implements
new ContainerOperationClient(conf);
final List<ContainerInfo> containerInfos =
- containerOperationClient.listContainer(0L, 1_000_000);
+ containerOperationClient.listContainer(0L,
1_000_000).getContainerInfoList();
//logic same as the download+import on the destination datanode
initializeReplicationSupervisor(conf, containerInfos.size() * 2);
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]