|
@@ -56,6 +56,7 @@ import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
|
|
import com.amazonaws.services.s3.model.CannedAccessControlList;
|
|
import com.amazonaws.services.s3.model.CannedAccessControlList;
|
|
import com.amazonaws.services.s3.model.CopyObjectRequest;
|
|
import com.amazonaws.services.s3.model.CopyObjectRequest;
|
|
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
|
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
|
|
|
+import com.amazonaws.services.s3.model.DeleteObjectsResult;
|
|
import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
|
|
import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
|
|
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
|
|
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
|
|
import com.amazonaws.services.s3.model.InitiateMultipartUploadResult;
|
|
import com.amazonaws.services.s3.model.InitiateMultipartUploadResult;
|
|
@@ -96,7 +97,10 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|
import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
|
|
import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
|
|
import org.apache.hadoop.fs.s3a.impl.ContextAccessors;
|
|
import org.apache.hadoop.fs.s3a.impl.ContextAccessors;
|
|
import org.apache.hadoop.fs.s3a.impl.CopyOutcome;
|
|
import org.apache.hadoop.fs.s3a.impl.CopyOutcome;
|
|
|
|
+import org.apache.hadoop.fs.s3a.impl.DeleteOperation;
|
|
|
|
+import org.apache.hadoop.fs.s3a.impl.InternalConstants;
|
|
import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport;
|
|
import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport;
|
|
|
|
+import org.apache.hadoop.fs.s3a.impl.OperationCallbacks;
|
|
import org.apache.hadoop.fs.s3a.impl.RenameOperation;
|
|
import org.apache.hadoop.fs.s3a.impl.RenameOperation;
|
|
import org.apache.hadoop.fs.s3a.impl.StoreContext;
|
|
import org.apache.hadoop.fs.s3a.impl.StoreContext;
|
|
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
|
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
|
@@ -115,7 +119,6 @@ import org.apache.hadoop.fs.LocatedFileStatus;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.PathFilter;
|
|
import org.apache.hadoop.fs.PathFilter;
|
|
import org.apache.hadoop.fs.PathIOException;
|
|
import org.apache.hadoop.fs.PathIOException;
|
|
-import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
|
|
|
|
import org.apache.hadoop.fs.RemoteIterator;
|
|
import org.apache.hadoop.fs.RemoteIterator;
|
|
import org.apache.hadoop.fs.StreamCapabilities;
|
|
import org.apache.hadoop.fs.StreamCapabilities;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
@@ -245,8 +248,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
/** Principal who created the FS; recorded during initialization. */
|
|
/** Principal who created the FS; recorded during initialization. */
|
|
private UserGroupInformation owner;
|
|
private UserGroupInformation owner;
|
|
|
|
|
|
- // The maximum number of entries that can be deleted in any call to s3
|
|
|
|
- private static final int MAX_ENTRIES_TO_DELETE = 1000;
|
|
|
|
private String blockOutputBuffer;
|
|
private String blockOutputBuffer;
|
|
private S3ADataBlocks.BlockFactory blockFactory;
|
|
private S3ADataBlocks.BlockFactory blockFactory;
|
|
private int blockOutputActiveBlocks;
|
|
private int blockOutputActiveBlocks;
|
|
@@ -259,6 +260,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
|
|
|
private ITtlTimeProvider ttlTimeProvider;
|
|
private ITtlTimeProvider ttlTimeProvider;
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Specific operations used by rename and delete operations.
|
|
|
|
+ */
|
|
|
|
+ private final S3AFileSystem.OperationCallbacksImpl
|
|
|
|
+ operationCallbacks = new OperationCallbacksImpl();
|
|
|
|
+
|
|
/** Add any deprecated keys. */
|
|
/** Add any deprecated keys. */
|
|
@SuppressWarnings("deprecation")
|
|
@SuppressWarnings("deprecation")
|
|
private static void addDeprecatedKeys() {
|
|
private static void addDeprecatedKeys() {
|
|
@@ -1309,17 +1316,16 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
createStoreContext(),
|
|
createStoreContext(),
|
|
src, srcKey, p.getLeft(),
|
|
src, srcKey, p.getLeft(),
|
|
dst, dstKey, p.getRight(),
|
|
dst, dstKey, p.getRight(),
|
|
- new RenameOperationCallbacksImpl());
|
|
|
|
- return renameOperation.executeRename();
|
|
|
|
|
|
+ operationCallbacks);
|
|
|
|
+ return renameOperation.execute();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * All the callbacks made by the rename operation of the filesystem.
|
|
|
|
|
|
+ * The callbacks made by the rename and delete operations.
|
|
* This separation allows the operation to be factored out and
|
|
* This separation allows the operation to be factored out and
|
|
* still avoid knowledge of the S3AFilesystem implementation.
|
|
* still avoid knowledge of the S3AFilesystem implementation.
|
|
*/
|
|
*/
|
|
- private class RenameOperationCallbacksImpl implements
|
|
|
|
- RenameOperation.RenameOperationCallbacks {
|
|
|
|
|
|
+ private class OperationCallbacksImpl implements OperationCallbacks {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public S3ObjectAttributes createObjectAttributes(final Path path,
|
|
public S3ObjectAttributes createObjectAttributes(final Path path,
|
|
@@ -1331,7 +1337,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public S3ObjectAttributes createObjectAttributes(final S3AFileStatus fileStatus) {
|
|
|
|
|
|
+ public S3ObjectAttributes createObjectAttributes(
|
|
|
|
+ final S3AFileStatus fileStatus) {
|
|
return S3AFileSystem.this.createObjectAttributes(fileStatus);
|
|
return S3AFileSystem.this.createObjectAttributes(fileStatus);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1343,18 +1350,32 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
|
|
+ @Retries.RetryTranslated
|
|
public void deleteObjectAtPath(final Path path,
|
|
public void deleteObjectAtPath(final Path path,
|
|
final String key,
|
|
final String key,
|
|
- final boolean isFile)
|
|
|
|
|
|
+ final boolean isFile,
|
|
|
|
+ final BulkOperationState operationState)
|
|
throws IOException {
|
|
throws IOException {
|
|
- S3AFileSystem.this.deleteObjectAtPath(path, key, isFile);
|
|
|
|
|
|
+ once("delete", key, () ->
|
|
|
|
+ S3AFileSystem.this.deleteObjectAtPath(path, key, isFile,
|
|
|
|
+ operationState));
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@Retries.RetryTranslated
|
|
@Retries.RetryTranslated
|
|
public RemoteIterator<S3ALocatedFileStatus> listFilesAndEmptyDirectories(
|
|
public RemoteIterator<S3ALocatedFileStatus> listFilesAndEmptyDirectories(
|
|
- final Path path) throws IOException {
|
|
|
|
- return S3AFileSystem.this.listFilesAndEmptyDirectories(path, true);
|
|
|
|
|
|
+ final Path path,
|
|
|
|
+ final S3AFileStatus status,
|
|
|
|
+ final boolean collectTombstones,
|
|
|
|
+ final boolean includeSelf) throws IOException {
|
|
|
|
+ return innerListFiles(
|
|
|
|
+ path,
|
|
|
|
+ true,
|
|
|
|
+ includeSelf
|
|
|
|
+ ? Listing.ACCEPT_ALL_BUT_S3N
|
|
|
|
+ : new Listing.AcceptAllButSelfAndS3nDirs(path),
|
|
|
|
+ status,
|
|
|
|
+ collectTombstones);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -1367,12 +1388,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public void removeKeys(final List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
|
|
|
|
|
+ public DeleteObjectsResult removeKeys(
|
|
|
|
+ final List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
|
final boolean deleteFakeDir,
|
|
final boolean deleteFakeDir,
|
|
- final List<Path> undeletedObjectsOnFailure)
|
|
|
|
|
|
+ final List<Path> undeletedObjectsOnFailure,
|
|
|
|
+ final BulkOperationState operationState,
|
|
|
|
+ final boolean quiet)
|
|
throws MultiObjectDeleteException, AmazonClientException, IOException {
|
|
throws MultiObjectDeleteException, AmazonClientException, IOException {
|
|
- S3AFileSystem.this.removeKeys(keysToDelete, deleteFakeDir,
|
|
|
|
- undeletedObjectsOnFailure);
|
|
|
|
|
|
+ return S3AFileSystem.this.removeKeys(keysToDelete, deleteFakeDir,
|
|
|
|
+ undeletedObjectsOnFailure, operationState, quiet);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -1385,6 +1409,25 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
maybeCreateFakeParentDirectory(sourceRenamed);
|
|
maybeCreateFakeParentDirectory(sourceRenamed);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public boolean allowAuthoritative(final Path p) {
|
|
|
|
+ return S3AFileSystem.this.allowAuthoritative(p);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ @Retries.RetryTranslated
|
|
|
|
+ public RemoteIterator<S3AFileStatus> listObjects(
|
|
|
|
+ final Path path,
|
|
|
|
+ final String key)
|
|
|
|
+ throws IOException {
|
|
|
|
+ return once("listObjects", key, () ->
|
|
|
|
+ listing.createFileStatusListingIterator(path,
|
|
|
|
+ createListObjectsRequest(key, null),
|
|
|
|
+ ACCEPT_ALL,
|
|
|
|
+ Listing.ACCEPT_ALL_BUT_S3N,
|
|
|
|
+ null));
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1738,14 +1781,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
throws AmazonClientException, IOException {
|
|
throws AmazonClientException, IOException {
|
|
blockRootDelete(key);
|
|
blockRootDelete(key);
|
|
incrementWriteOperations();
|
|
incrementWriteOperations();
|
|
- LOG.debug("DELETE {}", key);
|
|
|
|
- invoker.retryUntranslated("Delete "+ bucket + ":/" + key,
|
|
|
|
- DELETE_CONSIDERED_IDEMPOTENT,
|
|
|
|
- ()-> {
|
|
|
|
- incrementStatistic(OBJECT_DELETE_REQUESTS);
|
|
|
|
- s3.deleteObject(bucket, key);
|
|
|
|
- return null;
|
|
|
|
- });
|
|
|
|
|
|
+ try (DurationInfo ignored =
|
|
|
|
+ new DurationInfo(LOG, false,
|
|
|
|
+ "deleting %s", key)) {
|
|
|
|
+ invoker.retryUntranslated(String.format("Delete %s:/%s", bucket, key),
|
|
|
|
+ DELETE_CONSIDERED_IDEMPOTENT,
|
|
|
|
+ ()-> {
|
|
|
|
+ incrementStatistic(OBJECT_DELETE_REQUESTS);
|
|
|
|
+ s3.deleteObject(bucket, key);
|
|
|
|
+ return null;
|
|
|
|
+ });
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1755,11 +1801,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* @param f path path to delete
|
|
* @param f path path to delete
|
|
* @param key key of entry
|
|
* @param key key of entry
|
|
* @param isFile is the path a file (used for instrumentation only)
|
|
* @param isFile is the path a file (used for instrumentation only)
|
|
|
|
+ * @param operationState (nullable) operational state for a bulk update
|
|
* @throws AmazonClientException problems working with S3
|
|
* @throws AmazonClientException problems working with S3
|
|
* @throws IOException IO failure in the metastore
|
|
* @throws IOException IO failure in the metastore
|
|
*/
|
|
*/
|
|
@Retries.RetryMixed
|
|
@Retries.RetryMixed
|
|
- void deleteObjectAtPath(Path f, String key, boolean isFile)
|
|
|
|
|
|
+ void deleteObjectAtPath(Path f,
|
|
|
|
+ String key,
|
|
|
|
+ boolean isFile,
|
|
|
|
+ @Nullable final BulkOperationState operationState)
|
|
throws AmazonClientException, IOException {
|
|
throws AmazonClientException, IOException {
|
|
if (isFile) {
|
|
if (isFile) {
|
|
instrumentation.fileDeleted(1);
|
|
instrumentation.fileDeleted(1);
|
|
@@ -1767,7 +1817,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
instrumentation.directoryDeleted();
|
|
instrumentation.directoryDeleted();
|
|
}
|
|
}
|
|
deleteObject(key);
|
|
deleteObject(key);
|
|
- metadataStore.delete(f);
|
|
|
|
|
|
+ metadataStore.delete(f, operationState);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1789,18 +1839,19 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* operation statistics.
|
|
* operation statistics.
|
|
* Retry policy: retry untranslated; delete considered idempotent.
|
|
* Retry policy: retry untranslated; delete considered idempotent.
|
|
* @param deleteRequest keys to delete on the s3-backend
|
|
* @param deleteRequest keys to delete on the s3-backend
|
|
|
|
+ * @return the AWS response
|
|
* @throws MultiObjectDeleteException one or more of the keys could not
|
|
* @throws MultiObjectDeleteException one or more of the keys could not
|
|
* be deleted.
|
|
* be deleted.
|
|
* @throws AmazonClientException amazon-layer failure.
|
|
* @throws AmazonClientException amazon-layer failure.
|
|
*/
|
|
*/
|
|
@Retries.RetryRaw
|
|
@Retries.RetryRaw
|
|
- private void deleteObjects(DeleteObjectsRequest deleteRequest)
|
|
|
|
|
|
+ private DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteRequest)
|
|
throws MultiObjectDeleteException, AmazonClientException, IOException {
|
|
throws MultiObjectDeleteException, AmazonClientException, IOException {
|
|
incrementWriteOperations();
|
|
incrementWriteOperations();
|
|
try(DurationInfo ignored =
|
|
try(DurationInfo ignored =
|
|
new DurationInfo(LOG, false, "DELETE %d keys",
|
|
new DurationInfo(LOG, false, "DELETE %d keys",
|
|
deleteRequest.getKeys().size())) {
|
|
deleteRequest.getKeys().size())) {
|
|
- invoker.retryUntranslated("delete",
|
|
|
|
|
|
+ return invoker.retryUntranslated("delete",
|
|
DELETE_CONSIDERED_IDEMPOTENT,
|
|
DELETE_CONSIDERED_IDEMPOTENT,
|
|
() -> {
|
|
() -> {
|
|
incrementStatistic(OBJECT_DELETE_REQUESTS, 1);
|
|
incrementStatistic(OBJECT_DELETE_REQUESTS, 1);
|
|
@@ -2043,6 +2094,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* @param keysToDelete collection of keys to delete on the s3-backend.
|
|
* @param keysToDelete collection of keys to delete on the s3-backend.
|
|
* if empty, no request is made of the object store.
|
|
* if empty, no request is made of the object store.
|
|
* @param deleteFakeDir indicates whether this is for deleting fake dirs
|
|
* @param deleteFakeDir indicates whether this is for deleting fake dirs
|
|
|
|
+ * @param quiet should a bulk query be quiet, or should its result list
|
|
|
|
+ * all deleted keys?
|
|
|
|
+ * @return the deletion result if a multi object delete was invoked
|
|
|
|
+ * and it returned without a failure.
|
|
* @throws InvalidRequestException if the request was rejected due to
|
|
* @throws InvalidRequestException if the request was rejected due to
|
|
* a mistaken attempt to delete the root directory.
|
|
* a mistaken attempt to delete the root directory.
|
|
* @throws MultiObjectDeleteException one or more of the keys could not
|
|
* @throws MultiObjectDeleteException one or more of the keys could not
|
|
@@ -2052,22 +2107,26 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* @throws AmazonClientException other amazon-layer failure.
|
|
* @throws AmazonClientException other amazon-layer failure.
|
|
*/
|
|
*/
|
|
@Retries.RetryRaw
|
|
@Retries.RetryRaw
|
|
- private void removeKeysS3(List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
|
|
|
- boolean deleteFakeDir)
|
|
|
|
|
|
+ private DeleteObjectsResult removeKeysS3(
|
|
|
|
+ List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
|
|
|
+ boolean deleteFakeDir,
|
|
|
|
+ boolean quiet)
|
|
throws MultiObjectDeleteException, AmazonClientException,
|
|
throws MultiObjectDeleteException, AmazonClientException,
|
|
IOException {
|
|
IOException {
|
|
|
|
+ DeleteObjectsResult result = null;
|
|
if (keysToDelete.isEmpty()) {
|
|
if (keysToDelete.isEmpty()) {
|
|
// exit fast if there are no keys to delete
|
|
// exit fast if there are no keys to delete
|
|
- return;
|
|
|
|
|
|
+ return result;
|
|
}
|
|
}
|
|
for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
|
|
for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
|
|
blockRootDelete(keyVersion.getKey());
|
|
blockRootDelete(keyVersion.getKey());
|
|
}
|
|
}
|
|
try {
|
|
try {
|
|
if (enableMultiObjectsDelete) {
|
|
if (enableMultiObjectsDelete) {
|
|
- deleteObjects(new DeleteObjectsRequest(bucket)
|
|
|
|
- .withKeys(keysToDelete)
|
|
|
|
- .withQuiet(true));
|
|
|
|
|
|
+ result = deleteObjects(
|
|
|
|
+ new DeleteObjectsRequest(bucket)
|
|
|
|
+ .withKeys(keysToDelete)
|
|
|
|
+ .withQuiet(quiet));
|
|
} else {
|
|
} else {
|
|
for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
|
|
for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
|
|
deleteObject(keyVersion.getKey());
|
|
deleteObject(keyVersion.getKey());
|
|
@@ -2083,6 +2142,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
throw ex;
|
|
throw ex;
|
|
}
|
|
}
|
|
noteDeleted(keysToDelete.size(), deleteFakeDir);
|
|
noteDeleted(keysToDelete.size(), deleteFakeDir);
|
|
|
|
+ return result;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -2105,6 +2165,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* @param keysToDelete collection of keys to delete on the s3-backend.
|
|
* @param keysToDelete collection of keys to delete on the s3-backend.
|
|
* if empty, no request is made of the object store.
|
|
* if empty, no request is made of the object store.
|
|
* @param deleteFakeDir indicates whether this is for deleting fake dirs
|
|
* @param deleteFakeDir indicates whether this is for deleting fake dirs
|
|
|
|
+ * @param operationState (nullable) operational state for a bulk update
|
|
* @throws InvalidRequestException if the request was rejected due to
|
|
* @throws InvalidRequestException if the request was rejected due to
|
|
* a mistaken attempt to delete the root directory.
|
|
* a mistaken attempt to delete the root directory.
|
|
* @throws MultiObjectDeleteException one or more of the keys could not
|
|
* @throws MultiObjectDeleteException one or more of the keys could not
|
|
@@ -2116,14 +2177,16 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
@Retries.RetryMixed
|
|
@Retries.RetryMixed
|
|
void removeKeys(
|
|
void removeKeys(
|
|
final List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
|
final List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
|
- final boolean deleteFakeDir)
|
|
|
|
|
|
+ final boolean deleteFakeDir,
|
|
|
|
+ final BulkOperationState operationState)
|
|
throws MultiObjectDeleteException, AmazonClientException,
|
|
throws MultiObjectDeleteException, AmazonClientException,
|
|
IOException {
|
|
IOException {
|
|
- removeKeys(keysToDelete, deleteFakeDir, new ArrayList<>());
|
|
|
|
|
|
+ removeKeys(keysToDelete, deleteFakeDir, new ArrayList<>(), operationState,
|
|
|
|
+ true);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Invoke {@link #removeKeysS3(List, boolean)} with handling of
|
|
|
|
|
|
+ * Invoke {@link #removeKeysS3(List, boolean, boolean)} with handling of
|
|
* {@code MultiObjectDeleteException} before the exception is rethrown.
|
|
* {@code MultiObjectDeleteException} before the exception is rethrown.
|
|
* Specifically:
|
|
* Specifically:
|
|
* <ol>
|
|
* <ol>
|
|
@@ -2138,6 +2201,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* @param undeletedObjectsOnFailure List which will be built up of all
|
|
* @param undeletedObjectsOnFailure List which will be built up of all
|
|
* files that were not deleted. This happens even as an exception
|
|
* files that were not deleted. This happens even as an exception
|
|
* is raised.
|
|
* is raised.
|
|
|
|
+ * @param operationState (nullable) operational state for a bulk update
|
|
|
|
+ * @param quiet should a bulk query be quiet, or should its result list
|
|
|
|
+ * all deleted keys
|
|
|
|
+ * @return the deletion result if a multi object delete was invoked
|
|
|
|
+ * and it returned without a failure, else null.
|
|
* @throws InvalidRequestException if the request was rejected due to
|
|
* @throws InvalidRequestException if the request was rejected due to
|
|
* a mistaken attempt to delete the root directory.
|
|
* a mistaken attempt to delete the root directory.
|
|
* @throws MultiObjectDeleteException one or more of the keys could not
|
|
* @throws MultiObjectDeleteException one or more of the keys could not
|
|
@@ -2145,17 +2213,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* @throws AmazonClientException amazon-layer failure.
|
|
* @throws AmazonClientException amazon-layer failure.
|
|
* @throws IOException other IO Exception.
|
|
* @throws IOException other IO Exception.
|
|
*/
|
|
*/
|
|
- @VisibleForTesting
|
|
|
|
@Retries.RetryMixed
|
|
@Retries.RetryMixed
|
|
- void removeKeys(
|
|
|
|
|
|
+ DeleteObjectsResult removeKeys(
|
|
final List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
|
final List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
|
final boolean deleteFakeDir,
|
|
final boolean deleteFakeDir,
|
|
- final List<Path> undeletedObjectsOnFailure)
|
|
|
|
- throws MultiObjectDeleteException, AmazonClientException,
|
|
|
|
- IOException {
|
|
|
|
|
|
+ final List<Path> undeletedObjectsOnFailure,
|
|
|
|
+ final BulkOperationState operationState,
|
|
|
|
+ final boolean quiet)
|
|
|
|
+ throws MultiObjectDeleteException, AmazonClientException, IOException {
|
|
undeletedObjectsOnFailure.clear();
|
|
undeletedObjectsOnFailure.clear();
|
|
- try(DurationInfo ignored = new DurationInfo(LOG, false, "Deleting")) {
|
|
|
|
- removeKeysS3(keysToDelete, deleteFakeDir);
|
|
|
|
|
|
+ try (DurationInfo ignored = new DurationInfo(LOG, false, "Deleting")) {
|
|
|
|
+ return removeKeysS3(keysToDelete, deleteFakeDir, quiet);
|
|
} catch (MultiObjectDeleteException ex) {
|
|
} catch (MultiObjectDeleteException ex) {
|
|
LOG.debug("Partial delete failure");
|
|
LOG.debug("Partial delete failure");
|
|
// what to do if an IOE was raised? Given an exception was being
|
|
// what to do if an IOE was raised? Given an exception was being
|
|
@@ -2164,13 +2232,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
// when deleting fake directories we don't want to delete metastore
|
|
// when deleting fake directories we don't want to delete metastore
|
|
// entries so we only process these failures on "real" deletes.
|
|
// entries so we only process these failures on "real" deletes.
|
|
Triple<List<Path>, List<Path>, List<Pair<Path, IOException>>> results =
|
|
Triple<List<Path>, List<Path>, List<Pair<Path, IOException>>> results =
|
|
- new MultiObjectDeleteSupport(createStoreContext())
|
|
|
|
|
|
+ new MultiObjectDeleteSupport(createStoreContext(), operationState)
|
|
.processDeleteFailure(ex, keysToDelete);
|
|
.processDeleteFailure(ex, keysToDelete);
|
|
undeletedObjectsOnFailure.addAll(results.getMiddle());
|
|
undeletedObjectsOnFailure.addAll(results.getMiddle());
|
|
}
|
|
}
|
|
throw ex;
|
|
throw ex;
|
|
} catch (AmazonClientException | IOException ex) {
|
|
} catch (AmazonClientException | IOException ex) {
|
|
- List<Path> paths = new MultiObjectDeleteSupport(createStoreContext())
|
|
|
|
|
|
+ List<Path> paths = new MultiObjectDeleteSupport(
|
|
|
|
+ createStoreContext(),
|
|
|
|
+ operationState)
|
|
.processDeleteFailureGenericException(ex, keysToDelete);
|
|
.processDeleteFailureGenericException(ex, keysToDelete);
|
|
// other failures. Assume nothing was deleted
|
|
// other failures. Assume nothing was deleted
|
|
undeletedObjectsOnFailure.addAll(paths);
|
|
undeletedObjectsOnFailure.addAll(paths);
|
|
@@ -2195,7 +2265,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
public boolean delete(Path f, boolean recursive) throws IOException {
|
|
public boolean delete(Path f, boolean recursive) throws IOException {
|
|
try {
|
|
try {
|
|
entryPoint(INVOCATION_DELETE);
|
|
entryPoint(INVOCATION_DELETE);
|
|
- boolean outcome = innerDelete(innerGetFileStatus(f, true), recursive);
|
|
|
|
|
|
+ DeleteOperation deleteOperation = new DeleteOperation(
|
|
|
|
+ createStoreContext(),
|
|
|
|
+ innerGetFileStatus(f, true),
|
|
|
|
+ recursive,
|
|
|
|
+ operationCallbacks,
|
|
|
|
+ InternalConstants.MAX_ENTRIES_TO_DELETE);
|
|
|
|
+ boolean outcome = deleteOperation.execute();
|
|
if (outcome) {
|
|
if (outcome) {
|
|
try {
|
|
try {
|
|
maybeCreateFakeParentDirectory(f);
|
|
maybeCreateFakeParentDirectory(f);
|
|
@@ -2215,110 +2291,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Delete an object. See {@link #delete(Path, boolean)}.
|
|
|
|
- * This call does not create any fake parent directory; that is
|
|
|
|
- * left to the caller.
|
|
|
|
- * @param status fileStatus object
|
|
|
|
- * @param recursive if path is a directory and set to
|
|
|
|
- * true, the directory is deleted else throws an exception. In
|
|
|
|
- * case of a file the recursive can be set to either true or false.
|
|
|
|
- * @return true, except in the corner cases of root directory deletion
|
|
|
|
- * @throws IOException due to inability to delete a directory or file.
|
|
|
|
- * @throws AmazonClientException on failures inside the AWS SDK
|
|
|
|
- */
|
|
|
|
- @Retries.RetryMixed
|
|
|
|
- private boolean innerDelete(S3AFileStatus status, boolean recursive)
|
|
|
|
- throws IOException, AmazonClientException {
|
|
|
|
- Path f = status.getPath();
|
|
|
|
- LOG.debug("Delete path {} - recursive {}", f, recursive);
|
|
|
|
- LOG.debug("Type = {}",
|
|
|
|
- status.isFile() ? "File"
|
|
|
|
- : (status.isEmptyDirectory() == Tristate.TRUE
|
|
|
|
- ? "Empty Directory"
|
|
|
|
- : "Directory"));
|
|
|
|
-
|
|
|
|
- String key = pathToKey(f);
|
|
|
|
-
|
|
|
|
- if (status.isDirectory()) {
|
|
|
|
- LOG.debug("delete: Path is a directory: {}", f);
|
|
|
|
- Preconditions.checkArgument(
|
|
|
|
- status.isEmptyDirectory() != Tristate.UNKNOWN,
|
|
|
|
- "File status must have directory emptiness computed");
|
|
|
|
-
|
|
|
|
- if (!key.endsWith("/")) {
|
|
|
|
- key = key + "/";
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (key.equals("/")) {
|
|
|
|
- return rejectRootDirectoryDelete(status, recursive);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (!recursive && status.isEmptyDirectory() == Tristate.FALSE) {
|
|
|
|
- throw new PathIsNotEmptyDirectoryException(f.toString());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (status.isEmptyDirectory() == Tristate.TRUE) {
|
|
|
|
- LOG.debug("Deleting fake empty directory {}", key);
|
|
|
|
- // HADOOP-13761 s3guard: retries here
|
|
|
|
- deleteObjectAtPath(f, key, false);
|
|
|
|
- } else {
|
|
|
|
- LOG.debug("Getting objects for directory prefix {} to delete", key);
|
|
|
|
-
|
|
|
|
- S3ListRequest request = createListObjectsRequest(key, null);
|
|
|
|
-
|
|
|
|
- S3ListResult objects = listObjects(request);
|
|
|
|
- List<DeleteObjectsRequest.KeyVersion> keys =
|
|
|
|
- new ArrayList<>(objects.getObjectSummaries().size());
|
|
|
|
- while (true) {
|
|
|
|
- for (S3ObjectSummary summary : objects.getObjectSummaries()) {
|
|
|
|
- keys.add(new DeleteObjectsRequest.KeyVersion(summary.getKey()));
|
|
|
|
- LOG.debug("Got object to delete {}", summary.getKey());
|
|
|
|
-
|
|
|
|
- if (keys.size() == MAX_ENTRIES_TO_DELETE) {
|
|
|
|
- // delete a single page of keys
|
|
|
|
- removeKeys(keys, false);
|
|
|
|
- keys.clear();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (objects.isTruncated()) {
|
|
|
|
- objects = continueListObjects(request, objects);
|
|
|
|
- } else {
|
|
|
|
- // there is no more data: delete the final set of entries.
|
|
|
|
- removeKeys(keys, false);
|
|
|
|
- break;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- try(DurationInfo ignored =
|
|
|
|
- new DurationInfo(LOG, false, "Delete metastore")) {
|
|
|
|
- metadataStore.deleteSubtree(f);
|
|
|
|
- }
|
|
|
|
- } else {
|
|
|
|
- LOG.debug("delete: Path is a file: {}", key);
|
|
|
|
- deleteObjectAtPath(f, key, true);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- return true;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Implements the specific logic to reject root directory deletion.
|
|
|
|
- * The caller must return the result of this call, rather than
|
|
|
|
- * attempt to continue with the delete operation: deleting root
|
|
|
|
- * directories is never allowed.
|
|
|
|
- * @param status filesystem status
|
|
|
|
- * @param recursive recursive flag from command
|
|
|
|
- * @return a return code for the operation
|
|
|
|
- */
|
|
|
|
- private boolean rejectRootDirectoryDelete(S3AFileStatus status,
|
|
|
|
- boolean recursive) {
|
|
|
|
- LOG.error("S3A: Cannot delete the {} root directory. Path: {}. Recursive: "
|
|
|
|
- + "{}", bucket, status.getPath(), recursive);
|
|
|
|
- return false;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Create a fake directory if required.
|
|
* Create a fake directory if required.
|
|
* That is: it is not the root path and the path does not exist.
|
|
* That is: it is not the root path and the path does not exist.
|
|
@@ -2394,8 +2366,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
|
|
|
DirListingMetadata dirMeta =
|
|
DirListingMetadata dirMeta =
|
|
S3Guard.listChildrenWithTtl(metadataStore, path, ttlTimeProvider);
|
|
S3Guard.listChildrenWithTtl(metadataStore, path, ttlTimeProvider);
|
|
- boolean allowAuthoritative = S3Guard.allowAuthoritative(f, this,
|
|
|
|
- allowAuthoritativeMetadataStore, allowAuthoritativePaths);
|
|
|
|
|
|
+ boolean allowAuthoritative = allowAuthoritative(f);
|
|
if (allowAuthoritative && dirMeta != null && dirMeta.isAuthoritative()) {
|
|
if (allowAuthoritative && dirMeta != null && dirMeta.isAuthoritative()) {
|
|
return S3Guard.dirMetaToStatuses(dirMeta);
|
|
return S3Guard.dirMetaToStatuses(dirMeta);
|
|
}
|
|
}
|
|
@@ -2423,6 +2394,19 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Is a path to be considered as authoritative?
|
|
|
|
+ * True iff there is an authoritative metastore or if there
|
|
|
|
+ * is a non-auth store with the supplied path under
|
|
|
|
+ * one of the paths declared as authoritative.
|
|
|
|
+ * @param path path
|
|
|
|
+ * @return true if the path is auth
|
|
|
|
+ */
|
|
|
|
+ protected boolean allowAuthoritative(final Path path) {
|
|
|
|
+ return S3Guard.allowAuthoritative(path, this,
|
|
|
|
+ allowAuthoritativeMetadataStore, allowAuthoritativePaths);
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Create a {@code ListObjectsRequest} request against this bucket,
|
|
* Create a {@code ListObjectsRequest} request against this bucket,
|
|
* with the maximum keys returned in a query set by {@link #maxKeys}.
|
|
* with the maximum keys returned in a query set by {@link #maxKeys}.
|
|
@@ -2628,8 +2612,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
// dest is also a directory, there's no difference.
|
|
// dest is also a directory, there's no difference.
|
|
// TODO After HADOOP-16085 the modification detection can be done with
|
|
// TODO After HADOOP-16085 the modification detection can be done with
|
|
// etags or object version instead of modTime
|
|
// etags or object version instead of modTime
|
|
- boolean allowAuthoritative = S3Guard.allowAuthoritative(f, this,
|
|
|
|
- allowAuthoritativeMetadataStore, allowAuthoritativePaths);
|
|
|
|
|
|
+ boolean allowAuthoritative = allowAuthoritative(f);
|
|
if (!pm.getFileStatus().isDirectory() &&
|
|
if (!pm.getFileStatus().isDirectory() &&
|
|
!allowAuthoritative) {
|
|
!allowAuthoritative) {
|
|
LOG.debug("Metadata for {} found in the non-auth metastore.", path);
|
|
LOG.debug("Metadata for {} found in the non-auth metastore.", path);
|
|
@@ -3399,7 +3382,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
path = path.getParent();
|
|
path = path.getParent();
|
|
}
|
|
}
|
|
try {
|
|
try {
|
|
- removeKeys(keysToRemove, true);
|
|
|
|
|
|
+ removeKeys(keysToRemove, true, null);
|
|
} catch(AmazonClientException | IOException e) {
|
|
} catch(AmazonClientException | IOException e) {
|
|
instrumentation.errorIgnored();
|
|
instrumentation.errorIgnored();
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
@@ -3728,8 +3711,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* These are then translated into {@link LocatedFileStatus} instances.
|
|
* These are then translated into {@link LocatedFileStatus} instances.
|
|
*
|
|
*
|
|
* This is essentially a nested and wrapped set of iterators, with some
|
|
* This is essentially a nested and wrapped set of iterators, with some
|
|
- * generator classes; an architecture which may become less convoluted
|
|
|
|
- * using lambda-expressions.
|
|
|
|
|
|
+ * generator classes.
|
|
* @param f a path
|
|
* @param f a path
|
|
* @param recursive if the subdirectories need to be traversed recursively
|
|
* @param recursive if the subdirectories need to be traversed recursively
|
|
*
|
|
*
|
|
@@ -3739,11 +3721,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* @throws IOException if any I/O error occurred
|
|
* @throws IOException if any I/O error occurred
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
- @Retries.OnceTranslated
|
|
|
|
|
|
+ @Retries.RetryTranslated
|
|
public RemoteIterator<LocatedFileStatus> listFiles(Path f,
|
|
public RemoteIterator<LocatedFileStatus> listFiles(Path f,
|
|
boolean recursive) throws FileNotFoundException, IOException {
|
|
boolean recursive) throws FileNotFoundException, IOException {
|
|
return toLocatedFileStatusIterator(innerListFiles(f, recursive,
|
|
return toLocatedFileStatusIterator(innerListFiles(f, recursive,
|
|
- new Listing.AcceptFilesOnly(qualify(f))));
|
|
|
|
|
|
+ new Listing.AcceptFilesOnly(qualify(f)), null, true));
|
|
}
|
|
}
|
|
|
|
|
|
private static RemoteIterator<LocatedFileStatus> toLocatedFileStatusIterator(
|
|
private static RemoteIterator<LocatedFileStatus> toLocatedFileStatusIterator(
|
|
@@ -3770,19 +3752,55 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
@Retries.RetryTranslated
|
|
@Retries.RetryTranslated
|
|
public RemoteIterator<S3ALocatedFileStatus> listFilesAndEmptyDirectories(
|
|
public RemoteIterator<S3ALocatedFileStatus> listFilesAndEmptyDirectories(
|
|
Path f, boolean recursive) throws IOException {
|
|
Path f, boolean recursive) throws IOException {
|
|
- return invoker.retry("list", f.toString(), true,
|
|
|
|
- () -> innerListFiles(f, recursive, new Listing.AcceptAllButS3nDirs()));
|
|
|
|
|
|
+ return innerListFiles(f, recursive, Listing.ACCEPT_ALL_BUT_S3N, null, true);
|
|
}
|
|
}
|
|
|
|
|
|
- @Retries.OnceTranslated
|
|
|
|
- private RemoteIterator<S3ALocatedFileStatus> innerListFiles(Path f, boolean
|
|
|
|
- recursive, Listing.FileStatusAcceptor acceptor) throws IOException {
|
|
|
|
|
|
+ /**
|
|
|
|
+ * List files under the path.
|
|
|
|
+ * <ol>
|
|
|
|
+ * <li>
|
|
|
|
+ * If the path is authoritative on the client,
|
|
|
|
+ * only S3Guard will be queried.
|
|
|
|
+ * </li>
|
|
|
|
+ * <li>
|
|
|
|
+ * Otherwise, the S3Guard values are returned first, then the S3
|
|
|
|
+ * entries will be retrieved and returned if not already listed.</li>
|
|
|
|
+ * <li>
|
|
|
|
+ * when collectTombstones} is true, S3Guard tombstones will
|
|
|
|
+ * be used to filter out deleted files.
|
|
|
|
+ * They MUST be used for normal listings; it is only for
|
|
|
|
+ * deletion and low-level operations that they MAY be bypassed.
|
|
|
|
+ * </li>
|
|
|
|
+ * <li>
|
|
|
|
+ * The optional {@code status} parameter will be used to skip the
|
|
|
|
+ * initial getFileStatus call.
|
|
|
|
+ * </li>
|
|
|
|
+ * </ol>
|
|
|
|
+ *
|
|
|
|
+ * @param f path
|
|
|
|
+ * @param recursive recursive listing?
|
|
|
|
+ * @param acceptor file status filter
|
|
|
|
+ * @param status optional status of path to list.
|
|
|
|
+ * @param collectTombstones should tombstones be collected from S3Guard?
|
|
|
|
+ * @return an iterator over the listing.
|
|
|
|
+ * @throws IOException failure
|
|
|
|
+ */
|
|
|
|
+ @Retries.RetryTranslated
|
|
|
|
+ private RemoteIterator<S3ALocatedFileStatus> innerListFiles(
|
|
|
|
+ final Path f,
|
|
|
|
+ final boolean recursive,
|
|
|
|
+ final Listing.FileStatusAcceptor acceptor,
|
|
|
|
+ final S3AFileStatus status,
|
|
|
|
+ final boolean collectTombstones) throws IOException {
|
|
entryPoint(INVOCATION_LIST_FILES);
|
|
entryPoint(INVOCATION_LIST_FILES);
|
|
Path path = qualify(f);
|
|
Path path = qualify(f);
|
|
LOG.debug("listFiles({}, {})", path, recursive);
|
|
LOG.debug("listFiles({}, {})", path, recursive);
|
|
try {
|
|
try {
|
|
- // lookup dir triggers existence check
|
|
|
|
- final S3AFileStatus fileStatus = (S3AFileStatus) getFileStatus(path);
|
|
|
|
|
|
+ // if a status was given, that is used, otherwise
|
|
|
|
+ // call getFileStatus, which triggers an existence check
|
|
|
|
+ final S3AFileStatus fileStatus = status != null
|
|
|
|
+ ? status
|
|
|
|
+ : (S3AFileStatus) getFileStatus(path);
|
|
if (fileStatus.isFile()) {
|
|
if (fileStatus.isFile()) {
|
|
// simple case: File
|
|
// simple case: File
|
|
LOG.debug("Path is a file");
|
|
LOG.debug("Path is a file");
|
|
@@ -3796,8 +3814,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
key, delimiter);
|
|
key, delimiter);
|
|
final RemoteIterator<S3AFileStatus> cachedFilesIterator;
|
|
final RemoteIterator<S3AFileStatus> cachedFilesIterator;
|
|
final Set<Path> tombstones;
|
|
final Set<Path> tombstones;
|
|
- boolean allowAuthoritative = S3Guard.allowAuthoritative(f, this,
|
|
|
|
- allowAuthoritativeMetadataStore, allowAuthoritativePaths);
|
|
|
|
|
|
+ boolean allowAuthoritative = allowAuthoritative(f);
|
|
if (recursive) {
|
|
if (recursive) {
|
|
final PathMetadata pm = metadataStore.get(path, true);
|
|
final PathMetadata pm = metadataStore.get(path, true);
|
|
// shouldn't need to check pm.isDeleted() because that will have
|
|
// shouldn't need to check pm.isDeleted() because that will have
|
|
@@ -3829,7 +3846,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
ACCEPT_ALL,
|
|
ACCEPT_ALL,
|
|
acceptor,
|
|
acceptor,
|
|
cachedFilesIterator)),
|
|
cachedFilesIterator)),
|
|
- tombstones);
|
|
|
|
|
|
+ collectTombstones ? tombstones : null);
|
|
}
|
|
}
|
|
} catch (AmazonClientException e) {
|
|
} catch (AmazonClientException e) {
|
|
// TODO S3Guard: retry on file not found exception
|
|
// TODO S3Guard: retry on file not found exception
|
|
@@ -3890,8 +3907,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
final RemoteIterator<S3AFileStatus> cachedFileStatusIterator =
|
|
final RemoteIterator<S3AFileStatus> cachedFileStatusIterator =
|
|
listing.createProvidedFileStatusIterator(
|
|
listing.createProvidedFileStatusIterator(
|
|
S3Guard.dirMetaToStatuses(meta), filter, acceptor);
|
|
S3Guard.dirMetaToStatuses(meta), filter, acceptor);
|
|
- boolean allowAuthoritative = S3Guard.allowAuthoritative(f, this,
|
|
|
|
- allowAuthoritativeMetadataStore, allowAuthoritativePaths);
|
|
|
|
|
|
+ boolean allowAuthoritative = allowAuthoritative(f);
|
|
return (allowAuthoritative && meta != null
|
|
return (allowAuthoritative && meta != null
|
|
&& meta.isAuthoritative())
|
|
&& meta.isAuthoritative())
|
|
? listing.createLocatedFileStatusIterator(
|
|
? listing.createLocatedFileStatusIterator(
|
|
@@ -3918,7 +3934,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
return new S3ALocatedFileStatus(status,
|
|
return new S3ALocatedFileStatus(status,
|
|
status.isFile() ?
|
|
status.isFile() ?
|
|
getFileBlockLocations(status, 0, status.getLen())
|
|
getFileBlockLocations(status, 0, status.getLen())
|
|
- : null, status.getETag(), status.getVersionId());
|
|
|
|
|
|
+ : null);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|