This is an automated email from the ASF dual-hosted git repository.
captainzmc 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 49955dbeb2 HDDS-8822. [S3G] Improve list performance in LEGACY/OBS
bucket (#5003)
49955dbeb2 is described below
commit 49955dbeb2084e41ab03ca32c967df2bd4877183
Author: Hongbing Wang <[email protected]>
AuthorDate: Wed Aug 16 10:40:03 2023 +0800
HDDS-8822. [S3G] Improve list performance in LEGACY/OBS bucket (#5003)
* HDDS-8822. [S3G] Improve list performance in LEGACE/OBS bucket
---
.../common/src/main/resources/ozone-default.xml | 10 +
.../apache/hadoop/ozone/client/OzoneBucket.java | 185 ++++++++++-
.../org/apache/hadoop/ozone/om/TestListKeys.java | 341 +++++++++++++++++++++
.../hadoop/ozone/s3/S3GatewayConfigKeys.java | 9 +
.../hadoop/ozone/s3/endpoint/BucketEndpoint.java | 37 ++-
.../hadoop/ozone/client/OzoneBucketStub.java | 41 +++
.../hadoop/ozone/s3/endpoint/TestBucketList.java | 2 +
.../ozone/s3/endpoint/TestPermissionCheck.java | 4 +-
8 files changed, 605 insertions(+), 24 deletions(-)
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml
b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 5b3153632b..665ed24686 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -1787,6 +1787,16 @@
service principal. </description>
</property>
+ <property>
+ <name>ozone.s3g.list-keys.shallow.enabled</name>
+ <value>true</value>
+ <tag>OZONE, S3GATEWAY</tag>
+ <description>If this is true, there will be efficiency optimization effects
+ when calling s3g list interface with delimiter '/' parameter, especially
+ when there are a large number of keys.
+ </description>
+ </property>
+
<property>
<name>ozone.om.save.metrics.interval</name>
<value>5m</value>
diff --git
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
index cf847702ff..bc8dcdd0e5 100644
---
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
+++
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
@@ -58,6 +58,7 @@ import java.util.List;
import java.util.Map;
import java.util.Stack;
import java.util.NoSuchElementException;
+import java.util.stream.Collectors;
import static org.apache.hadoop.ozone.OzoneConsts.QUOTA_RESET;
import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
@@ -556,9 +557,25 @@ public class OzoneBucket extends WithMetadata {
*/
public Iterator<? extends OzoneKey> listKeys(String keyPrefix, String
prevKey)
throws IOException {
+ return listKeys(keyPrefix, prevKey, false);
+ }
+ /**
+ * Returns Iterator to iterate over all keys after prevKey in the bucket.
+ * If shallow is true, iterator will only contain immediate children.
+ * This applies to the aws s3 list with delimiter '/' scenario.
+ * Note: When shallow is true, whether keyPrefix ends with slash or not
+ * will affect the results, see {@code getNextShallowListOfKeys}.
+ *
+ * @param keyPrefix Bucket prefix to match
+ * @param prevKey Keys will be listed after this key name
+ * @param shallow If true, only list immediate children ozoneKeys
+ * @return {@code Iterator<OzoneKey>}
+ */
+ public Iterator<? extends OzoneKey> listKeys(String keyPrefix, String
prevKey,
+ boolean shallow) throws IOException {
return new KeyIteratorFactory()
- .getKeyIterator(keyPrefix, prevKey, bucketLayout);
+ .getKeyIterator(keyPrefix, prevKey, bucketLayout, shallow);
}
/**
@@ -1049,6 +1066,9 @@ public class OzoneBucket extends WithMetadata {
private String keyPrefix = null;
private Iterator<OzoneKey> currentIterator;
private OzoneKey currentValue;
+ private final boolean shallow;
+ private boolean addedKeyPrefix;
+ private String delimiterKeyPrefix;
String getKeyPrefix() {
return keyPrefix;
@@ -1058,15 +1078,27 @@ public class OzoneBucket extends WithMetadata {
keyPrefix = keyPrefixPath;
}
+ boolean addedKeyPrefix() {
+ return addedKeyPrefix;
+ }
+
+ void setAddedKeyPrefix(boolean addedKeyPrefix) {
+ this.addedKeyPrefix = addedKeyPrefix;
+ }
+
/**
* Creates an Iterator to iterate over all keys after prevKey in the
bucket.
* If prevKey is null it iterates from the first key in the bucket.
* The returned keys match key prefix.
* @param keyPrefix
+ * @param prevKey
+ * @param shallow
*/
- KeyIterator(String keyPrefix, String prevKey) throws IOException {
+ KeyIterator(String keyPrefix, String prevKey, boolean shallow)
+ throws IOException {
setKeyPrefix(keyPrefix);
this.currentValue = null;
+ this.shallow = shallow;
this.currentIterator = getNextListOfKeys(prevKey).iterator();
}
@@ -1099,9 +1131,139 @@ public class OzoneBucket extends WithMetadata {
*/
List<OzoneKey> getNextListOfKeys(String prevKey) throws
IOException {
+ // If shallow is true, only list immediate children
+ if (shallow) {
+ return getNextShallowListOfKeys(prevKey);
+ }
return proxy.listKeys(volumeName, name, keyPrefix, prevKey,
listCacheSize);
}
+
+ /**
+ * Using listStatus instead of listKeys avoiding listing all children keys.
+ * Giving the structure of keys delimited by "/":
+ *
+ * buck-1
+ * |
+ * a
+ * |
+ * -----------------------------------
+ * | | |
+ * b1 b2 b3
+ * ----- -------- ----------
+ * | | | | | | | |
+ * c1 c2 d1 d2 d3 e1 e2 e3
+ * | |
+ * -------- |
+ * | | |
+ * d21.txt d22.txt e11.txt
+ *
+ * For the above structure, the keys listed delimited "/" in order are
+ * as follows:
+ * a/
+ * a/b1/
+ * a/b1/c1/
+ * a/b1/c2/
+ * a/b2/
+ * a/b2/d1/
+ * a/b2/d2/
+ * a/b2/d2/d21.txt
+ * a/b2/d2/d22.txt
+ * a/b2/d3/
+ * a/b3/
+ * a/b3/e1/
+ * a/b3/e1/e11.txt
+ * a/b3/e2/
+ * a/b3/e3/
+ *
+ * When keyPrefix ends without slash (/), the result as Example 1:
+ * Example 1: keyPrefix="a/b2", prevKey=""
+ * result: [a/b2/]
+ * Example 2: keyPrefix="a/b2/", prevKey=""
+ * result: [a/b2/d1/, a/b2/d2/, a/b2/d3/]
+ * Example 3: keyPrefix="a/b2/", prevKey="a/b2/d2/d21.txt"
+ * result: [a/b2/d2/, a/b2/d3/]
+ * Example 4: keyPrefix="a/b2/", prevKey="a/b2/d2/d22.txt"
+ * result: [a/b2/d3/]
+ * Say, keyPrefix="a/b" and prevKey="", the results will be
+ * [a/b1/, a/b2/, a/b3/]
+ * In implementation, the keyPrefix "a/b" can be identified in listKeys,
+ * but cannot be identified in listStatus. Therefore, keyPrefix "a/b"
+ * needs to be split into keyPrefix "a" and call listKeys method to get
+ * the next one key as the startKey in listStatus.
+ */
+ protected List<OzoneKey> getNextShallowListOfKeys(String prevKey)
+ throws IOException {
+ List<OzoneKey> resultList = new ArrayList<>();
+ String startKey = prevKey;
+
+ // handle for first round
+ if (!addedKeyPrefix) {
+ // prepare startKey
+ List<OzoneKey> nextOneKeys =
+ proxy.listKeys(volumeName, name, getKeyPrefix(), prevKey, 1);
+ if (nextOneKeys.isEmpty()) {
+ return nextOneKeys;
+ }
+ // Special case: ListKey expects keyPrefix element should present in
+ // the resultList if startKey is blank or equals to keyPrefix.
+ // The nextOneKey needs be added to the result because it will not be
+ // present when using the 'listStatus' method.
+ // Consider the case, keyPrefix="test/", prevKey="" or 'test1/',
+ // then 'test/' will be added to the list result.
+ startKey = nextOneKeys.get(0).getName();
+ if (getKeyPrefix().endsWith(OZONE_URI_DELIMITER) &&
+ startKey.equals(getKeyPrefix())) {
+ resultList.add(nextOneKeys.get(0));
+ }
+
+ // prepare delimiterKeyPrefix
+ delimiterKeyPrefix = getKeyPrefix();
+ if (!getKeyPrefix().endsWith(OZONE_URI_DELIMITER)) {
+ delimiterKeyPrefix = OzoneFSUtils.getParentDir(getKeyPrefix());
+ }
+ }
+
+ // Elements in statuses must be sorted after startKey,
+ // which means they come after the keyPrefix.
+ List<OzoneFileStatus> statuses = proxy.listStatus(volumeName, name,
+ delimiterKeyPrefix, false, startKey, listCacheSize);
+
+ if (addedKeyPrefix) {
+ // previous round already include the startKey, so remove it
+ statuses.remove(0);
+ } else {
+ setAddedKeyPrefix(true);
+ }
+
+ List<OzoneKey> ozoneKeys = buildOzoneKeysFromFileStatus(statuses)
+ .stream()
+ .filter(key -> StringUtils.startsWith(key.getName(), getKeyPrefix()))
+ .collect(Collectors.toList());
+
+ resultList.addAll(ozoneKeys);
+ return resultList;
+ }
+
+ private List<OzoneKey> buildOzoneKeysFromFileStatus(
+ List<OzoneFileStatus> statuses) {
+ return statuses.stream()
+ .map(status -> {
+ OmKeyInfo keyInfo = status.getKeyInfo();
+ String keyName = keyInfo.getKeyName();
+ if (status.isDirectory()) {
+ // add trailing slash to represent directory
+ keyName = OzoneFSUtils.addTrailingSlashIfNeeded(keyName);
+ }
+ return new OzoneKey(keyInfo.getVolumeName(),
+ keyInfo.getBucketName(), keyName,
+ keyInfo.getDataSize(), keyInfo.getCreationTime(),
+ keyInfo.getModificationTime(),
+ keyInfo.getReplicationConfig(), keyInfo.isFile());
+ })
+ .collect(Collectors.toList());
+ }
+
}
@@ -1136,7 +1298,6 @@ public class OzoneBucket extends WithMetadata {
private class KeyIteratorWithFSO extends KeyIterator {
private Stack<Pair<String, String>> stack;
- private boolean addedKeyPrefix;
private String removeStartKey = "";
/**
@@ -1146,9 +1307,11 @@ public class OzoneBucket extends WithMetadata {
*
* @param keyPrefix
* @param prevKey
+ * @param shallow
*/
- KeyIteratorWithFSO(String keyPrefix, String prevKey) throws IOException {
- super(keyPrefix, prevKey);
+ KeyIteratorWithFSO(String keyPrefix, String prevKey, boolean shallow)
+ throws IOException {
+ super(keyPrefix, prevKey, shallow);
}
/**
@@ -1189,7 +1352,7 @@ public class OzoneBucket extends WithMetadata {
}
// normalize paths
- if (!addedKeyPrefix) {
+ if (!addedKeyPrefix()) {
prevKey = OmUtils.normalizeKey(prevKey, true);
String keyPrefixName = "";
if (StringUtils.isNotBlank(getKeyPrefix())) {
@@ -1421,12 +1584,12 @@ public class OzoneBucket extends WithMetadata {
private void addKeyPrefixInfoToResultList(String keyPrefix,
String startKey, List<OzoneKey> keysResultList) throws IOException {
- if (addedKeyPrefix) {
+ if (addedKeyPrefix()) {
return;
}
// setting flag to true.
- addedKeyPrefix = true;
+ setAddedKeyPrefix(true);
// not required to addKeyPrefix
// case-1) if keyPrefix is null/empty/just contains snapshot indicator
@@ -1481,11 +1644,11 @@ public class OzoneBucket extends WithMetadata {
private class KeyIteratorFactory {
KeyIterator getKeyIterator(String keyPrefix, String prevKey,
- BucketLayout bType) throws IOException {
+ BucketLayout bType, boolean shallow) throws IOException {
if (bType.isFileSystemOptimized()) {
- return new KeyIteratorWithFSO(keyPrefix, prevKey);
+ return new KeyIteratorWithFSO(keyPrefix, prevKey, shallow);
} else {
- return new KeyIterator(keyPrefix, prevKey);
+ return new KeyIterator(keyPrefix, prevKey, shallow);
}
}
}
diff --git
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestListKeys.java
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestListKeys.java
new file mode 100644
index 0000000000..344623b760
--- /dev/null
+++
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestListKeys.java
@@ -0,0 +1,341 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
under
+ * the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.junit.rules.Timeout;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.stream.Stream;
+
+import static com.google.common.collect.Lists.newLinkedList;
+import static
org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_LIST_CACHE_SIZE;
+import static
org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
+import static org.junit.jupiter.params.provider.Arguments.of;
+
+/**
+ * Test covers listKeys(keyPrefix, startKey, shallow) combinations
+ * in a legacy/OBS bucket layout type.
+ */
+public class TestListKeys {
+
+ private static MiniOzoneCluster cluster = null;
+
+ private static OzoneConfiguration conf;
+ private static String clusterId;
+ private static String scmId;
+ private static String omId;
+
+ private static OzoneBucket legacyOzoneBucket;
+ private static OzoneClient client;
+
+ @Rule
+ public Timeout timeout = new Timeout(1200000);
+
+ /**
+ * Create a MiniDFSCluster for testing.
+ * <p>
+ *
+ * @throws IOException
+ */
+ @BeforeAll
+ public static void init() throws Exception {
+ conf = new OzoneConfiguration();
+ conf.setBoolean(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS, true);
+ clusterId = UUID.randomUUID().toString();
+ scmId = UUID.randomUUID().toString();
+ omId = UUID.randomUUID().toString();
+ // Set the number of keys to be processed during batch operate.
+ conf.setInt(OZONE_FS_ITERATE_BATCH_SIZE, 3);
+ conf.setInt(OZONE_CLIENT_LIST_CACHE_SIZE, 3);
+ cluster = MiniOzoneCluster.newBuilder(conf).setClusterId(clusterId)
+ .setScmId(scmId).setOmId(omId).build();
+ cluster.waitForClusterToBeReady();
+ client = cluster.newClient();
+
+ // create a volume and a LEGACY bucket
+ legacyOzoneBucket = TestDataUtil
+ .createVolumeAndBucket(client, BucketLayout.LEGACY);
+
+ initFSNameSpace();
+ }
+
+ @AfterAll
+ public static void teardownClass() {
+ IOUtils.closeQuietly(client);
+ if (cluster != null) {
+ cluster.shutdown();
+ }
+ }
+
+ private static void initFSNameSpace() throws Exception {
+ buildNameSpaceTree(legacyOzoneBucket);
+ }
+
+ /**
+ * Verify listKeys at different levels.
+ *
+ * buck-1
+ * |
+ * a1
+ * |
+ * -----------------------------------
+ * | | |
+ * b1 b2 b3
+ * ------- --------- -----------
+ * | | | | | | | |
+ * c1 c2 d1 d2 d3 e1 e2 e3
+ * | | | | | | | |
+ * c1.tx c2.tx d11.tx | d31.tx | | e31.tx
+ * --------- | e21.tx
+ * | | |
+ * d21.tx d22.tx e11.tx
+ *
+ * Above is the key namespace tree structure.
+ */
+ private static void buildNameSpaceTree(OzoneBucket ozoneBucket)
+ throws Exception {
+ LinkedList<String> keys = new LinkedList<>();
+ keys.add("/a1/b1/c1111.tx");
+ keys.add("/a1/b1/c1222.tx");
+ keys.add("/a1/b1/c1333.tx");
+ keys.add("/a1/b1/c1444.tx");
+ keys.add("/a1/b1/c1555.tx");
+ keys.add("/a1/b1/c1/c1.tx");
+ keys.add("/a1/b1/c12/c2.tx");
+ keys.add("/a1/b1/c12/c3.tx");
+
+ keys.add("/a1/b2/d1/d11.tx");
+ keys.add("/a1/b2/d2/d21.tx");
+ keys.add("/a1/b2/d2/d22.tx");
+ keys.add("/a1/b2/d3/d31.tx");
+
+ keys.add("/a1/b3/e1/e11.tx");
+ keys.add("/a1/b3/e2/e21.tx");
+ keys.add("/a1/b3/e3/e31.tx");
+
+ createKeys(ozoneBucket, keys);
+ }
+
+ private static Stream<Arguments> shallowListDataWithTrailingSlash() {
+ return Stream.of(
+
+ // Case-1: StartKey is less than prefixKey, return emptyList.
+ of("a1/b2/", "a1", newLinkedList(Collections.emptyList())),
+
+ // Case-2: StartKey is empty, return all immediate node.
+ of("a1/b2/", "", newLinkedList(Arrays.asList(
+ "a1/b2/",
+ "a1/b2/d1/",
+ "a1/b2/d2/",
+ "a1/b2/d3/"
+ ))),
+
+ // Case-3: StartKey is same as prefixKey, return all immediate nodes.
+ of("a1/b2/", "a1/b2", newLinkedList(Arrays.asList(
+ "a1/b2/",
+ "a1/b2/d1/",
+ "a1/b2/d2/",
+ "a1/b2/d3/"
+ ))),
+
+ // Case-4: StartKey is greater than prefixKey
+ of("a1/b2/", "a1/b2/d2/d21.tx", newLinkedList(Arrays.asList(
+ "a1/b2/d2/",
+ "a1/b2/d3/"
+ ))),
+
+ // Case-5: StartKey reaches last element, return emptyList
+ of("a1/b2/", "a1/b2/d3/d31.tx", newLinkedList(
+ Collections.emptyList()
+ )),
+
+ // Case-6: Mix result
+ of("a1/b1/", "a1/b1/c12", newLinkedList(Arrays.asList(
+ "a1/b1/c12/",
+ "a1/b1/c1222.tx",
+ "a1/b1/c1333.tx",
+ "a1/b1/c1444.tx",
+ "a1/b1/c1555.tx"
+ )))
+ );
+ }
+
+ private static Stream<Arguments> shallowListDataWithoutTrailingSlash() {
+ return Stream.of(
+
+ // Case-1: StartKey is less than prefixKey, return emptyList.
+ of("a1/b2", "a1", newLinkedList(Collections.emptyList())),
+
+ // Case-2: StartKey is empty, return all immediate node.
+ of("a1/b2", "", newLinkedList(Arrays.asList(
+ "a1/b2/"
+ ))),
+
+ // Case-3: StartKey is same as prefixKey.
+ of("a1/b2", "a1/b2", newLinkedList(Arrays.asList(
+ "a1/b2/"
+ ))),
+
+ // Case-4: StartKey is greater than prefixKey, return immediate
+ // nodes which after startKey.
+ of("a1/b2", "a1/b2/d2/d21.tx", newLinkedList(Arrays.asList(
+ "a1/b2/"
+ ))),
+
+ // Case-5: StartKey reaches last element, return emptyList
+ of("a1/b2", "a1/b2/d3/d31.tx", newLinkedList(
+ Collections.emptyList()
+ )),
+
+ // Case-6: StartKey is invalid (less than last element)
+ of("a1/b1/c1", "a1/b1/c1/c0invalid", newLinkedList(Arrays.asList(
+ "a1/b1/c1/",
+ "a1/b1/c1111.tx",
+ "a1/b1/c12/",
+ "a1/b1/c1222.tx",
+ "a1/b1/c1333.tx",
+ "a1/b1/c1444.tx",
+ "a1/b1/c1555.tx"
+ ))),
+
+ // Case-7: StartKey reaches last element
+ of("a1/b1/c1", "a1/b1/c1/c2.tx", newLinkedList(Arrays.asList(
+ "a1/b1/c1111.tx",
+ "a1/b1/c12/",
+ "a1/b1/c1222.tx",
+ "a1/b1/c1333.tx",
+ "a1/b1/c1444.tx",
+ "a1/b1/c1555.tx"
+ ))),
+
+ // Case-8: StartKey is invalid (greater than last element)
+ of("a1/b1/c1", "a1/b1/c1/c2invalid", newLinkedList(Arrays.asList(
+ "a1/b1/c1111.tx",
+ "a1/b1/c12/",
+ "a1/b1/c1222.tx",
+ "a1/b1/c1333.tx",
+ "a1/b1/c1444.tx",
+ "a1/b1/c1555.tx"
+ ))),
+
+ // Case-9:
+ of("a1/b1/c12", "", newLinkedList(Arrays.asList(
+ "a1/b1/c12/",
+ "a1/b1/c1222.tx"
+ )))
+
+ );
+ }
+
+ @ParameterizedTest
+ @MethodSource("shallowListDataWithTrailingSlash")
+ public void testShallowListKeysWithPrefixTrailingSlash(String keyPrefix,
+ String startKey, List<String> expectedKeys) throws Exception {
+ checkKeyShallowList(keyPrefix, startKey, expectedKeys, legacyOzoneBucket);
+ }
+
+ @ParameterizedTest
+ @MethodSource("shallowListDataWithoutTrailingSlash")
+ public void testShallowListKeysWithoutPrefixTrailingSlash(String keyPrefix,
+ String startKey, List<String> expectedKeys) throws Exception {
+ checkKeyShallowList(keyPrefix, startKey, expectedKeys, legacyOzoneBucket);
+ }
+
+ private void checkKeyShallowList(String keyPrefix, String startKey,
+ List<String> keys, OzoneBucket bucket)
+ throws Exception {
+
+ Iterator<? extends OzoneKey> ozoneKeyIterator =
+ bucket.listKeys(keyPrefix, startKey, true);
+ ReplicationConfig expectedReplication =
+ Optional.ofNullable(bucket.getReplicationConfig())
+ .orElse(cluster.getOzoneManager().getDefaultReplicationConfig());
+
+ List <String> keyLists = new ArrayList<>();
+ while (ozoneKeyIterator.hasNext()) {
+ OzoneKey ozoneKey = ozoneKeyIterator.next();
+ Assert.assertEquals(expectedReplication,
ozoneKey.getReplicationConfig());
+ keyLists.add(ozoneKey.getName());
+ }
+ LinkedList outputKeysList = new LinkedList(keyLists);
+ System.out.println("BEGIN:::keyPrefix---> " + keyPrefix + ":::---> " +
+ startKey);
+ for (String key : keys) {
+ System.out.println(" " + key);
+ }
+ System.out.println("END:::keyPrefix---> " + keyPrefix + ":::---> " +
+ startKey);
+ Assert.assertEquals(keys, outputKeysList);
+ }
+
+ private static void createKeys(OzoneBucket ozoneBucket, List<String> keys)
+ throws Exception {
+ int length = 10;
+ byte[] input = new byte[length];
+ Arrays.fill(input, (byte) 96);
+ for (String key : keys) {
+ createKey(ozoneBucket, key, 10, input);
+ }
+ }
+
+ private static void createKey(OzoneBucket ozoneBucket, String key, int
length,
+ byte[] input) throws Exception {
+
+ OzoneOutputStream ozoneOutputStream =
+ ozoneBucket.createKey(key, length);
+
+ ozoneOutputStream.write(input);
+ ozoneOutputStream.write(input, 0, 10);
+ ozoneOutputStream.close();
+
+ // Read the key with given key name.
+ OzoneInputStream ozoneInputStream = ozoneBucket.readKey(key);
+ byte[] read = new byte[length];
+ ozoneInputStream.read(read, 0, length);
+ ozoneInputStream.close();
+
+ Assert.assertEquals(new String(input, StandardCharsets.UTF_8),
+ new String(read, StandardCharsets.UTF_8));
+ }
+}
diff --git
a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/S3GatewayConfigKeys.java
b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/S3GatewayConfigKeys.java
index 3f80087ebb..179c5eeee7 100644
---
a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/S3GatewayConfigKeys.java
+++
b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/S3GatewayConfigKeys.java
@@ -72,6 +72,15 @@ public final class S3GatewayConfigKeys {
public static final boolean OZONE_S3G_FSO_DIRECTORY_CREATION_ENABLED_DEFAULT
=
true;
+ /**
+ * Configuration key that enables shallow listing of Keys when results
+ * with delimiter by '/'.
+ */
+ public static final String OZONE_S3G_LIST_KEYS_SHALLOW_ENABLED =
+ "ozone.s3g.list-keys.shallow.enabled";
+ public static final boolean OZONE_S3G_LIST_KEYS_SHALLOW_ENABLED_DEFAULT =
+ true;
+
/**
* Never constructed.
*/
diff --git
a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
index 9842ccaee0..4c0f055ecb 100644
---
a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
+++
b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.s3.endpoint;
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.audit.S3GAction;
import org.apache.hadoop.ozone.client.OzoneBucket;
@@ -45,6 +46,7 @@ import org.apache.http.HttpStatus;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import javax.inject.Inject;
import javax.ws.rs.DELETE;
import javax.ws.rs.DefaultValue;
import javax.ws.rs.GET;
@@ -69,6 +71,9 @@ import java.util.List;
import java.util.Set;
import static org.apache.hadoop.ozone.OzoneAcl.AclScope.ACCESS;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+import static
org.apache.hadoop.ozone.s3.S3GatewayConfigKeys.OZONE_S3G_LIST_KEYS_SHALLOW_ENABLED;
+import static
org.apache.hadoop.ozone.s3.S3GatewayConfigKeys.OZONE_S3G_LIST_KEYS_SHALLOW_ENABLED_DEFAULT;
import static
org.apache.hadoop.ozone.s3.exception.S3ErrorTable.NOT_IMPLEMENTED;
import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.newError;
import static org.apache.hadoop.ozone.s3.util.S3Consts.ENCODING_TYPE;
@@ -82,6 +87,11 @@ public class BucketEndpoint extends EndpointBase {
private static final Logger LOG =
LoggerFactory.getLogger(BucketEndpoint.class);
+ private boolean listKeysShallowEnabled;
+
+ @Inject
+ private OzoneConfiguration ozoneConfiguration;
+
/**
* Rest endpoint to list objects in a specific bucket.
* <p>
@@ -133,18 +143,19 @@ public class BucketEndpoint extends EndpointBase {
startAfter = marker;
}
+ // If continuation token and start after both are provided, then we
+ // ignore start After
+ String prevKey = continueToken != null ? decodedToken.getLastKey()
+ : startAfter;
+
+ // If shallow is true, only list immediate children
+ // delimited by OZONE_URI_DELIMITER
+ boolean shallow = listKeysShallowEnabled
+ && OZONE_URI_DELIMITER.equals(delimiter);
+
OzoneBucket bucket = getBucket(bucketName);
- if (startAfter != null && continueToken != null) {
- // If continuation token and start after both are provided, then we
- // ignore start After
- ozoneKeyIterator = bucket.listKeys(prefix, decodedToken.getLastKey());
- } else if (startAfter != null && continueToken == null) {
- ozoneKeyIterator = bucket.listKeys(prefix, startAfter);
- } else if (startAfter == null && continueToken != null) {
- ozoneKeyIterator = bucket.listKeys(prefix, decodedToken.getLastKey());
- } else {
- ozoneKeyIterator = bucket.listKeys(prefix);
- }
+ ozoneKeyIterator = bucket.listKeys(prefix, prevKey, shallow);
+
} catch (OMException ex) {
AUDIT.logReadFailure(
buildAuditMessageForFailure(s3GAction, getAuditParameters(), ex));
@@ -701,6 +712,8 @@ public class BucketEndpoint extends EndpointBase {
@Override
public void init() {
-
+ listKeysShallowEnabled = ozoneConfiguration.getBoolean(
+ OZONE_S3G_LIST_KEYS_SHALLOW_ENABLED,
+ OZONE_S3G_LIST_KEYS_SHALLOW_ENABLED_DEFAULT);
}
}
diff --git
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
index d613297fc0..e7ed9face4 100644
---
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
+++
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
@@ -52,6 +52,8 @@ import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
import org.apache.hadoop.util.Time;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+
/**
* In-memory ozone bucket for testing.
*/
@@ -339,6 +341,45 @@ public class OzoneBucketStub extends OzoneBucket {
.iterator();
}
+ public Iterator<? extends OzoneKey> listKeys(String keyPrefix,
+ String prevKey, boolean shallow) throws IOException {
+ if (!shallow) {
+ return prevKey == null ? listKeys(keyPrefix)
+ : listKeys(keyPrefix, prevKey);
+ }
+
+ Map<String, OzoneKey> sortedKey = new TreeMap<>(keyDetails);
+ List<OzoneKey> ozoneKeys = sortedKey.values()
+ .stream()
+ .filter(key -> key.getName().startsWith(keyPrefix))
+ .map(key -> {
+ String[] res = key.getName().split(OZONE_URI_DELIMITER);
+ String newKeyName;
+ if (res.length < 2) {
+ newKeyName = key.getName();
+ } else if (res.length == 2) {
+ newKeyName = res[0] + OZONE_URI_DELIMITER + res[1];
+ } else {
+ newKeyName =
+ res[0] + OZONE_URI_DELIMITER + res[1] + OZONE_URI_DELIMITER;
+ }
+ return new OzoneKey(key.getVolumeName(),
+ key.getBucketName(), newKeyName,
+ key.getDataSize(),
+ key.getCreationTime().getEpochSecond() * 1000,
+ key.getModificationTime().getEpochSecond() * 1000,
+ key.getReplicationConfig(), key.isFile());
+ }).collect(Collectors.toList());
+
+ if (prevKey != null) {
+ return ozoneKeys.stream()
+ .filter(key -> key.getName().compareTo(prevKey) > 0)
+ .collect(Collectors.toList())
+ .iterator();
+ }
+ return ozoneKeys.iterator();
+ }
+
@Override
public void deleteKey(String key) throws IOException {
keyDetails.remove(key);
diff --git
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketList.java
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketList.java
index f77c970f5f..e011c1d597 100644
---
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketList.java
+++
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketList.java
@@ -36,6 +36,8 @@ import org.junit.Test;
/**
* Testing basic object list browsing.
+ * Note: delimiter with '/' will call shallow list logic,
+ * just list immediate subdir of prefix.
*/
public class TestBucketList {
diff --git
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
index edf1488f16..a04af4be6b 100644
---
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
+++
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
@@ -45,6 +45,7 @@ import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.ArgumentMatchers.isNull;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.when;
@@ -168,7 +169,8 @@ public class TestPermissionCheck {
public void testListKey() throws IOException {
Mockito.when(objectStore.getVolume(anyString())).thenReturn(volume);
Mockito.when(objectStore.getS3Bucket(anyString())).thenReturn(bucket);
- doThrow(exception).when(bucket).listKeys(anyString());
+ doThrow(exception).when(bucket).listKeys(anyString(), isNull(),
+ anyBoolean());
BucketEndpoint bucketEndpoint = new BucketEndpoint();
bucketEndpoint.setClient(client);
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]