|
@@ -28,6 +28,8 @@ import com.amazonaws.services.s3.model.DeleteObjectRequest;
|
|
|
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
|
|
import com.amazonaws.services.s3.model.DeleteObjectsResult;
|
|
|
import com.amazonaws.services.s3.model.ListObjectsRequest;
|
|
|
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
|
|
|
+import com.amazonaws.services.s3.model.ListObjectsV2Result;
|
|
|
import com.amazonaws.services.s3.model.ObjectListing;
|
|
|
import com.amazonaws.services.s3.model.PutObjectRequest;
|
|
|
import com.amazonaws.services.s3.model.PutObjectResult;
|
|
@@ -109,8 +111,10 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /** Map of key to delay -> time it was deleted + object summary (object
|
|
|
- * summary is null for prefixes. */
|
|
|
+ /**
|
|
|
+ * Map of key to delay -> time it was deleted + object summary (object summary
|
|
|
+ * is null for prefixes.
|
|
|
+ */
|
|
|
private Map<String, Delete> delayedDeletes = new HashMap<>();
|
|
|
|
|
|
/** Map of key to delay -> time it was created. */
|
|
@@ -196,17 +200,29 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
|
|
|
return super.putObject(putObjectRequest);
|
|
|
}
|
|
|
|
|
|
- /* We should only need to override this version of listObjects() */
|
|
|
+ /* We should only need to override these versions of listObjects() */
|
|
|
@Override
|
|
|
public ObjectListing listObjects(ListObjectsRequest listObjectsRequest)
|
|
|
throws AmazonClientException, AmazonServiceException {
|
|
|
LOG.debug("prefix {}", listObjectsRequest.getPrefix());
|
|
|
ObjectListing listing = super.listObjects(listObjectsRequest);
|
|
|
- listing = filterListObjects(listObjectsRequest, listing);
|
|
|
+ listing = filterListObjects(listing);
|
|
|
listing = restoreListObjects(listObjectsRequest, listing);
|
|
|
return listing;
|
|
|
}
|
|
|
|
|
|
+ /* We should only need to override these versions of listObjects() */
|
|
|
+ @Override
|
|
|
+ public ListObjectsV2Result listObjectsV2(ListObjectsV2Request request)
|
|
|
+ throws AmazonClientException, AmazonServiceException {
|
|
|
+ LOG.debug("prefix {}", request.getPrefix());
|
|
|
+ ListObjectsV2Result listing = super.listObjectsV2(request);
|
|
|
+ listing = filterListObjectsV2(listing);
|
|
|
+ listing = restoreListObjectsV2(request, listing);
|
|
|
+ return listing;
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
private void addSummaryIfNotPresent(List<S3ObjectSummary> list,
|
|
|
S3ObjectSummary item) {
|
|
|
// Behavior of S3ObjectSummary
|
|
@@ -282,21 +298,58 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
|
|
|
// recursive list has no delimiter, returns everything that matches a
|
|
|
// prefix.
|
|
|
boolean recursiveObjectList = !("/".equals(request.getDelimiter()));
|
|
|
+ String prefix = request.getPrefix();
|
|
|
+
|
|
|
+ restoreDeleted(outputList, outputPrefixes, recursiveObjectList, prefix);
|
|
|
+ return new CustomObjectListing(rawListing, outputList, outputPrefixes);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * V2 list API variant of
|
|
|
+ * {@link #restoreListObjects(ListObjectsRequest, ObjectListing)}.
|
|
|
+ * @param request original v2 list request
|
|
|
+ * @param result raw s3 result
|
|
|
+ */
|
|
|
+ private ListObjectsV2Result restoreListObjectsV2(ListObjectsV2Request request,
|
|
|
+ ListObjectsV2Result result) {
|
|
|
+ List<S3ObjectSummary> outputList = result.getObjectSummaries();
|
|
|
+ List<String> outputPrefixes = result.getCommonPrefixes();
|
|
|
+ // recursive list has no delimiter, returns everything that matches a
|
|
|
+ // prefix.
|
|
|
+ boolean recursiveObjectList = !("/".equals(request.getDelimiter()));
|
|
|
+ String prefix = request.getPrefix();
|
|
|
+
|
|
|
+ restoreDeleted(outputList, outputPrefixes, recursiveObjectList, prefix);
|
|
|
+ return new CustomListObjectsV2Result(result, outputList, outputPrefixes);
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Main logic for
|
|
|
+ * {@link #restoreListObjects(ListObjectsRequest, ObjectListing)} and
|
|
|
+ * the v2 variant above.
|
|
|
+ * @param summaries object summary list to modify.
|
|
|
+ * @param prefixes prefix list to modify
|
|
|
+ * @param recursive true if recursive list request
|
|
|
+ * @param prefix prefix for original list request
|
|
|
+ */
|
|
|
+ private void restoreDeleted(List<S3ObjectSummary> summaries,
|
|
|
+ List<String> prefixes, boolean recursive, String prefix) {
|
|
|
|
|
|
// Go through all deleted keys
|
|
|
for (String key : new HashSet<>(delayedDeletes.keySet())) {
|
|
|
Delete delete = delayedDeletes.get(key);
|
|
|
if (isKeyDelayed(delete.time(), key)) {
|
|
|
- if (isDescendant(request.getPrefix(), key, recursiveObjectList)) {
|
|
|
+ if (isDescendant(prefix, key, recursive)) {
|
|
|
if (delete.summary() != null) {
|
|
|
- addSummaryIfNotPresent(outputList, delete.summary());
|
|
|
+ addSummaryIfNotPresent(summaries, delete.summary());
|
|
|
}
|
|
|
}
|
|
|
// Non-recursive list has delimiter: will return rolled-up prefixes for
|
|
|
// all keys that are not direct children
|
|
|
- if (!recursiveObjectList) {
|
|
|
- if (isDescendant(request.getPrefix(), key, true)) {
|
|
|
- addPrefixIfNotPresent(outputPrefixes, request.getPrefix(), key);
|
|
|
+ if (!recursive) {
|
|
|
+ if (isDescendant(prefix, key, true)) {
|
|
|
+ addPrefixIfNotPresent(prefixes, prefix, key);
|
|
|
}
|
|
|
}
|
|
|
} else {
|
|
@@ -304,31 +357,52 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
|
|
|
delayedDeletes.remove(key);
|
|
|
}
|
|
|
}
|
|
|
+ }
|
|
|
+
|
|
|
+ private ObjectListing filterListObjects(ObjectListing rawListing) {
|
|
|
+
|
|
|
+ // Filter object listing
|
|
|
+ List<S3ObjectSummary> outputList = filterSummaries(
|
|
|
+ rawListing.getObjectSummaries());
|
|
|
+
|
|
|
+ // Filter prefixes (directories)
|
|
|
+ List<String> outputPrefixes = filterPrefixes(
|
|
|
+ rawListing.getCommonPrefixes());
|
|
|
|
|
|
return new CustomObjectListing(rawListing, outputList, outputPrefixes);
|
|
|
}
|
|
|
|
|
|
- private ObjectListing filterListObjects(ListObjectsRequest request,
|
|
|
- ObjectListing rawListing) {
|
|
|
-
|
|
|
+ private ListObjectsV2Result filterListObjectsV2(ListObjectsV2Result raw) {
|
|
|
// Filter object listing
|
|
|
+ List<S3ObjectSummary> outputList = filterSummaries(
|
|
|
+ raw.getObjectSummaries());
|
|
|
+
|
|
|
+ // Filter prefixes (directories)
|
|
|
+ List<String> outputPrefixes = filterPrefixes(raw.getCommonPrefixes());
|
|
|
+
|
|
|
+ return new CustomListObjectsV2Result(raw, outputList, outputPrefixes);
|
|
|
+ }
|
|
|
+
|
|
|
+ private List<S3ObjectSummary> filterSummaries(
|
|
|
+ List<S3ObjectSummary> summaries) {
|
|
|
List<S3ObjectSummary> outputList = new ArrayList<>();
|
|
|
- for (S3ObjectSummary s : rawListing.getObjectSummaries()) {
|
|
|
+ for (S3ObjectSummary s : summaries) {
|
|
|
String key = s.getKey();
|
|
|
if (!isKeyDelayed(delayedPutKeys.get(key), key)) {
|
|
|
outputList.add(s);
|
|
|
}
|
|
|
}
|
|
|
+ return outputList;
|
|
|
+ }
|
|
|
|
|
|
- // Filter prefixes (directories)
|
|
|
+ private List<String> filterPrefixes(List<String> prefixes) {
|
|
|
List<String> outputPrefixes = new ArrayList<>();
|
|
|
- for (String key : rawListing.getCommonPrefixes()) {
|
|
|
+ for (String key : prefixes) {
|
|
|
if (!isKeyDelayed(delayedPutKeys.get(key), key)) {
|
|
|
outputPrefixes.add(key);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- return new CustomObjectListing(rawListing, outputList, outputPrefixes);
|
|
|
+ return outputPrefixes;
|
|
|
}
|
|
|
|
|
|
private boolean isKeyDelayed(Long enqueueTime, String key) {
|
|
@@ -342,7 +416,7 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
|
|
|
delayedDeletes.remove(key);
|
|
|
LOG.debug("no longer delaying {}", key);
|
|
|
return false;
|
|
|
- } else {
|
|
|
+ } else {
|
|
|
LOG.info("delaying {}", key);
|
|
|
return true;
|
|
|
}
|
|
@@ -431,4 +505,37 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
|
|
|
return customPrefixes;
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private static class CustomListObjectsV2Result extends ListObjectsV2Result {
|
|
|
+
|
|
|
+ private final List<S3ObjectSummary> customListing;
|
|
|
+ private final List<String> customPrefixes;
|
|
|
+
|
|
|
+ CustomListObjectsV2Result(ListObjectsV2Result raw,
|
|
|
+ List<S3ObjectSummary> customListing, List<String> customPrefixes) {
|
|
|
+ super();
|
|
|
+ this.customListing = customListing;
|
|
|
+ this.customPrefixes = customPrefixes;
|
|
|
+
|
|
|
+ this.setBucketName(raw.getBucketName());
|
|
|
+ this.setCommonPrefixes(raw.getCommonPrefixes());
|
|
|
+ this.setDelimiter(raw.getDelimiter());
|
|
|
+ this.setEncodingType(raw.getEncodingType());
|
|
|
+ this.setStartAfter(raw.getStartAfter());
|
|
|
+ this.setMaxKeys(raw.getMaxKeys());
|
|
|
+ this.setContinuationToken(raw.getContinuationToken());
|
|
|
+ this.setPrefix(raw.getPrefix());
|
|
|
+ this.setTruncated(raw.isTruncated());
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public List<S3ObjectSummary> getObjectSummaries() {
|
|
|
+ return customListing;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public List<String> getCommonPrefixes() {
|
|
|
+ return customPrefixes;
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|