HADOOP-11572. s3a delete() operation fails during a concurrent delete of child entries.
Contributed by Steve Loughran.
(cherry picked from commit 2ac5aab8d7
)
This commit is contained in:
parent
b46cd315f7
commit
ba70225cf6
|
@ -49,6 +49,7 @@ import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
|||
import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
|
||||
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
|
||||
import com.amazonaws.services.s3.model.ListObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
|
||||
import com.amazonaws.services.s3.model.ObjectListing;
|
||||
import com.amazonaws.services.s3.model.ObjectMetadata;
|
||||
import com.amazonaws.services.s3.model.PartETag;
|
||||
|
@ -455,7 +456,8 @@ public class S3AFileSystem extends FileSystem {
|
|||
* @param path input path, may be relative to the working dir
|
||||
* @return a key excluding the leading "/", or, if it is the root path, ""
|
||||
*/
|
||||
private String pathToKey(Path path) {
|
||||
@VisibleForTesting
|
||||
String pathToKey(Path path) {
|
||||
if (!path.isAbsolute()) {
|
||||
path = new Path(workingDir, path);
|
||||
}
|
||||
|
@ -1011,11 +1013,26 @@ public class S3AFileSystem extends FileSystem {
|
|||
* Increments the {@code OBJECT_DELETE_REQUESTS} and write
|
||||
* operation statistics.
|
||||
* @param deleteRequest keys to delete on the s3-backend
|
||||
* @throws MultiObjectDeleteException one or more of the keys could not
|
||||
* be deleted.
|
||||
* @throws AmazonClientException amazon-layer failure.
|
||||
*/
|
||||
private void deleteObjects(DeleteObjectsRequest deleteRequest) {
|
||||
private void deleteObjects(DeleteObjectsRequest deleteRequest)
|
||||
throws MultiObjectDeleteException, AmazonClientException {
|
||||
incrementWriteOperations();
|
||||
incrementStatistic(OBJECT_DELETE_REQUESTS, 1);
|
||||
s3.deleteObjects(deleteRequest);
|
||||
try {
|
||||
s3.deleteObjects(deleteRequest);
|
||||
} catch (MultiObjectDeleteException e) {
|
||||
// one or more of the operations failed.
|
||||
List<MultiObjectDeleteException.DeleteError> errors = e.getErrors();
|
||||
LOG.error("Partial failure of delete, {} errors", errors.size(), e);
|
||||
for (MultiObjectDeleteException.DeleteError error : errors) {
|
||||
LOG.error("{}: \"{}\" - {}",
|
||||
error.getKey(), error.getCode(), error.getMessage());
|
||||
}
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1225,10 +1242,15 @@ public class S3AFileSystem extends FileSystem {
|
|||
* @param deleteFakeDir indicates whether this is for deleting fake dirs
|
||||
* @throws InvalidRequestException if the request was rejected due to
|
||||
* a mistaken attempt to delete the root directory.
|
||||
* @throws MultiObjectDeleteException one or more of the keys could not
|
||||
* be deleted in a multiple object delete operation.
|
||||
* @throws AmazonClientException amazon-layer failure.
|
||||
*/
|
||||
private void removeKeys(List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
||||
@VisibleForTesting
|
||||
void removeKeys(List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
||||
boolean clearKeys, boolean deleteFakeDir)
|
||||
throws AmazonClientException, InvalidRequestException {
|
||||
throws MultiObjectDeleteException, AmazonClientException,
|
||||
InvalidRequestException {
|
||||
if (keysToDelete.isEmpty()) {
|
||||
// exit fast if there are no keys to delete
|
||||
return;
|
||||
|
|
|
@ -1874,6 +1874,31 @@ Consult [Cleaning up After Incremental Upload Failures](#s3a_multipart_purge) fo
|
|||
details on how the multipart purge timeout can be set. If multipart uploads
|
||||
are failing with the message above, it may be a sign that this value is too low.
|
||||
|
||||
### `MultiObjectDeleteException` during delete or rename of files
|
||||
|
||||
```
|
||||
Exception in thread "main" com.amazonaws.services.s3.model.MultiObjectDeleteException:
|
||||
Status Code: 0, AWS Service: null, AWS Request ID: null, AWS Error Code: null,
|
||||
AWS Error Message: One or more objects could not be deleted, S3 Extended Request ID: null
|
||||
at com.amazonaws.services.s3.AmazonS3Client.deleteObjects(AmazonS3Client.java:1745)
|
||||
```
|
||||
This happens when trying to delete multiple objects, and one of the objects
|
||||
could not be deleted. It *should not occur* just because the object is missing.
|
||||
More specifically: at the time this document was written, we could not create
|
||||
such a failure.
|
||||
|
||||
It will occur if the caller lacks the permission to delete any of the objects.
|
||||
|
||||
Consult the log to see the specifics of which objects could not be deleted.
|
||||
Do you have permission to do so?
|
||||
|
||||
If this operation is failing for reasons other than the caller lacking
|
||||
permissions:
|
||||
|
||||
1. Try setting `fs.s3a.multiobjectdelete.enable` to `false`.
|
||||
1. Consult [HADOOP-11572](https://issues.apache.org/jira/browse/HADOOP-11572)
|
||||
for up to date advice.
|
||||
|
||||
### When writing to S3A, HTTP Exceptions logged at info from `AmazonHttpClient`
|
||||
|
||||
```
|
||||
|
|
|
@ -18,17 +18,24 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.InvalidRequestException;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.test.LambdaTestUtils;
|
||||
|
||||
import org.junit.Assume;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.Callable;
|
||||
|
||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
|
||||
|
@ -41,6 +48,13 @@ public class ITestS3AFailureHandling extends AbstractS3ATestBase {
|
|||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(ITestS3AFailureHandling.class);
|
||||
|
||||
@Override
|
||||
protected Configuration createConfiguration() {
|
||||
Configuration conf = super.createConfiguration();
|
||||
S3ATestUtils.disableFilesystemCaching(conf);
|
||||
conf.setBoolean(Constants.ENABLE_MULTI_DELETE, true);
|
||||
return conf;
|
||||
}
|
||||
@Test
|
||||
public void testReadFileChanged() throws Throwable {
|
||||
describe("overwrite a file with a shorter one during a read, seek");
|
||||
|
@ -127,4 +141,41 @@ public class ITestS3AFailureHandling extends AbstractS3ATestBase {
|
|||
assertEquals("Expected EOF from "+ operation
|
||||
+ "; got char " + (char) readResult, -1, readResult);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiObjectDeleteNoFile() throws Throwable {
|
||||
describe("Deleting a missing object");
|
||||
removeKeys(getFileSystem(), "ITestS3AFailureHandling/missingFile");
|
||||
}
|
||||
|
||||
private void removeKeys(S3AFileSystem fileSystem, String... keys)
|
||||
throws InvalidRequestException {
|
||||
List<DeleteObjectsRequest.KeyVersion> request = new ArrayList<>(
|
||||
keys.length);
|
||||
for (String key : keys) {
|
||||
request.add(new DeleteObjectsRequest.KeyVersion(key));
|
||||
}
|
||||
fileSystem.removeKeys(request, false, false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiObjectDeleteSomeFiles() throws Throwable {
|
||||
Path valid = path("ITestS3AFailureHandling/validFile");
|
||||
touch(getFileSystem(), valid);
|
||||
NanoTimer timer = new NanoTimer();
|
||||
removeKeys(getFileSystem(), getFileSystem().pathToKey(valid),
|
||||
"ITestS3AFailureHandling/missingFile");
|
||||
timer.end("removeKeys");
|
||||
}
|
||||
|
||||
@Test(expected = MultiObjectDeleteException.class)
|
||||
public void testMultiObjectDeleteNoPermissions() throws Throwable {
|
||||
Configuration conf = getConfiguration();
|
||||
String csvFile = conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE);
|
||||
Assume.assumeTrue("CSV test file is not the default",
|
||||
DEFAULT_CSVTEST_FILE.equals(csvFile));
|
||||
Path testFile = new Path(csvFile);
|
||||
S3AFileSystem fs = (S3AFileSystem)FileSystem.newInstance(testFile.toUri(), conf);
|
||||
removeKeys(fs, fs.pathToKey(testFile));
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue