Skip to content

Commit 4f5f68a

Browse files
authored
HDDS-11696. Limit max number of entries in list keys/status response (apache#7431)
(cherry picked from commit 2cef393)
1 parent 3862ff8 commit 4f5f68a

15 files changed

Lines changed: 271 additions & 31 deletions

File tree

hadoop-hdds/common/src/main/resources/ozone-default.xml

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4556,4 +4556,12 @@
45564556
</description>
45574557
</property>
45584558

4559+
<property>
4560+
<name>ozone.om.server.list.max.size</name>
4561+
<value>1000</value>
4562+
<tag>OZONE, OM</tag>
4563+
<description>
4564+
Configuration property to configure the max server side response size for list calls on om.
4565+
</description>
4566+
</property>
45594567
</configuration>

hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1808,7 +1808,6 @@ private boolean getChildrenKeys(String keyPrefix, String startKey,
18081808
// 1. Get immediate children of keyPrefix, starting with startKey
18091809
List<OzoneFileStatusLight> statuses = proxy.listStatusLight(volumeName,
18101810
name, keyPrefix, false, startKey, listCacheSize, true);
1811-
boolean reachedLimitCacheSize = statuses.size() == listCacheSize;
18121811

18131812
// 2. Special case: ListKey expects keyPrefix element should present in
18141813
// the resultList, only if startKey is blank. If startKey is not blank
@@ -1840,7 +1839,7 @@ private boolean getChildrenKeys(String keyPrefix, String startKey,
18401839
// Return it so that the next iteration will be
18411840
// started using the stacked items.
18421841
return true;
1843-
} else if (reachedLimitCacheSize && indx == statuses.size() - 1) {
1842+
} else if (indx == statuses.size() - 1) {
18441843
// The last element is a FILE and reaches the listCacheSize.
18451844
// Now, sets next seek key to this element
18461845
stack.push(new ImmutablePair<>(keyPrefix, keyInfo.getKeyName()));

hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OFSPath.java

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -301,6 +301,19 @@ public boolean isSnapshotPath() {
301301
return false;
302302
}
303303

304+
/**
305+
* If the path is a snapshot path get the snapshot name from the key name.
306+
*/
307+
public String getSnapshotName() {
308+
if (keyName.startsWith(OM_SNAPSHOT_INDICATOR)) {
309+
if (!bucketName.isEmpty() && !volumeName.isEmpty()) {
310+
String[] keyNames = keyName.split(OZONE_URI_DELIMITER);
311+
return keyNames.length > 1 ? keyNames[1] : null;
312+
}
313+
}
314+
return null;
315+
}
316+
304317
/**
305318
* If key name is not empty, the given path is a key.
306319
* e.g. /volume1/bucket2/key3 is a key.

hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -625,4 +625,9 @@ private OMConfigKeys() {
625625
public static final String OZONE_OM_MAX_BUCKET =
626626
"ozone.om.max.buckets";
627627
public static final int OZONE_OM_MAX_BUCKET_DEFAULT = 100000;
628+
/**
629+
* Configuration property to configure the max server side response size for list calls.
630+
*/
631+
public static final String OZONE_OM_SERVER_LIST_MAX_SIZE = "ozone.om.server.list.max.size";
632+
public static final int OZONE_OM_SERVER_LIST_MAX_SIZE_DEFAULT = 1000;
628633
}

hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/AbstractOzoneFileSystemTest.java

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -117,6 +117,7 @@
117117
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
118118
import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
119119
import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
120+
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVER_LIST_MAX_SIZE;
120121
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
121122
import static org.apache.hadoop.ozone.om.helpers.BucketLayout.FILE_SYSTEM_OPTIMIZED;
122123
import static org.assertj.core.api.Assertions.assertThat;
@@ -184,7 +185,7 @@ void init() throws Exception {
184185
conf.setFloat(OMConfigKeys.OZONE_FS_TRASH_INTERVAL_KEY, TRASH_INTERVAL);
185186
conf.setFloat(FS_TRASH_INTERVAL_KEY, TRASH_INTERVAL);
186187
conf.setFloat(FS_TRASH_CHECKPOINT_INTERVAL_KEY, TRASH_INTERVAL / 2);
187-
188+
conf.setInt(OZONE_OM_SERVER_LIST_MAX_SIZE, 2);
188189
conf.setBoolean(OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY, omRatisEnabled);
189190
conf.setBoolean(OZONE_ACL_ENABLED, true);
190191
conf.setBoolean(OzoneConfigKeys.OZONE_HBASE_ENHANCEMENTS_ALLOWED, true);
@@ -2093,8 +2094,8 @@ void testListStatus2() throws IOException {
20932094
final long initialListStatusCount = omMetrics.getNumListStatus();
20942095
FileStatus[] statusList = fs.listStatus(createPath("/"));
20952096
assertEquals(1, statusList.length);
2096-
assertChange(initialStats, statistics, Statistic.OBJECTS_LIST.getSymbol(), 1);
2097-
assertEquals(initialListStatusCount + 1, omMetrics.getNumListStatus());
2097+
assertChange(initialStats, statistics, Statistic.OBJECTS_LIST.getSymbol(), 2);
2098+
assertEquals(initialListStatusCount + 2, omMetrics.getNumListStatus());
20982099
assertEquals(fs.getFileStatus(path), statusList[0]);
20992100

21002101
dirPath = RandomStringUtils.randomAlphanumeric(5);
@@ -2105,8 +2106,8 @@ void testListStatus2() throws IOException {
21052106

21062107
statusList = fs.listStatus(createPath("/"));
21072108
assertEquals(2, statusList.length);
2108-
assertChange(initialStats, statistics, Statistic.OBJECTS_LIST.getSymbol(), 2);
2109-
assertEquals(initialListStatusCount + 2, omMetrics.getNumListStatus());
2109+
assertChange(initialStats, statistics, Statistic.OBJECTS_LIST.getSymbol(), 4);
2110+
assertEquals(initialListStatusCount + 4, omMetrics.getNumListStatus());
21102111
for (Path p : paths) {
21112112
assertThat(Arrays.asList(statusList)).contains(fs.getFileStatus(p));
21122113
}

hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsSnapshot.java

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,8 @@
2424
import java.nio.file.Files;
2525
import java.nio.file.Path;
2626
import java.nio.file.Paths;
27+
import java.util.ArrayList;
28+
import java.util.List;
2729
import java.util.concurrent.TimeUnit;
2830
import java.util.stream.Stream;
2931
import java.util.concurrent.atomic.AtomicInteger;
@@ -49,10 +51,13 @@
4951

5052
import static java.nio.charset.StandardCharsets.UTF_8;
5153
import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY;
54+
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_LISTING_PAGE_SIZE;
55+
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_LISTING_PAGE_SIZE_DEFAULT;
5256
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL;
5357
import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
5458
import static org.apache.hadoop.ozone.OzoneConsts.OZONE_OFS_URI_SCHEME;
5559
import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_INDICATOR;
60+
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVER_LIST_MAX_SIZE;
5661
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL;
5762
import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPath;
5863
import static org.assertj.core.api.Assertions.assertThat;
@@ -91,6 +96,8 @@ static void initClass() throws Exception {
9196
conf.setBoolean(OMConfigKeys.OZONE_FILESYSTEM_SNAPSHOT_ENABLED_KEY, true);
9297
conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL, 1, TimeUnit.SECONDS);
9398
conf.setInt(OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL, KeyManagerImpl.DISABLE_VALUE);
99+
conf.setInt(OZONE_OM_SERVER_LIST_MAX_SIZE, 20);
100+
conf.setInt(OZONE_FS_LISTING_PAGE_SIZE, 30);
94101

95102
// Start the cluster
96103
cluster = MiniOzoneCluster.newHABuilder(conf)
@@ -289,6 +296,13 @@ void testFsLsSnapshot(@TempDir Path tempDir) throws Exception {
289296
String snapshotPath2 = BUCKET_WITH_SNAPSHOT_INDICATOR_PATH +
290297
OM_KEY_PREFIX + snapshotName2;
291298
String snapshotKeyPath2 = snapshotPath2 + OM_KEY_PREFIX + key2;
299+
List<String> snapshotNames = new ArrayList<>();
300+
for (int i = 0; i < cluster.getConf().getInt(OZONE_FS_LISTING_PAGE_SIZE,
301+
OZONE_FS_LISTING_PAGE_SIZE_DEFAULT) * 2; i++) {
302+
snapshotNames.add(createSnapshot());
303+
}
304+
String snapshotName3 = createSnapshot();
305+
292306

293307
int res = ToolRunner.run(shell,
294308
new String[]{"-deleteSnapshot", BUCKET_PATH, snapshotName1});
@@ -313,6 +327,10 @@ void testFsLsSnapshot(@TempDir Path tempDir) throws Exception {
313327

314328
assertThat(listSnapOut).doesNotContain(snapshotName1);
315329
assertThat(listSnapOut).contains(snapshotName2);
330+
assertThat(listSnapOut).contains(snapshotName3);
331+
for (String snapshotName : snapshotNames) {
332+
assertThat(listSnapOut).contains(snapshotName);
333+
}
316334

317335
// Check for snapshot keys with "ozone fs -ls"
318336
String listSnapKeyOut = execShellCommandAndGetOutput(1,

hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestHadoopDirTreeGenerator.java

Lines changed: 9 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -46,6 +46,9 @@
4646
import java.io.IOException;
4747
import java.net.URI;
4848
import java.util.ArrayList;
49+
import java.util.Arrays;
50+
import java.util.Collections;
51+
import java.util.List;
4952

5053
import static org.assertj.core.api.Assertions.assertThat;
5154
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -150,7 +153,7 @@ private void verifyDirTree(String volumeName, String bucketName, int depth,
150153
FileStatus[] fileStatuses = fileSystem.listStatus(rootDir);
151154
// verify the num of peer directories, expected span count is 1
152155
// as it has only one dir at root.
153-
verifyActualSpan(1, fileStatuses);
156+
verifyActualSpan(1, Arrays.asList(fileStatuses));
154157
for (FileStatus fileStatus : fileStatuses) {
155158
int actualDepth =
156159
traverseToLeaf(fileSystem, fileStatus.getPath(), 1, depth, span,
@@ -164,14 +167,16 @@ private int traverseToLeaf(FileSystem fs, Path dirPath, int depth,
164167
int expectedFileCnt, StorageSize perFileSize)
165168
throws IOException {
166169
FileStatus[] fileStatuses = fs.listStatus(dirPath);
170+
List<FileStatus> fileStatusList = new ArrayList<>();
171+
Collections.addAll(fileStatusList, fileStatuses);
167172
// check the num of peer directories except root and leaf as both
168173
// has less dirs.
169174
if (depth < expectedDepth - 1) {
170-
verifyActualSpan(expectedSpanCnt, fileStatuses);
175+
verifyActualSpan(expectedSpanCnt, fileStatusList);
171176
}
172177
int actualNumFiles = 0;
173178
ArrayList <String> files = new ArrayList<>();
174-
for (FileStatus fileStatus : fileStatuses) {
179+
for (FileStatus fileStatus : fileStatusList) {
175180
if (fileStatus.isDirectory()) {
176181
++depth;
177182
return traverseToLeaf(fs, fileStatus.getPath(), depth, expectedDepth,
@@ -192,7 +197,7 @@ private int traverseToLeaf(FileSystem fs, Path dirPath, int depth,
192197
}
193198

194199
private int verifyActualSpan(int expectedSpanCnt,
195-
FileStatus[] fileStatuses) {
200+
List<FileStatus> fileStatuses) {
196201
int actualSpan = 0;
197202
for (FileStatus fileStatus : fileStatuses) {
198203
if (fileStatus.isDirectory()) {

hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestListKeys.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -48,6 +48,7 @@
4848
import static com.google.common.collect.Lists.newLinkedList;
4949
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_LIST_CACHE_SIZE;
5050
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
51+
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVER_LIST_MAX_SIZE;
5152
import static org.junit.jupiter.params.provider.Arguments.of;
5253
import static org.junit.jupiter.api.Assertions.assertEquals;
5354

@@ -80,6 +81,7 @@ public static void init() throws Exception {
8081
// Set the number of keys to be processed during batch operate.
8182
conf.setInt(OZONE_FS_ITERATE_BATCH_SIZE, 3);
8283
conf.setInt(OZONE_CLIENT_LIST_CACHE_SIZE, 3);
84+
conf.setInt(OZONE_OM_SERVER_LIST_MAX_SIZE, 2);
8385
cluster = MiniOzoneCluster.newBuilder(conf).build();
8486
cluster.waitForClusterToBeReady();
8587
client = cluster.newClient();

hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestListKeysWithFSO.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -47,6 +47,7 @@
4747

4848
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_LIST_CACHE_SIZE;
4949
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
50+
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVER_LIST_MAX_SIZE;
5051
import static org.junit.jupiter.api.Assertions.assertEquals;
5152

5253
/**
@@ -81,6 +82,7 @@ public static void init() throws Exception {
8182
// Set the number of keys to be processed during batch operate.
8283
conf.setInt(OZONE_FS_ITERATE_BATCH_SIZE, 3);
8384
conf.setInt(OZONE_CLIENT_LIST_CACHE_SIZE, 3);
85+
conf.setInt(OZONE_OM_SERVER_LIST_MAX_SIZE, 2);
8486
cluster = MiniOzoneCluster.newBuilder(conf).build();
8587
cluster.waitForClusterToBeReady();
8688
client = cluster.newClient();

hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java

Lines changed: 14 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -146,6 +146,8 @@
146146

147147
import com.google.common.collect.Lists;
148148

149+
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVER_LIST_MAX_SIZE;
150+
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVER_LIST_MAX_SIZE_DEFAULT;
149151
import static org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature.HBASE_SUPPORT;
150152
import static org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature.MULTITENANCY_SCHEMA;
151153
import static org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature.FILESYSTEM_SNAPSHOT;
@@ -181,9 +183,16 @@ public class OzoneManagerRequestHandler implements RequestHandler {
181183
LoggerFactory.getLogger(OzoneManagerRequestHandler.class);
182184
private final OzoneManager impl;
183185
private FaultInjector injector;
186+
private long maxKeyListSize;
187+
184188

185189
public OzoneManagerRequestHandler(OzoneManager om) {
186190
this.impl = om;
191+
this.maxKeyListSize = om.getConfiguration().getLong(OZONE_OM_SERVER_LIST_MAX_SIZE,
192+
OZONE_OM_SERVER_LIST_MAX_SIZE_DEFAULT);
193+
if (this.maxKeyListSize <= 0) {
194+
this.maxKeyListSize = OZONE_OM_SERVER_LIST_MAX_SIZE_DEFAULT;
195+
}
187196
}
188197

189198
//TODO simplify it to make it shorter
@@ -745,7 +754,7 @@ private ListKeysResponse listKeys(ListKeysRequest request, int clientVersion)
745754
request.getBucketName(),
746755
request.getStartKey(),
747756
request.getPrefix(),
748-
request.getCount());
757+
(int)Math.min(this.maxKeyListSize, request.getCount()));
749758
for (OmKeyInfo key : listKeysResult.getKeys()) {
750759
resp.addKeyInfo(key.getProtobuf(true, clientVersion));
751760
}
@@ -763,7 +772,7 @@ private ListKeysLightResponse listKeysLight(ListKeysRequest request)
763772
request.getBucketName(),
764773
request.getStartKey(),
765774
request.getPrefix(),
766-
request.getCount());
775+
(int)Math.min(this.maxKeyListSize, request.getCount()));
767776
for (BasicOmKeyInfo key : listKeysLightResult.getKeys()) {
768777
resp.addBasicKeyInfo(key.getProtobuf());
769778
}
@@ -1234,7 +1243,7 @@ private ListStatusResponse listStatus(
12341243
request.hasAllowPartialPrefix() && request.getAllowPartialPrefix();
12351244
List<OzoneFileStatus> statuses =
12361245
impl.listStatus(omKeyArgs, request.getRecursive(),
1237-
request.getStartKey(), request.getNumEntries(),
1246+
request.getStartKey(), Math.min(this.maxKeyListSize, request.getNumEntries()),
12381247
allowPartialPrefixes);
12391248
ListStatusResponse.Builder
12401249
listStatusResponseBuilder =
@@ -1260,7 +1269,7 @@ private ListStatusLightResponse listStatusLight(
12601269
request.hasAllowPartialPrefix() && request.getAllowPartialPrefix();
12611270
List<OzoneFileStatusLight> statuses =
12621271
impl.listStatusLight(omKeyArgs, request.getRecursive(),
1263-
request.getStartKey(), request.getNumEntries(),
1272+
request.getStartKey(), Math.min(this.maxKeyListSize, request.getNumEntries()),
12641273
allowPartialPrefixes);
12651274
ListStatusLightResponse.Builder
12661275
listStatusLightResponseBuilder =
@@ -1488,7 +1497,7 @@ private OzoneManagerProtocolProtos.ListSnapshotResponse getSnapshots(
14881497
throws IOException {
14891498
ListSnapshotResponse implResponse = impl.listSnapshot(
14901499
request.getVolumeName(), request.getBucketName(), request.getPrefix(),
1491-
request.getPrevSnapshot(), request.getMaxListResult());
1500+
request.getPrevSnapshot(), (int)Math.min(request.getMaxListResult(), maxKeyListSize));
14921501

14931502
List<OzoneManagerProtocolProtos.SnapshotInfo> snapshotInfoList = implResponse.getSnapshotInfos()
14941503
.stream().map(SnapshotInfo::getProtobuf).collect(Collectors.toList());

0 commit comments

Comments
 (0)