|
@@ -38,6 +38,7 @@ import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
|
|
|
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.GetObjectRequest;
|
|
|
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
|
|
|
import com.amazonaws.services.s3.model.InitiateMultipartUploadResult;
|
|
|
import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
|
|
@@ -48,6 +49,7 @@ import com.amazonaws.services.s3.model.MultipartUploadListing;
|
|
|
import com.amazonaws.services.s3.model.ObjectListing;
|
|
|
import com.amazonaws.services.s3.model.PutObjectRequest;
|
|
|
import com.amazonaws.services.s3.model.PutObjectResult;
|
|
|
+import com.amazonaws.services.s3.model.S3Object;
|
|
|
import com.amazonaws.services.s3.model.S3ObjectSummary;
|
|
|
import com.amazonaws.services.s3.model.UploadPartRequest;
|
|
|
import com.amazonaws.services.s3.model.UploadPartResult;
|
|
@@ -60,8 +62,6 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
|
|
|
-import static org.apache.hadoop.fs.s3a.Constants.*;
|
|
|
-
|
|
|
/**
|
|
|
* A wrapper around {@link com.amazonaws.services.s3.AmazonS3} that injects
|
|
|
* inconsistency and/or errors. Used for testing S3Guard.
|
|
@@ -71,49 +71,16 @@ import static org.apache.hadoop.fs.s3a.Constants.*;
|
|
|
@InterfaceStability.Unstable
|
|
|
public class InconsistentAmazonS3Client extends AmazonS3Client {
|
|
|
|
|
|
- /**
|
|
|
- * Keys containing this substring will be subject to delayed visibility.
|
|
|
- */
|
|
|
- public static final String DEFAULT_DELAY_KEY_SUBSTRING = "DELAY_LISTING_ME";
|
|
|
-
|
|
|
- /**
|
|
|
- * How many seconds affected keys will be delayed from appearing in listing.
|
|
|
- * This should probably be a config value.
|
|
|
- */
|
|
|
- public static final long DEFAULT_DELAY_KEY_MSEC = 5 * 1000;
|
|
|
-
|
|
|
- public static final float DEFAULT_DELAY_KEY_PROBABILITY = 1.0f;
|
|
|
-
|
|
|
- /** Special config value since we can't store empty strings in XML. */
|
|
|
- public static final String MATCH_ALL_KEYS = "*";
|
|
|
-
|
|
|
private static final Logger LOG =
|
|
|
LoggerFactory.getLogger(InconsistentAmazonS3Client.class);
|
|
|
|
|
|
- /** Empty string matches all keys. */
|
|
|
- private String delayKeySubstring;
|
|
|
-
|
|
|
- /** Probability to delay visibility of a matching key. */
|
|
|
- private float delayKeyProbability;
|
|
|
-
|
|
|
- /** Time in milliseconds to delay visibility of newly modified object. */
|
|
|
- private long delayKeyMsec;
|
|
|
-
|
|
|
- /**
|
|
|
- * Probability of throttling a request.
|
|
|
- */
|
|
|
- private float throttleProbability;
|
|
|
+ private FailureInjectionPolicy policy;
|
|
|
|
|
|
/**
|
|
|
* Counter of failures since last reset.
|
|
|
*/
|
|
|
private final AtomicLong failureCounter = new AtomicLong(0);
|
|
|
|
|
|
- /**
|
|
|
- * limit for failures before operations succeed; if 0 then "no limit".
|
|
|
- */
|
|
|
- private int failureLimit = 0;
|
|
|
-
|
|
|
/**
|
|
|
* Composite of data we need to track about recently deleted objects:
|
|
|
* when it was deleted (same was with recently put objects) and the object
|
|
@@ -150,36 +117,42 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
|
|
|
public InconsistentAmazonS3Client(AWSCredentialsProvider credentials,
|
|
|
ClientConfiguration clientConfiguration, Configuration conf) {
|
|
|
super(credentials, clientConfiguration);
|
|
|
- setupConfig(conf);
|
|
|
+ policy = new FailureInjectionPolicy(conf);
|
|
|
}
|
|
|
|
|
|
- protected void setupConfig(Configuration conf) {
|
|
|
|
|
|
- delayKeySubstring = conf.get(FAIL_INJECT_INCONSISTENCY_KEY,
|
|
|
- DEFAULT_DELAY_KEY_SUBSTRING);
|
|
|
- // "" is a substring of all strings, use it to match all keys.
|
|
|
- if (delayKeySubstring.equals(MATCH_ALL_KEYS)) {
|
|
|
- delayKeySubstring = "";
|
|
|
- }
|
|
|
- delayKeyProbability = validProbability(
|
|
|
- conf.getFloat(FAIL_INJECT_INCONSISTENCY_PROBABILITY,
|
|
|
- DEFAULT_DELAY_KEY_PROBABILITY));
|
|
|
- delayKeyMsec = conf.getLong(FAIL_INJECT_INCONSISTENCY_MSEC,
|
|
|
- DEFAULT_DELAY_KEY_MSEC);
|
|
|
- setThrottleProbability(conf.getFloat(FAIL_INJECT_THROTTLE_PROBABILITY,
|
|
|
- 0.0f));
|
|
|
- LOG.info("{}", this);
|
|
|
+ /**
|
|
|
+ * Clear any accumulated inconsistency state. Used by tests to make paths
|
|
|
+ * visible again.
|
|
|
+ * @param fs S3AFileSystem under test
|
|
|
+ * @throws Exception on failure
|
|
|
+ */
|
|
|
+ public static void clearInconsistency(S3AFileSystem fs) throws Exception {
|
|
|
+ AmazonS3 s3 = fs.getAmazonS3ClientForTesting("s3guard");
|
|
|
+ InconsistentAmazonS3Client ic = InconsistentAmazonS3Client.castFrom(s3);
|
|
|
+ ic.clearInconsistency();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * A way for tests to patch in a different fault injection policy at runtime.
|
|
|
+ * @param fs filesystem under test
|
|
|
+ *
|
|
|
+ */
|
|
|
+ public static void setFailureInjectionPolicy(S3AFileSystem fs,
|
|
|
+ FailureInjectionPolicy policy) throws Exception {
|
|
|
+ AmazonS3 s3 = fs.getAmazonS3ClientForTesting("s3guard");
|
|
|
+ InconsistentAmazonS3Client ic = InconsistentAmazonS3Client.castFrom(s3);
|
|
|
+ ic.replacePolicy(policy);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void replacePolicy(FailureInjectionPolicy pol) {
|
|
|
+ this.policy = pol;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public String toString() {
|
|
|
- return String.format(
|
|
|
- "Inconsistent S3 Client with"
|
|
|
- + " %s msec delay, substring %s, delay probability %s;"
|
|
|
- + " throttle probability %s"
|
|
|
- + "; failure limit %d, failure count %d",
|
|
|
- delayKeyMsec, delayKeySubstring, delayKeyProbability,
|
|
|
- throttleProbability, failureLimit, failureCounter.get());
|
|
|
+ return String.format("Inconsistent S3 Client: %s; failure count %d",
|
|
|
+ policy, failureCounter.get());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -470,7 +443,7 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
|
|
|
return false;
|
|
|
}
|
|
|
long currentTime = System.currentTimeMillis();
|
|
|
- long deadline = enqueueTime + delayKeyMsec;
|
|
|
+ long deadline = enqueueTime + policy.getDelayKeyMsec();
|
|
|
if (currentTime >= deadline) {
|
|
|
delayedDeletes.remove(key);
|
|
|
LOG.debug("no longer delaying {}", key);
|
|
@@ -482,7 +455,7 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
|
|
|
}
|
|
|
|
|
|
private void registerDeleteObject(String key, String bucket) {
|
|
|
- if (shouldDelay(key)) {
|
|
|
+ if (policy.shouldDelay(key)) {
|
|
|
// Record summary so we can add it back for some time post-deletion
|
|
|
ListObjectsRequest request = new ListObjectsRequest()
|
|
|
.withBucketName(bucket)
|
|
@@ -498,28 +471,11 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
|
|
|
|
|
|
private void registerPutObject(PutObjectRequest req) {
|
|
|
String key = req.getKey();
|
|
|
- if (shouldDelay(key)) {
|
|
|
+ if (policy.shouldDelay(key)) {
|
|
|
enqueueDelayedPut(key);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Should we delay listing visibility for this key?
|
|
|
- * @param key key which is being put
|
|
|
- * @return true if we should delay
|
|
|
- */
|
|
|
- private boolean shouldDelay(String key) {
|
|
|
- boolean delay = key.contains(delayKeySubstring);
|
|
|
- delay = delay && trueWithProbability(delayKeyProbability);
|
|
|
- LOG.debug("{} -> {}", key, delay);
|
|
|
- return delay;
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
- private boolean trueWithProbability(float p) {
|
|
|
- return Math.random() < p;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Record this key as something that should not become visible in
|
|
|
* listObject replies for a while, to simulate eventual list consistency.
|
|
@@ -561,20 +517,8 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
|
|
|
return super.listMultipartUploads(listMultipartUploadsRequest);
|
|
|
}
|
|
|
|
|
|
- public float getDelayKeyProbability() {
|
|
|
- return delayKeyProbability;
|
|
|
- }
|
|
|
-
|
|
|
public long getDelayKeyMsec() {
|
|
|
- return delayKeyMsec;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Get the probability of the request being throttled.
|
|
|
- * @return a value 0 - 1.0f.
|
|
|
- */
|
|
|
- public float getThrottleProbability() {
|
|
|
- return throttleProbability;
|
|
|
+ return policy.getDelayKeyMsec();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -582,36 +526,28 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
|
|
|
* @param throttleProbability the probability of a request being throttled.
|
|
|
*/
|
|
|
public void setThrottleProbability(float throttleProbability) {
|
|
|
- this.throttleProbability = validProbability(throttleProbability);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Validate a probability option.
|
|
|
- * @param p probability
|
|
|
- * @return the probability, if valid
|
|
|
- * @throws IllegalArgumentException if the probability is out of range.
|
|
|
- */
|
|
|
- private float validProbability(float p) {
|
|
|
- Preconditions.checkArgument(p >= 0.0f && p <= 1.0f,
|
|
|
- "Probability out of range 0 to 1 %s", p);
|
|
|
- return p;
|
|
|
+ policy.setThrottleProbability(throttleProbability);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Conditionally fail the operation.
|
|
|
+ * @param errorMsg description of failure
|
|
|
+ * @param statusCode http status code for error
|
|
|
* @throws AmazonClientException if the client chooses to fail
|
|
|
* the request.
|
|
|
*/
|
|
|
- private void maybeFail() throws AmazonClientException {
|
|
|
+ private void maybeFail(String errorMsg, int statusCode)
|
|
|
+ throws AmazonClientException {
|
|
|
// code structure here is to line up for more failures later
|
|
|
AmazonServiceException ex = null;
|
|
|
- if (trueWithProbability(throttleProbability)) {
|
|
|
+ if (policy.trueWithProbability(policy.getThrottleProbability())) {
|
|
|
// throttle the request
|
|
|
- ex = new AmazonServiceException("throttled"
|
|
|
+ ex = new AmazonServiceException(errorMsg
|
|
|
+ " count = " + (failureCounter.get() + 1), null);
|
|
|
- ex.setStatusCode(503);
|
|
|
+ ex.setStatusCode(statusCode);
|
|
|
}
|
|
|
|
|
|
+ int failureLimit = policy.getFailureLimit();
|
|
|
if (ex != null) {
|
|
|
long count = failureCounter.incrementAndGet();
|
|
|
if (failureLimit == 0
|
|
@@ -621,16 +557,37 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private void maybeFail() {
|
|
|
+ maybeFail("throttled", 503);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Set the limit on failures before all operations pass through.
|
|
|
* This resets the failure count.
|
|
|
* @param limit limit; "0" means "no limit"
|
|
|
*/
|
|
|
public void setFailureLimit(int limit) {
|
|
|
- this.failureLimit = limit;
|
|
|
+ policy.setFailureLimit(limit);
|
|
|
failureCounter.set(0);
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public S3Object getObject(GetObjectRequest var1) throws SdkClientException,
|
|
|
+ AmazonServiceException {
|
|
|
+ maybeFail("file not found", 404);
|
|
|
+ S3Object o = super.getObject(var1);
|
|
|
+ LOG.debug("Wrapping in InconsistentS3Object for key {}", var1.getKey());
|
|
|
+ return new InconsistentS3Object(o, policy);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public S3Object getObject(String bucketName, String key)
|
|
|
+ throws SdkClientException, AmazonServiceException {
|
|
|
+ S3Object o = super.getObject(bucketName, key);
|
|
|
+ LOG.debug("Wrapping in InconsistentS3Object for key {}", key);
|
|
|
+ return new InconsistentS3Object(o, policy);
|
|
|
+ }
|
|
|
+
|
|
|
/** Since ObjectListing is immutable, we just override it with wrapper. */
|
|
|
@SuppressWarnings("serial")
|
|
|
private static class CustomObjectListing extends ObjectListing {
|