diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java index ca0029cad998c..2500cd981b47d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java @@ -20,7 +20,9 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.util.Arrays; import java.util.List; +import java.util.UUID; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -402,7 +404,8 @@ private Path createDirWithEmptySubFolder() throws IOException { Path path = getContract().getTestPath(); fs.delete(path, true); // create a - non-qualified - Path for a subdir - Path subfolder = path.suffix('/' + this.methodName.getMethodName()); + Path subfolder = path.suffix('/' + this.methodName.getMethodName() + + "-" + UUID.randomUUID()); mkdirs(subfolder); return subfolder; } @@ -527,7 +530,8 @@ private FileStatus[] verifyListStatus(int expected, Path path, PathFilter filter) throws IOException { FileStatus[] result = getFileSystem().listStatus(path, filter); - assertEquals("length of listStatus(" + path + ", " + filter + " )", + assertEquals("length of listStatus(" + path + ", " + filter + " ) " + + Arrays.toString(result), expected, result.length); return result; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 2d980593e9e83..cfd628be4b92d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -87,6 +87,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; +import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -2172,6 +2173,8 @@ S3AFileStatus innerGetFileStatus(final Path f, FileStatus msStatus = pm.getFileStatus(); if (needEmptyDirectoryFlag && msStatus.isDirectory()) { + // the caller needs to know if a directory is empty, + // and that this is a directory. if (pm.isEmptyDirectory() != Tristate.UNKNOWN) { // We have a definitive true / false from MetadataStore, we are done. return S3AFileStatus.fromFileStatus(msStatus, pm.isEmptyDirectory()); @@ -2180,28 +2183,33 @@ S3AFileStatus innerGetFileStatus(final Path f, if (children != null) { tombstones = children.listTombstones(); } - LOG.debug("MetadataStore doesn't know if dir is empty, using S3."); + LOG.debug("MetadataStore doesn't know if {} is empty, using S3.", + path); } } else { // Either this is not a directory, or we don't care if it is empty return S3AFileStatus.fromFileStatus(msStatus, pm.isEmptyDirectory()); } - // If the metadata store has no children for it and it's not listed in - // S3 yet, we'll assume the empty directory is true; - S3AFileStatus s3FileStatus; + // now issue the S3 getFileStatus call. try { - s3FileStatus = s3GetFileStatus(path, key, tombstones); + S3AFileStatus s3FileStatus = s3GetFileStatus(path, key, + StatusProbeEnum.ALL, + tombstones, + true); + // entry was found, so save in S3Guard and return the final value. + return S3Guard.putAndReturn(metadataStore, s3FileStatus, + instrumentation); } catch (FileNotFoundException e) { return S3AFileStatus.fromFileStatus(msStatus, Tristate.TRUE); } - // entry was found, save in S3Guard - return S3Guard.putAndReturn(metadataStore, s3FileStatus, instrumentation); } else { // there was no entry in S3Guard // retrieve the data and update the metadata store in the process. return S3Guard.putAndReturn(metadataStore, - s3GetFileStatus(path, key, tombstones), instrumentation); + s3GetFileStatus(path, key, StatusProbeEnum.ALL, + tombstones, needEmptyDirectoryFlag), + instrumentation); } } @@ -2212,87 +2220,96 @@ S3AFileStatus innerGetFileStatus(final Path f, * Retry policy: retry translated. * @param path Qualified path * @param key Key string for the path + * @param probes probes to make + * @param tombstones tombstones to filter + * @param needEmptyDirectoryFlag if true, implementation will calculate + * a TRUE or FALSE value for {@link S3AFileStatus#isEmptyDirectory()} * @return Status - * @throws FileNotFoundException when the path does not exist + * @throws FileNotFoundException the supplied probes failed. * @throws IOException on other problems. */ + @VisibleForTesting @Retries.RetryTranslated - private S3AFileStatus s3GetFileStatus(final Path path, String key, - Set tombstones) throws IOException { - if (!key.isEmpty()) { + S3AFileStatus s3GetFileStatus(final Path path, + final String key, + final Set probes, + @Nullable final Set tombstones, + final boolean needEmptyDirectoryFlag) throws IOException { + LOG.debug("S3GetFileStatus {}", path); + Preconditions.checkArgument(!needEmptyDirectoryFlag + || probes.contains(StatusProbeEnum.List), + "s3GetFileStatus(%s) wants to know if a directory is empty but" + + " does not request a list probe", path); + + if (!key.isEmpty() && !key.endsWith("/") + && probes.contains(StatusProbeEnum.Head)) { try { + // look for the simple file ObjectMetadata meta = getObjectMetadata(key); - - if (objectRepresentsDirectory(key, meta.getContentLength())) { - LOG.debug("Found exact file: fake directory"); - return new S3AFileStatus(Tristate.TRUE, path, username); - } else { - LOG.debug("Found exact file: normal file"); + LOG.debug("Found exact file: normal file {}", key); return new S3AFileStatus(meta.getContentLength(), dateToLong(meta.getLastModified()), path, getDefaultBlockSize(path), username); - } } catch (AmazonServiceException e) { + // if the response is a 404 error, it just means that there is + // no file at that path...the remaining checks will be needed. if (e.getStatusCode() != 404) { throw translateException("getFileStatus", path, e); } } catch (AmazonClientException e) { throw translateException("getFileStatus", path, e); } + } - // Necessary? - if (!key.endsWith("/")) { - String newKey = key + "/"; + // execute the list + if (probes.contains(StatusProbeEnum.List)) { try { - ObjectMetadata meta = getObjectMetadata(newKey); - - if (objectRepresentsDirectory(newKey, meta.getContentLength())) { - LOG.debug("Found file (with /): fake directory"); - return new S3AFileStatus(Tristate.TRUE, path, username); + // this will find a marker dir / as well as an entry. + // When making a simple "is this a dir check" all is good. + // but when looking for an empty dir, we need to verify there are no + // children, so ask for two entries, so as to find + // a child + String dirKey = maybeAddTrailingSlash(key); + // list size is dir marker + at least one non-tombstone entry + // there's a corner case: more tombstones than you have in a + // single page list. We assume that if you have been deleting + // that many files, then the AWS listing will have purged some + // by the time of listing so that the response includes some + // which have not. + + int listSize; + if (tombstones == null) { + // no tombstones so look for a marker and at least one child. + listSize = 2; } else { - LOG.warn("Found file (with /): real file? should not happen: {}", - key); - - return new S3AFileStatus(meta.getContentLength(), - dateToLong(meta.getLastModified()), - path, - getDefaultBlockSize(path), - username); - } - } catch (AmazonServiceException e) { - if (e.getStatusCode() != 404) { - throw translateException("getFileStatus", newKey, e); - } - } catch (AmazonClientException e) { - throw translateException("getFileStatus", newKey, e); - } + // build a listing > tombstones. If the caller has many thousands + // of tombstones this won't work properly, which is why pruning + // of expired tombstones matters. + listSize = Math.min(2 + tombstones.size(), Math.max(2, maxKeys)); } - } - - try { - key = maybeAddTrailingSlash(key); - S3ListRequest request = createListObjectsRequest(key, "/", 1); + S3ListRequest request = createListObjectsRequest(dirKey, "/", + listSize); + // execute the request + S3ListResult listResult = listObjects(request); - S3ListResult objects = listObjects(request); - Collection prefixes = objects.getCommonPrefixes(); - Collection summaries = objects.getObjectSummaries(); - if (!isEmptyOfKeys(prefixes, tombstones) || - !isEmptyOfObjects(summaries, tombstones)) { + if (listResult.hasPrefixesOrObjects(this::keyToPath, tombstones)) { if (LOG.isDebugEnabled()) { - LOG.debug("Found path as directory (with /): {}/{}", - prefixes.size(), summaries.size()); - - for (S3ObjectSummary summary : summaries) { - LOG.debug("Summary: {} {}", summary.getKey(), summary.getSize()); + LOG.debug("Found path as directory (with /)"); + listResult.logAtDebug(LOG); } - for (String prefix : prefixes) { - LOG.debug("Prefix: {}", prefix); + // At least one entry has been found. + // If looking for an empty directory, the marker must exist but no children. + // So the listing must contain the marker entry only. + if (needEmptyDirectoryFlag + && listResult.representsEmptyDirectory( + this::keyToPath, dirKey, tombstones)) { + return new S3AFileStatus(Tristate.TRUE, path, username); } - } - + // either an empty directory is not needed, or the + // listing does not meet the requirements. return new S3AFileStatus(Tristate.FALSE, path, username); } else if (key.isEmpty()) { LOG.debug("Found root directory"); @@ -2305,53 +2322,12 @@ private S3AFileStatus s3GetFileStatus(final Path path, String key, } catch (AmazonClientException e) { throw translateException("getFileStatus", path, e); } + } LOG.debug("Not Found: {}", path); throw new FileNotFoundException("No such file or directory: " + path); } - /** - * Helper function to determine if a collection of paths is empty - * after accounting for tombstone markers (if provided). - * @param keys Collection of path (prefixes / directories or keys). - * @param tombstones Set of tombstone markers, or null if not applicable. - * @return false if summaries contains objects not accounted for by - * tombstones. - */ - private boolean isEmptyOfKeys(Collection keys, Set - tombstones) { - if (tombstones == null) { - return keys.isEmpty(); - } - for (String key : keys) { - Path qualified = keyToQualifiedPath(key); - if (!tombstones.contains(qualified)) { - return false; - } - } - return true; - } - - /** - * Helper function to determine if a collection of object summaries is empty - * after accounting for tombstone markers (if provided). - * @param summaries Collection of objects as returned by listObjects. - * @param tombstones Set of tombstone markers, or null if not applicable. - * @return false if summaries contains objects not accounted for by - * tombstones. - */ - private boolean isEmptyOfObjects(Collection summaries, - Set tombstones) { - if (tombstones == null) { - return summaries.isEmpty(); - } - Collection stringCollection = new ArrayList<>(summaries.size()); - for (S3ObjectSummary summary : summaries) { - stringCollection.add(summary.getKey()); - } - return isEmptyOfKeys(stringCollection, tombstones); - } - /** * Raw version of {@link FileSystem#exists(Path)} which uses S3 only: * S3Guard MetadataStore, if any, will be skipped. @@ -2364,7 +2340,8 @@ private boolean s3Exists(final Path f) throws IOException { Path path = qualify(f); String key = pathToKey(path); try { - s3GetFileStatus(path, key, null); + s3GetFileStatus(path, key, StatusProbeEnum.ALL, + null, false); return true; } catch (FileNotFoundException e) { return false; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java index e8aff329070ef..eec47705edec3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java @@ -18,11 +18,18 @@ package org.apache.hadoop.fs.s3a; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + import com.amazonaws.services.s3.model.ListObjectsV2Result; import com.amazonaws.services.s3.model.ObjectListing; import com.amazonaws.services.s3.model.S3ObjectSummary; +import org.slf4j.Logger; -import java.util.List; +import org.apache.hadoop.fs.Path; /** * API version-independent container for S3 List responses. @@ -92,6 +99,109 @@ public List getCommonPrefixes() { } else { return v2Result.getCommonPrefixes(); } + } + + /** + * Is the list of object summaries empty + * after accounting for tombstone markers (if provided)? + * @param keyToPath callback for key to path mapping. + * @param tombstones Set of tombstone markers, or null if not applicable. + * @return false if summaries contains objects not accounted for by + * tombstones. + */ + public boolean isEmptyOfObjects( + final Function keyToPath, + final Set tombstones) { + if (tombstones == null) { + return getObjectSummaries().isEmpty(); + } + return isEmptyOfKeys(keyToPath, + objectSummaryKeys(), + tombstones); + } + + /** + * Get the list of keys in the object summary. + * @return a possibly empty list + */ + private List objectSummaryKeys() { + return getObjectSummaries().stream() + .map(S3ObjectSummary::getKey) + .collect(Collectors.toList()); + } + + /** + * Does this listing have prefixes or objects? + * @param keyToPath callback for key to path mapping. + * @param tombstones Set of tombstone markers, or null if not applicable. + * @return true if the reconciled list is non-empty + */ + public boolean hasPrefixesOrObjects( + final Function keyToPath, + final Set tombstones) { + + return !isEmptyOfKeys(keyToPath, getCommonPrefixes(), tombstones) + || !isEmptyOfObjects(keyToPath, tombstones); + } + + /** + * Helper function to determine if a collection of keys is empty + * after accounting for tombstone markers (if provided). + * @param keyToPath callback for key to path mapping. + * @param keys Collection of path (prefixes / directories or keys). + * @param tombstones Set of tombstone markers, or null if not applicable. + * @return true if the list is considered empty. + */ + public boolean isEmptyOfKeys( + final Function keyToPath, + final Collection keys, + final Set tombstones) { + if (tombstones == null) { + return keys.isEmpty(); + } + for (String key : keys) { + Path qualified = keyToPath.apply(key); + if (!tombstones.contains(qualified)) { + return false; + } + } + return true; + } + /** + * Does this listing represent an empty directory? + * @param keyToPath callback for key to path mapping. + * @param dirKey directory key + * @param tombstones Set of tombstone markers, or null if not applicable. + * @return true if the list is considered empty. + */ + public boolean representsEmptyDirectory( + final Function keyToPath, + final String dirKey, + final Set tombstones) { + // If looking for an empty directory, the marker must exist but + // no children. + // So the listing must contain the marker entry only as an object, + // and prefixes is null + List keys = objectSummaryKeys(); + return keys.size() == 1 && keys.contains(dirKey) + && getCommonPrefixes().isEmpty(); + } + + /** + * dump the result at debug level only. + * @param log log to use + */ + public void logAtDebug(Logger log) { + Collection prefixes = getCommonPrefixes(); + Collection summaries = getObjectSummaries(); + log.debug("Prefix count = {}; object count={}", + prefixes.size(), summaries.size()); + for (S3ObjectSummary summary : summaries) { + log.debug("Summary: {} {}", summary.getKey(), summary.getSize()); + } + for (String prefix : prefixes) { + log.debug("Prefix: {}", prefix); + } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StatusProbeEnum.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StatusProbeEnum.java new file mode 100644 index 0000000000000..f9749a14a3d7d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StatusProbeEnum.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.fs.s3a.impl; + +import java.util.EnumSet; +import java.util.Set; + +/** + * Enum of probes which can be made of S3. + */ +public enum StatusProbeEnum { + + /** The actual path. */ + Head, + /** HEAD of the path + /. */ + DirMarker, + /** LIST under the path. */ + List; + + /** Look for files and directories. */ + public static final Set ALL = + EnumSet.of(Head, List); + + /** We only want the HEAD. */ + public static final Set HEAD_ONLY = + EnumSet.of(Head); + + /** List operation only. */ + public static final Set LIST_ONLY = + EnumSet.of(List); + + /** Look for files and directories. */ + public static final Set FILE = + HEAD_ONLY; + + /** Skip the HEAD and only look for directories. */ + public static final Set DIRECTORIES = + LIST_ONLY; + + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java index 279ec9cd6a80c..6ff2e506f9b92 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java @@ -35,6 +35,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; +import static org.apache.hadoop.fs.s3a.test.costs.HeadListCosts.*; import static org.apache.hadoop.test.GenericTestUtils.getTestDir; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -90,9 +91,9 @@ public void testCostOfGetFileStatusOnEmptyDir() throws Throwable { Tristate.TRUE); if (!fs.hasMetadataStore()) { - metadataRequests.assertDiffEquals(2); + metadataRequests.assertDiffEquals(GETFILESTATUS_EMPTY_DIR_H); } - listRequests.assertDiffEquals(0); + listRequests.assertDiffEquals(GETFILESTATUS_EMPTY_DIR_L); } @Test @@ -103,8 +104,8 @@ public void testCostOfGetFileStatusOnMissingFile() throws Throwable { resetMetricDiffs(); intercept(FileNotFoundException.class, () -> fs.getFileStatus(path)); - metadataRequests.assertDiffEquals(2); - listRequests.assertDiffEquals(1); + metadataRequests.assertDiffEquals(GETFILESTATUS_FNFE_H); + listRequests.assertDiffEquals(GETFILESTATUS_FNFE_L); } @Test @@ -115,8 +116,8 @@ public void testCostOfGetFileStatusOnMissingSubPath() throws Throwable { resetMetricDiffs(); intercept(FileNotFoundException.class, () -> fs.getFileStatus(path)); - metadataRequests.assertDiffEquals(2); - listRequests.assertDiffEquals(1); + metadataRequests.assertDiffEquals(GETFILESTATUS_FNFE_H); + listRequests.assertDiffEquals(GETFILESTATUS_FNFE_L); } @Test @@ -137,8 +138,8 @@ public void testCostOfGetFileStatusOnNonEmptyDir() throws Throwable { + "\n" + fsState); } if (!fs.hasMetadataStore()) { - metadataRequests.assertDiffEquals(2); - listRequests.assertDiffEquals(1); + metadataRequests.assertDiffEquals(GETFILESTATUS_FNFE_H); + listRequests.assertDiffEquals(GETFILESTATUS_FNFE_L); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java index 586264d1e7628..b5615faed4469 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java @@ -76,11 +76,15 @@ public void testFakeDirectory() throws Exception { String key = path.toUri().getPath().substring(1); when(s3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key)))) .thenThrow(NOT_FOUND); - ObjectMetadata meta = new ObjectMetadata(); - meta.setContentLength(0L); - when(s3.getObjectMetadata(argThat( - correctGetMetadataRequest(BUCKET, key + "/")) - )).thenReturn(meta); + String keyDir = key + "/"; + ListObjectsV2Result listResult = new ListObjectsV2Result(); + S3ObjectSummary objectSummary = new S3ObjectSummary(); + objectSummary.setKey(keyDir); + objectSummary.setSize(0L); + listResult.getObjectSummaries().add(objectSummary); + when(s3.listObjectsV2(argThat( + matchListV2Request(BUCKET, keyDir)) + )).thenReturn(listResult); FileStatus stat = fs.getFileStatus(path); assertNotNull(stat); assertEquals(fs.makeQualified(path), stat.getPath()); @@ -176,4 +180,28 @@ public boolean matches(Object o) { } }; } + + private Matcher matchListV2Request( + String bucket, String key) { + return new BaseMatcher() { + + @Override + public void describeTo(Description description) { + description.appendText("bucket and key match"); + } + + @Override + public boolean matches(Object o) { + if(o instanceof ListObjectsV2Request) { + ListObjectsV2Request request = + (ListObjectsV2Request)o; + return request.getBucketName().equals(bucket) + && request.getPrefix().equals(key); + } + return false; + } + }; + } + + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/HeadListCosts.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/HeadListCosts.java new file mode 100644 index 0000000000000..ef15a725d7acf --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/HeadListCosts.java @@ -0,0 +1,123 @@ +/* + * 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.fs.s3a.test.costs; + +/** + * Declaration of the costs of head and list calls for various FS IO operations. + */ +public class HeadListCosts { + + /** Head costs for getFileStatus() directory probe: {@value}. */ + public static final int FILESTATUS_DIR_PROBE_H = 0; + + /** List costs for getFileStatus() directory probe: {@value}. */ + public static final int FILESTATUS_DIR_PROBE_L = 1; + + + /** Head cost getFileStatus() file probe only. */ + public static final int FILESTATUS_FILE_PROBE_H = 1; + + /** Liast cost getFileStatus() file probe only. */ + + public static final int FILESTATUS_FILE_PROBE_L = 0; + + /** Head costs getFileStatus() no file or dir. */ + public static final int GETFILESTATUS_FNFE_H = FILESTATUS_FILE_PROBE_H; + + /** List costs for getFileStatus() on an empty path: {@value}. */ + + public static final int GETFILESTATUS_FNFE_L = FILESTATUS_DIR_PROBE_L; + + /** getFileStatus() directory which is non-empty. */ + public static final int GETFILESTATUS_DIR_H = FILESTATUS_FILE_PROBE_H; + + /** List costs for getFileStatus() on a non-empty directory: {@value}. */ + public static final int GETFILESTATUS_DIR_L = FILESTATUS_DIR_PROBE_L; + + /** List costs for getFileStatus() on an non-empty directory: {@value}. */ + public static final int GETFILESTATUS_EMPTY_DIR_L = FILESTATUS_DIR_PROBE_L; + /** List costs for getFileStatus() on an non-empty directory: {@value}. */ + public static final int GETFILESTATUS_EMPTY_DIR_H = GETFILESTATUS_DIR_H; + + /** getFileStatus() directory marker which exists. */ + public static final int GETFILESTATUS_MARKER_H = FILESTATUS_FILE_PROBE_H; + + /** getFileStatus() on a file which exists. */ + public static final int GETFILESTATUS_SINGLE_FILE_H = FILESTATUS_FILE_PROBE_H; + + + public static final int GETFILESTATUS_SINGLE_FILE_L = FILESTATUS_FILE_PROBE_L; + + public static final int DELETE_OBJECT_REQUEST = 1; + + public static final int DELETE_MARKER_REQUEST = 1; + + /** listLocatedStatus always does a list. */ + public static final int LIST_LOCATED_STATUS_L = 1; + + public static final int LIST_FILES_L = 1; + + /** + * Cost of renaming a file to a different directory. + *

+ * LIST on dest not found, look for dest dir, and then, at + * end of rename, whether a parent dir needs to be created. + */ + public static final int RENAME_SINGLE_FILE_RENAME_DIFFERENT_DIR_L = + GETFILESTATUS_FNFE_L + GETFILESTATUS_DIR_L * 2; + + /** + * Cost of renaming a file to a different directory. + *

+ * LIST on dest not found, look for dest dir, and then, at + * end of rename, whether a parent dir needs to be created. + */ + public static final int RENAME_SINGLE_FILE_RENAME_SAME_DIR_L = + GETFILESTATUS_FNFE_L; + + /** + * Rename a single file. + *

+ * source is found, dest not found, copy adds a + * metadata request. + */ + public static final int RENAME_SINGLE_FILE_RENAME_H = + FILESTATUS_FILE_PROBE_H + GETFILESTATUS_FNFE_H + 1; + + /** + * Create file no overwrite head : {@value}. + */ + public static final int CREATE_FILE_OVERWRITE_H = 0; + + /** + * Create file no overwrite list : {@value}. + */ + public static final int CREATE_FILE_OVERWRITE_L = FILESTATUS_DIR_PROBE_L; + + /** + * Create file no overwrite head : {@value}. + */ + public static final int CREATE_FILE_NO_OVERWRITE_H = FILESTATUS_FILE_PROBE_H; + + /** + * Create file no overwrite list : {@value}. + */ + public static final int CREATE_FILE_NO_OVERWRITE_L = FILESTATUS_DIR_PROBE_L; + +}