|
@@ -36,10 +36,12 @@ import java.util.Set;
|
|
|
import java.util.TreeMap;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
+import java.util.concurrent.atomic.AtomicLong;
|
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
|
import com.amazonaws.AmazonClientException;
|
|
|
+import com.amazonaws.AmazonServiceException;
|
|
|
import com.amazonaws.auth.AWSCredentialsProvider;
|
|
|
import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
|
|
|
import com.amazonaws.services.dynamodbv2.document.BatchWriteItemOutcome;
|
|
@@ -77,12 +79,12 @@ import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
|
|
|
+import org.apache.hadoop.fs.s3a.AWSServiceThrottledException;
|
|
|
import org.apache.hadoop.fs.s3a.Constants;
|
|
|
import org.apache.hadoop.fs.s3a.Invoker;
|
|
|
import org.apache.hadoop.fs.s3a.Retries;
|
|
|
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
|
|
import org.apache.hadoop.fs.s3a.S3AInstrumentation;
|
|
|
-import org.apache.hadoop.fs.s3a.S3ARetryPolicy;
|
|
|
import org.apache.hadoop.fs.s3a.S3AUtils;
|
|
|
import org.apache.hadoop.fs.s3a.Tristate;
|
|
|
import org.apache.hadoop.fs.s3a.auth.RolePolicies;
|
|
@@ -198,10 +200,6 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
public static final String E_INCOMPATIBLE_VERSION
|
|
|
= "Database table is from an incompatible S3Guard version.";
|
|
|
|
|
|
- /** Initial delay for retries when batched operations get throttled by
|
|
|
- * DynamoDB. Value is {@value} msec. */
|
|
|
- public static final long MIN_RETRY_SLEEP_MSEC = 100;
|
|
|
-
|
|
|
@VisibleForTesting
|
|
|
static final String DESCRIPTION
|
|
|
= "S3Guard metadata store in DynamoDB";
|
|
@@ -214,6 +212,13 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
@VisibleForTesting
|
|
|
static final String TABLE = "table";
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ static final String HINT_DDB_IOPS_TOO_LOW
|
|
|
+ = " This may be because the write threshold of DynamoDB is set too low.";
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ static final String THROTTLING = "Throttling";
|
|
|
+
|
|
|
private static ValueMap deleteTrackingValueMap =
|
|
|
new ValueMap().withBoolean(":false", false);
|
|
|
|
|
@@ -226,7 +231,14 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
private Configuration conf;
|
|
|
private String username;
|
|
|
|
|
|
- private RetryPolicy dataAccessRetryPolicy;
|
|
|
+ /**
|
|
|
+ * This policy is mostly for batched writes, not for processing
|
|
|
+ * exceptions in invoke() calls.
|
|
|
+ * It also has a role purpose in {@link #getVersionMarkerItem()};
|
|
|
+ * look at that method for the details.
|
|
|
+ */
|
|
|
+ private RetryPolicy batchWriteRetryPolicy;
|
|
|
+
|
|
|
private S3AInstrumentation.S3GuardInstrumentation instrumentation;
|
|
|
|
|
|
/** Owner FS: only valid if configured with an owner FS. */
|
|
@@ -237,8 +249,15 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
Invoker.NO_OP
|
|
|
);
|
|
|
|
|
|
- /** Data access can have its own policies. */
|
|
|
- private Invoker dataAccess;
|
|
|
+ /** Invoker for read operations. */
|
|
|
+ private Invoker readOp;
|
|
|
+
|
|
|
+ /** Invoker for write operations. */
|
|
|
+ private Invoker writeOp;
|
|
|
+
|
|
|
+ private final AtomicLong readThrottleEvents = new AtomicLong(0);
|
|
|
+ private final AtomicLong writeThrottleEvents = new AtomicLong(0);
|
|
|
+ private final AtomicLong batchWriteCapacityExceededEvents = new AtomicLong(0);
|
|
|
|
|
|
/**
|
|
|
* Total limit on the number of throttle events after which
|
|
@@ -292,10 +311,8 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
Preconditions.checkNotNull(fs, "Null filesystem");
|
|
|
Preconditions.checkArgument(fs instanceof S3AFileSystem,
|
|
|
"DynamoDBMetadataStore only supports S3A filesystem.");
|
|
|
- owner = (S3AFileSystem) fs;
|
|
|
- instrumentation = owner.getInstrumentation().getS3GuardInstrumentation();
|
|
|
+ bindToOwnerFilesystem((S3AFileSystem) fs);
|
|
|
final String bucket = owner.getBucket();
|
|
|
- conf = owner.getConf();
|
|
|
String confRegion = conf.getTrimmed(S3GUARD_DDB_REGION_KEY);
|
|
|
if (!StringUtils.isEmpty(confRegion)) {
|
|
|
region = confRegion;
|
|
@@ -316,7 +333,6 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
}
|
|
|
LOG.debug("Inferring DynamoDB region from S3 bucket: {}", region);
|
|
|
}
|
|
|
- username = owner.getUsername();
|
|
|
credentials = owner.shareCredentials("s3guard");
|
|
|
dynamoDB = createDynamoDB(conf, region, bucket, credentials);
|
|
|
|
|
@@ -325,7 +341,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
initDataAccessRetries(conf);
|
|
|
|
|
|
// set up a full retry policy
|
|
|
- invoker = new Invoker(new S3ARetryPolicy(conf),
|
|
|
+ invoker = new Invoker(new S3GuardDataAccessRetryPolicy(conf),
|
|
|
this::retryEvent
|
|
|
);
|
|
|
|
|
@@ -334,6 +350,20 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
instrumentation.initialized();
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Declare that this table is owned by the specific S3A FS instance.
|
|
|
+ * This will bind some fields to the values provided by the owner,
|
|
|
+ * including wiring up the instrumentation.
|
|
|
+ * @param fs owner filesystem
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ void bindToOwnerFilesystem(final S3AFileSystem fs) {
|
|
|
+ owner = fs;
|
|
|
+ conf = owner.getConf();
|
|
|
+ instrumentation = owner.getInstrumentation().getS3GuardInstrumentation();
|
|
|
+ username = owner.getUsername();
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Performs one-time initialization of the metadata store via configuration.
|
|
|
*
|
|
@@ -382,16 +412,23 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
/**
|
|
|
* Set retry policy. This is driven by the value of
|
|
|
* {@link Constants#S3GUARD_DDB_MAX_RETRIES} with an exponential backoff
|
|
|
- * between each attempt of {@link #MIN_RETRY_SLEEP_MSEC} milliseconds.
|
|
|
+ * between each attempt of {@link Constants#S3GUARD_DDB_THROTTLE_RETRY_INTERVAL}
|
|
|
+ * milliseconds.
|
|
|
* @param config configuration for data access
|
|
|
*/
|
|
|
private void initDataAccessRetries(Configuration config) {
|
|
|
- int maxRetries = config.getInt(S3GUARD_DDB_MAX_RETRIES,
|
|
|
- S3GUARD_DDB_MAX_RETRIES_DEFAULT);
|
|
|
- dataAccessRetryPolicy = RetryPolicies
|
|
|
- .exponentialBackoffRetry(maxRetries, MIN_RETRY_SLEEP_MSEC,
|
|
|
+ batchWriteRetryPolicy = RetryPolicies
|
|
|
+ .exponentialBackoffRetry(
|
|
|
+ config.getInt(S3GUARD_DDB_MAX_RETRIES,
|
|
|
+ S3GUARD_DDB_MAX_RETRIES_DEFAULT),
|
|
|
+ conf.getTimeDuration(S3GUARD_DDB_THROTTLE_RETRY_INTERVAL,
|
|
|
+ S3GUARD_DDB_THROTTLE_RETRY_INTERVAL_DEFAULT,
|
|
|
+ TimeUnit.MILLISECONDS),
|
|
|
TimeUnit.MILLISECONDS);
|
|
|
- dataAccess = new Invoker(dataAccessRetryPolicy, this::retryEvent);
|
|
|
+ final RetryPolicy throttledRetryRetryPolicy
|
|
|
+ = new S3GuardDataAccessRetryPolicy(config);
|
|
|
+ readOp = new Invoker(throttledRetryRetryPolicy, this::readRetryEvent);
|
|
|
+ writeOp = new Invoker(throttledRetryRetryPolicy, this::writeRetryEvent);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -432,11 +469,17 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
if (tombstone) {
|
|
|
Item item = PathMetadataDynamoDBTranslation.pathMetadataToItem(
|
|
|
new DDBPathMetadata(PathMetadata.tombstone(path)));
|
|
|
- invoker.retry("Put tombstone", path.toString(), idempotent,
|
|
|
+ writeOp.retry(
|
|
|
+ "Put tombstone",
|
|
|
+ path.toString(),
|
|
|
+ idempotent,
|
|
|
() -> table.putItem(item));
|
|
|
} else {
|
|
|
PrimaryKey key = pathToKey(path);
|
|
|
- invoker.retry("Delete key", path.toString(), idempotent,
|
|
|
+ writeOp.retry(
|
|
|
+ "Delete key",
|
|
|
+ path.toString(),
|
|
|
+ idempotent,
|
|
|
() -> table.deleteItem(key));
|
|
|
}
|
|
|
}
|
|
@@ -460,28 +503,38 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Retries.OnceRaw
|
|
|
- private Item getConsistentItem(PrimaryKey key) {
|
|
|
+ /**
|
|
|
+ * Get a consistent view of an item.
|
|
|
+ * @param path path to look up in the database
|
|
|
+ * @param path entry
|
|
|
+ * @return the result
|
|
|
+ * @throws IOException failure
|
|
|
+ */
|
|
|
+ @Retries.RetryTranslated
|
|
|
+ private Item getConsistentItem(final Path path) throws IOException {
|
|
|
+ PrimaryKey key = pathToKey(path);
|
|
|
final GetItemSpec spec = new GetItemSpec()
|
|
|
.withPrimaryKey(key)
|
|
|
.withConsistentRead(true); // strictly consistent read
|
|
|
- return table.getItem(spec);
|
|
|
+ return readOp.retry("get",
|
|
|
+ path.toString(),
|
|
|
+ true,
|
|
|
+ () -> table.getItem(spec));
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- @Retries.OnceTranslated
|
|
|
+ @Retries.RetryTranslated
|
|
|
public DDBPathMetadata get(Path path) throws IOException {
|
|
|
return get(path, false);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- @Retries.OnceTranslated
|
|
|
+ @Retries.RetryTranslated
|
|
|
public DDBPathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
|
|
|
throws IOException {
|
|
|
checkPath(path);
|
|
|
LOG.debug("Get from table {} in region {}: {}", tableName, region, path);
|
|
|
- return Invoker.once("get", path.toString(),
|
|
|
- () -> innerGet(path, wantEmptyDirectoryFlag));
|
|
|
+ return innerGet(path, wantEmptyDirectoryFlag);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -491,9 +544,8 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
* MetadataStore that it should try to compute the empty directory flag.
|
|
|
* @return metadata for {@code path}, {@code null} if not found
|
|
|
* @throws IOException IO problem
|
|
|
- * @throws AmazonClientException dynamo DB level problem
|
|
|
*/
|
|
|
- @Retries.OnceRaw
|
|
|
+ @Retries.RetryTranslated
|
|
|
private DDBPathMetadata innerGet(Path path, boolean wantEmptyDirectoryFlag)
|
|
|
throws IOException {
|
|
|
final DDBPathMetadata meta;
|
|
@@ -502,7 +554,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
meta =
|
|
|
new DDBPathMetadata(makeDirStatus(username, path));
|
|
|
} else {
|
|
|
- final Item item = getConsistentItem(pathToKey(path));
|
|
|
+ final Item item = getConsistentItem(path);
|
|
|
meta = itemToPathMetadata(item, username);
|
|
|
LOG.debug("Get from table {} in region {} returning for {}: {}",
|
|
|
tableName, region, path, meta);
|
|
@@ -517,8 +569,10 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
.withConsistentRead(true)
|
|
|
.withFilterExpression(IS_DELETED + " = :false")
|
|
|
.withValueMap(deleteTrackingValueMap);
|
|
|
- final ItemCollection<QueryOutcome> items = table.query(spec);
|
|
|
- boolean hasChildren = items.iterator().hasNext();
|
|
|
+ boolean hasChildren = readOp.retry("get/hasChildren",
|
|
|
+ path.toString(),
|
|
|
+ true,
|
|
|
+ () -> table.query(spec).iterator().hasNext());
|
|
|
// When this class has support for authoritative
|
|
|
// (fully-cached) directory listings, we may also be able to answer
|
|
|
// TRUE here. Until then, we don't know if we have full listing or
|
|
@@ -545,13 +599,16 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- @Retries.OnceTranslated
|
|
|
+ @Retries.RetryTranslated
|
|
|
public DirListingMetadata listChildren(final Path path) throws IOException {
|
|
|
checkPath(path);
|
|
|
LOG.debug("Listing table {} in region {}: {}", tableName, region, path);
|
|
|
|
|
|
// find the children in the table
|
|
|
- return Invoker.once("listChildren", path.toString(),
|
|
|
+ return readOp.retry(
|
|
|
+ "listChildren",
|
|
|
+ path.toString(),
|
|
|
+ true,
|
|
|
() -> {
|
|
|
final QuerySpec spec = new QuerySpec()
|
|
|
.withHashKey(pathToParentKeyAttribute(path))
|
|
@@ -610,7 +667,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- @Retries.OnceTranslated
|
|
|
+ @Retries.RetryTranslated
|
|
|
public void move(Collection<Path> pathsToDelete,
|
|
|
Collection<PathMetadata> pathsToCreate) throws IOException {
|
|
|
if (pathsToDelete == null && pathsToCreate == null) {
|
|
@@ -639,25 +696,25 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- Invoker.once("move", tableName,
|
|
|
- () -> processBatchWriteRequest(null, pathMetadataToItem(newItems)));
|
|
|
+ processBatchWriteRequest(null, pathMetadataToItem(newItems));
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Helper method to issue a batch write request to DynamoDB.
|
|
|
*
|
|
|
- * The retry logic here is limited to repeating the write operations
|
|
|
- * until all items have been written; there is no other attempt
|
|
|
- * at recovery/retry. Throttling is handled internally.
|
|
|
+ * As well as retrying on the operation invocation, incomplete
|
|
|
+ * batches are retried until all have been deleted.
|
|
|
* @param keysToDelete primary keys to be deleted; can be null
|
|
|
* @param itemsToPut new items to be put; can be null
|
|
|
+ * @return the number of iterations needed to complete the call.
|
|
|
*/
|
|
|
- @Retries.OnceRaw("Outstanding batch items are updated with backoff")
|
|
|
- private void processBatchWriteRequest(PrimaryKey[] keysToDelete,
|
|
|
+ @Retries.RetryTranslated("Outstanding batch items are updated with backoff")
|
|
|
+ private int processBatchWriteRequest(PrimaryKey[] keysToDelete,
|
|
|
Item[] itemsToPut) throws IOException {
|
|
|
final int totalToDelete = (keysToDelete == null ? 0 : keysToDelete.length);
|
|
|
final int totalToPut = (itemsToPut == null ? 0 : itemsToPut.length);
|
|
|
int count = 0;
|
|
|
+ int batches = 0;
|
|
|
while (count < totalToDelete + totalToPut) {
|
|
|
final TableWriteItems writeItems = new TableWriteItems(tableName);
|
|
|
int numToDelete = 0;
|
|
@@ -682,35 +739,66 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
count += numToPut;
|
|
|
}
|
|
|
|
|
|
- BatchWriteItemOutcome res = dynamoDB.batchWriteItem(writeItems);
|
|
|
+ // if there's a retry and another process updates things then it's not
|
|
|
+ // quite idempotent, but this was the case anyway
|
|
|
+ batches++;
|
|
|
+ BatchWriteItemOutcome res = writeOp.retry(
|
|
|
+ "batch write",
|
|
|
+ "",
|
|
|
+ true,
|
|
|
+ () -> dynamoDB.batchWriteItem(writeItems));
|
|
|
// Check for unprocessed keys in case of exceeding provisioned throughput
|
|
|
Map<String, List<WriteRequest>> unprocessed = res.getUnprocessedItems();
|
|
|
int retryCount = 0;
|
|
|
while (!unprocessed.isEmpty()) {
|
|
|
- retryBackoff(retryCount++);
|
|
|
- res = dynamoDB.batchWriteItemUnprocessed(unprocessed);
|
|
|
+ batchWriteCapacityExceededEvents.incrementAndGet();
|
|
|
+ batches++;
|
|
|
+ retryBackoffOnBatchWrite(retryCount++);
|
|
|
+ // use a different reference to keep the compiler quiet
|
|
|
+ final Map<String, List<WriteRequest>> upx = unprocessed;
|
|
|
+ res = writeOp.retry(
|
|
|
+ "batch write",
|
|
|
+ "",
|
|
|
+ true,
|
|
|
+ () -> dynamoDB.batchWriteItemUnprocessed(upx));
|
|
|
unprocessed = res.getUnprocessedItems();
|
|
|
}
|
|
|
}
|
|
|
+ return batches;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Put the current thread to sleep to implement exponential backoff
|
|
|
* depending on retryCount. If max retries are exceeded, throws an
|
|
|
* exception instead.
|
|
|
+ *
|
|
|
* @param retryCount number of retries so far
|
|
|
* @throws IOException when max retryCount is exceeded.
|
|
|
*/
|
|
|
- private void retryBackoff(int retryCount) throws IOException {
|
|
|
+ private void retryBackoffOnBatchWrite(int retryCount) throws IOException {
|
|
|
try {
|
|
|
// Our RetryPolicy ignores everything but retryCount here.
|
|
|
- RetryPolicy.RetryAction action = dataAccessRetryPolicy.shouldRetry(null,
|
|
|
+ RetryPolicy.RetryAction action = batchWriteRetryPolicy.shouldRetry(
|
|
|
+ null,
|
|
|
retryCount, 0, true);
|
|
|
if (action.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
|
|
|
- throw new IOException(
|
|
|
- String.format("Max retries exceeded (%d) for DynamoDB. This may be"
|
|
|
- + " because write threshold of DynamoDB is set too low.",
|
|
|
- retryCount));
|
|
|
+ // Create an AWSServiceThrottledException, with a fake inner cause
|
|
|
+ // which we fill in to look like a real exception so
|
|
|
+ // error messages look sensible
|
|
|
+ AmazonServiceException cause = new AmazonServiceException(
|
|
|
+ "Throttling");
|
|
|
+ cause.setServiceName("S3Guard");
|
|
|
+ cause.setStatusCode(AWSServiceThrottledException.STATUS_CODE);
|
|
|
+ cause.setErrorCode(THROTTLING); // used in real AWS errors
|
|
|
+ cause.setErrorType(AmazonServiceException.ErrorType.Service);
|
|
|
+ cause.setErrorMessage(THROTTLING);
|
|
|
+ cause.setRequestId("n/a");
|
|
|
+ throw new AWSServiceThrottledException(
|
|
|
+ String.format("Max retries during batch write exceeded"
|
|
|
+ + " (%d) for DynamoDB."
|
|
|
+ + HINT_DDB_IOPS_TOO_LOW,
|
|
|
+ retryCount),
|
|
|
+ cause);
|
|
|
} else {
|
|
|
LOG.debug("Sleeping {} msec before next retry", action.delayMillis);
|
|
|
Thread.sleep(action.delayMillis);
|
|
@@ -720,12 +808,12 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
} catch (IOException e) {
|
|
|
throw e;
|
|
|
} catch (Exception e) {
|
|
|
- throw new IOException("Unexpected exception", e);
|
|
|
+ throw new IOException("Unexpected exception " + e, e);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- @Retries.OnceRaw
|
|
|
+ @Retries.RetryTranslated
|
|
|
public void put(PathMetadata meta) throws IOException {
|
|
|
// For a deeply nested path, this method will automatically create the full
|
|
|
// ancestry and save respective item in DynamoDB table.
|
|
@@ -741,7 +829,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- @Retries.OnceRaw
|
|
|
+ @Retries.RetryTranslated
|
|
|
public void put(Collection<PathMetadata> metas) throws IOException {
|
|
|
innerPut(pathMetaToDDBPathMeta(metas));
|
|
|
}
|
|
@@ -757,8 +845,9 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
/**
|
|
|
* Helper method to get full path of ancestors that are nonexistent in table.
|
|
|
*/
|
|
|
- @Retries.OnceRaw
|
|
|
- private Collection<DDBPathMetadata> fullPathsToPut(DDBPathMetadata meta)
|
|
|
+ @VisibleForTesting
|
|
|
+ @Retries.RetryTranslated
|
|
|
+ Collection<DDBPathMetadata> fullPathsToPut(DDBPathMetadata meta)
|
|
|
throws IOException {
|
|
|
checkPathMetadata(meta);
|
|
|
final Collection<DDBPathMetadata> metasToPut = new ArrayList<>();
|
|
@@ -771,7 +860,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
// first existent ancestor
|
|
|
Path path = meta.getFileStatus().getPath().getParent();
|
|
|
while (path != null && !path.isRoot()) {
|
|
|
- final Item item = getConsistentItem(pathToKey(path));
|
|
|
+ final Item item = getConsistentItem(path);
|
|
|
if (!itemExists(item)) {
|
|
|
final FileStatus status = makeDirStatus(path, username);
|
|
|
metasToPut.add(new DDBPathMetadata(status, Tristate.FALSE, false,
|
|
@@ -810,7 +899,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
* @throws IOException IO problem
|
|
|
*/
|
|
|
@Override
|
|
|
- @Retries.OnceTranslated("retry(listFullPaths); once(batchWrite)")
|
|
|
+ @Retries.RetryTranslated
|
|
|
public void put(DirListingMetadata meta) throws IOException {
|
|
|
LOG.debug("Saving to table {} in region {}: {}", tableName, region, meta);
|
|
|
|
|
@@ -821,15 +910,12 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
false, meta.isAuthoritative());
|
|
|
|
|
|
// First add any missing ancestors...
|
|
|
- final Collection<DDBPathMetadata> metasToPut = invoker.retry(
|
|
|
- "paths to put", path.toString(), true,
|
|
|
- () -> fullPathsToPut(ddbPathMeta));
|
|
|
+ final Collection<DDBPathMetadata> metasToPut = fullPathsToPut(ddbPathMeta);
|
|
|
|
|
|
// next add all children of the directory
|
|
|
metasToPut.addAll(pathMetaToDDBPathMeta(meta.getListing()));
|
|
|
|
|
|
- Invoker.once("put", path.toString(),
|
|
|
- () -> processBatchWriteRequest(null, pathMetadataToItem(metasToPut)));
|
|
|
+ processBatchWriteRequest(null, pathMetadataToItem(metasToPut));
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -847,10 +933,10 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
closeAutocloseables(LOG, credentials);
|
|
|
credentials = null;
|
|
|
}
|
|
|
-}
|
|
|
+ }
|
|
|
|
|
|
@Override
|
|
|
- @Retries.OnceTranslated
|
|
|
+ @Retries.RetryTranslated
|
|
|
public void destroy() throws IOException {
|
|
|
if (table == null) {
|
|
|
LOG.info("In destroy(): no table to delete");
|
|
@@ -859,10 +945,11 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
LOG.info("Deleting DynamoDB table {} in region {}", tableName, region);
|
|
|
Preconditions.checkNotNull(dynamoDB, "Not connected to DynamoDB");
|
|
|
try {
|
|
|
- table.delete();
|
|
|
+ invoker.retry("delete", null, true,
|
|
|
+ () -> table.delete());
|
|
|
table.waitForDelete();
|
|
|
- } catch (ResourceNotFoundException rnfe) {
|
|
|
- LOG.info("ResourceNotFoundException while deleting DynamoDB table {} in "
|
|
|
+ } catch (FileNotFoundException rnfe) {
|
|
|
+ LOG.info("FileNotFoundException while deleting DynamoDB table {} in "
|
|
|
+ "region {}. This may indicate that the table does not exist, "
|
|
|
+ "or has been deleted by another concurrent thread or process.",
|
|
|
tableName, region);
|
|
@@ -872,39 +959,50 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
tableName, ie);
|
|
|
throw new InterruptedIOException("Table " + tableName
|
|
|
+ " in region " + region + " has not been deleted");
|
|
|
- } catch (AmazonClientException e) {
|
|
|
- throw translateException("destroy", tableName, e);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Retries.OnceRaw
|
|
|
+ @Retries.RetryTranslated
|
|
|
private ItemCollection<ScanOutcome> expiredFiles(long modTime,
|
|
|
- String keyPrefix) {
|
|
|
+ String keyPrefix) throws IOException {
|
|
|
String filterExpression =
|
|
|
"mod_time < :mod_time and begins_with(parent, :parent)";
|
|
|
String projectionExpression = "parent,child";
|
|
|
ValueMap map = new ValueMap()
|
|
|
.withLong(":mod_time", modTime)
|
|
|
.withString(":parent", keyPrefix);
|
|
|
- return table.scan(filterExpression, projectionExpression, null, map);
|
|
|
+ return readOp.retry(
|
|
|
+ "scan",
|
|
|
+ keyPrefix,
|
|
|
+ true,
|
|
|
+ () -> table.scan(filterExpression, projectionExpression, null, map));
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- @Retries.OnceRaw("once(batchWrite)")
|
|
|
+ @Retries.RetryTranslated
|
|
|
public void prune(long modTime) throws IOException {
|
|
|
prune(modTime, "/");
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Prune files, in batches. There's a sleep between each batch.
|
|
|
+ * @param modTime Oldest modification time to allow
|
|
|
+ * @param keyPrefix The prefix for the keys that should be removed
|
|
|
+ * @throws IOException Any IO/DDB failure.
|
|
|
+ * @throws InterruptedIOException if the prune was interrupted
|
|
|
+ */
|
|
|
@Override
|
|
|
- @Retries.OnceRaw("once(batchWrite)")
|
|
|
+ @Retries.RetryTranslated
|
|
|
public void prune(long modTime, String keyPrefix) throws IOException {
|
|
|
int itemCount = 0;
|
|
|
try {
|
|
|
Collection<Path> deletionBatch =
|
|
|
new ArrayList<>(S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT);
|
|
|
- int delay = conf.getInt(S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY,
|
|
|
- S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_DEFAULT);
|
|
|
- Set<Path> parentPathSet = new HashSet<>();
|
|
|
+ long delay = conf.getTimeDuration(
|
|
|
+ S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY,
|
|
|
+ S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_DEFAULT,
|
|
|
+ TimeUnit.MILLISECONDS);
|
|
|
+ Set<Path> parentPathSet = new HashSet<>();
|
|
|
for (Item item : expiredFiles(modTime, keyPrefix)) {
|
|
|
DDBPathMetadata md = PathMetadataDynamoDBTranslation
|
|
|
.itemToPathMetadata(item, username);
|
|
@@ -929,7 +1027,8 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
deletionBatch.clear();
|
|
|
}
|
|
|
}
|
|
|
- if (deletionBatch.size() > 0) {
|
|
|
+ // final batch of deletes
|
|
|
+ if (!deletionBatch.isEmpty()) {
|
|
|
Thread.sleep(delay);
|
|
|
processBatchWriteRequest(pathToKey(deletionBatch), null);
|
|
|
|
|
@@ -1093,19 +1192,34 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
* Get the version mark item in the existing DynamoDB table.
|
|
|
*
|
|
|
* As the version marker item may be created by another concurrent thread or
|
|
|
- * process, we sleep and retry a limited times before we fail to get it.
|
|
|
- * This does not include handling any failure other than "item not found",
|
|
|
- * so this method is tagged as "OnceRaw"
|
|
|
+ * process, we sleep and retry a limited number times if the lookup returns
|
|
|
+ * with a null value.
|
|
|
+ * DDB throttling is always retried.
|
|
|
*/
|
|
|
- @Retries.OnceRaw
|
|
|
- private Item getVersionMarkerItem() throws IOException {
|
|
|
+ @VisibleForTesting
|
|
|
+ @Retries.RetryTranslated
|
|
|
+ Item getVersionMarkerItem() throws IOException {
|
|
|
final PrimaryKey versionMarkerKey =
|
|
|
createVersionMarkerPrimaryKey(VERSION_MARKER);
|
|
|
int retryCount = 0;
|
|
|
- Item versionMarker = table.getItem(versionMarkerKey);
|
|
|
+ // look for a version marker, with usual throttling/failure retries.
|
|
|
+ Item versionMarker = queryVersionMarker(versionMarkerKey);
|
|
|
while (versionMarker == null) {
|
|
|
+ // The marker was null.
|
|
|
+ // Two possibilities
|
|
|
+ // 1. This isn't a S3Guard table.
|
|
|
+ // 2. This is a S3Guard table in construction; another thread/process
|
|
|
+ // is about to write/actively writing the version marker.
|
|
|
+ // So that state #2 is handled, batchWriteRetryPolicy is used to manage
|
|
|
+ // retries.
|
|
|
+ // This will mean that if the cause is actually #1, failure will not
|
|
|
+ // be immediate. As this will ultimately result in a failure to
|
|
|
+ // init S3Guard and the S3A FS, this isn't going to be a performance
|
|
|
+ // bottleneck -simply a slightly slower failure report than would otherwise
|
|
|
+ // be seen.
|
|
|
+ // "if your settings are broken, performance is not your main issue"
|
|
|
try {
|
|
|
- RetryPolicy.RetryAction action = dataAccessRetryPolicy.shouldRetry(null,
|
|
|
+ RetryPolicy.RetryAction action = batchWriteRetryPolicy.shouldRetry(null,
|
|
|
retryCount, 0, true);
|
|
|
if (action.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
|
|
|
break;
|
|
@@ -1114,14 +1228,29 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
Thread.sleep(action.delayMillis);
|
|
|
}
|
|
|
} catch (Exception e) {
|
|
|
- throw new IOException("initTable: Unexpected exception", e);
|
|
|
+ throw new IOException("initTable: Unexpected exception " + e, e);
|
|
|
}
|
|
|
retryCount++;
|
|
|
- versionMarker = table.getItem(versionMarkerKey);
|
|
|
+ versionMarker = queryVersionMarker(versionMarkerKey);
|
|
|
}
|
|
|
return versionMarker;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Issue the query to get the version marker, with throttling for overloaded
|
|
|
+ * DDB tables.
|
|
|
+ * @param versionMarkerKey key to look up
|
|
|
+ * @return the marker
|
|
|
+ * @throws IOException failure
|
|
|
+ */
|
|
|
+ @Retries.RetryTranslated
|
|
|
+ private Item queryVersionMarker(final PrimaryKey versionMarkerKey)
|
|
|
+ throws IOException {
|
|
|
+ return readOp.retry("getVersionMarkerItem",
|
|
|
+ VERSION_MARKER, true,
|
|
|
+ () -> table.getItem(versionMarkerKey));
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Verify that a table version is compatible with this S3Guard client.
|
|
|
* @param tableName name of the table (for error messages)
|
|
@@ -1207,7 +1336,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
* @return the outcome.
|
|
|
*/
|
|
|
@Retries.OnceRaw
|
|
|
- PutItemOutcome putItem(Item item) {
|
|
|
+ private PutItemOutcome putItem(Item item) {
|
|
|
LOG.debug("Putting item {}", item);
|
|
|
return table.putItem(item);
|
|
|
}
|
|
@@ -1254,6 +1383,11 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
return region;
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ public String getTableName() {
|
|
|
+ return tableName;
|
|
|
+ }
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
DynamoDB getDynamoDB() {
|
|
|
return dynamoDB;
|
|
@@ -1312,8 +1446,8 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
}
|
|
|
map.put("description", DESCRIPTION);
|
|
|
map.put("region", region);
|
|
|
- if (dataAccessRetryPolicy != null) {
|
|
|
- map.put("retryPolicy", dataAccessRetryPolicy.toString());
|
|
|
+ if (batchWriteRetryPolicy != null) {
|
|
|
+ map.put("retryPolicy", batchWriteRetryPolicy.toString());
|
|
|
}
|
|
|
return map;
|
|
|
}
|
|
@@ -1368,6 +1502,38 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Callback on a read operation retried.
|
|
|
+ * @param text text of the operation
|
|
|
+ * @param ex exception
|
|
|
+ * @param attempts number of attempts
|
|
|
+ * @param idempotent is the method idempotent (this is assumed to be true)
|
|
|
+ */
|
|
|
+ void readRetryEvent(
|
|
|
+ String text,
|
|
|
+ IOException ex,
|
|
|
+ int attempts,
|
|
|
+ boolean idempotent) {
|
|
|
+ readThrottleEvents.incrementAndGet();
|
|
|
+ retryEvent(text, ex, attempts, true);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Callback on a write operation retried.
|
|
|
+ * @param text text of the operation
|
|
|
+ * @param ex exception
|
|
|
+ * @param attempts number of attempts
|
|
|
+ * @param idempotent is the method idempotent (this is assumed to be true)
|
|
|
+ */
|
|
|
+ void writeRetryEvent(
|
|
|
+ String text,
|
|
|
+ IOException ex,
|
|
|
+ int attempts,
|
|
|
+ boolean idempotent) {
|
|
|
+ writeThrottleEvents.incrementAndGet();
|
|
|
+ retryEvent(text, ex, attempts, idempotent);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Callback from {@link Invoker} when an operation is retried.
|
|
|
* @param text text of the operation
|
|
@@ -1410,4 +1576,31 @@ public class DynamoDBMetadataStore implements MetadataStore {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Get the count of read throttle events.
|
|
|
+ * @return the current count of read throttle events.
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ public long getReadThrottleEventCount() {
|
|
|
+ return readThrottleEvents.get();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get the count of write throttle events.
|
|
|
+ * @return the current count of write throttle events.
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ public long getWriteThrottleEventCount() {
|
|
|
+ return writeThrottleEvents.get();
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public long getBatchWriteCapacityExceededCount() {
|
|
|
+ return batchWriteCapacityExceededEvents.get();
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public Invoker getInvoker() {
|
|
|
+ return invoker;
|
|
|
+ }
|
|
|
}
|