You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by ra...@apache.org on 2021/04/08 03:56:26 UTC
[ozone] 20/32: HDDS-4683. [FSO]ListKeys: do lookup in dir and file
tables (#1954)
This is an automated email from the ASF dual-hosted git repository.
rakeshr pushed a commit to branch HDDS-2939
in repository https://gitbox.apache.org/repos/asf/ozone.git
commit 0dfad20e86bb535269f8eee485afc3d5ec4feb20
Author: Rakesh Radhakrishnan <ra...@apache.org>
AuthorDate: Fri Feb 26 07:43:40 2021 +0530
HDDS-4683. [FSO]ListKeys: do lookup in dir and file tables (#1954)
---
.../apache/hadoop/ozone/client/OzoneBucket.java | 299 ++++++++++++++++++++-
.../hadoop/ozone/om/helpers/OzoneFSUtils.java | 10 +-
.../apache/hadoop/ozone/om/TestObjectStoreV1.java | 261 +++++++++++++++++-
.../org/apache/hadoop/ozone/om/KeyManagerImpl.java | 8 +-
.../ozone/om/request/file/OMFileRequest.java | 3 +-
.../S3MultipartUploadCompleteRequest.java | 1 -
6 files changed, 575 insertions(+), 7 deletions(-)
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 c1877b4..bfe6456 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
@@ -21,12 +21,14 @@ package org.apache.hadoop.ozone.client;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
+import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.hdds.client.OzoneQuota;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.StorageType;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
+import org.apache.hadoop.ozone.OmUtils;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
@@ -35,7 +37,9 @@ import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
import org.apache.hadoop.ozone.om.helpers.WithMetadata;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
import org.apache.hadoop.ozone.security.acl.OzoneObjInfo;
@@ -43,13 +47,16 @@ import org.apache.hadoop.util.Time;
import java.io.IOException;
import java.time.Instant;
+import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.Stack;
import java.util.NoSuchElementException;
import static org.apache.hadoop.ozone.OzoneConsts.QUOTA_RESET;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
/**
* A class that encapsulates OzoneBucket.
@@ -544,6 +551,10 @@ public class OzoneBucket extends WithMetadata {
*/
public Iterator<? extends OzoneKey> listKeys(String keyPrefix,
String prevKey) throws IOException {
+
+ if(OzoneFSUtils.isFSOptimizedBucket(getMetadata())){
+ return new KeyIteratorV1(keyPrefix, prevKey);
+ }
return new KeyIterator(keyPrefix, prevKey);
}
@@ -788,6 +799,13 @@ public class OzoneBucket extends WithMetadata {
private Iterator<OzoneKey> currentIterator;
private OzoneKey currentValue;
+ String getKeyPrefix() {
+ return keyPrefix;
+ }
+
+ void setKeyPrefix(String keyPrefixPath) {
+ keyPrefix = keyPrefixPath;
+ }
/**
* Creates an Iterator to iterate over all keys after prevKey in the bucket.
@@ -796,7 +814,7 @@ public class OzoneBucket extends WithMetadata {
* @param keyPrefix
*/
KeyIterator(String keyPrefix, String prevKey) throws IOException{
- this.keyPrefix = keyPrefix;
+ setKeyPrefix(keyPrefix);
this.currentValue = null;
this.currentIterator = getNextListOfKeys(prevKey).iterator();
}
@@ -828,10 +846,287 @@ public class OzoneBucket extends WithMetadata {
* @param prevKey
* @return {@code List<OzoneKey>}
*/
- private List<OzoneKey> getNextListOfKeys(String prevKey) throws
+ List<OzoneKey> getNextListOfKeys(String prevKey) throws
IOException {
return proxy.listKeys(volumeName, name, keyPrefix, prevKey,
listCacheSize);
}
}
+
+
+ /**
+ * An Iterator to iterate over {@link OzoneKey} list.
+ *
+ * buck-1
+ * |
+ * a
+ * |
+ * -----------------------------------
+ * | | |
+ * b1 b2 b3
+ * ----- -------- ----------
+ * | | | | | | | |
+ * c1 c2 d1 d2 d3 e1 e2 e3
+ * | |
+ * -------- |
+ * | | |
+ * d21.txt d22.txt e11.txt
+ *
+ * Say, keyPrefix="a" and prevKey="", then will do Depth-First-Traversal and
+ * visit node to getChildren in below fashion:-
+ * 1. getChildren("a/") 2. getChildren("a/b1") 3. getChildren("a/b1/c1")
+ * 4. getChildren("a/b1/c2") 5. getChildren("a/b2/d1")
+ * 6. getChildren("a/b2/d2") 7. getChildren("a/b2/d3")
+ * 8. getChildren("a/b3/e1") 9. getChildren("a/b3/e2")
+ * 10. getChildren("a/b3/e3")
+ *
+ * Note: Does not guarantee to return the list of keys in a sorted order.
+ */
+ private class KeyIteratorV1 extends KeyIterator{
+
+ private Stack<String> stack;
+ private List<OzoneKey> pendingItemsToBeBatched;
+ private boolean 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
+ */
+ KeyIteratorV1(String keyPrefix, String prevKey) throws IOException {
+ super(keyPrefix, prevKey);
+ }
+
+ @Override
+ List<OzoneKey> getNextListOfKeys(String prevKey) throws IOException {
+ if (stack == null) {
+ stack = new Stack();
+ pendingItemsToBeBatched = new ArrayList<>();
+ }
+
+ // normalize paths
+ if (!addedKeyPrefix) {
+ prevKey = OmUtils.normalizeKey(prevKey, true);
+ String keyPrefixName = "";
+ if (StringUtils.isNotBlank(getKeyPrefix())) {
+ keyPrefixName = OmUtils.normalizeKey(getKeyPrefix(), true);
+ }
+ setKeyPrefix(keyPrefixName);
+ }
+
+ // Get immediate children
+ List<OzoneKey> keysResultList = new ArrayList<>();
+ getChildrenKeys(getKeyPrefix(), prevKey, keysResultList);
+
+ // TODO: Back and Forth seek all the files & dirs, starting from
+ // startKey till keyPrefix.
+
+ return keysResultList;
+ }
+
+ /**
+ * List children under the given keyPrefix and startKey path. It does
+ * recursive #listStatus calls to list all the sub-keys resultList.
+ *
+ * buck-1
+ * |
+ * a
+ * |
+ * -----------------------------------
+ * | | |
+ * b1 b2 b3
+ * ----- -------- ----------
+ * | | | | | | | |
+ * c1 c2 d1 d2 d3 e1 e2 e3
+ * | |
+ * -------- |
+ * | | |
+ * d21.txt d22.txt e11.txt
+ *
+ * Say, KeyPrefix = "a" and startKey = null;
+ *
+ * Iteration-1) RPC call proxy#listStatus("a").
+ * Add b3, b2 and b1 to stack.
+ * Iteration-2) pop b1 and do RPC call proxy#listStatus("b1")
+ * Add c2, c1 to stack.
+ * Iteration-3) pop c1 and do RPC call proxy#listStatus("c1"). Empty list.
+ * Iteration-4) pop c2 and do RPC call proxy#listStatus("c2"). Empty list.
+ * Iteration-5) pop b2 and do RPC call proxy#listStatus("b2")
+ * Add d3, d2 and d1 to stack.
+ * ..........
+ * ..........
+ * Iteration-n) pop e3 and do RPC call proxy#listStatus("e3")
+ * Reached end of the FS tree.
+ *
+ * @param keyPrefix
+ * @param startKey
+ * @param keysResultList
+ * @return true represents it reached limit batch size, false otherwise.
+ * @throws IOException
+ */
+ private boolean getChildrenKeys(String keyPrefix, String startKey,
+ List<OzoneKey> keysResultList) throws IOException {
+
+ // listStatus API expects a not null 'startKey' value
+ startKey = startKey == null ? "" : startKey;
+
+ // 1. Add pending items to the user key resultList
+ if (addAllPendingItemsToResultList(keysResultList)) {
+ // reached limit batch size.
+ return true;
+ }
+
+ // 2. Get immediate children of keyPrefix, starting with startKey
+ List<OzoneFileStatus> statuses = proxy.listStatus(volumeName, name,
+ keyPrefix, false, startKey, listCacheSize);
+
+ // 3. Special case: ListKey expects keyPrefix element should present in
+ // the resultList, only if startKey is blank. If startKey is not blank
+ // then resultList shouldn't contain the startKey element.
+ // Since proxy#listStatus API won't return keyPrefix element in the
+ // resultList. So, this is to add user given keyPrefix to the return list.
+ addKeyPrefixInfoToResultList(keyPrefix, startKey, keysResultList);
+
+ // 4. Special case: ListKey expects startKey shouldn't present in the
+ // resultList. Since proxy#listStatus API returns startKey element to
+ // the returnList, this function is to remove the startKey element.
+ removeStartKeyIfExistsInStatusList(startKey, statuses);
+
+ boolean reachedLimitCacheSize = false;
+ // This dirList is used to store paths elements in left-to-right order.
+ List<String> dirList = new ArrayList<>();
+
+ // 5. Iterating over the resultStatuses list and add each key to the
+ // resultList. If the listCacheSize reaches then it will add the rest
+ // of the statuses to pendingItemsToBeBatched
+ for (int indx = 0; indx < statuses.size(); indx++) {
+ OzoneFileStatus status = statuses.get(indx);
+ OmKeyInfo keyInfo = status.getKeyInfo();
+ String keyName = keyInfo.getKeyName();
+
+ // Add dir to the dirList
+ if (status.isDirectory()) {
+ dirList.add(keyInfo.getKeyName());
+ // add trailing slash to represent directory
+ keyName = OzoneFSUtils.addTrailingSlashIfNeeded(keyName);
+ }
+
+ OzoneKey ozoneKey = new OzoneKey(keyInfo.getVolumeName(),
+ keyInfo.getBucketName(), keyName,
+ keyInfo.getDataSize(), keyInfo.getCreationTime(),
+ keyInfo.getModificationTime(),
+ ReplicationType.valueOf(keyInfo.getType().toString()),
+ keyInfo.getFactor().getNumber());
+
+ // 5.1) Add to the resultList till it reaches limit batch size.
+ // Once it reaches limit, then add rest of the items to
+ // pendingItemsToBeBatched and this will picked in next batch iteration
+ if (!reachedLimitCacheSize && listCacheSize > keysResultList.size()) {
+ keysResultList.add(ozoneKey);
+ reachedLimitCacheSize = listCacheSize <= keysResultList.size();
+ } else {
+ pendingItemsToBeBatched.add(ozoneKey);
+ }
+ }
+
+ // 6. Push elements in reverse order so that the FS tree traversal will
+ // occur in left-to-right fashion.
+ for (int indx = dirList.size() - 1; indx >= 0; indx--) {
+ String dirPathComponent = dirList.get(indx);
+ stack.push(dirPathComponent);
+ }
+
+ if (reachedLimitCacheSize) {
+ return true;
+ }
+
+ // 7. Pop element and seek for its sub-child path(s). Basically moving
+ // seek pointer to next level(depth) in FS tree.
+ while (!stack.isEmpty()) {
+ keyPrefix = stack.pop();
+ if (getChildrenKeys(keyPrefix, "", keysResultList)) {
+ // reached limit batch size.
+ return true;
+ }
+ }
+
+ return false;
+ }
+
+ private void removeStartKeyIfExistsInStatusList(String startKey,
+ List<OzoneFileStatus> statuses) {
+
+ if (StringUtils.isNotBlank(startKey) && !statuses.isEmpty()) {
+ String startKeyPath = startKey;
+ if (startKey.endsWith(OZONE_URI_DELIMITER)) {
+ startKeyPath = OzoneFSUtils.removeTrailingSlashIfNeeded(startKey);
+ }
+ if (StringUtils.equals(statuses.get(0).getKeyInfo().getKeyName(),
+ startKeyPath)) {
+ // remove the duplicateKey from the list.
+ statuses.remove(0);
+ }
+ }
+ }
+
+ private boolean addAllPendingItemsToResultList(List<OzoneKey> keys) {
+
+ Iterator<OzoneKey> ozoneKeyItr = pendingItemsToBeBatched.iterator();
+ while (ozoneKeyItr.hasNext()) {
+ if (listCacheSize <= keys.size()) {
+ // reached limit batch size.
+ return true;
+ }
+ keys.add(ozoneKeyItr.next());
+ ozoneKeyItr.remove();
+ }
+ return false;
+ }
+
+ private void addKeyPrefixInfoToResultList(String keyPrefix,
+ String startKey, List<OzoneKey> keysResultList) throws IOException {
+
+ if (addedKeyPrefix) {
+ return;
+ }
+
+ // setting flag to true.
+ addedKeyPrefix = true;
+
+ // not required to addKeyPrefix
+ // case-1) if keyPrefix is null or empty
+ // case-2) if startKey is not null or empty
+ if (StringUtils.isBlank(keyPrefix) || StringUtils.isNotBlank(startKey)) {
+ return;
+ }
+
+ // TODO: HDDS-4859 will fix the case where startKey not started with
+ // keyPrefix.
+
+ OzoneFileStatus status = proxy.getOzoneFileStatus(volumeName, name,
+ keyPrefix);
+
+ if (status != null) {
+ OmKeyInfo keyInfo = status.getKeyInfo();
+ String keyName = keyInfo.getKeyName();
+ if (status.isDirectory()) {
+ // add trailing slash to represent directory
+ keyName =
+ OzoneFSUtils.addTrailingSlashIfNeeded(keyInfo.getKeyName());
+ }
+
+ OzoneKey ozoneKey = new OzoneKey(keyInfo.getVolumeName(),
+ keyInfo.getBucketName(), keyName,
+ keyInfo.getDataSize(), keyInfo.getCreationTime(),
+ keyInfo.getModificationTime(),
+ ReplicationType.valueOf(keyInfo.getType().toString()),
+ keyInfo.getFactor().getNumber());
+ keysResultList.add(ozoneKey);
+ }
+ }
+
+ }
}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneFSUtils.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneFSUtils.java
index e7e2eb0..c63c21f 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneFSUtils.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneFSUtils.java
@@ -179,7 +179,7 @@ public final class OzoneFSUtils {
return fileName.toString();
}
// failed to find a parent directory.
- return keyName;
+ return "";
}
/**
@@ -230,4 +230,12 @@ public final class OzoneFSUtils {
return layoutVersionEnabled && fsEnabled;
}
+ public static String removeTrailingSlashIfNeeded(String key) {
+ if (key.endsWith(OZONE_URI_DELIMITER)) {
+ java.nio.file.Path keyPath = Paths.get(key);
+ return keyPath.toString();
+ } else {
+ return key;
+ }
+ }
}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreV1.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreV1.java
index b877e29..0e44c9d 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreV1.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreV1.java
@@ -17,15 +17,18 @@
package org.apache.hadoop.ozone.om;
import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.fs.ozone.OzoneFileSystem;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.utils.db.Table;
import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OmUtils;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.TestDataUtil;
import org.apache.hadoop.ozone.client.ObjectStore;
@@ -35,6 +38,7 @@ import org.apache.hadoop.ozone.client.OzoneKey;
import org.apache.hadoop.ozone.client.OzoneKeyDetails;
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.ozone.client.io.KeyOutputStream;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
@@ -51,8 +55,13 @@ import org.junit.Test;
import org.junit.rules.Timeout;
import java.io.IOException;
+import java.net.URI;
+import java.util.Arrays;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
import java.util.UUID;
import java.util.concurrent.TimeoutException;
@@ -60,6 +69,7 @@ import static org.apache.hadoop.hdds.client.ReplicationFactor.ONE;
import static org.apache.hadoop.hdds.client.ReplicationType.STAND_ALONE;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_SCHEME;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_ALREADY_EXISTS;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
import static org.junit.Assert.assertEquals;
@@ -77,7 +87,7 @@ public class TestObjectStoreV1 {
private static FileSystem fs;
@Rule
- public Timeout timeout = new Timeout(240000);
+ public Timeout timeout = new Timeout(1200000);
/**
* Create a MiniDFSCluster for testing.
@@ -275,6 +285,255 @@ public class TestObjectStoreV1 {
dirPathC.getObjectID(), true);
}
+ /**
+ * Verify listKeys at different levels.
+ *
+ * buck-1
+ * |
+ * a
+ * |
+ * -----------------------------------
+ * | | |
+ * 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 FS tree structure.
+ */
+ @Test
+ public void testListKeysAtDifferentLevels() throws Exception {
+ OzoneClient client = cluster.getClient();
+
+ ObjectStore objectStore = client.getObjectStore();
+ OzoneVolume ozoneVolume = objectStore.getVolume(volumeName);
+ Assert.assertTrue(ozoneVolume.getName().equals(volumeName));
+ OzoneBucket ozoneBucket = ozoneVolume.getBucket(bucketName);
+ Assert.assertTrue(ozoneBucket.getName().equals(bucketName));
+
+ String keyc1 = "/a/b1/c1/c1.tx";
+ String keyc2 = "/a/b1/c2/c2.tx";
+
+ String keyd13 = "/a/b2/d1/d11.tx";
+ String keyd21 = "/a/b2/d2/d21.tx";
+ String keyd22 = "/a/b2/d2/d22.tx";
+ String keyd31 = "/a/b2/d3/d31.tx";
+
+ String keye11 = "/a/b3/e1/e11.tx";
+ String keye21 = "/a/b3/e2/e21.tx";
+ String keye31 = "/a/b3/e3/e31.tx";
+
+ LinkedList<String> keys = new LinkedList<>();
+ keys.add(keyc1);
+ keys.add(keyc2);
+
+ keys.add(keyd13);
+ keys.add(keyd21);
+ keys.add(keyd22);
+ keys.add(keyd31);
+
+ keys.add(keye11);
+ keys.add(keye21);
+ keys.add(keye31);
+
+ int length = 10;
+ byte[] input = new byte[length];
+ Arrays.fill(input, (byte)96);
+
+ createKeys(ozoneBucket, keys);
+
+ // Root level listing keys
+ Iterator<? extends OzoneKey> ozoneKeyIterator =
+ ozoneBucket.listKeys(null, null);
+ verifyFullTreeStructure(ozoneKeyIterator);
+
+ ozoneKeyIterator =
+ ozoneBucket.listKeys("a/", null);
+ verifyFullTreeStructure(ozoneKeyIterator);
+
+ LinkedList<String> expectedKeys;
+
+ // Intermediate level keyPrefix - 2nd level
+ ozoneKeyIterator =
+ ozoneBucket.listKeys("a///b2///", null);
+ expectedKeys = new LinkedList<>();
+ expectedKeys.add("a/b2/");
+ expectedKeys.add("a/b2/d1/");
+ expectedKeys.add("a/b2/d2/");
+ expectedKeys.add("a/b2/d3/");
+ expectedKeys.add("a/b2/d1/d11.tx");
+ expectedKeys.add("a/b2/d2/d21.tx");
+ expectedKeys.add("a/b2/d2/d22.tx");
+ expectedKeys.add("a/b2/d3/d31.tx");
+ checkKeyList(ozoneKeyIterator, expectedKeys);
+
+ // Intermediate level keyPrefix - 3rd level
+ ozoneKeyIterator =
+ ozoneBucket.listKeys("a/b2/d1", null);
+ expectedKeys = new LinkedList<>();
+ expectedKeys.add("a/b2/d1/");
+ expectedKeys.add("a/b2/d1/d11.tx");
+ checkKeyList(ozoneKeyIterator, expectedKeys);
+
+ // Boundary of a level
+ ozoneKeyIterator =
+ ozoneBucket.listKeys("a/b2/d2", "a/b2/d2/d21.tx");
+ expectedKeys = new LinkedList<>();
+ expectedKeys.add("a/b2/d2/d22.tx");
+ checkKeyList(ozoneKeyIterator, expectedKeys);
+
+ // Boundary case - last node in the depth-first-traversal
+ ozoneKeyIterator =
+ ozoneBucket.listKeys("a/b3/e3", "a/b3/e3/e31.tx");
+ expectedKeys = new LinkedList<>();
+ checkKeyList(ozoneKeyIterator, expectedKeys);
+ }
+
+ private void verifyFullTreeStructure(Iterator<? extends OzoneKey> keyItr) {
+ LinkedList<String> expectedKeys = new LinkedList<>();
+ expectedKeys.add("a/");
+ expectedKeys.add("a/b1/");
+ expectedKeys.add("a/b2/");
+ expectedKeys.add("a/b3/");
+ expectedKeys.add("a/b1/c1/");
+ expectedKeys.add("a/b1/c2/");
+ expectedKeys.add("a/b1/c1/c1.tx");
+ expectedKeys.add("a/b1/c2/c2.tx");
+ expectedKeys.add("a/b2/d1/");
+ expectedKeys.add("a/b2/d2/");
+ expectedKeys.add("a/b2/d3/");
+ expectedKeys.add("a/b2/d1/d11.tx");
+ expectedKeys.add("a/b2/d2/d21.tx");
+ expectedKeys.add("a/b2/d2/d22.tx");
+ expectedKeys.add("a/b2/d3/d31.tx");
+ expectedKeys.add("a/b3/e1/");
+ expectedKeys.add("a/b3/e2/");
+ expectedKeys.add("a/b3/e3/");
+ expectedKeys.add("a/b3/e1/e11.tx");
+ expectedKeys.add("a/b3/e2/e21.tx");
+ expectedKeys.add("a/b3/e3/e31.tx");
+ checkKeyList(keyItr, expectedKeys);
+ }
+
+ @Test
+ public void testListKeysWithNotNormalizedPath() throws Exception {
+ OzoneClient client = cluster.getClient();
+
+ ObjectStore objectStore = client.getObjectStore();
+ OzoneVolume ozoneVolume = objectStore.getVolume(volumeName);
+ Assert.assertTrue(ozoneVolume.getName().equals(volumeName));
+ OzoneBucket ozoneBucket = ozoneVolume.getBucket(bucketName);
+ Assert.assertTrue(ozoneBucket.getName().equals(bucketName));
+
+ String key1 = "/dir1///dir2/file1/";
+ String key2 = "/dir1///dir2/file2/";
+ String key3 = "/dir1///dir2/file3/";
+
+ LinkedList<String> keys = new LinkedList<>();
+ keys.add("dir1/");
+ keys.add("dir1/dir2/");
+ keys.add(OmUtils.normalizeKey(key1, false));
+ keys.add(OmUtils.normalizeKey(key2, false));
+ keys.add(OmUtils.normalizeKey(key3, false));
+
+ int length = 10;
+ byte[] input = new byte[length];
+ Arrays.fill(input, (byte)96);
+
+ createKey(ozoneBucket, key1, 10, input);
+ createKey(ozoneBucket, key2, 10, input);
+ createKey(ozoneBucket, key3, 10, input);
+
+ // Iterator with key name as prefix.
+
+ Iterator<? extends OzoneKey> ozoneKeyIterator =
+ ozoneBucket.listKeys("/dir1//", null);
+
+ checkKeyList(ozoneKeyIterator, keys);
+
+ // Iterator with with normalized key prefix.
+ ozoneKeyIterator =
+ ozoneBucket.listKeys("dir1/");
+
+ checkKeyList(ozoneKeyIterator, keys);
+
+ // Iterator with key name as previous key.
+ ozoneKeyIterator = ozoneBucket.listKeys(null,
+ "/dir1///dir2/file1/");
+
+ // Remove keys before //dir1/dir2/file1
+ keys.remove("dir1/");
+ keys.remove("dir1/dir2/");
+ keys.remove("dir1/dir2/file1");
+
+ checkKeyList(ozoneKeyIterator, keys);
+
+ // Iterator with normalized key as previous key.
+ ozoneKeyIterator = ozoneBucket.listKeys(null,
+ OmUtils.normalizeKey(key1, false));
+
+ checkKeyList(ozoneKeyIterator, keys);
+ }
+
+ private void checkKeyList(Iterator<? extends OzoneKey > ozoneKeyIterator,
+ List<String> keys) {
+
+ LinkedList<String> outputKeys = new LinkedList<>();
+ while (ozoneKeyIterator.hasNext()) {
+ OzoneKey ozoneKey = ozoneKeyIterator.next();
+ outputKeys.add(ozoneKey.getName());
+ }
+
+ Assert.assertEquals(keys, outputKeys);
+ }
+
+ private 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 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();
+
+ String inputString = new String(input, StandardCharsets.UTF_8);
+ Assert.assertEquals(inputString, new String(read, StandardCharsets.UTF_8));
+
+ // Read using filesystem.
+ String rootPath = String.format("%s://%s.%s/", OZONE_URI_SCHEME,
+ bucketName, volumeName, StandardCharsets.UTF_8);
+ OzoneFileSystem o3fs = (OzoneFileSystem) FileSystem.get(new URI(rootPath),
+ conf);
+ FSDataInputStream fsDataInputStream = o3fs.open(new Path(key));
+ read = new byte[length];
+ fsDataInputStream.read(read, 0, length);
+ ozoneInputStream.close();
+
+ Assert.assertEquals(inputString, new String(read, StandardCharsets.UTF_8));
+ }
+
@Test
public void testRenameKey() throws IOException {
String fromKeyName = UUID.randomUUID().toString();
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
index 604f7d2..f6ae506 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
@@ -2420,7 +2420,8 @@ public class KeyManagerImpl implements KeyManager {
// Check startKey is an immediate child of keyName. For example,
// keyName=/a/ and expected startKey=/a/b. startKey can't be /xyz/b.
- if (!OzoneFSUtils.isImmediateChild(keyName, startKey)) {
+ if (StringUtils.isNotBlank(keyName) &&
+ !OzoneFSUtils.isImmediateChild(keyName, startKey)) {
if (LOG.isDebugEnabled()) {
LOG.debug("StartKey {} is not an immediate child of keyName {}. " +
"Returns empty list", startKey, keyName);
@@ -2428,6 +2429,11 @@ public class KeyManagerImpl implements KeyManager {
return Collections.emptyList();
}
+ // assign startKeyPath if prefixPath is empty string.
+ if (StringUtils.isBlank(prefixPath)) {
+ prefixPath = OzoneFSUtils.getParentDir(startKey);
+ }
+
OzoneFileStatus fileStatusInfo = getOzoneFileStatusV1(volumeName,
bucketName, startKey, false, null, true);
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java
index ebf86ce..e42bc6b 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java
@@ -697,7 +697,8 @@ public final class OMFileRequest {
if (Strings.isNullOrEmpty(prefixName)) {
return fileName;
}
- return prefixName.concat(OzoneConsts.OZONE_URI_DELIMITER).concat(fileName);
+ prefixName = OzoneFSUtils.addTrailingSlashIfNeeded(prefixName);
+ return prefixName.concat(fileName);
}
/**
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCompleteRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCompleteRequest.java
index f008ac2..d396e8e 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCompleteRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/multipart/S3MultipartUploadCompleteRequest.java
@@ -61,7 +61,6 @@ import java.util.List;
import java.util.Map;
import java.util.TreeMap;
-import static org.apache.hadoop.ozone.OzoneConsts.OM_MULTIPART_MIN_SIZE;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_A_FILE;
import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@ozone.apache.org
For additional commands, e-mail: commits-help@ozone.apache.org