|
@@ -258,6 +258,7 @@ import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDura
|
|
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier;
|
|
|
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
|
|
import static org.apache.hadoop.util.Preconditions.checkArgument;
|
|
|
+import static org.apache.hadoop.util.functional.RemoteIterators.foreach;
|
|
|
import static org.apache.hadoop.util.functional.RemoteIterators.typeCastingRemoteIterator;
|
|
|
|
|
|
/**
|
|
@@ -384,6 +385,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
private SignerManager signerManager;
|
|
|
private S3AInternals s3aInternals;
|
|
|
|
|
|
+ /**
|
|
|
+ * Do directory operations purge pending uploads?
|
|
|
+ */
|
|
|
+ private boolean dirOperationsPurgeUploads;
|
|
|
+
|
|
|
/**
|
|
|
* Page size for deletions.
|
|
|
*/
|
|
@@ -565,6 +571,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
//check but do not store the block size
|
|
|
longBytesOption(conf, FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE, 1);
|
|
|
enableMultiObjectsDelete = conf.getBoolean(ENABLE_MULTI_DELETE, true);
|
|
|
+ // should the delete also purge uploads.
|
|
|
+ dirOperationsPurgeUploads = conf.getBoolean(DIRECTORY_OPERATIONS_PURGE_UPLOADS,
|
|
|
+ DIRECTORY_OPERATIONS_PURGE_UPLOADS_DEFAULT);
|
|
|
|
|
|
this.prefetchEnabled = conf.getBoolean(PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT);
|
|
|
long prefetchBlockSizeLong =
|
|
@@ -1230,7 +1239,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
purgeBefore);
|
|
|
invoker.retry("Purging multipart uploads", bucket, true,
|
|
|
() -> {
|
|
|
- MultipartUtils.UploadIterator uploadIterator =
|
|
|
+ RemoteIterator<MultipartUpload> uploadIterator =
|
|
|
MultipartUtils.listMultipartUploads(createStoreContext(), s3Client, null, maxKeys);
|
|
|
|
|
|
while (uploadIterator.hasNext()) {
|
|
@@ -2283,12 +2292,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
|
|
|
// Initiate the rename.
|
|
|
// this will call back into this class via the rename callbacks
|
|
|
+ final StoreContext storeContext = createStoreContext();
|
|
|
RenameOperation renameOperation = new RenameOperation(
|
|
|
- createStoreContext(),
|
|
|
+ storeContext,
|
|
|
src, srcKey, p.getLeft(),
|
|
|
dst, dstKey, p.getRight(),
|
|
|
- new OperationCallbacksImpl(),
|
|
|
- pageSize);
|
|
|
+ new OperationCallbacksImpl(storeContext),
|
|
|
+ pageSize,
|
|
|
+ dirOperationsPurgeUploads);
|
|
|
return renameOperation.execute();
|
|
|
}
|
|
|
|
|
@@ -2309,8 +2320,19 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
/** Audit Span at time of creation. */
|
|
|
private final AuditSpan auditSpan;
|
|
|
|
|
|
- private OperationCallbacksImpl() {
|
|
|
- auditSpan = getActiveAuditSpan();
|
|
|
+ private final StoreContext storeContext;
|
|
|
+
|
|
|
+ private OperationCallbacksImpl(final StoreContext storeContext) {
|
|
|
+ this.storeContext = requireNonNull(storeContext);
|
|
|
+ this.auditSpan = storeContext.getActiveAuditSpan();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get the audit span.
|
|
|
+ * @return the span
|
|
|
+ */
|
|
|
+ private AuditSpan getAuditSpan() {
|
|
|
+ return auditSpan;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -2410,7 +2432,29 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
Listing.ACCEPT_ALL_BUT_S3N,
|
|
|
auditSpan));
|
|
|
}
|
|
|
- }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Abort multipart uploads under a path.
|
|
|
+ * @param prefix prefix for uploads to abort
|
|
|
+ * @return a count of aborts
|
|
|
+ * @throws IOException trouble; FileNotFoundExceptions are swallowed.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ @Retries.RetryTranslated
|
|
|
+ public long abortMultipartUploadsUnderPrefix(String prefix)
|
|
|
+ throws IOException {
|
|
|
+ getAuditSpan().activate();
|
|
|
+ // this deactivates the audit span somehow
|
|
|
+ final RemoteIterator<MultipartUpload> uploads =
|
|
|
+ S3AFileSystem.this.listUploadsUnderPrefix(storeContext, prefix);
|
|
|
+ // so reactivate it.
|
|
|
+ getAuditSpan().activate();
|
|
|
+ return foreach(uploads, upload ->
|
|
|
+ invoker.retry("Aborting multipart commit", upload.key(), true, () ->
|
|
|
+ S3AFileSystem.this.abortMultipartUpload(upload)));
|
|
|
+ }
|
|
|
+
|
|
|
+ } // end OperationCallbacksImpl
|
|
|
|
|
|
/**
|
|
|
* Callbacks from {@link Listing}.
|
|
@@ -3371,14 +3415,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
// span covers delete, getFileStatus, fake directory operations.
|
|
|
try (AuditSpan span = createSpan(INVOCATION_DELETE.getSymbol(),
|
|
|
path.toString(), null)) {
|
|
|
+ // SC will include active span
|
|
|
+ final StoreContext storeContext = createStoreContext();
|
|
|
boolean outcome = trackDuration(getDurationTrackerFactory(),
|
|
|
INVOCATION_DELETE.getSymbol(),
|
|
|
new DeleteOperation(
|
|
|
- createStoreContext(),
|
|
|
+ storeContext,
|
|
|
innerGetFileStatus(path, true, StatusProbeEnum.ALL),
|
|
|
recursive,
|
|
|
- new OperationCallbacksImpl(),
|
|
|
- pageSize));
|
|
|
+ new OperationCallbacksImpl(storeContext),
|
|
|
+ pageSize,
|
|
|
+ dirOperationsPurgeUploads));
|
|
|
if (outcome) {
|
|
|
try {
|
|
|
maybeCreateFakeParentDirectory(path);
|
|
@@ -5151,13 +5198,39 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
@InterfaceAudience.Private
|
|
|
@Retries.RetryTranslated
|
|
|
@AuditEntryPoint
|
|
|
- public MultipartUtils.UploadIterator listUploads(@Nullable String prefix)
|
|
|
+ public RemoteIterator<MultipartUpload> listUploads(@Nullable String prefix)
|
|
|
+ throws IOException {
|
|
|
+ // span is picked up retained in the listing.
|
|
|
+ checkNotClosed();
|
|
|
+ try (AuditSpan span = createSpan(MULTIPART_UPLOAD_LIST.getSymbol(),
|
|
|
+ prefix, null)) {
|
|
|
+ return listUploadsUnderPrefix(createStoreContext(), prefix);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * List any pending multipart uploads whose keys begin with prefix, using
|
|
|
+ * an iterator that can handle an unlimited number of entries.
|
|
|
+ * See {@link #listMultipartUploads(String)} for a non-iterator version of
|
|
|
+ * this.
|
|
|
+ * @param storeContext store conext.
|
|
|
+ * @param prefix optional key prefix to search
|
|
|
+ * @return Iterator over multipart uploads.
|
|
|
+ * @throws IOException on failure
|
|
|
+ */
|
|
|
+ @InterfaceAudience.Private
|
|
|
+ @Retries.RetryTranslated
|
|
|
+ public RemoteIterator<MultipartUpload> listUploadsUnderPrefix(
|
|
|
+ final StoreContext storeContext,
|
|
|
+ final @Nullable String prefix)
|
|
|
throws IOException {
|
|
|
// span is picked up retained in the listing.
|
|
|
- return trackDurationAndSpan(MULTIPART_UPLOAD_LIST, prefix, null, () ->
|
|
|
- MultipartUtils.listMultipartUploads(
|
|
|
- createStoreContext(), s3Client, prefix, maxKeys
|
|
|
- ));
|
|
|
+ String p = prefix;
|
|
|
+ if (prefix != null && !prefix.isEmpty() && !prefix.endsWith("/")) {
|
|
|
+ p = prefix + "/";
|
|
|
+ }
|
|
|
+ // duration tracking is done in iterator.
|
|
|
+ return MultipartUtils.listMultipartUploads(storeContext, s3Client, p, maxKeys);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -5179,9 +5252,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
}
|
|
|
String p = prefix;
|
|
|
return invoker.retry("listMultipartUploads", p, true, () -> {
|
|
|
- ListMultipartUploadsRequest.Builder requestBuilder = getRequestFactory()
|
|
|
- .newListMultipartUploadsRequestBuilder(p);
|
|
|
- return s3Client.listMultipartUploads(requestBuilder.build()).uploads();
|
|
|
+ final ListMultipartUploadsRequest request = getRequestFactory()
|
|
|
+ .newListMultipartUploadsRequestBuilder(p).build();
|
|
|
+ return trackDuration(getInstrumentation(), MULTIPART_UPLOAD_LIST.getSymbol(), () ->
|
|
|
+ s3Client.listMultipartUploads(request).uploads());
|
|
|
});
|
|
|
}
|
|
|
|
|
@@ -5190,37 +5264,35 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
* Retry policy: none.
|
|
|
* @param destKey destination key
|
|
|
* @param uploadId Upload ID
|
|
|
+ * @throws IOException IO failure, including any uprated SdkException
|
|
|
*/
|
|
|
- @Retries.OnceRaw
|
|
|
- void abortMultipartUpload(String destKey, String uploadId) {
|
|
|
- LOG.info("Aborting multipart upload {} to {}", uploadId, destKey);
|
|
|
- s3Client.abortMultipartUpload(
|
|
|
- getRequestFactory().newAbortMultipartUploadRequestBuilder(
|
|
|
- destKey,
|
|
|
- uploadId).build());
|
|
|
+ @Retries.OnceTranslated
|
|
|
+ public void abortMultipartUpload(String destKey, String uploadId) throws IOException {
|
|
|
+ LOG.debug("Aborting multipart upload {} to {}", uploadId, destKey);
|
|
|
+ trackDuration(getInstrumentation(), OBJECT_MULTIPART_UPLOAD_ABORTED.getSymbol(), () ->
|
|
|
+ s3Client.abortMultipartUpload(
|
|
|
+ getRequestFactory().newAbortMultipartUploadRequestBuilder(
|
|
|
+ destKey,
|
|
|
+ uploadId).build()));
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Abort a multipart upload.
|
|
|
* Retry policy: none.
|
|
|
* @param upload the listed upload to abort.
|
|
|
+ * @throws IOException IO failure, including any uprated SdkException
|
|
|
*/
|
|
|
- @Retries.OnceRaw
|
|
|
- void abortMultipartUpload(MultipartUpload upload) {
|
|
|
- String destKey;
|
|
|
- String uploadId;
|
|
|
- destKey = upload.key();
|
|
|
- uploadId = upload.uploadId();
|
|
|
- if (LOG.isInfoEnabled()) {
|
|
|
+ @Retries.OnceTranslated
|
|
|
+ public void abortMultipartUpload(MultipartUpload upload) throws IOException {
|
|
|
+ String destKey = upload.key();
|
|
|
+ String uploadId = upload.uploadId();
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
|
|
|
LOG.debug("Aborting multipart upload {} to {} initiated by {} on {}",
|
|
|
uploadId, destKey, upload.initiator(),
|
|
|
df.format(Date.from(upload.initiated())));
|
|
|
}
|
|
|
- s3Client.abortMultipartUpload(
|
|
|
- getRequestFactory().newAbortMultipartUploadRequestBuilder(
|
|
|
- destKey,
|
|
|
- uploadId).build());
|
|
|
+ abortMultipartUpload(destKey, uploadId);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -5266,13 +5338,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
case STORE_CAPABILITY_DIRECTORY_MARKER_AWARE:
|
|
|
return true;
|
|
|
|
|
|
+ // Do directory operations purge uploads.
|
|
|
+ case DIRECTORY_OPERATIONS_PURGE_UPLOADS:
|
|
|
+ return dirOperationsPurgeUploads;
|
|
|
+
|
|
|
// etags are avaialable in listings, but they
|
|
|
// are not consistent across renames.
|
|
|
// therefore, only availability is declared
|
|
|
case CommonPathCapabilities.ETAGS_AVAILABLE:
|
|
|
return true;
|
|
|
|
|
|
- /*
|
|
|
+ /*
|
|
|
* Marker policy capabilities are handed off.
|
|
|
*/
|
|
|
case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_KEEP:
|
|
@@ -5545,7 +5621,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
throws IOException {
|
|
|
createSpan("marker-tool-scan", target,
|
|
|
null);
|
|
|
- return new MarkerToolOperationsImpl(new OperationCallbacksImpl());
|
|
|
+ return new MarkerToolOperationsImpl(new OperationCallbacksImpl(createStoreContext()));
|
|
|
}
|
|
|
|
|
|
/**
|