HADOOP-15183. S3Guard store becomes inconsistent after partial failure of rename.

Contributed by Steve Loughran.

Change-Id: I825b0bc36be960475d2d259b1cdab45ae1bb78eb
This commit is contained in:
Steve Loughran 2019-06-20 09:56:40 +01:00
parent 28291a9e8a
commit e02eb24e0a
No known key found for this signature in database
GPG Key ID: D22CF846DBB162A0
71 changed files with 7575 additions and 1062 deletions

View File

@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.impl;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Evolving support for functional programming/lambda-expressions.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public final class FunctionsRaisingIOE {
private FunctionsRaisingIOE() {
}
/**
* Function of arity 1 which may raise an IOException.
* @param <T> type of arg1
* @param <R> type of return value.
*/
@FunctionalInterface
public interface FunctionRaisingIOE<T, R> {
R apply(T t) throws IOException;
}
/**
* Function of arity 2 which may raise an IOException.
* @param <T> type of arg1
* @param <U> type of arg2
* @param <R> type of return value.
*/
@FunctionalInterface
public interface BiFunctionRaisingIOE<T, U, R> {
R apply(T t, U u) throws IOException;
}
/**
* This is a callable which only raises an IOException.
* @param <R> return type
*/
@FunctionalInterface
public interface CallableRaisingIOE<R> {
R apply() throws IOException;
}
}

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.fs.impl;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException; import java.io.InterruptedIOException;
import java.util.Map; import java.util.Map;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -108,20 +109,55 @@ public final class FutureIOSupport {
*/ */
public static <T> T raiseInnerCause(final ExecutionException e) public static <T> T raiseInnerCause(final ExecutionException e)
throws IOException { throws IOException {
throw unwrapInnerException(e);
}
/**
* Extract the cause of a completion failure and rethrow it if an IOE
* or RTE.
* @param e exception.
* @param <T> type of return value.
* @return nothing, ever.
* @throws IOException either the inner IOException, or a wrapper around
* any non-Runtime-Exception
* @throws RuntimeException if that is the inner cause.
*/
public static <T> T raiseInnerCause(final CompletionException e)
throws IOException {
throw unwrapInnerException(e);
}
/**
* From the inner cause of an execution exception, extract the inner cause.
* If it is an RTE: throw immediately.
* If it is an IOE: Return.
* If it is a WrappedIOException: Unwrap and return
* Else: create a new IOException.
*
* Recursively handles wrapped Execution and Completion Exceptions in
* case something very complicated has happened.
* @param e exception.
* @return an IOException extracted or built from the cause.
* @throws RuntimeException if that is the inner cause.
*/
private static IOException unwrapInnerException(final Throwable e) {
Throwable cause = e.getCause(); Throwable cause = e.getCause();
if (cause instanceof IOException) { if (cause instanceof IOException) {
throw (IOException) cause; return (IOException) cause;
} else if (cause instanceof WrappedIOException){ } else if (cause instanceof WrappedIOException){
throw ((WrappedIOException) cause).getCause(); return ((WrappedIOException) cause).getCause();
} else if (cause instanceof CompletionException){
return unwrapInnerException(cause);
} else if (cause instanceof ExecutionException){
return unwrapInnerException(cause);
} else if (cause instanceof RuntimeException){ } else if (cause instanceof RuntimeException){
throw (RuntimeException) cause; throw (RuntimeException) cause;
} else if (cause != null) { } else if (cause != null) {
// other type: wrap with a new IOE // other type: wrap with a new IOE
throw new IOException(cause); return new IOException(cause);
} else { } else {
// this only happens if somebody deliberately raises // this only happens if there was no cause.
// an ExecutionException return new IOException(e);
throw new IOException(e);
} }
} }

View File

@ -1213,8 +1213,12 @@
<property> <property>
<name>fs.s3a.connection.maximum</name> <name>fs.s3a.connection.maximum</name>
<value>15</value> <value>48</value>
<description>Controls the maximum number of simultaneous connections to S3.</description> <description>Controls the maximum number of simultaneous connections to S3.
This must be bigger than the value of fs.s3a.threads.max so as to stop
threads being blocked waiting for new HTTPS connections.
Why not equal? The AWS SDK transfer manager also uses these connections.
</description>
</property> </property>
<property> <property>
@ -1312,7 +1316,7 @@
<property> <property>
<name>fs.s3a.threads.max</name> <name>fs.s3a.threads.max</name>
<value>10</value> <value>64</value>
<description>The total number of threads available in the filesystem for data <description>The total number of threads available in the filesystem for data
uploads *or any other queued filesystem operation*.</description> uploads *or any other queued filesystem operation*.</description>
</property> </property>
@ -1326,8 +1330,25 @@
<property> <property>
<name>fs.s3a.max.total.tasks</name> <name>fs.s3a.max.total.tasks</name>
<value>5</value> <value>32</value>
<description>The number of operations which can be queued for execution</description> <description>The number of operations which can be queued for execution.
This is in addition to the number of active threads in fs.s3a.threads.max.
</description>
</property>
<property>
<name>fs.s3a.executor.capacity</name>
<value>16</value>
<description>The maximum number of submitted tasks which is a single
operation (e.g. rename(), delete()) may submit simultaneously for
execution -excluding the IO-heavy block uploads, whose capacity
is set in "fs.s3a.fast.upload.active.blocks"
All tasks are submitted to the shared thread pool whose size is
set in "fs.s3a.threads.max"; the value of capacity should be less than that
of the thread pool itself, as the goal is to stop a single operation
from overloading that thread pool.
</description>
</property> </property>
<property> <property>

View File

@ -268,7 +268,7 @@ public abstract class AbstractContractRenameTest extends
* @param dst the destination root to move * @param dst the destination root to move
* @param nestedPath the nested path to move * @param nestedPath the nested path to move
*/ */
private void validateAncestorsMoved(Path src, Path dst, String nestedPath) protected void validateAncestorsMoved(Path src, Path dst, String nestedPath)
throws IOException { throws IOException {
assertIsDirectory(dst); assertIsDirectory(dst);
assertPathDoesNotExist("src path should not exist", path(src + nestedPath)); assertPathDoesNotExist("src path should not exist", path(src + nestedPath));

View File

@ -417,6 +417,11 @@
<artifactId>aws-java-sdk-bundle</artifactId> <artifactId>aws-java-sdk-bundle</artifactId>
<scope>compile</scope> <scope>compile</scope>
</dependency> </dependency>
<dependency>
<groupId>org.assertj</groupId>
<artifactId>assertj-core</artifactId>
<scope>test</scope>
</dependency>
<dependency> <dependency>
<groupId>junit</groupId> <groupId>junit</groupId>
<artifactId>junit</artifactId> <artifactId>junit</artifactId>

View File

@ -138,9 +138,15 @@ public final class Constants {
public static final String ASSUMED_ROLE_CREDENTIALS_DEFAULT = public static final String ASSUMED_ROLE_CREDENTIALS_DEFAULT =
SimpleAWSCredentialsProvider.NAME; SimpleAWSCredentialsProvider.NAME;
// the maximum number of tasks cached if all threads are already uploading
public static final String MAX_TOTAL_TASKS = "fs.s3a.max.total.tasks";
public static final int DEFAULT_MAX_TOTAL_TASKS = 32;
// number of simultaneous connections to s3 // number of simultaneous connections to s3
public static final String MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum"; public static final String MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum";
public static final int DEFAULT_MAXIMUM_CONNECTIONS = 15; public static final int DEFAULT_MAXIMUM_CONNECTIONS = 48;
// connect to s3 over ssl? // connect to s3 over ssl?
public static final String SECURE_CONNECTIONS = public static final String SECURE_CONNECTIONS =
@ -194,10 +200,6 @@ public final class Constants {
public static final String KEEPALIVE_TIME = "fs.s3a.threads.keepalivetime"; public static final String KEEPALIVE_TIME = "fs.s3a.threads.keepalivetime";
public static final int DEFAULT_KEEPALIVE_TIME = 60; public static final int DEFAULT_KEEPALIVE_TIME = 60;
// the maximum number of tasks cached if all threads are already uploading
public static final String MAX_TOTAL_TASKS = "fs.s3a.max.total.tasks";
public static final int DEFAULT_MAX_TOTAL_TASKS = 5;
// size of each of or multipart pieces in bytes // size of each of or multipart pieces in bytes
public static final String MULTIPART_SIZE = "fs.s3a.multipart.size"; public static final String MULTIPART_SIZE = "fs.s3a.multipart.size";
public static final long DEFAULT_MULTIPART_SIZE = 104857600; // 100 MB public static final long DEFAULT_MULTIPART_SIZE = 104857600; // 100 MB
@ -283,6 +285,22 @@ public final class Constants {
@InterfaceStability.Unstable @InterfaceStability.Unstable
public static final int DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS = 4; public static final int DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS = 4;
/**
* The capacity of executor queues for operations other than block
* upload, where {@link #FAST_UPLOAD_ACTIVE_BLOCKS} is used instead.
* This should be less than {@link #MAX_THREADS} for fair
* submission.
* Value: {@value}.
*/
public static final String EXECUTOR_CAPACITY = "fs.s3a.executor.capacity";
/**
* The capacity of executor queues for operations other than block
* upload, where {@link #FAST_UPLOAD_ACTIVE_BLOCKS} is used instead.
* Value: {@value}
*/
public static final int DEFAULT_EXECUTOR_CAPACITY = 16;
// Private | PublicRead | PublicReadWrite | AuthenticatedRead | // Private | PublicRead | PublicReadWrite | AuthenticatedRead |
// LogDeliveryWrite | BucketOwnerRead | BucketOwnerFullControl // LogDeliveryWrite | BucketOwnerRead | BucketOwnerFullControl
public static final String CANNED_ACL = "fs.s3a.acl.default"; public static final String CANNED_ACL = "fs.s3a.acl.default";

View File

@ -128,23 +128,22 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
* This does not attempt to open it; that is only done on the first * This does not attempt to open it; that is only done on the first
* actual read() operation. * actual read() operation.
* @param ctx operation context * @param ctx operation context
* @param s3Attributes object attributes from a HEAD request * @param s3Attributes object attributes
* @param contentLength length of content
* @param client S3 client to use * @param client S3 client to use
*/ */
public S3AInputStream(S3AReadOpContext ctx, public S3AInputStream(S3AReadOpContext ctx,
S3ObjectAttributes s3Attributes, S3ObjectAttributes s3Attributes,
long contentLength,
AmazonS3 client) { AmazonS3 client) {
Preconditions.checkArgument(isNotEmpty(s3Attributes.getBucket()), Preconditions.checkArgument(isNotEmpty(s3Attributes.getBucket()),
"No Bucket"); "No Bucket");
Preconditions.checkArgument(isNotEmpty(s3Attributes.getKey()), "No Key"); Preconditions.checkArgument(isNotEmpty(s3Attributes.getKey()), "No Key");
Preconditions.checkArgument(contentLength >= 0, "Negative content length"); long l = s3Attributes.getLen();
Preconditions.checkArgument(l >= 0, "Negative content length");
this.context = ctx; this.context = ctx;
this.bucket = s3Attributes.getBucket(); this.bucket = s3Attributes.getBucket();
this.key = s3Attributes.getKey(); this.key = s3Attributes.getKey();
this.pathStr = ctx.dstFileStatus.getPath().toString(); this.pathStr = ctx.dstFileStatus.getPath().toString();
this.contentLength = contentLength; this.contentLength = l;
this.client = client; this.client = client;
this.uri = "s3a://" + this.bucket + "/" + this.key; this.uri = "s3a://" + this.bucket + "/" + this.key;
this.streamStatistics = ctx.instrumentation.newInputStreamStatistics(); this.streamStatistics = ctx.instrumentation.newInputStreamStatistics();

View File

@ -183,10 +183,14 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
COMMITTER_MAGIC_FILES_CREATED, COMMITTER_MAGIC_FILES_CREATED,
S3GUARD_METADATASTORE_PUT_PATH_REQUEST, S3GUARD_METADATASTORE_PUT_PATH_REQUEST,
S3GUARD_METADATASTORE_INITIALIZATION, S3GUARD_METADATASTORE_INITIALIZATION,
S3GUARD_METADATASTORE_RECORD_DELETES,
S3GUARD_METADATASTORE_RECORD_READS,
S3GUARD_METADATASTORE_RECORD_WRITES,
S3GUARD_METADATASTORE_RETRY, S3GUARD_METADATASTORE_RETRY,
S3GUARD_METADATASTORE_THROTTLED, S3GUARD_METADATASTORE_THROTTLED,
STORE_IO_THROTTLED, STORE_IO_THROTTLED,
DELEGATION_TOKENS_ISSUED DELEGATION_TOKENS_ISSUED,
FILES_DELETE_REJECTED
}; };
private static final Statistic[] GAUGES_TO_CREATE = { private static final Statistic[] GAUGES_TO_CREATE = {
@ -1144,6 +1148,31 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
public void retrying() { public void retrying() {
// counters are incremented by owner. // counters are incremented by owner.
} }
/**
* Records have been read.
* @param count the number of records read
*/
public void recordsDeleted(int count) {
incrementCounter(S3GUARD_METADATASTORE_RECORD_DELETES, count);
}
/**
* Records have been read.
* @param count the number of records read
*/
public void recordsRead(int count) {
incrementCounter(S3GUARD_METADATASTORE_RECORD_READS, count);
}
/**
* records have been written (including deleted).
* @param count number of records written.
*/
public void recordsWritten(int count) {
incrementCounter(S3GUARD_METADATASTORE_RECORD_WRITES, count);
}
} }
/** /**

View File

@ -84,6 +84,7 @@ import java.util.concurrent.ExecutionException;
import static org.apache.commons.lang3.StringUtils.isEmpty; import static org.apache.commons.lang3.StringUtils.isEmpty;
import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.translateDeleteException;
/** /**
* Utility methods for S3A code. * Utility methods for S3A code.
@ -284,7 +285,7 @@ public final class S3AUtils {
case 200: case 200:
if (exception instanceof MultiObjectDeleteException) { if (exception instanceof MultiObjectDeleteException) {
// failure during a bulk delete // failure during a bulk delete
return translateMultiObjectDeleteException(message, return translateDeleteException(message,
(MultiObjectDeleteException) exception); (MultiObjectDeleteException) exception);
} }
// other 200: FALL THROUGH // other 200: FALL THROUGH
@ -448,40 +449,6 @@ public final class S3AUtils {
return result; return result;
} }
/**
* A MultiObjectDeleteException is raised if one or more delete objects
* listed in a bulk DELETE operation failed.
* The top-level exception is therefore just "something wasn't deleted",
* but doesn't include the what or the why.
* This translation will extract an AccessDeniedException if that's one of
* the causes, otherwise grabs the status code and uses it in the
* returned exception.
* @param message text for the exception
* @param ex exception to translate
* @return an IOE with more detail.
*/
public static IOException translateMultiObjectDeleteException(String message,
MultiObjectDeleteException ex) {
List<String> keys;
StringBuffer result = new StringBuffer(ex.getErrors().size() * 100);
result.append(message).append(": ");
String exitCode = "";
for (MultiObjectDeleteException.DeleteError error : ex.getErrors()) {
String code = error.getCode();
result.append(String.format("%s: %s: %s%n", code, error.getKey(),
error.getMessage()));
if (exitCode.isEmpty() || "AccessDenied".equals(code)) {
exitCode = code;
}
}
if ("AccessDenied".equals(exitCode)) {
return (IOException) new AccessDeniedException(result.toString())
.initCause(ex);
} else {
return new AWSS3IOException(result.toString(), ex);
}
}
/** /**
* Get low level details of an amazon exception for logging; multi-line. * Get low level details of an amazon exception for logging; multi-line.
* @param e exception * @param e exception
@ -1004,7 +971,7 @@ public final class S3AUtils {
* @return the value * @return the value
* @throws IllegalArgumentException if the value is below the minimum * @throws IllegalArgumentException if the value is below the minimum
*/ */
static int intOption(Configuration conf, String key, int defVal, int min) { public static int intOption(Configuration conf, String key, int defVal, int min) {
int v = conf.getInt(key, defVal); int v = conf.getInt(key, defVal);
Preconditions.checkArgument(v >= min, Preconditions.checkArgument(v >= min,
String.format("Value of %s: %d is below the minimum value %d", String.format("Value of %s: %d is below the minimum value %d",
@ -1022,7 +989,7 @@ public final class S3AUtils {
* @return the value * @return the value
* @throws IllegalArgumentException if the value is below the minimum * @throws IllegalArgumentException if the value is below the minimum
*/ */
static long longOption(Configuration conf, public static long longOption(Configuration conf,
String key, String key,
long defVal, long defVal,
long min) { long min) {
@ -1384,7 +1351,7 @@ public final class S3AUtils {
* @throws IOException anything in the closure, or iteration logic. * @throws IOException anything in the closure, or iteration logic.
*/ */
public static long applyLocatedFiles( public static long applyLocatedFiles(
RemoteIterator<LocatedFileStatus> iterator, RemoteIterator<? extends LocatedFileStatus> iterator,
CallOnLocatedFileStatus eval) throws IOException { CallOnLocatedFileStatus eval) throws IOException {
long count = 0; long count = 0;
while (iterator.hasNext()) { while (iterator.hasNext()) {
@ -1404,7 +1371,7 @@ public final class S3AUtils {
* @throws IOException anything in the closure, or iteration logic. * @throws IOException anything in the closure, or iteration logic.
*/ */
public static <T> List<T> mapLocatedFiles( public static <T> List<T> mapLocatedFiles(
RemoteIterator<LocatedFileStatus> iterator, RemoteIterator<? extends LocatedFileStatus> iterator,
LocatedFileStatusMap<T> eval) throws IOException { LocatedFileStatusMap<T> eval) throws IOException {
final List<T> results = new ArrayList<>(); final List<T> results = new ArrayList<>();
applyLocatedFiles(iterator, applyLocatedFiles(iterator,

View File

@ -18,38 +18,74 @@
package org.apache.hadoop.fs.s3a; package org.apache.hadoop.fs.s3a;
import com.amazonaws.services.s3.transfer.model.CopyResult;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.Path;
/** /**
* This class is only a holder for bucket, key, SSE Algorithm and SSE key * This class holds attributed of an object independent of the
* attributes. It is used in {@link S3AInputStream} and the select equivalent. * file status type.
* It is used in {@link S3AInputStream} and the select equivalent.
* as a way to reduce parameters being passed * as a way to reduce parameters being passed
* to the constructor of such class. * to the constructor of such class,
* and elsewhere to be a source-neutral representation of a file status.
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class S3ObjectAttributes { public class S3ObjectAttributes {
private final String bucket; private final String bucket;
private final Path path;
private final String key; private final String key;
private final S3AEncryptionMethods serverSideEncryptionAlgorithm; private final S3AEncryptionMethods serverSideEncryptionAlgorithm;
private final String serverSideEncryptionKey; private final String serverSideEncryptionKey;
private final String eTag; private final String eTag;
private final String versionId; private final String versionId;
private final long len;
public S3ObjectAttributes( public S3ObjectAttributes(
String bucket, String bucket,
Path path,
String key, String key,
S3AEncryptionMethods serverSideEncryptionAlgorithm, S3AEncryptionMethods serverSideEncryptionAlgorithm,
String serverSideEncryptionKey, String serverSideEncryptionKey,
String eTag, String eTag,
String versionId) { String versionId,
long len) {
this.bucket = bucket; this.bucket = bucket;
this.path = path;
this.key = key; this.key = key;
this.serverSideEncryptionAlgorithm = serverSideEncryptionAlgorithm; this.serverSideEncryptionAlgorithm = serverSideEncryptionAlgorithm;
this.serverSideEncryptionKey = serverSideEncryptionKey; this.serverSideEncryptionKey = serverSideEncryptionKey;
this.eTag = eTag; this.eTag = eTag;
this.versionId = versionId; this.versionId = versionId;
this.len = len;
}
/**
* Construct from the result of a copy and those parameters
* which aren't included in an AWS SDK response.
* @param path
* @param copyResult copy result.
* @param serverSideEncryptionAlgorithm current encryption algorithm
* @param serverSideEncryptionKey any server side encryption key?
* @param len
*/
public S3ObjectAttributes(
final Path path,
final CopyResult copyResult,
final S3AEncryptionMethods serverSideEncryptionAlgorithm,
final String serverSideEncryptionKey,
final long len) {
this.bucket = copyResult.getDestinationBucketName();
this.key = copyResult.getDestinationKey();
this.path = path;
this.serverSideEncryptionAlgorithm = serverSideEncryptionAlgorithm;
this.serverSideEncryptionKey = serverSideEncryptionKey;
this.eTag = copyResult.getETag();
this.versionId = copyResult.getVersionId();
this.len = len;
} }
public String getBucket() { public String getBucket() {
@ -75,4 +111,12 @@ public class S3ObjectAttributes {
public String getVersionId() { public String getVersionId() {
return versionId; return versionId;
} }
public long getLen() {
return len;
}
public Path getPath() {
return path;
}
} }

View File

@ -42,6 +42,8 @@ public enum Statistic {
"Total number of files created through the object store."), "Total number of files created through the object store."),
FILES_DELETED("files_deleted", FILES_DELETED("files_deleted",
"Total number of files deleted from the object store."), "Total number of files deleted from the object store."),
FILES_DELETE_REJECTED("files_delete_rejected",
"Total number of files whose delete request was rejected"),
FAKE_DIRECTORIES_CREATED("fake_directories_created", FAKE_DIRECTORIES_CREATED("fake_directories_created",
"Total number of fake directory entries created in the object store."), "Total number of fake directory entries created in the object store."),
FAKE_DIRECTORIES_DELETED("fake_directories_deleted", FAKE_DIRECTORIES_DELETED("fake_directories_deleted",
@ -207,6 +209,15 @@ public enum Statistic {
"S3Guard metadata store put one metadata path latency"), "S3Guard metadata store put one metadata path latency"),
S3GUARD_METADATASTORE_INITIALIZATION("s3guard_metadatastore_initialization", S3GUARD_METADATASTORE_INITIALIZATION("s3guard_metadatastore_initialization",
"S3Guard metadata store initialization times"), "S3Guard metadata store initialization times"),
S3GUARD_METADATASTORE_RECORD_DELETES(
"s3guard_metadatastore_record_deletes",
"S3Guard metadata store records deleted"),
S3GUARD_METADATASTORE_RECORD_READS(
"s3guard_metadatastore_record_reads",
"S3Guard metadata store records read"),
S3GUARD_METADATASTORE_RECORD_WRITES(
"s3guard_metadatastore_record_writes",
"S3Guard metadata store records written"),
S3GUARD_METADATASTORE_RETRY("s3guard_metadatastore_retry", S3GUARD_METADATASTORE_RETRY("s3guard_metadatastore_retry",
"S3Guard metadata store retry events"), "S3Guard metadata store retry events"),
S3GUARD_METADATASTORE_THROTTLED("s3guard_metadatastore_throttled", S3GUARD_METADATASTORE_THROTTLED("s3guard_metadatastore_throttled",

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.fs.s3a; package org.apache.hadoop.fs.s3a;
import javax.annotation.Nullable;
import java.io.File; import java.io.File;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
@ -41,13 +42,15 @@ import com.amazonaws.services.s3.model.UploadPartRequest;
import com.amazonaws.services.s3.model.UploadPartResult; import com.amazonaws.services.s3.model.UploadPartResult;
import com.amazonaws.services.s3.transfer.model.UploadResult; import com.amazonaws.services.s3.transfer.model.UploadResult;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
import org.apache.hadoop.fs.s3a.select.SelectBinding; import org.apache.hadoop.fs.s3a.select.SelectBinding;
import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.DurationInfo;
@ -226,7 +229,7 @@ public class WriteOperationHelper {
/** /**
* Finalize a multipart PUT operation. * Finalize a multipart PUT operation.
* This completes the upload, and, if that works, calls * This completes the upload, and, if that works, calls
* {@link S3AFileSystem#finishedWrite(String, long, String, String)} * {@link S3AFileSystem#finishedWrite(String, long, String, String, BulkOperationState)}
* to update the filesystem. * to update the filesystem.
* Retry policy: retrying, translated. * Retry policy: retrying, translated.
* @param destKey destination of the commit * @param destKey destination of the commit
@ -234,6 +237,7 @@ public class WriteOperationHelper {
* @param partETags list of partial uploads * @param partETags list of partial uploads
* @param length length of the upload * @param length length of the upload
* @param retrying retrying callback * @param retrying retrying callback
* @param operationState (nullable) operational state for a bulk update
* @return the result of the operation. * @return the result of the operation.
* @throws IOException on problems. * @throws IOException on problems.
*/ */
@ -243,7 +247,8 @@ public class WriteOperationHelper {
String uploadId, String uploadId,
List<PartETag> partETags, List<PartETag> partETags,
long length, long length,
Retried retrying) throws IOException { Retried retrying,
@Nullable BulkOperationState operationState) throws IOException {
if (partETags.isEmpty()) { if (partETags.isEmpty()) {
throw new IOException( throw new IOException(
"No upload parts in multipart upload to " + destKey); "No upload parts in multipart upload to " + destKey);
@ -263,7 +268,7 @@ public class WriteOperationHelper {
} }
); );
owner.finishedWrite(destKey, length, uploadResult.getETag(), owner.finishedWrite(destKey, length, uploadResult.getETag(),
uploadResult.getVersionId()); uploadResult.getVersionId(), operationState);
return uploadResult; return uploadResult;
} }
@ -298,7 +303,8 @@ public class WriteOperationHelper {
uploadId, uploadId,
partETags, partETags,
length, length,
(text, e, r, i) -> errorCount.incrementAndGet()); (text, e, r, i) -> errorCount.incrementAndGet(),
null);
} }
/** /**
@ -489,6 +495,52 @@ public class WriteOperationHelper {
); );
} }
/**
* This completes a multipart upload to the destination key via
* {@code finalizeMultipartUpload()}.
* Retry policy: retrying, translated.
* Retries increment the {@code errorCount} counter.
* @param destKey destination
* @param uploadId multipart operation Id
* @param partETags list of partial uploads
* @param length length of the upload
* @param operationState operational state for a bulk update
* @return the result of the operation.
* @throws IOException if problems arose which could not be retried, or
* the retry count was exceeded
*/
@Retries.RetryTranslated
public CompleteMultipartUploadResult commitUpload(
String destKey,
String uploadId,
List<PartETag> partETags,
long length,
@Nullable BulkOperationState operationState)
throws IOException {
checkNotNull(uploadId);
checkNotNull(partETags);
LOG.debug("Completing multipart upload {} with {} parts",
uploadId, partETags.size());
return finalizeMultipartUpload(destKey,
uploadId,
partETags,
length,
Invoker.NO_OP,
operationState);
}
/**
* Initiate a commit operation through any metastore.
* @param path path under which the writes will all take place.
* @return an possibly null operation state from the metastore.
* @throws IOException failure to instantiate.
*/
public BulkOperationState initiateCommitOperation(
Path path) throws IOException {
return S3Guard.initiateBulkWrite(owner.getMetadataStore(),
BulkOperationState.OperationType.Put, path);
}
/** /**
* Upload part of a multi-partition file. * Upload part of a multi-partition file.
* @param request request * @param request request

View File

@ -442,14 +442,27 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
} }
LOG.debug("{}: committing the output of {} task(s)", LOG.debug("{}: committing the output of {} task(s)",
getRole(), pending.size()); getRole(), pending.size());
Tasks.foreach(pending) try(CommitOperations.CommitContext commitContext
.stopOnFailure() = initiateCommitOperation()) {
.executeWith(buildThreadPool(context)) Tasks.foreach(pending)
.onFailure((commit, exception) -> .stopOnFailure()
getCommitOperations().abortSingleCommit(commit)) .executeWith(buildThreadPool(context))
.abortWith(commit -> getCommitOperations().abortSingleCommit(commit)) .onFailure((commit, exception) ->
.revertWith(commit -> getCommitOperations().revertCommit(commit)) commitContext.abortSingleCommit(commit))
.run(commit -> getCommitOperations().commitOrFail(commit)); .abortWith(commitContext::abortSingleCommit)
.revertWith(commitContext::revertCommit)
.run(commitContext::commitOrFail);
}
}
/**
* Start the final commit/abort commit operations.
* @return a commit context through which the operations can be invoked.
* @throws IOException failure.
*/
protected CommitOperations.CommitContext initiateCommitOperation()
throws IOException {
return getCommitOperations().initiateCommitOperation(getOutputPath());
} }
/** /**
@ -531,7 +544,9 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
Path dest = getOutputPath(); Path dest = getOutputPath();
try (DurationInfo d = try (DurationInfo d =
new DurationInfo(LOG, "Aborting all pending commits under %s", new DurationInfo(LOG, "Aborting all pending commits under %s",
dest)) { dest);
CommitOperations.CommitContext commitContext
= initiateCommitOperation()) {
CommitOperations ops = getCommitOperations(); CommitOperations ops = getCommitOperations();
List<MultipartUpload> pending; List<MultipartUpload> pending;
try { try {
@ -544,7 +559,8 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
Tasks.foreach(pending) Tasks.foreach(pending)
.executeWith(buildThreadPool(getJobContext())) .executeWith(buildThreadPool(getJobContext()))
.suppressExceptions(suppressExceptions) .suppressExceptions(suppressExceptions)
.run(u -> ops.abortMultipartCommit(u.getKey(), u.getUploadId())); .run(u -> commitContext.abortMultipartCommit(
u.getKey(), u.getUploadId()));
} }
} }
@ -752,11 +768,13 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
LOG.info("{}: no pending commits to abort", getRole()); LOG.info("{}: no pending commits to abort", getRole());
} else { } else {
try (DurationInfo d = new DurationInfo(LOG, try (DurationInfo d = new DurationInfo(LOG,
"Aborting %s uploads", pending.size())) { "Aborting %s uploads", pending.size());
CommitOperations.CommitContext commitContext
= initiateCommitOperation()) {
Tasks.foreach(pending) Tasks.foreach(pending)
.executeWith(buildThreadPool(context)) .executeWith(buildThreadPool(context))
.suppressExceptions(suppressExceptions) .suppressExceptions(suppressExceptions)
.run(commit -> getCommitOperations().abortSingleCommit(commit)); .run(commitContext::abortSingleCommit);
} }
} }
} }

View File

@ -18,13 +18,14 @@
package org.apache.hadoop.fs.s3a.commit; package org.apache.hadoop.fs.s3a.commit;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.io.File; import java.io.File;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.IntStream; import java.util.stream.IntStream;
@ -49,6 +50,9 @@ import org.apache.hadoop.fs.s3a.WriteOperationHelper;
import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.DurationInfo;
import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.S3AUtils.*;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
@ -128,10 +132,13 @@ public class CommitOperations {
/** /**
* Commit the operation, throwing an exception on any failure. * Commit the operation, throwing an exception on any failure.
* @param commit commit to execute * @param commit commit to execute
* @param operationState S3Guard state of ongoing operation.
* @throws IOException on a failure * @throws IOException on a failure
*/ */
public void commitOrFail(SinglePendingCommit commit) throws IOException { private void commitOrFail(
commit(commit, commit.getFilename()).maybeRethrow(); final SinglePendingCommit commit,
final BulkOperationState operationState) throws IOException {
commit(commit, commit.getFilename(), operationState).maybeRethrow();
} }
/** /**
@ -139,16 +146,20 @@ public class CommitOperations {
* and converted to an outcome. * and converted to an outcome.
* @param commit entry to commit * @param commit entry to commit
* @param origin origin path/string for outcome text * @param origin origin path/string for outcome text
* @param operationState S3Guard state of ongoing operation.
* @return the outcome * @return the outcome
*/ */
public MaybeIOE commit(SinglePendingCommit commit, String origin) { private MaybeIOE commit(
final SinglePendingCommit commit,
final String origin,
final BulkOperationState operationState) {
LOG.debug("Committing single commit {}", commit); LOG.debug("Committing single commit {}", commit);
MaybeIOE outcome; MaybeIOE outcome;
String destKey = "unknown destination"; String destKey = "unknown destination";
try { try {
commit.validate(); commit.validate();
destKey = commit.getDestinationKey(); destKey = commit.getDestinationKey();
long l = innerCommit(commit); long l = innerCommit(commit, operationState);
LOG.debug("Successful commit of file length {}", l); LOG.debug("Successful commit of file length {}", l);
outcome = MaybeIOE.NONE; outcome = MaybeIOE.NONE;
statistics.commitCompleted(commit.getLength()); statistics.commitCompleted(commit.getLength());
@ -171,17 +182,20 @@ public class CommitOperations {
/** /**
* Inner commit operation. * Inner commit operation.
* @param commit entry to commit * @param commit entry to commit
* @param operationState S3Guard state of ongoing operation.
* @return bytes committed. * @return bytes committed.
* @throws IOException failure * @throws IOException failure
*/ */
private long innerCommit(SinglePendingCommit commit) throws IOException { private long innerCommit(
final SinglePendingCommit commit,
final BulkOperationState operationState) throws IOException {
// finalize the commit // finalize the commit
writeOperations.completeMPUwithRetries( writeOperations.commitUpload(
commit.getDestinationKey(), commit.getDestinationKey(),
commit.getUploadId(), commit.getUploadId(),
toPartEtags(commit.getEtags()), toPartEtags(commit.getEtags()),
commit.getLength(), commit.getLength(),
new AtomicInteger(0)); operationState);
return commit.getLength(); return commit.getLength();
} }
@ -249,7 +263,7 @@ public class CommitOperations {
* @throws FileNotFoundException if the abort ID is unknown * @throws FileNotFoundException if the abort ID is unknown
* @throws IOException on any failure * @throws IOException on any failure
*/ */
public void abortSingleCommit(SinglePendingCommit commit) private void abortSingleCommit(SinglePendingCommit commit)
throws IOException { throws IOException {
String destKey = commit.getDestinationKey(); String destKey = commit.getDestinationKey();
String origin = commit.getFilename() != null String origin = commit.getFilename() != null
@ -268,7 +282,7 @@ public class CommitOperations {
* @throws FileNotFoundException if the abort ID is unknown * @throws FileNotFoundException if the abort ID is unknown
* @throws IOException on any failure * @throws IOException on any failure
*/ */
public void abortMultipartCommit(String destKey, String uploadId) private void abortMultipartCommit(String destKey, String uploadId)
throws IOException { throws IOException {
try { try {
writeOperations.abortMultipartCommit(destKey, uploadId); writeOperations.abortMultipartCommit(destKey, uploadId);
@ -392,7 +406,10 @@ public class CommitOperations {
Path markerPath = new Path(outputPath, _SUCCESS); Path markerPath = new Path(outputPath, _SUCCESS);
LOG.debug("Touching success marker for job {}: {}", markerPath, LOG.debug("Touching success marker for job {}: {}", markerPath,
successData); successData);
successData.save(fs, markerPath, true); try (DurationInfo ignored = new DurationInfo(LOG,
"Writing success file %s", markerPath)) {
successData.save(fs, markerPath, true);
}
} }
/** /**
@ -401,7 +418,7 @@ public class CommitOperations {
* @throws IOException failure * @throws IOException failure
*/ */
public void revertCommit(SinglePendingCommit commit) throws IOException { public void revertCommit(SinglePendingCommit commit) throws IOException {
LOG.warn("Revert {}", commit); LOG.info("Revert {}", commit);
try { try {
writeOperations.revertCommit(commit.getDestinationKey()); writeOperations.revertCommit(commit.getDestinationKey());
} finally { } finally {
@ -520,6 +537,120 @@ public class CommitOperations {
statistics.jobCompleted(success); statistics.jobCompleted(success);
} }
/**
* Begin the final commit.
* @param path path for all work.
* @return the commit context to pass in.
* @throws IOException failure.
*/
public CommitContext initiateCommitOperation(Path path) throws IOException {
return new CommitContext(writeOperations.initiateCommitOperation(path));
}
/**
* Commit context.
*
* It is used to manage the final commit sequence where files become
* visible. It contains a {@link BulkOperationState} field, which, if
* there is a metastore, will be requested from the store so that it
* can track multiple creation operations within the same overall operation.
* This will be null if there is no metastore, or the store chooses not
* to provide one.
*
* This can only be created through {@link #initiateCommitOperation(Path)}.
*
* Once the commit operation has completed, it must be closed.
* It must not be reused.
*/
public final class CommitContext implements Closeable {
/**
* State of any metastore.
*/
private final BulkOperationState operationState;
/**
* Create.
* @param operationState any S3Guard bulk state.
*/
private CommitContext(@Nullable final BulkOperationState operationState) {
this.operationState = operationState;
}
/**
* Commit the operation, throwing an exception on any failure.
* See {@link CommitOperations#commitOrFail(SinglePendingCommit, BulkOperationState)}.
* @param commit commit to execute
* @throws IOException on a failure
*/
public void commitOrFail(SinglePendingCommit commit) throws IOException {
CommitOperations.this.commitOrFail(commit, operationState);
}
/**
* Commit a single pending commit; exceptions are caught
* and converted to an outcome.
* See {@link CommitOperations#commit(SinglePendingCommit, String, BulkOperationState)}.
* @param commit entry to commit
* @param origin origin path/string for outcome text
* @return the outcome
*/
public MaybeIOE commit(SinglePendingCommit commit,
String origin) {
return CommitOperations.this.commit(commit, origin, operationState);
}
/**
* See {@link CommitOperations#abortSingleCommit(SinglePendingCommit)}.
* @param commit pending commit to abort
* @throws FileNotFoundException if the abort ID is unknown
* @throws IOException on any failure
*/
public void abortSingleCommit(final SinglePendingCommit commit)
throws IOException {
CommitOperations.this.abortSingleCommit(commit);
}
/**
* See {@link CommitOperations#revertCommit(SinglePendingCommit)}.
* @param commit pending commit
* @throws IOException failure
*/
public void revertCommit(final SinglePendingCommit commit)
throws IOException {
CommitOperations.this.revertCommit(commit);
}
/**
* See {@link CommitOperations#abortMultipartCommit(String, String)}..
* @param destKey destination key
* @param uploadId upload to cancel
* @throws FileNotFoundException if the abort ID is unknown
* @throws IOException on any failure
*/
public void abortMultipartCommit(
final String destKey,
final String uploadId)
throws IOException {
CommitOperations.this.abortMultipartCommit(destKey, uploadId);
}
@Override
public void close() throws IOException {
IOUtils.cleanupWithLogger(LOG, operationState);
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder(
"CommitContext{");
sb.append("operationState=").append(operationState);
sb.append('}');
return sb.toString();
}
}
/** /**
* A holder for a possible IOException; the call {@link #maybeRethrow()} * A holder for a possible IOException; the call {@link #maybeRethrow()}
* will throw any exception passed into the constructor, and be a no-op * will throw any exception passed into the constructor, and be a no-op

View File

@ -123,8 +123,11 @@ public class MagicS3GuardCommitter extends AbstractS3ACommitter {
*/ */
public void cleanupStagingDirs() { public void cleanupStagingDirs() {
Path path = magicSubdir(getOutputPath()); Path path = magicSubdir(getOutputPath());
Invoker.ignoreIOExceptions(LOG, "cleanup magic directory", path.toString(), try(DurationInfo ignored = new DurationInfo(LOG, true,
() -> deleteWithWarning(getDestFS(), path, true)); "Deleting magic directory %s", path)) {
Invoker.ignoreIOExceptions(LOG, "cleanup magic directory", path.toString(),
() -> deleteWithWarning(getDestFS(), path, true));
}
} }
@Override @Override

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter; import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter;
import org.apache.hadoop.fs.s3a.commit.CommitConstants; import org.apache.hadoop.fs.s3a.commit.CommitConstants;
import org.apache.hadoop.fs.s3a.commit.CommitOperations;
import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants;
import org.apache.hadoop.fs.s3a.commit.Tasks; import org.apache.hadoop.fs.s3a.commit.Tasks;
import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
@ -729,9 +730,14 @@ public class StagingCommitter extends AbstractS3ACommitter {
LOG.error( LOG.error(
"{}: Exception during commit process, aborting {} commit(s)", "{}: Exception during commit process, aborting {} commit(s)",
getRole(), commits.size()); getRole(), commits.size());
Tasks.foreach(commits) try(CommitOperations.CommitContext commitContext
.suppressExceptions() = initiateCommitOperation();
.run(commit -> getCommitOperations().abortSingleCommit(commit)); DurationInfo ignored = new DurationInfo(LOG,
"Aborting %s uploads", commits.size())) {
Tasks.foreach(commits)
.suppressExceptions()
.run(commitContext::abortSingleCommit);
}
deleteTaskAttemptPathQuietly(context); deleteTaskAttemptPathQuietly(context);
} }
} }

View File

@ -0,0 +1,49 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.impl;
import static com.google.common.base.Preconditions.checkNotNull;
/**
* Base class of operations in the store.
* An operation is something which executes against the context to
* perform a single function.
* It is expected to have a limited lifespan.
*/
public abstract class AbstractStoreOperation {
private final StoreContext storeContext;
/**
* constructor.
* @param storeContext store context.
*/
protected AbstractStoreOperation(final StoreContext storeContext) {
this.storeContext = checkNotNull(storeContext);
}
/**
* Get the store context.
* @return the context.
*/
public final StoreContext getStoreContext() {
return storeContext;
}
}

View File

@ -0,0 +1,126 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.impl;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.CancellationException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.Executor;
import java.util.function.Supplier;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.impl.WrappedIOException;
import org.apache.hadoop.util.DurationInfo;
import static org.apache.hadoop.fs.impl.FutureIOSupport.raiseInnerCause;
/**
* A bridge from Callable to Supplier; catching exceptions
* raised by the callable and wrapping them as appropriate.
* @param <T> return type.
*/
public final class CallableSupplier<T> implements Supplier {
private static final Logger LOG =
LoggerFactory.getLogger(CallableSupplier.class);
private final Callable<T> call;
/**
* Create.
* @param call call to invoke.
*/
public CallableSupplier(final Callable<T> call) {
this.call = call;
}
@Override
public Object get() {
try {
return call.call();
} catch (RuntimeException e) {
throw e;
} catch (IOException e) {
throw new WrappedIOException(e);
} catch (Exception e) {
throw new WrappedIOException(new IOException(e));
}
}
/**
* Submit a callable into a completable future.
* RTEs are rethrown.
* Non RTEs are caught and wrapped; IOExceptions to
* {@link WrappedIOException} instances.
* @param executor executor.
* @param call call to invoke
* @param <T> type
* @return the future to wait for
*/
@SuppressWarnings("unchecked")
public static <T> CompletableFuture<T> submit(
final Executor executor,
final Callable<T> call) {
return CompletableFuture.supplyAsync(
new CallableSupplier<T>(call), executor);
}
/**
* Wait for a list of futures to complete. If the list is empty,
* return immediately.
* @param futures list of futures.
* @throws IOException if one of the called futures raised an IOE.
* @throws RuntimeException if one of the futures raised one.
*/
public static <T> void waitForCompletion(
final List<CompletableFuture<T>> futures)
throws IOException {
if (futures.isEmpty()) {
return;
}
// await completion
waitForCompletion(CompletableFuture.allOf(
futures.toArray(new CompletableFuture[0])));
}
/**
* Wait for a single of future to complete, extracting IOEs afterwards.
* @param future future to wait for.
* @throws IOException if one of the called futures raised an IOE.
* @throws RuntimeException if one of the futures raised one.
*/
public static <T> void waitForCompletion(
final CompletableFuture<T> future)
throws IOException {
try (DurationInfo ignore =
new DurationInfo(LOG, false, "Waiting for task completion")) {
future.join();
} catch (CancellationException e) {
throw new IOException(e);
} catch (CompletionException e) {
raiseInnerCause(e);
}
}
}

View File

@ -0,0 +1,74 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.impl;
import java.io.File;
import java.io.IOException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.Retries;
/**
* An interface to implement for providing accessors to
* S3AFileSystem-level API calls.
* <p>
* This is used to avoid giving any explicit reference to the owning
* FS in the store context; there are enough calls that using lambda-expressions
* gets over-complex.
* <ol>
* <li>Test suites are free to provide their own implementation, using
* * the S3AFileSystem methods as the normative reference.</li>
* <li>All implementations <i>MUST</i> translate exceptions.</li>
* </ol>
*/
public interface ContextAccessors {
/**
* Convert a key to a fully qualified path.
* @param key input key
* @return the fully qualified path including URI scheme and bucket name.
*/
Path keyToPath(String key);
/**
* Turns a path (relative or otherwise) into an S3 key.
*
* @param path input path, may be relative to the working dir
* @return a key excluding the leading "/", or, if it is the root path, ""
*/
String pathToKey(Path path);
/**
* Create a temporary file.
* @param prefix prefix for the temporary file
* @param size the size of the file that is going to be written
* @return a unique temporary file
* @throws IOException IO problems
*/
File createTempFile(String prefix, long size) throws IOException;
/**
* Get the region of a bucket. This may be via an S3 API call if not
* already cached.
* @return the region in which a bucket is located
* @throws IOException on any failure.
*/
@Retries.RetryTranslated
String getBucketLocation() throws IOException;
}

View File

@ -0,0 +1,60 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.impl;
/**
* Internal constants private only to the S3A codebase.
*/
public final class InternalConstants {
private InternalConstants() {
}
/**
* This is an arbitrary value: {@value}.
* It declares how many parallel copy operations
* in a single rename can be queued before the operation pauses
* and awaits completion.
* A very large value wouldn't just starve other threads from
* performing work, there's a risk that the S3 store itself would
* throttle operations (which all go to the same shard).
* It is not currently configurable just to avoid people choosing values
* which work on a microbenchmark (single rename, no other work, ...)
* but don't scale well to execution in a large process against a common
* store, all while separate processes are working with the same shard
* of storage.
*
* It should be a factor of {@link #MAX_ENTRIES_TO_DELETE} so that
* all copies will have finished before deletion is contemplated.
* (There's always a block for that, it just makes more sense to
* perform the bulk delete after another block of copies have completed).
*/
public static final int RENAME_PARALLEL_LIMIT = 10;
/**
* The maximum number of entries that can be deleted in any bulk delete
* call to S3: {@value}.
*/
public static final int MAX_ENTRIES_TO_DELETE = 1000;
/**
* Default blocksize as used in blocksize and FS status queries: {@value}.
*/
public static final int DEFAULT_BLOCKSIZE = 32 * 1024 * 1024;
}

View File

@ -0,0 +1,244 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.impl;
import java.io.IOException;
import java.nio.file.AccessDeniedException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.function.Function;
import java.util.stream.Collectors;
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.commons.lang3.tuple.Triple;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.AWSS3IOException;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
import static com.google.common.base.Preconditions.checkNotNull;
/**
* Support for Multi Object Deletion.
*/
public final class MultiObjectDeleteSupport extends AbstractStoreOperation {
private static final Logger LOG = LoggerFactory.getLogger(
MultiObjectDeleteSupport.class);
/**
* Initiate with a store context.
* @param context store context.
*/
public MultiObjectDeleteSupport(final StoreContext context) {
super(context);
}
/**
* This is the exception exit code if access was denied on a delete.
* {@value}.
*/
public static final String ACCESS_DENIED = "AccessDenied";
/**
* A {@code MultiObjectDeleteException} is raised if one or more
* paths listed in a bulk DELETE operation failed.
* The top-level exception is therefore just "something wasn't deleted",
* but doesn't include the what or the why.
* This translation will extract an AccessDeniedException if that's one of
* the causes, otherwise grabs the status code and uses it in the
* returned exception.
* @param message text for the exception
* @param deleteException the delete exception. to translate
* @return an IOE with more detail.
*/
public static IOException translateDeleteException(
final String message,
final MultiObjectDeleteException deleteException) {
final StringBuilder result = new StringBuilder(
deleteException.getErrors().size() * 256);
result.append(message).append(": ");
String exitCode = "";
for (MultiObjectDeleteException.DeleteError error :
deleteException.getErrors()) {
String code = error.getCode();
result.append(String.format("%s: %s: %s%n", code, error.getKey(),
error.getMessage()));
if (exitCode.isEmpty() || ACCESS_DENIED.equals(code)) {
exitCode = code;
}
}
if (ACCESS_DENIED.equals(exitCode)) {
return (IOException) new AccessDeniedException(result.toString())
.initCause(deleteException);
} else {
return new AWSS3IOException(result.toString(), deleteException);
}
}
/**
* Process a multi object delete exception by building two paths from
* the delete request: one of all deleted files, one of all undeleted values.
* The latter are those rejected in the delete call.
* @param deleteException the delete exception.
* @param keysToDelete the keys in the delete request
* @return tuple of (undeleted, deleted) paths.
*/
public Pair<List<Path>, List<Path>> splitUndeletedKeys(
final MultiObjectDeleteException deleteException,
final Collection<DeleteObjectsRequest.KeyVersion> keysToDelete) {
LOG.debug("Processing delete failure; keys to delete count = {};"
+ " errors in exception {}; successful deletions = {}",
keysToDelete.size(),
deleteException.getErrors().size(),
deleteException.getDeletedObjects().size());
// convert the collection of keys being deleted into paths
final List<Path> pathsBeingDeleted = keysToPaths(keysToDelete);
// Take this is list of paths
// extract all undeleted entries contained in the exception and
// then removes them from the original list.
List<Path> undeleted = removeUndeletedPaths(deleteException,
pathsBeingDeleted,
getStoreContext()::keyToPath);
return Pair.of(undeleted, pathsBeingDeleted);
}
/**
* Given a list of delete requests, convert them all to paths.
* @param keysToDelete list of keys for the delete operation.
* @return the paths.
*/
public List<Path> keysToPaths(
final Collection<DeleteObjectsRequest.KeyVersion> keysToDelete) {
return convertToPaths(keysToDelete,
getStoreContext()::keyToPath);
}
/**
* Given a list of delete requests, convert them all to paths.
* @param keysToDelete list of keys for the delete operation.
* @param qualifier path qualifier
* @return the paths.
*/
public static List<Path> convertToPaths(
final Collection<DeleteObjectsRequest.KeyVersion> keysToDelete,
final Function<String, Path> qualifier) {
return keysToDelete.stream()
.map((keyVersion) ->
qualifier.apply(keyVersion.getKey()))
.collect(Collectors.toList());
}
/**
* Process a delete failure by removing from the metastore all entries
* which where deleted, as inferred from the delete failures exception
* and the original list of files to delete declares to have been deleted.
* @param deleteException the delete exception.
* @param keysToDelete collection of keys which had been requested.
* @return a tuple of (undeleted, deleted, failures)
*/
public Triple<List<Path>, List<Path>, List<Pair<Path, IOException>>>
processDeleteFailure(
final MultiObjectDeleteException deleteException,
final List<DeleteObjectsRequest.KeyVersion> keysToDelete) {
final MetadataStore metadataStore =
checkNotNull(getStoreContext().getMetadataStore(),
"context metadatastore");
final List<Pair<Path, IOException>> failures = new ArrayList<>();
final Pair<List<Path>, List<Path>> outcome =
splitUndeletedKeys(deleteException, keysToDelete);
List<Path> deleted = outcome.getRight();
List<Path> undeleted = outcome.getLeft();
// delete the paths but recover
// TODO: handle the case where a parent path is deleted but not a child.
// TODO: in a fake object delete, we don't actually want to delete
// metastore entries
deleted.forEach(path -> {
try {
metadataStore.delete(path, getStoreContext().getTimeProvider());
} catch (IOException e) {
// trouble: we failed to delete the far end entry
// try with the next one.
// if this is a big network failure, this is going to be noisy.
LOG.warn("Failed to update S3Guard store with deletion of {}", path);
failures.add(Pair.of(path, e));
}
});
if (LOG.isDebugEnabled()) {
undeleted.forEach(p -> LOG.debug("Deleted {}", p));
}
return Triple.of(undeleted, deleted, failures);
}
/**
* Build a list of undeleted paths from a {@code MultiObjectDeleteException}.
* Outside of unit tests, the qualifier function should be
* {@link S3AFileSystem#keyToQualifiedPath(String)}.
* @param deleteException the delete exception.
* @param qualifierFn function to qualify paths
* @return the possibly empty list of paths.
*/
@VisibleForTesting
public static List<Path> extractUndeletedPaths(
final MultiObjectDeleteException deleteException,
final Function<String, Path> qualifierFn) {
return deleteException.getErrors().stream()
.map((e) -> qualifierFn.apply(e.getKey()))
.collect(Collectors.toList());
}
/**
* Process a {@code MultiObjectDeleteException} by
* removing all undeleted paths from the list of paths being deleted.
* The original list is updated, and so becomes the list of successfully
* deleted paths.
* @param deleteException the delete exception.
* @param pathsBeingDeleted list of paths which were being deleted.
* This has all undeleted paths removed, leaving only those deleted.
* @return the list of undeleted entries
*/
@VisibleForTesting
static List<Path> removeUndeletedPaths(
final MultiObjectDeleteException deleteException,
final Collection<Path> pathsBeingDeleted,
final Function<String, Path> qualifier) {
List<Path> undeleted = extractUndeletedPaths(deleteException, qualifier);
pathsBeingDeleted.removeAll(undeleted);
return undeleted;
}
/**
* A delete operation failed.
* Currently just returns the list of all paths.
* @param ex exception.
* @param keysToDelete the keys which were being deleted.
* @return all paths which were not deleted.
*/
public List<Path> processDeleteFailureGenericException(Exception ex,
final List<DeleteObjectsRequest.KeyVersion> keysToDelete) {
return keysToPaths(keysToDelete);
}
}

View File

@ -0,0 +1,634 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.impl;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import com.amazonaws.AmazonClientException;
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
import com.amazonaws.services.s3.transfer.model.CopyResult;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.InvalidRequestException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.s3a.RenameFailedException;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
import org.apache.hadoop.fs.s3a.Tristate;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
import org.apache.hadoop.fs.s3a.s3guard.RenameTracker;
import org.apache.hadoop.util.DurationInfo;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_BLOCK_SIZE;
import static org.apache.hadoop.fs.s3a.S3AUtils.objectRepresentsDirectory;
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_BLOCKSIZE;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.MAX_ENTRIES_TO_DELETE;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.RENAME_PARALLEL_LIMIT;
/**
* A parallelized rename operation which updates the metastore in the
* process, through whichever {@link RenameTracker} the store provides.
* The parallel execution is in groups of size
* {@link InternalConstants#RENAME_PARALLEL_LIMIT}; it is only
* after one group completes that the next group is initiated.
* Once enough files have been copied that they meet the
* {@link InternalConstants#MAX_ENTRIES_TO_DELETE} threshold, a delete
* is initiated.
* If it succeeds, the rename continues with the next group of files.
*
* The RenameTracker has the task of keeping the metastore up to date
* as the rename proceeds.
*
* The rename operation implements the classic HDFS rename policy of
* rename(file, dir) renames the file under the directory.
*
* There is <i>no</i> validation of input and output paths.
* Callers are required to themselves verify that destination is not under
* the source, above the source, the source itself, etc, etc.
*/
public class RenameOperation extends AbstractStoreOperation {
private static final Logger LOG = LoggerFactory.getLogger(
RenameOperation.class);
/**
* Used to stop any re-entrancy of the rename.
* This is an execute-once operation.
*/
private final AtomicBoolean executed = new AtomicBoolean(false);
private final Path sourcePath;
private final String sourceKey;
private final S3AFileStatus sourceStatus;
private final Path destPath;
private final String destKey;
private final S3AFileStatus destStatus;
/**
* Callbacks into the filesystem.
*/
private final RenameOperationCallbacks callbacks;
/**
* Counter of bytes copied.
*/
private final AtomicLong bytesCopied = new AtomicLong();
/**
* Rename tracker.
*/
private RenameTracker renameTracker;
/**
* List of active copies.
*/
private final List<CompletableFuture<Path>> activeCopies =
new ArrayList<>(RENAME_PARALLEL_LIMIT);
/**
* list of keys to delete on the next (bulk) delete call.
*/
private final List<DeleteObjectsRequest.KeyVersion> keysToDelete =
new ArrayList<>();
/**
* List of paths to delete, which will be passed to the rename
* tracker after the deletion succeeds.
*/
private final List<Path> pathsToDelete = new ArrayList<>();
private final long blocksize;
/**
* Initiate the rename.
*
* @param storeContext store context
* @param sourcePath source path
* @param sourceKey key of source
* @param sourceStatus pre-fetched source status
* @param destPath destination path.
* @param destKey destination key
* @param destStatus destination status.
* @param callbacks callback provider
*/
public RenameOperation(
final StoreContext storeContext,
final Path sourcePath,
final String sourceKey,
final S3AFileStatus sourceStatus,
final Path destPath,
final String destKey,
final S3AFileStatus destStatus,
final RenameOperationCallbacks callbacks) {
super(storeContext);
this.sourcePath = sourcePath;
this.sourceKey = sourceKey;
this.sourceStatus = sourceStatus;
this.destPath = destPath;
this.destKey = destKey;
this.destStatus = destStatus;
this.callbacks = callbacks;
blocksize = storeContext.getConfiguration()
.getLongBytes(FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE);
}
/**
* Wait for the active copies to complete then reset the list.
* @param reason for messages
*/
private void completeActiveCopies(String reason) throws IOException {
LOG.debug("Waiting for {} active copies to complete: {}",
activeCopies.size(), reason);
waitForCompletion(activeCopies);
activeCopies.clear();
}
/**
* Queue and object for deletion.
* @param path path to the object
* @param key key of the object.
*/
private void queueToDelete(Path path, String key) {
pathsToDelete.add(path);
keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key));
}
/**
* Block waiting for ay active copies to finish
* then delete all queued keys + paths to delete.
* @param reason reason for logs
* @throws IOException failure.
*/
private void completeActiveCopiesAndDeleteSources(String reason)
throws IOException {
completeActiveCopies(reason);
removeSourceObjects(
keysToDelete,
pathsToDelete);
// now reset the lists.
keysToDelete.clear();
pathsToDelete.clear();
}
@Retries.RetryMixed
public long executeRename() throws IOException {
Preconditions.checkState(
!executed.getAndSet(true),
"Rename attempted twice");
final StoreContext storeContext = getStoreContext();
final MetadataStore metadataStore = checkNotNull(
storeContext.getMetadataStore(),
"No metadata store in context");
// Validation completed: time to begin the operation.
// The store-specific rename tracker is used to keep the store
// to date with the in-progress operation.
// for the null store, these are all no-ops.
renameTracker = metadataStore.initiateRenameOperation(
storeContext,
sourcePath, sourceStatus, destPath);
// Ok! Time to start
try {
if (sourceStatus.isFile()) {
renameFileToDest();
} else {
recursiveDirectoryRename();
}
} catch (AmazonClientException | IOException ex) {
// rename failed.
// block for all ongoing copies to complete, successfully or not
try {
completeActiveCopies("failure handling");
} catch (IOException e) {
// a failure to update the metastore after a rename failure is what
// we'd see on a network problem, expired credentials and other
// unrecoverable errors.
// Downgrading to warn because an exception is already
// about to be thrown.
LOG.warn("While completing all active copies", e);
}
// notify the rename tracker of the failure
throw renameTracker.renameFailed(ex);
}
// At this point the rename has completed successfully in the S3 store.
// Tell the metastore this fact and let it complete its changes
renameTracker.completeRename();
callbacks.finishRename(sourcePath, destPath);
return bytesCopied.get();
}
/**
* The source is a file: rename it to the destination.
* @throws IOException failure
*/
protected void renameFileToDest() throws IOException {
final StoreContext storeContext = getStoreContext();
// the source is a file.
Path copyDestinationPath = destPath;
String copyDestinationKey = destKey;
S3ObjectAttributes sourceAttributes =
callbacks.createObjectAttributes(sourceStatus);
S3AReadOpContext readContext = callbacks.createReadContext(sourceStatus);
if (destStatus != null && destStatus.isDirectory()) {
// destination is a directory: build the final destination underneath
String newDestKey = maybeAddTrailingSlash(destKey);
String filename = sourceKey.substring(
storeContext.pathToKey(sourcePath.getParent()).length() + 1);
newDestKey = newDestKey + filename;
copyDestinationKey = newDestKey;
copyDestinationPath = storeContext.keyToPath(newDestKey);
}
// destination either does not exist or is a file to overwrite.
LOG.debug("rename: renaming file {} to {}", sourcePath,
copyDestinationPath);
copySourceAndUpdateTracker(
sourcePath,
sourceKey,
sourceAttributes,
readContext,
copyDestinationPath,
copyDestinationKey,
false);
bytesCopied.addAndGet(sourceStatus.getLen());
// delete the source
callbacks.deleteObjectAtPath(sourcePath, sourceKey, true);
// and update the tracker
renameTracker.sourceObjectsDeleted(Lists.newArrayList(sourcePath));
}
/**
* Execute a full recursive rename.
* The source is a file: rename it to the destination.
* @throws IOException failure
*/
protected void recursiveDirectoryRename() throws IOException {
final StoreContext storeContext = getStoreContext();
LOG.debug("rename: renaming directory {} to {}", sourcePath, destPath);
// This is a directory-to-directory copy
String dstKey = maybeAddTrailingSlash(destKey);
String srcKey = maybeAddTrailingSlash(sourceKey);
// Verify dest is not a child of the source directory
if (dstKey.startsWith(srcKey)) {
throw new RenameFailedException(srcKey, dstKey,
"cannot rename a directory to a subdirectory of itself ");
}
if (destStatus != null
&& destStatus.isEmptyDirectory() == Tristate.TRUE) {
// delete unnecessary fake directory at the destination.
// this MUST be done before anything else so that
// rollback code doesn't get confused and insert a tombstone
// marker.
LOG.debug("Deleting fake directory marker at destination {}",
destStatus.getPath());
callbacks.deleteObjectAtPath(destStatus.getPath(), dstKey, false);
}
Path parentPath = storeContext.keyToPath(srcKey);
final RemoteIterator<S3ALocatedFileStatus> iterator =
callbacks.listFilesAndEmptyDirectories(parentPath);
while (iterator.hasNext()) {
// get the next entry in the listing.
S3ALocatedFileStatus child = iterator.next();
// convert it to an S3 key.
String k = storeContext.pathToKey(child.getPath());
// possibly adding a "/" if it represents directory and it does
// not have a trailing slash already.
String key = (child.isDirectory() && !k.endsWith("/"))
? k + "/"
: k;
// the source object to copy as a path.
Path childSourcePath = storeContext.keyToPath(key);
// mark for deletion on a successful copy.
queueToDelete(childSourcePath, key);
// the destination key is that of the key under the source tree,
// remapped under the new destination path.
String newDestKey =
dstKey + key.substring(srcKey.length());
Path childDestPath = storeContext.keyToPath(newDestKey);
// now begin the single copy
CompletableFuture<Path> copy = initiateCopy(child, key,
childSourcePath, newDestKey, childDestPath);
activeCopies.add(copy);
bytesCopied.addAndGet(sourceStatus.getLen());
if (activeCopies.size() == RENAME_PARALLEL_LIMIT) {
// the limit of active copies has been reached;
// wait for completion or errors to surface.
LOG.debug("Waiting for active copies to complete");
completeActiveCopies("batch threshold reached");
}
if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) {
// finish ongoing copies then delete all queued keys.
// provided the parallel limit is a factor of the max entry
// constant, this will not need to block for the copy, and
// simply jump straight to the delete.
completeActiveCopiesAndDeleteSources("paged delete");
}
} // end of iteration through the list
// await the final set of copies and their deletion
// This will notify the renameTracker that these objects
// have been deleted.
completeActiveCopiesAndDeleteSources("final copy and delete");
// We moved all the children, now move the top-level dir
// Empty directory should have been added as the object summary
renameTracker.moveSourceDirectory();
}
/**
* Initiate a copy operation in the executor.
* @param source status of the source object.
* @param key source key
* @param childSourcePath source as a path.
* @param newDestKey destination key
* @param childDestPath destination path.
* @return the future.
*/
protected CompletableFuture<Path> initiateCopy(
final S3ALocatedFileStatus source,
final String key,
final Path childSourcePath,
final String newDestKey,
final Path childDestPath) {
S3ObjectAttributes sourceAttributes =
callbacks.createObjectAttributes(
source.getPath(),
source.getETag(),
source.getVersionId(),
source.getLen());
// queue the copy operation for execution in the thread pool
return submit(getStoreContext().getExecutor(), () ->
copySourceAndUpdateTracker(
childSourcePath,
key,
sourceAttributes,
callbacks.createReadContext(source),
childDestPath,
newDestKey,
true));
}
/**
* This invoked to copy a file or directory marker then update the
* rename operation on success.
* It may be called in its own thread.
* @param sourceFile source path of the copy; may have a trailing / on it.
* @param srcKey source key
* @param srcAttributes status of the source object
* @param destination destination as a qualified path.
* @param destinationKey destination key
* @param addAncestors should ancestors be added to the metastore?
* @return the destination path.
* @throws IOException failure
*/
@Retries.RetryTranslated
private Path copySourceAndUpdateTracker(
final Path sourceFile,
final String srcKey,
final S3ObjectAttributes srcAttributes,
final S3AReadOpContext readContext,
final Path destination,
final String destinationKey,
final boolean addAncestors) throws IOException {
long len = srcAttributes.getLen();
CopyResult copyResult;
try (DurationInfo ignored = new DurationInfo(LOG, false,
"Copy file from %s to %s (length=%d)", srcKey, destinationKey, len)) {
copyResult = callbacks.copyFile(srcKey, destinationKey,
srcAttributes, readContext);
}
if (objectRepresentsDirectory(srcKey, len)) {
renameTracker.directoryMarkerCopied(
sourceFile,
destination,
addAncestors);
} else {
S3ObjectAttributes destAttributes = new S3ObjectAttributes(
destination,
copyResult,
srcAttributes.getServerSideEncryptionAlgorithm(),
srcAttributes.getServerSideEncryptionKey(),
len);
renameTracker.fileCopied(
sourceFile,
srcAttributes,
destAttributes,
destination,
blocksize,
addAncestors);
}
return destination;
}
/**
* Remove source objects and update the metastore by way of
* the rename tracker.
* @param keys list of keys to delete
* @param paths list of paths matching the keys to delete 1:1.
* @throws IOException failure
*/
@Retries.RetryMixed
private void removeSourceObjects(
final List<DeleteObjectsRequest.KeyVersion> keys,
final List<Path> paths)
throws IOException {
List<Path> undeletedObjects = new ArrayList<>();
try {
// remove the keys
// this will update the metastore on a failure, but on
// a successful operation leaves the store as is.
callbacks.removeKeys(keys, false, undeletedObjects);
// and clear the list.
} catch (AmazonClientException | IOException e) {
// Failed.
// Notify the rename operation.
// removeKeys will have already purged the metastore of
// all keys it has known to delete; this is just a final
// bit of housekeeping and a chance to tune exception
// reporting
throw renameTracker.deleteFailed(e, paths, undeletedObjects);
}
renameTracker.sourceObjectsDeleted(paths);
}
/**
* Turns a path (relative or otherwise) into an S3 key, adding a trailing
* "/" if the path is not the root <i>and</i> does not already have a "/"
* at the end.
*
* @param key s3 key or ""
* @return the with a trailing "/", or, if it is the root key, "",
*/
private String maybeAddTrailingSlash(String key) {
if (!key.isEmpty() && !key.endsWith("/")) {
return key + '/';
} else {
return key;
}
}
/**
* These are all the callbacks which the rename operation needs,
* derived from the appropriate S3AFileSystem methods.
*/
public interface RenameOperationCallbacks {
/**
* Create the attributes of an object for subsequent use.
* @param path path path of the request.
* @param eTag the eTag of the S3 object
* @param versionId S3 object version ID
* @param len length of the file
* @return attributes to use when building the query.
*/
S3ObjectAttributes createObjectAttributes(
Path path,
String eTag,
String versionId,
long len);
/**
* Create the attributes of an object for subsequent use.
* @param fileStatus file status to build from.
* @return attributes to use when building the query.
*/
S3ObjectAttributes createObjectAttributes(
S3AFileStatus fileStatus);
/**
* Create the read context for reading from the referenced file,
* using FS state as well as the status.
* @param fileStatus file status.
* @return a context for read and select operations.
*/
S3AReadOpContext createReadContext(
FileStatus fileStatus);
/**
* The rename has finished; perform any store cleanup operations
* such as creating/deleting directory markers.
* @param sourceRenamed renamed source
* @param destCreated destination file created.
* @throws IOException failure
*/
void finishRename(Path sourceRenamed, Path destCreated) throws IOException;
/**
* Delete an object, also updating the metastore.
* This call does <i>not</i> create any mock parent entries.
* Retry policy: retry untranslated; delete considered idempotent.
* @param path path to delete
* @param key key of entry
* @param isFile is the path a file (used for instrumentation only)
* @throws AmazonClientException problems working with S3
* @throws IOException IO failure in the metastore
*/
@Retries.RetryMixed
void deleteObjectAtPath(Path path, String key, boolean isFile)
throws IOException;
/**
* Recursive list of files and empty directories.
* @param path path to list from
* @return an iterator.
* @throws IOException failure
*/
RemoteIterator<S3ALocatedFileStatus> listFilesAndEmptyDirectories(
Path path) throws IOException;
/**
* Copy a single object in the bucket via a COPY operation.
* There's no update of metadata, directory markers, etc.
* Callers must implement.
* @param srcKey source object path
* @param srcAttributes S3 attributes of the source object
* @param readContext the read context
* @return the result of the copy
* @throws InterruptedIOException the operation was interrupted
* @throws IOException Other IO problems
*/
@Retries.RetryTranslated
CopyResult copyFile(String srcKey,
String destKey,
S3ObjectAttributes srcAttributes,
S3AReadOpContext readContext)
throws IOException;
/**
* Remove keys from the store, updating the metastore on a
* partial delete represented as a MultiObjectDeleteException failure by
* deleting all those entries successfully deleted and then rethrowing
* the MultiObjectDeleteException.
* @param keysToDelete collection of keys to delete on the s3-backend.
* if empty, no request is made of the object store.
* @param deleteFakeDir indicates whether this is for deleting fake dirs.
* @param undeletedObjectsOnFailure List which will be built up of all
* files that were not deleted. This happens even as an exception
* is raised.
* @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.
* @throws IOException other IO Exception.
*/
@Retries.RetryMixed
void removeKeys(
List<DeleteObjectsRequest.KeyVersion> keysToDelete,
boolean deleteFakeDir,
List<Path> undeletedObjectsOnFailure)
throws MultiObjectDeleteException, AmazonClientException,
IOException;
}
}

View File

@ -0,0 +1,335 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.impl;
import java.io.File;
import java.io.IOException;
import java.net.URI;
import com.google.common.util.concurrent.ListeningExecutorService;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.S3AInputPolicy;
import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
import org.apache.hadoop.fs.s3a.Statistic;
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
/**
* This class provides the core context of the S3A filesystem to subsidiary
* components, without exposing the entire parent class.
* This is eliminate explicit recursive coupling.
*
* Where methods on the FS are to be invoked, they are referenced
* via the {@link ContextAccessors} interface, so tests can implement
* their own.
*
* <i>Warning:</i> this really is private and unstable. Do not use
* outside the org.apache.hadoop.fs.s3a package.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class StoreContext {
/** Filesystem URI. */
private final URI fsURI;
/** Bucket name. */
private final String bucket;
/** FS configuration after all per-bucket overrides applied. */
private final Configuration configuration;
/** Username. */
private final String username;
/** Principal who created the FS. */
private final UserGroupInformation owner;
/**
* Bounded thread pool for async operations.
*/
private final ListeningExecutorService executor;
/**
* Capacity of new executors created.
*/
private final int executorCapacity;
/** Invoker of operations. */
private final Invoker invoker;
/** Instrumentation and statistics. */
private final S3AInstrumentation instrumentation;
private final S3AStorageStatistics storageStatistics;
/** Seek policy. */
private final S3AInputPolicy inputPolicy;
/** How to react to changes in etags and versions. */
private final ChangeDetectionPolicy changeDetectionPolicy;
/** Evaluated options. */
private final boolean multiObjectDeleteEnabled;
/** List algorithm. */
private final boolean useListV1;
/**
* To allow this context to be passed down to the metastore, this field
* wll be null until initialized.
*/
private final MetadataStore metadataStore;
private final ContextAccessors contextAccessors;
/**
* Source of time.
*/
private ITtlTimeProvider timeProvider;
/**
* Instantiate.
* No attempt to use a builder here as outside tests
* this should only be created in the S3AFileSystem.
*/
public StoreContext(
final URI fsURI,
final String bucket,
final Configuration configuration,
final String username,
final UserGroupInformation owner,
final ListeningExecutorService executor,
final int executorCapacity,
final Invoker invoker,
final S3AInstrumentation instrumentation,
final S3AStorageStatistics storageStatistics,
final S3AInputPolicy inputPolicy,
final ChangeDetectionPolicy changeDetectionPolicy,
final boolean multiObjectDeleteEnabled,
final MetadataStore metadataStore,
final boolean useListV1,
final ContextAccessors contextAccessors,
final ITtlTimeProvider timeProvider) {
this.fsURI = fsURI;
this.bucket = bucket;
this.configuration = configuration;
this.username = username;
this.owner = owner;
this.executor = executor;
this.executorCapacity = executorCapacity;
this.invoker = invoker;
this.instrumentation = instrumentation;
this.storageStatistics = storageStatistics;
this.inputPolicy = inputPolicy;
this.changeDetectionPolicy = changeDetectionPolicy;
this.multiObjectDeleteEnabled = multiObjectDeleteEnabled;
this.metadataStore = metadataStore;
this.useListV1 = useListV1;
this.contextAccessors = contextAccessors;
this.timeProvider = timeProvider;
}
@Override
protected Object clone() throws CloneNotSupportedException {
return super.clone();
}
public URI getFsURI() {
return fsURI;
}
public String getBucket() {
return bucket;
}
public Configuration getConfiguration() {
return configuration;
}
public String getUsername() {
return username;
}
public ListeningExecutorService getExecutor() {
return executor;
}
public Invoker getInvoker() {
return invoker;
}
public S3AInstrumentation getInstrumentation() {
return instrumentation;
}
public S3AInputPolicy getInputPolicy() {
return inputPolicy;
}
public ChangeDetectionPolicy getChangeDetectionPolicy() {
return changeDetectionPolicy;
}
public boolean isMultiObjectDeleteEnabled() {
return multiObjectDeleteEnabled;
}
public MetadataStore getMetadataStore() {
return metadataStore;
}
public boolean isUseListV1() {
return useListV1;
}
/**
* Convert a key to a fully qualified path.
* @param key input key
* @return the fully qualified path including URI scheme and bucket name.
*/
public Path keyToPath(String key) {
return contextAccessors.keyToPath(key);
}
/**
* Turns a path (relative or otherwise) into an S3 key.
*
* @param path input path, may be relative to the working dir
* @return a key excluding the leading "/", or, if it is the root path, ""
*/
public String pathToKey(Path path) {
return contextAccessors.pathToKey(path);
}
/**
* Get the storage statistics of this filesystem.
* @return the storage statistics
*/
public S3AStorageStatistics getStorageStatistics() {
return storageStatistics;
}
/**
* Increment a statistic by 1.
* This increments both the instrumentation and storage statistics.
* @param statistic The operation to increment
*/
public void incrementStatistic(Statistic statistic) {
incrementStatistic(statistic, 1);
}
/**
* Increment a statistic by a specific value.
* This increments both the instrumentation and storage statistics.
* @param statistic The operation to increment
* @param count the count to increment
*/
public void incrementStatistic(Statistic statistic, long count) {
instrumentation.incrementCounter(statistic, count);
storageStatistics.incrementCounter(statistic, count);
}
/**
* Decrement a gauge by a specific value.
* @param statistic The operation to decrement
* @param count the count to decrement
*/
public void decrementGauge(Statistic statistic, long count) {
instrumentation.decrementGauge(statistic, count);
}
/**
* Increment a gauge by a specific value.
* @param statistic The operation to increment
* @param count the count to increment
*/
public void incrementGauge(Statistic statistic, long count) {
instrumentation.incrementGauge(statistic, count);
}
/**
* Create a new executor service with a given capacity.
* This executor submits works to the {@link #executor}, using a
* {@link SemaphoredDelegatingExecutor} to limit the number
* of requests coming in from a specific client.
*
* Because this delegates to an existing thread pool, the cost of
* creating a new instance here is low.
* As the throttling is per instance, separate instances
* should be created for each operation which wishes to execute work in
* parallel <i>without</i> saturating the base executor.
* This is important if either the duration of each operation is long
* or the submission rate of work is high.
* @param capacity maximum capacity of this executor.
* @return an executor for submitting work.
*/
public ListeningExecutorService createThrottledExecutor(int capacity) {
return new SemaphoredDelegatingExecutor(executor,
capacity, true);
}
/**
* Create a new executor with the capacity defined in
* {@link #executorCapacity}.
* @return a new executor for exclusive use by the caller.
*/
public ListeningExecutorService createThrottledExecutor() {
return createThrottledExecutor(executorCapacity);
}
public UserGroupInformation getOwner() {
return owner;
}
/**
* Create a temporary file somewhere.
* @param prefix prefix for the temporary file
* @param size expected size.
* @return a file reference.
* @throws IOException failure.
*/
public File createTempFile(String prefix, long size) throws IOException {
return contextAccessors.createTempFile(prefix, size);
}
/**
* Get the location of the bucket.
* @return the bucket location.
* @throws IOException failure.
*/
public String getBucketLocation() throws IOException {
return contextAccessors.getBucketLocation();
}
/**
* Get the time provider.
* @return the time source.
*/
public ITtlTimeProvider getTimeProvider() {
return timeProvider;
}
}

View File

@ -0,0 +1,82 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.s3guard;
import java.io.Closeable;
import java.io.IOException;
/**
* This represents state which may be passed to bulk IO operations
* to enable them to store information about the state of the ongoing
* operation across invocations.
* <p>
* A bulk operation state <i>MUST</i> only be be used for the single store
* from which it was created, and <i>MUST</i>only for the duration of a single
* bulk update operation.
* <p>
* Passing in the state is to allow the stores to maintain state about
* updates they have already made to their store during this single operation:
* a cache of what has happened. It is not a list of operations to be applied.
* If a list of operations to perform is built up (e.g. during rename)
* that is the duty of the caller, not this state.
* <p>
* After the operation has completed, it <i>MUST</i> be closed so
* as to guarantee that all state is released.
*/
public class BulkOperationState implements Closeable {
private final OperationType operation;
/**
* Constructor.
* @param operation the type of the operation.
*/
public BulkOperationState(final OperationType operation) {
this.operation = operation;
}
/**
* Get the operation type.
* @return the operation type.
*/
public OperationType getOperation() {
return operation;
}
@Override
public void close() throws IOException {
}
/**
* Enumeration of operations which can be performed in bulk.
* This can be used by the stores however they want.
* One special aspect: renames are to be done through a {@link RenameTracker}.
* Callers will be blocked from initiating a rename through
* {@code S3Guard#initiateBulkWrite()}
*/
public enum OperationType {
/** Writing data. */
Put,
/** Rename: add and delete. */
Rename,
/** Pruning: deleting entries and updating parents. */
Prune,
}
}

View File

@ -0,0 +1,192 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.s3guard;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import com.amazonaws.SdkBaseException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
import org.apache.hadoop.fs.s3a.Tristate;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
import org.apache.hadoop.util.DurationInfo;
import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveAncestors;
import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveDir;
import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveFile;
/**
* This is the rename updating strategy originally used:
* a collection of source paths and a list of destinations are created,
* then updated at the end (possibly slow).
* <p>
* It is not currently instantiated by any of the active trackers,
* but is preserved to show that the original rename strategy
* can be implemented via the tracker model.
*/
public class DelayedUpdateRenameTracker extends RenameTracker {
private final MetadataStore metadataStore;
private final Collection<Path> sourcePaths = new HashSet<>();
private final List<PathMetadata> destMetas = new ArrayList<>();
private final List<Path> deletedPaths = new ArrayList<>();
public DelayedUpdateRenameTracker(
final StoreContext storeContext,
final MetadataStore metadataStore,
final Path sourceRoot,
final Path dest,
final BulkOperationState operationState) {
super("DelayedUpdateRenameTracker", storeContext, metadataStore,
sourceRoot, dest, operationState);
this.metadataStore = storeContext.getMetadataStore();
}
@Override
public synchronized void fileCopied(
final Path sourcePath,
final S3ObjectAttributes sourceAttributes,
final S3ObjectAttributes destAttributes,
final Path destPath,
final long blockSize,
final boolean addAncestors) throws IOException {
addMoveFile(metadataStore,
sourcePaths,
destMetas,
sourcePath,
destPath,
sourceAttributes.getLen(),
blockSize,
getOwner(),
destAttributes.getETag(),
destAttributes.getVersionId());
// Ancestor directories may not be listed, so we explicitly add them
if (addAncestors) {
addMoveAncestors(metadataStore,
sourcePaths,
destMetas,
getSourceRoot(),
sourcePath,
destPath,
getOwner());
}
}
@Override
public synchronized void directoryMarkerCopied(final Path sourcePath,
final Path destPath,
final boolean addAncestors) throws IOException {
addMoveDir(metadataStore, sourcePaths, destMetas,
sourcePath,
destPath, getOwner());
// Ancestor directories may not be listed, so we explicitly add them
if (addAncestors) {
addMoveAncestors(metadataStore,
sourcePaths,
destMetas,
getSourceRoot(),
sourcePath,
destPath,
getOwner());
}
}
@Override
public synchronized void moveSourceDirectory() throws IOException {
if (!sourcePaths.contains(getSourceRoot())) {
addMoveDir(metadataStore, sourcePaths, destMetas,
getSourceRoot(),
getDest(), getOwner());
}
}
@Override
public synchronized void sourceObjectsDeleted(
final Collection<Path> paths) throws IOException {
// add to the list of deleted paths.
deletedPaths.addAll(paths);
}
@Override
public void completeRename() throws IOException {
metadataStore.move(sourcePaths, destMetas,
getStoreContext().getTimeProvider(),
getOperationState());
super.completeRename();
}
@Override
public IOException renameFailed(final Exception ex) {
LOG.warn("Rename has failed; updating s3guard with destination state");
try (DurationInfo ignored = new DurationInfo(LOG,
"Cleaning up deleted paths")) {
// the destination paths are updated; the source is left alone.
metadataStore.move(new ArrayList<>(0), destMetas,
getStoreContext().getTimeProvider(),
getOperationState());
for (Path deletedPath : deletedPaths) {
// this is not ideal in that it may leave parent stuff around.
metadataStore.delete(deletedPath, getStoreContext().getTimeProvider());
}
deleteParentPaths();
} catch (IOException | SdkBaseException e) {
LOG.warn("Ignoring error raised in AWS SDK ", e);
}
return super.renameFailed(ex);
}
/**
* Delete all the parent paths we know to be empty (by walking up the tree
* deleting as appropriate).
* @throws IOException failure
*/
private void deleteParentPaths() throws IOException {
Set<Path> parentPaths = new HashSet<>();
for (Path deletedPath : deletedPaths) {
Path parent = deletedPath.getParent();
if (!parent.equals(getSourceRoot())) {
parentPaths.add(parent);
}
}
// now there's a set of parent paths. We now want to
// get them ordered by depth, so that deeper entries come first
// that way: when we check for a parent path existing we can
// see if it really is empty.
List<Path> parents = new ArrayList<>(parentPaths);
parents.sort(PathOrderComparators.TOPMOST_PATH_LAST);
for (Path parent : parents) {
PathMetadata md = metadataStore.get(parent, true);
if (md != null && md.isEmptyDirectory() == Tristate.TRUE) {
// if were confident that this is empty: delete it.
metadataStore.delete(parent, getStoreContext().getTimeProvider());
}
}
}
}

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.fs.s3a.s3guard; package org.apache.hadoop.fs.s3a.s3guard;
import javax.annotation.Nullable;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
@ -30,12 +32,17 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.Tristate; import org.apache.hadoop.fs.s3a.Tristate;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
import org.apache.hadoop.security.UserGroupInformation;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.io.IOException; import java.io.IOException;
import java.net.URI; import java.net.URI;
import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.Map; import java.util.Map;
@ -70,6 +77,8 @@ public class LocalMetadataStore implements MetadataStore {
/* Null iff this FS does not have an associated URI host. */ /* Null iff this FS does not have an associated URI host. */
private String uriHost; private String uriHost;
private String username;
@Override @Override
public void initialize(FileSystem fileSystem) throws IOException { public void initialize(FileSystem fileSystem) throws IOException {
Preconditions.checkNotNull(fileSystem); Preconditions.checkNotNull(fileSystem);
@ -100,6 +109,7 @@ public class LocalMetadataStore implements MetadataStore {
} }
localCache = builder.build(); localCache = builder.build();
username = UserGroupInformation.getCurrentUser().getShortUserName();
} }
@Override @Override
@ -192,15 +202,19 @@ public class LocalMetadataStore implements MetadataStore {
} }
@Override @Override
public void move(Collection<Path> pathsToDelete, public void move(
Collection<PathMetadata> pathsToCreate, @Nullable Collection<Path> pathsToDelete,
ITtlTimeProvider ttlTimeProvider) throws IOException { @Nullable Collection<PathMetadata> pathsToCreate,
ITtlTimeProvider ttlTimeProvider,
@Nullable final BulkOperationState operationState) throws IOException {
LOG.info("Move {} to {}", pathsToDelete, pathsToCreate); LOG.info("Move {} to {}", pathsToDelete, pathsToCreate);
Preconditions.checkNotNull(pathsToDelete, "pathsToDelete is null"); if (pathsToCreate == null) {
Preconditions.checkNotNull(pathsToCreate, "pathsToCreate is null"); pathsToCreate = Collections.emptyList();
Preconditions.checkArgument(pathsToDelete.size() == pathsToCreate.size(), }
"Must supply same number of paths to delete/create."); if (pathsToDelete == null) {
pathsToDelete = Collections.emptyList();
}
// I feel dirty for using reentrant lock. :-| // I feel dirty for using reentrant lock. :-|
synchronized (this) { synchronized (this) {
@ -214,7 +228,7 @@ public class LocalMetadataStore implements MetadataStore {
// 2. Create new destination path metadata // 2. Create new destination path metadata
for (PathMetadata meta : pathsToCreate) { for (PathMetadata meta : pathsToCreate) {
LOG.debug("move: adding metadata {}", meta); LOG.debug("move: adding metadata {}", meta);
put(meta); put(meta, null);
} }
// 3. We now know full contents of all dirs in destination subtree // 3. We now know full contents of all dirs in destination subtree
@ -232,7 +246,13 @@ public class LocalMetadataStore implements MetadataStore {
} }
@Override @Override
public void put(PathMetadata meta) throws IOException { public void put(final PathMetadata meta) throws IOException {
put(meta, null);
}
@Override
public void put(PathMetadata meta,
final BulkOperationState operationState) throws IOException {
Preconditions.checkNotNull(meta); Preconditions.checkNotNull(meta);
S3AFileStatus status = meta.getFileStatus(); S3AFileStatus status = meta.getFileStatus();
@ -301,7 +321,8 @@ public class LocalMetadataStore implements MetadataStore {
} }
@Override @Override
public synchronized void put(DirListingMetadata meta) throws IOException { public synchronized void put(DirListingMetadata meta,
final BulkOperationState operationState) throws IOException {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("put dirMeta {}", meta.prettyPrint()); LOG.debug("put dirMeta {}", meta.prettyPrint());
} }
@ -312,13 +333,14 @@ public class LocalMetadataStore implements MetadataStore {
} else { } else {
entry.setDirListingMetadata(meta); entry.setDirListingMetadata(meta);
} }
put(meta.getListing()); put(meta.getListing(), null);
} }
public synchronized void put(Collection<PathMetadata> metas) throws public synchronized void put(Collection<? extends PathMetadata> metas,
final BulkOperationState operationState) throws
IOException { IOException {
for (PathMetadata meta : metas) { for (PathMetadata meta : metas) {
put(meta); put(meta, operationState);
} }
} }
@ -564,4 +586,35 @@ public class LocalMetadataStore implements MetadataStore {
} }
} }
@Override
public RenameTracker initiateRenameOperation(final StoreContext storeContext,
final Path source,
final S3AFileStatus sourceStatus, final Path dest) throws IOException {
return new ProgressiveRenameTracker(storeContext, this, source, dest,
null);
}
@Override
public void addAncestors(final Path qualifiedPath,
ITtlTimeProvider ttlTimeProvider,
@Nullable final BulkOperationState operationState) throws IOException {
Collection<PathMetadata> newDirs = new ArrayList<>();
Path parent = qualifiedPath.getParent();
while (!parent.isRoot()) {
PathMetadata directory = get(parent);
if (directory == null || directory.isDeleted()) {
S3AFileStatus status = new S3AFileStatus(Tristate.FALSE, parent,
username);
PathMetadata meta = new PathMetadata(status, Tristate.FALSE, false);
newDirs.add(meta);
} else {
break;
}
parent = parent.getParent();
}
if (!newDirs.isEmpty()) {
put(newDirs, operationState);
}
}
} }

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.fs.s3a.s3guard; package org.apache.hadoop.fs.s3a.s3guard;
import javax.annotation.Nullable;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.util.Collection; import java.util.Collection;
@ -30,6 +31,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.Retries.RetryTranslated; import org.apache.hadoop.fs.s3a.Retries.RetryTranslated;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
/** /**
* {@code MetadataStore} defines the set of operations that any metadata store * {@code MetadataStore} defines the set of operations that any metadata store
@ -142,6 +145,23 @@ public interface MetadataStore extends Closeable {
*/ */
DirListingMetadata listChildren(Path path) throws IOException; DirListingMetadata listChildren(Path path) throws IOException;
/**
* This adds all new ancestors of a path as directories.
* <p>
* Important: to propagate TTL information, any new ancestors added
* must have their last updated timestamps set through
* {@link S3Guard#patchLastUpdated(Collection, ITtlTimeProvider)}.
* @param qualifiedPath path to update
* @param timeProvider time provider for timestamps
* @param operationState (nullable) operational state for a bulk update
* @throws IOException failure
*/
@RetryTranslated
void addAncestors(
Path qualifiedPath,
@Nullable ITtlTimeProvider timeProvider,
@Nullable BulkOperationState operationState) throws IOException;
/** /**
* Record the effects of a {@link FileSystem#rename(Path, Path)} in the * Record the effects of a {@link FileSystem#rename(Path, Path)} in the
* MetadataStore. Clients provide explicit enumeration of the affected * MetadataStore. Clients provide explicit enumeration of the affected
@ -163,15 +183,18 @@ public interface MetadataStore extends Closeable {
* @param pathsToDelete Collection of all paths that were removed from the * @param pathsToDelete Collection of all paths that were removed from the
* source directory tree of the move. * source directory tree of the move.
* @param pathsToCreate Collection of all PathMetadata for the new paths * @param pathsToCreate Collection of all PathMetadata for the new paths
* that were created at the destination of the rename * that were created at the destination of the rename().
* ().
* @param ttlTimeProvider the time provider to set last_updated. Must not * @param ttlTimeProvider the time provider to set last_updated. Must not
* be null. * be null.
* @param operationState Any ongoing state supplied to the rename tracker
* which is to be passed in with each move operation.
* @throws IOException if there is an error * @throws IOException if there is an error
*/ */
void move(Collection<Path> pathsToDelete, void move(
Collection<PathMetadata> pathsToCreate, @Nullable Collection<Path> pathsToDelete,
ITtlTimeProvider ttlTimeProvider) throws IOException; @Nullable Collection<PathMetadata> pathsToCreate,
ITtlTimeProvider ttlTimeProvider,
@Nullable BulkOperationState operationState) throws IOException;
/** /**
* Saves metadata for exactly one path. * Saves metadata for exactly one path.
@ -186,15 +209,33 @@ public interface MetadataStore extends Closeable {
@RetryTranslated @RetryTranslated
void put(PathMetadata meta) throws IOException; void put(PathMetadata meta) throws IOException;
/**
* Saves metadata for exactly one path, potentially
* using any bulk operation state to eliminate duplicate work.
*
* Implementations may pre-create all the path's ancestors automatically.
* Implementations must update any {@code DirListingMetadata} objects which
* track the immediate parent of this file.
*
* @param meta the metadata to save
* @param operationState operational state for a bulk update
* @throws IOException if there is an error
*/
@RetryTranslated
void put(PathMetadata meta,
@Nullable BulkOperationState operationState) throws IOException;
/** /**
* Saves metadata for any number of paths. * Saves metadata for any number of paths.
* *
* Semantics are otherwise the same as single-path puts. * Semantics are otherwise the same as single-path puts.
* *
* @param metas the metadata to save * @param metas the metadata to save
* @param operationState (nullable) operational state for a bulk update
* @throws IOException if there is an error * @throws IOException if there is an error
*/ */
void put(Collection<PathMetadata> metas) throws IOException; void put(Collection<? extends PathMetadata> metas,
@Nullable BulkOperationState operationState) throws IOException;
/** /**
* Save directory listing metadata. Callers may save a partial directory * Save directory listing metadata. Callers may save a partial directory
@ -211,9 +252,11 @@ public interface MetadataStore extends Closeable {
* another process. * another process.
* *
* @param meta Directory listing metadata. * @param meta Directory listing metadata.
* @param operationState operational state for a bulk update
* @throws IOException if there is an error * @throws IOException if there is an error
*/ */
void put(DirListingMetadata meta) throws IOException; void put(DirListingMetadata meta,
@Nullable BulkOperationState operationState) throws IOException;
/** /**
* Destroy all resources associated with the metadata store. * Destroy all resources associated with the metadata store.
@ -303,4 +346,36 @@ public interface MetadataStore extends Closeable {
ALL_BY_MODTIME, ALL_BY_MODTIME,
TOMBSTONES_BY_LASTUPDATED TOMBSTONES_BY_LASTUPDATED
} }
/**
* Start a rename operation.
*
* @param storeContext store context.
* @param source source path
* @param sourceStatus status of the source file/dir
* @param dest destination path.
* @return the rename tracker
* @throws IOException Failure.
*/
RenameTracker initiateRenameOperation(
StoreContext storeContext,
Path source,
S3AFileStatus sourceStatus,
Path dest)
throws IOException;
/**
* Initiate a bulk update and create an operation state for it.
* This may then be passed into put operations.
* @param operation the type of the operation.
* @param dest path under which updates will be explicitly put.
* @return null or a store-specific state to pass into the put operations.
* @throws IOException failure
*/
default BulkOperationState initiateBulkWrite(
BulkOperationState.OperationType operation,
Path dest) throws IOException {
return null;
}
} }

View File

@ -18,9 +18,14 @@
package org.apache.hadoop.fs.s3a.s3guard; package org.apache.hadoop.fs.s3a.s3guard;
import javax.annotation.Nullable;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
import java.io.IOException; import java.io.IOException;
import java.util.Collection; import java.util.Collection;
@ -79,19 +84,27 @@ public class NullMetadataStore implements MetadataStore {
@Override @Override
public void move(Collection<Path> pathsToDelete, public void move(Collection<Path> pathsToDelete,
Collection<PathMetadata> pathsToCreate, Collection<PathMetadata> pathsToCreate,
ITtlTimeProvider ttlTimeProvider) throws IOException { ITtlTimeProvider ttlTimeProvider,
final BulkOperationState operationState) throws IOException {
} }
@Override @Override
public void put(PathMetadata meta) throws IOException { public void put(final PathMetadata meta) throws IOException {
} }
@Override @Override
public void put(Collection<PathMetadata> meta) throws IOException { public void put(PathMetadata meta,
final BulkOperationState operationState) throws IOException {
} }
@Override @Override
public void put(DirListingMetadata meta) throws IOException { public void put(Collection<? extends PathMetadata> meta,
final BulkOperationState operationState) throws IOException {
}
@Override
public void put(DirListingMetadata meta,
final BulkOperationState operationState) throws IOException {
} }
@Override @Override
@ -123,4 +136,40 @@ public class NullMetadataStore implements MetadataStore {
public void updateParameters(Map<String, String> parameters) public void updateParameters(Map<String, String> parameters)
throws IOException { throws IOException {
} }
@Override
public RenameTracker initiateRenameOperation(final StoreContext storeContext,
final Path source,
final S3AFileStatus sourceStatus,
final Path dest)
throws IOException {
return new NullRenameTracker(storeContext, source, dest, this);
}
@Override
public void addAncestors(final Path qualifiedPath,
final ITtlTimeProvider timeProvider,
@Nullable final BulkOperationState operationState) throws IOException {
}
private static final class NullRenameTracker extends RenameTracker {
private NullRenameTracker(
final StoreContext storeContext,
final Path source,
final Path dest, MetadataStore metadataStore) {
super("null tracker", storeContext, metadataStore, source, dest, null);
}
@Override
public void fileCopied(final Path childSource,
final S3ObjectAttributes sourceAttributes,
final S3ObjectAttributes destAttributes,
final Path destPath,
final long blockSize,
final boolean addAncestors) throws IOException {
}
}
} }

View File

@ -51,7 +51,8 @@ import org.apache.hadoop.fs.s3a.Tristate;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
@InterfaceStability.Evolving @InterfaceStability.Evolving
final class PathMetadataDynamoDBTranslation { @VisibleForTesting
public final class PathMetadataDynamoDBTranslation {
/** The HASH key name of each item. */ /** The HASH key name of each item. */
@VisibleForTesting @VisibleForTesting
@ -289,7 +290,8 @@ final class PathMetadataDynamoDBTranslation {
* @param path path to convert * @param path path to convert
* @return string for parent key * @return string for parent key
*/ */
static String pathToParentKey(Path path) { @VisibleForTesting
public static String pathToParentKey(Path path) {
Preconditions.checkNotNull(path); Preconditions.checkNotNull(path);
Preconditions.checkArgument(path.isUriPathAbsolute(), "Path not absolute"); Preconditions.checkArgument(path.isUriPathAbsolute(), "Path not absolute");
URI uri = path.toUri(); URI uri = path.toUri();
@ -343,9 +345,21 @@ final class PathMetadataDynamoDBTranslation {
private PathMetadataDynamoDBTranslation() { private PathMetadataDynamoDBTranslation() {
} }
/**
* Convert a collection of metadata entries to a list
* of DDBPathMetadata entries.
* If the sources are already DDBPathMetadata instances, they
* are copied directly into the new list, otherwise new
* instances are created.
* @param pathMetadatas source data
* @return the converted list.
*/
static List<DDBPathMetadata> pathMetaToDDBPathMeta( static List<DDBPathMetadata> pathMetaToDDBPathMeta(
Collection<PathMetadata> pathMetadatas) { Collection<? extends PathMetadata> pathMetadatas) {
return pathMetadatas.stream().map(p -> new DDBPathMetadata(p)) return pathMetadatas.stream().map(p ->
(p instanceof DDBPathMetadata)
? (DDBPathMetadata) p
: new DDBPathMetadata(p))
.collect(Collectors.toList()); .collect(Collectors.toList());
} }

View File

@ -0,0 +1,133 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.s3guard;
import java.io.Serializable;
import java.util.Comparator;
import org.apache.hadoop.fs.Path;
/**
* Comparator of path ordering for sorting collections.
*
* The definition of "topmost" is:
* <ol>
* <li>The depth of a path is the primary comparator.</li>
* <li>Root is topmost, "0"</li>
* <li>If two paths are of equal depth, {@link Path#compareTo(Path)}</li>
* is used. This delegates to URI compareTo.
* <li>repeated sorts do not change the order</li>
* </ol>
*/
final class PathOrderComparators {
private PathOrderComparators() {
}
/**
* The shallowest paths come first.
* This is to be used when adding entries.
*/
static final Comparator<Path> TOPMOST_PATH_FIRST
= new TopmostFirst();
/**
* The leaves come first.
* This is to be used when deleting entries.
*/
static final Comparator<Path> TOPMOST_PATH_LAST
= new TopmostLast();
/**
* The shallowest paths come first.
* This is to be used when adding entries.
*/
static final Comparator<PathMetadata> TOPMOST_PM_FIRST
= new PathMetadataComparator(TOPMOST_PATH_FIRST);
/**
* The leaves come first.
* This is to be used when deleting entries.
*/
static final Comparator<PathMetadata> TOPMOST_PM_LAST
= new PathMetadataComparator(TOPMOST_PATH_LAST);
private static class TopmostFirst implements Comparator<Path>, Serializable {
@Override
public int compare(Path pathL, Path pathR) {
// exit fast on equal values.
if (pathL.equals(pathR)) {
return 0;
}
int depthL = pathL.depth();
int depthR = pathR.depth();
if (depthL < depthR) {
// left is higher up than the right.
return -1;
}
if (depthR < depthL) {
// right is higher up than the left
return 1;
}
// and if they are of equal depth, use the "classic" comparator
// of paths.
return pathL.compareTo(pathR);
}
}
/**
* Compare the topmost last.
* For some reason the .reverse() option wasn't giving the
* correct outcome.
*/
private static final class TopmostLast extends TopmostFirst {
@Override
public int compare(final Path pathL, final Path pathR) {
int compare = super.compare(pathL, pathR);
if (compare < 0) {
return 1;
}
if (compare > 0) {
return -1;
}
return 0;
}
}
/**
* Compare on path status.
*/
private static final class PathMetadataComparator implements
Comparator<PathMetadata>, Serializable {
private final Comparator<Path> inner;
private PathMetadataComparator(final Comparator<Path> inner) {
this.inner = inner;
}
@Override
public int compare(final PathMetadata o1, final PathMetadata o2) {
return inner.compare(o1.getFileStatus().getPath(),
o2.getFileStatus().getPath());
}
}
}

View File

@ -0,0 +1,252 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.s3guard;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
import org.apache.hadoop.util.DurationInfo;
import static com.google.common.base.Preconditions.checkArgument;
import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveAncestors;
import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveDir;
/**
* This rename tracker progressively updates the metadata store
* as it proceeds, during the parallelized copy operation.
* <p>
* Algorithm
* <ol>
* <li>
* As {@code RenameTracker.fileCopied()} callbacks
* are raised, the metastore is updated with the new file entry.
* </li>
* <li>
* Including parent entries, as appropriate.
* </li>
* <li>
* All directories which have been created are tracked locally,
* to avoid needing to read the store; this is a thread-safe structure.
* </li>
* <li>
* The actual update is performed out of any synchronized block.
* </li>
* <li>
* When deletes are executed, the store is also updated.
* </li>
* <li>
* And at the completion of a successful rename, the source directory
* is also removed.
* </li>
* </ol>
* <pre>
*
* </pre>
*/
public class ProgressiveRenameTracker extends RenameTracker {
/**
* The collection of paths to delete; this is added as individual files
* are renamed.
* <p>
* The metastore is only updated with these entries after the DELETE
* call containing these paths succeeds.
* <p>
* If the DELETE fails; the filesystem will use
* {@code MultiObjectDeleteSupport} to remove all successfully deleted
* entries from the metastore.
*/
private final Collection<Path> pathsToDelete = new HashSet<>();
public ProgressiveRenameTracker(
final StoreContext storeContext,
final MetadataStore metadataStore,
final Path sourceRoot,
final Path dest,
final BulkOperationState operationState) {
super("ProgressiveRenameTracker",
storeContext, metadataStore, sourceRoot, dest, operationState);
}
/**
* When a file is copied, any ancestors
* are calculated and then the store is updated with
* the destination entries.
* <p>
* The source entries are added to the {@link #pathsToDelete} list.
* @param sourcePath path of source
* @param sourceAttributes status of source.
* @param destAttributes destination attributes
* @param destPath destination path.
* @param blockSize block size.
* @param addAncestors should ancestors be added?
* @throws IOException failure
*/
@Override
public void fileCopied(
final Path sourcePath,
final S3ObjectAttributes sourceAttributes,
final S3ObjectAttributes destAttributes,
final Path destPath,
final long blockSize,
final boolean addAncestors) throws IOException {
// build the list of entries to add in a synchronized block.
final List<PathMetadata> entriesToAdd = new ArrayList<>(1);
LOG.debug("Updating store with copied file {}", sourcePath);
MetadataStore store = getMetadataStore();
synchronized (this) {
checkArgument(!pathsToDelete.contains(sourcePath),
"File being renamed is already processed %s", destPath);
// create the file metadata and update the lists
// the pathsToDelete field is incremented with the new source path,
// for deletion after the DELETE operation succeeds;
// the entriesToAdd variable is filled in with all entries
// to add within this method
S3Guard.addMoveFile(
store,
pathsToDelete,
entriesToAdd,
sourcePath,
destPath,
sourceAttributes.getLen(),
blockSize,
getOwner(),
destAttributes.getETag(),
destAttributes.getVersionId());
LOG.debug("New metastore entry : {}", entriesToAdd.get(0));
if (addAncestors) {
// add all new ancestors to the lists
addMoveAncestors(
store,
pathsToDelete,
entriesToAdd,
getSourceRoot(),
sourcePath,
destPath,
getOwner());
}
}
// outside the lock, the entriesToAdd variable has all the new entries to
// create. ...so update the store.
// no entries are deleted at this point.
try (DurationInfo ignored = new DurationInfo(LOG, false,
"Adding new metastore entries")) {
store.move(null, entriesToAdd,
getStoreContext().getTimeProvider(),
getOperationState());
}
}
/**
* A directory marker has been added.
* Add the new entry and record the source path as another entry to delete.
* @param sourcePath status of source.
* @param destPath destination path.
* @param addAncestors should ancestors be added?
* @throws IOException failure.
*/
@Override
public void directoryMarkerCopied(
final Path sourcePath,
final Path destPath,
final boolean addAncestors) throws IOException {
// this list is created on demand.
final List<PathMetadata> entriesToAdd = new ArrayList<>(1);
MetadataStore store = getMetadataStore();
synchronized (this) {
addMoveDir(store,
pathsToDelete,
entriesToAdd,
sourcePath,
destPath,
getOwner());
// Ancestor directories may not be listed, so we explicitly add them
if (addAncestors) {
addMoveAncestors(store,
pathsToDelete,
entriesToAdd,
getSourceRoot(),
sourcePath,
destPath,
getOwner());
}
}
// outside the lock, the entriesToAdd list has all new files to create.
// ...so update the store.
try (DurationInfo ignored = new DurationInfo(LOG, false,
"adding %s metastore entries", entriesToAdd.size())) {
store.move(null, entriesToAdd,
getStoreContext().getTimeProvider(),
getOperationState());
}
}
@Override
public synchronized void moveSourceDirectory() throws IOException {
// this moves the source directory in the metastore if it has not
// already been processed.
// TODO S3Guard: performance: mark destination dirs as authoritative
if (!pathsToDelete.contains(getSourceRoot())) {
final List<Path> toDelete = new ArrayList<>(1);
final List<PathMetadata> toAdd = new ArrayList<>(1);
addMoveDir(getMetadataStore(), pathsToDelete, toAdd,
getSourceRoot(),
getDest(),
getOwner());
getMetadataStore().move(toDelete, toAdd,
getStoreContext().getTimeProvider(),
getOperationState());
}
}
/**
* As source objects are deleted, so is the list of entries.
* @param paths path of objects deleted.
* @throws IOException failure.
*/
@Override
public void sourceObjectsDeleted(
final Collection<Path> paths) throws IOException {
// delete the paths from the metastore
try (DurationInfo ignored = new DurationInfo(LOG, false,
"delete %s metastore entries", paths.size())) {
getMetadataStore().move(paths, null,
getStoreContext().getTimeProvider(),
getOperationState());
}
}
@Override
public synchronized void completeRename() throws IOException {
// and finish off; by deleting source directories.
sourceObjectsDeleted(pathsToDelete);
super.completeRename();
}
}

View File

@ -0,0 +1,275 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.s3guard;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import com.amazonaws.SdkBaseException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
import org.apache.hadoop.fs.s3a.impl.AbstractStoreOperation;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.DurationInfo;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.hadoop.fs.s3a.S3AUtils.translateException;
/**
* A class which manages updating the metastore with the rename process
* as initiated in the S3AFilesystem rename.
* <p>
* Subclasses must provide an implementation and return it in
* {@code MetadataStore.initiateRenameOperation()}.
* <p>
* The {@link #operationState} field/constructor argument is an opaque state to
* be passed down to the metastore in its move operations; this allows the
* stores to manage ongoing state -while still being able to share
* rename tracker implementations.
* <p>
* This is to avoid performance problems wherein the progressive rename
* tracker causes the store to repeatedly create and write duplicate
* ancestor entries for every file added.
*/
public abstract class RenameTracker extends AbstractStoreOperation {
public static final Logger LOG = LoggerFactory.getLogger(
RenameTracker.class);
/** source path. */
private final Path sourceRoot;
/** destination path. */
private final Path dest;
/**
* Track the duration of this operation.
*/
private final DurationInfo durationInfo;
/**
* Generated name for strings.
*/
private final String name;
/**
* Any ongoing state supplied to the rename tracker
* which is to be passed in with each move operation.
* This must be closed at the end of the tracker's life.
*/
private final BulkOperationState operationState;
/**
* The metadata store for this tracker.
* Always non-null.
* <p>
* This is passed in separate from the store context to guarantee
* that whichever store creates a tracker is explicitly bound to that
* instance.
*/
private final MetadataStore metadataStore;
/**
* Constructor.
* @param name tracker name for logs.
* @param storeContext store context.
* @param metadataStore the stopre
* @param sourceRoot source path.
* @param dest destination path.
* @param operationState ongoing move state.
*/
protected RenameTracker(
final String name,
final StoreContext storeContext,
final MetadataStore metadataStore,
final Path sourceRoot,
final Path dest,
final BulkOperationState operationState) {
super(checkNotNull(storeContext));
checkNotNull(storeContext.getUsername(), "No username");
this.metadataStore = checkNotNull(metadataStore);
this.sourceRoot = checkNotNull(sourceRoot);
this.dest = checkNotNull(dest);
this.operationState = operationState;
this.name = String.format("%s (%s, %s)", name, sourceRoot, dest);
durationInfo = new DurationInfo(LOG, false,
name +" (%s, %s)", sourceRoot, dest);
}
@Override
public String toString() {
return name;
}
public Path getSourceRoot() {
return sourceRoot;
}
public Path getDest() {
return dest;
}
public String getOwner() {
return getStoreContext().getUsername();
}
public BulkOperationState getOperationState() {
return operationState;
}
/**
* Get the metadata store.
* @return a non-null store.
*/
protected MetadataStore getMetadataStore() {
return metadataStore;
}
/**
* A file has been copied.
*
* @param childSource source of the file. This may actually be different
* from the path of the sourceAttributes. (HOW?)
* @param sourceAttributes status of source.
* @param destAttributes destination attributes
* @param destPath destination path.
* @param blockSize block size.
* @param addAncestors should ancestors be added?
* @throws IOException failure.
*/
public abstract void fileCopied(
Path childSource,
S3ObjectAttributes sourceAttributes,
S3ObjectAttributes destAttributes,
Path destPath,
long blockSize,
boolean addAncestors) throws IOException;
/**
* A directory marker has been copied.
* @param sourcePath source path.
* @param destPath destination path.
* @param addAncestors should ancestors be added?
* @throws IOException failure.
*/
public void directoryMarkerCopied(
Path sourcePath,
Path destPath,
boolean addAncestors) throws IOException {
}
/**
* The delete failed.
* <p>
* By the time this is called, the metastore will already have
* been updated with the results of any partial delete failure,
* such that all files known to have been deleted will have been
* removed.
* @param e exception
* @param pathsToDelete paths which were to be deleted.
* @param undeletedObjects list of objects which were not deleted.
*/
public IOException deleteFailed(
final Exception e,
final List<Path> pathsToDelete,
final List<Path> undeletedObjects) {
return convertToIOException(e);
}
/**
* Top level directory move.
* This is invoked after all child entries have been copied
* @throws IOException on failure
*/
public void moveSourceDirectory() throws IOException {
}
/**
* Note that source objects have been deleted.
* The metastore will already have been updated.
* @param paths path of objects deleted.
*/
public void sourceObjectsDeleted(
final Collection<Path> paths) throws IOException {
}
/**
* Complete the operation.
* @throws IOException failure.
*/
public void completeRename() throws IOException {
IOUtils.cleanupWithLogger(LOG, operationState);
noteRenameFinished();
}
/**
* Note that the rename has finished by closing the duration info;
* this will log the duration of the operation at debug.
*/
protected void noteRenameFinished() {
durationInfo.close();
}
/**
* Rename has failed.
* <p>
* The metastore now needs to be updated with its current state
* even though the operation is incomplete.
* Implementations MUST NOT throw exceptions here, as this is going to
* be invoked in an exception handler.
* catch and log or catch and return/wrap.
* <p>
* The base implementation returns the IOE passed in and translates
* any AWS exception into an IOE.
* @param ex the exception which caused the failure.
* This is either an IOException or and AWS exception
* @return an IOException to throw in an exception.
*/
public IOException renameFailed(Exception ex) {
LOG.debug("Rename has failed", ex);
IOUtils.cleanupWithLogger(LOG, operationState);
noteRenameFinished();
return convertToIOException(ex);
}
/**
* Convert a passed in exception (expected to be an IOE or AWS exception)
* into an IOException.
* @param ex exception caught
* @return the exception to throw in the failure handler.
*/
protected IOException convertToIOException(final Exception ex) {
if (ex instanceof IOException) {
return (IOException) ex;
} else if (ex instanceof SdkBaseException) {
return translateException("rename " + sourceRoot + " to " + dest,
sourceRoot.toString(),
(SdkBaseException) ex);
} else {
// should never happen, but for completeness
return new IOException(ex);
}
}
}

View File

@ -47,7 +47,6 @@ import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.Retries.RetryTranslated; import org.apache.hadoop.fs.s3a.Retries.RetryTranslated;
import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.Tristate;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ReflectionUtils;
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_METADATASTORE_METADATA_TTL; import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_METADATASTORE_METADATA_TTL;
@ -132,7 +131,7 @@ public final class S3Guard {
} }
if (conf.get(S3_METADATA_STORE_IMPL) != null && LOG.isDebugEnabled()) { if (conf.get(S3_METADATA_STORE_IMPL) != null && LOG.isDebugEnabled()) {
LOG.debug("Metastore option source {}", LOG.debug("Metastore option source {}",
conf.getPropertySources(S3_METADATA_STORE_IMPL)); (Object)conf.getPropertySources(S3_METADATA_STORE_IMPL));
} }
Class<? extends MetadataStore> aClass = conf.getClass( Class<? extends MetadataStore> aClass = conf.getClass(
@ -157,14 +156,64 @@ public final class S3Guard {
S3AFileStatus status, S3AFileStatus status,
S3AInstrumentation instrumentation, S3AInstrumentation instrumentation,
ITtlTimeProvider timeProvider) throws IOException { ITtlTimeProvider timeProvider) throws IOException {
return putAndReturn(ms, status, instrumentation, timeProvider, null);
}
/**
* Helper function which puts a given S3AFileStatus into the MetadataStore and
* returns the same S3AFileStatus. Instrumentation monitors the put operation.
* @param ms MetadataStore to {@code put()} into.
* @param status status to store
* @param instrumentation instrumentation of the s3a file system
* @param timeProvider Time provider to use when writing entries
* @param operationState possibly-null metastore state tracker.
* @return The same status as passed in
* @throws IOException if metadata store update failed
*/
@RetryTranslated
public static S3AFileStatus putAndReturn(
final MetadataStore ms,
final S3AFileStatus status,
final S3AInstrumentation instrumentation,
final ITtlTimeProvider timeProvider,
@Nullable final BulkOperationState operationState) throws IOException {
long startTimeNano = System.nanoTime(); long startTimeNano = System.nanoTime();
S3Guard.putWithTtl(ms, new PathMetadata(status), timeProvider); try {
instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_PUT_PATH_LATENCY, putWithTtl(ms, new PathMetadata(status), timeProvider, operationState);
(System.nanoTime() - startTimeNano)); } finally {
instrumentation.incrementCounter(S3GUARD_METADATASTORE_PUT_PATH_REQUEST, 1); instrumentation.addValueToQuantiles(
S3GUARD_METADATASTORE_PUT_PATH_LATENCY,
(System.nanoTime() - startTimeNano));
instrumentation.incrementCounter(
S3GUARD_METADATASTORE_PUT_PATH_REQUEST,
1);
}
return status; return status;
} }
/**
* Initiate a bulk write and create an operation state for it.
* This may then be passed into put operations.
* @param metastore store
* @param operation the type of the operation.
* @param path path under which updates will be explicitly put.
* @return a store-specific state to pass into the put operations, or null
* @throws IOException failure
*/
public static BulkOperationState initiateBulkWrite(
@Nullable final MetadataStore metastore,
final BulkOperationState.OperationType operation,
final Path path) throws IOException {
Preconditions.checkArgument(
operation != BulkOperationState.OperationType.Rename,
"Rename operations cannot be started through initiateBulkWrite");
if (metastore == null || isNullMetadataStore(metastore)) {
return null;
} else {
return metastore.initiateBulkWrite(operation, path);
}
}
/** /**
* Convert the data of a directory listing to an array of {@link FileStatus} * Convert the data of a directory listing to an array of {@link FileStatus}
* entries. Tombstones are filtered out at this point. If the listing is null * entries. Tombstones are filtered out at this point. If the listing is null
@ -250,7 +299,7 @@ public final class S3Guard {
if (status != null if (status != null
&& s.getModificationTime() > status.getModificationTime()) { && s.getModificationTime() > status.getModificationTime()) {
LOG.debug("Update ms with newer metadata of: {}", status); LOG.debug("Update ms with newer metadata of: {}", status);
S3Guard.putWithTtl(ms, new PathMetadata(s), timeProvider); S3Guard.putWithTtl(ms, new PathMetadata(s), timeProvider, null);
} }
} }
@ -271,7 +320,7 @@ public final class S3Guard {
if (changed && isAuthoritative) { if (changed && isAuthoritative) {
dirMeta.setAuthoritative(true); // This is the full directory contents dirMeta.setAuthoritative(true); // This is the full directory contents
S3Guard.putWithTtl(ms, dirMeta, timeProvider); S3Guard.putWithTtl(ms, dirMeta, timeProvider, null);
} }
return dirMetaToStatuses(dirMeta); return dirMetaToStatuses(dirMeta);
@ -308,7 +357,7 @@ public final class S3Guard {
* dir. * dir.
* @param owner Hadoop user name. * @param owner Hadoop user name.
* @param authoritative Whether to mark new directories as authoritative. * @param authoritative Whether to mark new directories as authoritative.
* @param timeProvider Time provider for testing. * @param timeProvider Time provider.
*/ */
@Deprecated @Deprecated
@Retries.OnceExceptionsSwallowed @Retries.OnceExceptionsSwallowed
@ -357,7 +406,7 @@ public final class S3Guard {
children.add(new PathMetadata(prevStatus)); children.add(new PathMetadata(prevStatus));
} }
dirMeta = new DirListingMetadata(f, children, authoritative); dirMeta = new DirListingMetadata(f, children, authoritative);
S3Guard.putWithTtl(ms, dirMeta, timeProvider); S3Guard.putWithTtl(ms, dirMeta, timeProvider, null);
} }
pathMetas.add(new PathMetadata(status)); pathMetas.add(new PathMetadata(status));
@ -365,7 +414,7 @@ public final class S3Guard {
} }
// Batched put // Batched put
S3Guard.putWithTtl(ms, pathMetas, timeProvider); S3Guard.putWithTtl(ms, pathMetas, timeProvider, null);
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.error("MetadataStore#put() failure:", ioe); LOG.error("MetadataStore#put() failure:", ioe);
} }
@ -432,7 +481,7 @@ public final class S3Guard {
* take care of those inferred directories of this path explicitly. * take care of those inferred directories of this path explicitly.
* *
* As {@link #addMoveFile} and {@link #addMoveDir}, this method adds resulting * As {@link #addMoveFile} and {@link #addMoveDir}, this method adds resulting
* metadata to the supplied lists. It does not store in MetadataStore. * metadata to the supplied lists. It does not update the MetadataStore.
* *
* @param ms MetadataStore, no-op if it is NullMetadataStore * @param ms MetadataStore, no-op if it is NullMetadataStore
* @param srcPaths stores the source path here * @param srcPaths stores the source path here
@ -469,25 +518,36 @@ public final class S3Guard {
} }
} }
public static void addAncestors(MetadataStore metadataStore, /**
Path qualifiedPath, String username, ITtlTimeProvider timeProvider) * This adds all new ancestors of a path as directories.
throws IOException { * This forwards to
Collection<PathMetadata> newDirs = new ArrayList<>(); * {@link MetadataStore#addAncestors(Path, ITtlTimeProvider, BulkOperationState)}.
Path parent = qualifiedPath.getParent(); * <p>
while (!parent.isRoot()) { * Originally it implemented the logic to probe for an add ancestors,
PathMetadata directory = metadataStore.get(parent); * but with the addition of a store-specific bulk operation state
if (directory == null || directory.isDeleted()) { * it became unworkable.
S3AFileStatus s3aStatus = new S3AFileStatus(Tristate.FALSE, parent, username); *
PathMetadata meta = new PathMetadata(s3aStatus, Tristate.FALSE, false); * @param metadataStore store
newDirs.add(meta); * @param qualifiedPath path to update
} else { * @param operationState (nullable) operational state for a bulk update
break; * @throws IOException failure
} */
parent = parent.getParent(); @Retries.RetryTranslated
} public static void addAncestors(
S3Guard.putWithTtl(metadataStore, newDirs, timeProvider); final MetadataStore metadataStore,
final Path qualifiedPath,
final ITtlTimeProvider timeProvider,
@Nullable final BulkOperationState operationState) throws IOException {
metadataStore.addAncestors(qualifiedPath, timeProvider, operationState);
} }
/**
* Add the fact that a file was moved from a source path to a destination.
* @param srcPaths collection of source paths to update
* @param dstMetas collection of destination meta data entries to update.
* @param srcPath path of the source file.
* @param dstStatus status of the source file after it was copied.
*/
private static void addMoveStatus(Collection<Path> srcPaths, private static void addMoveStatus(Collection<Path> srcPaths,
Collection<PathMetadata> dstMetas, Collection<PathMetadata> dstMetas,
Path srcPath, Path srcPath,
@ -570,30 +630,72 @@ public final class S3Guard {
} }
} }
/**
* Put a directory entry, setting the updated timestamp of the
* directory and its children.
* @param ms metastore
* @param dirMeta directory
* @param timeProvider nullable time provider
* @throws IOException failure.
*/
public static void putWithTtl(MetadataStore ms, DirListingMetadata dirMeta, public static void putWithTtl(MetadataStore ms, DirListingMetadata dirMeta,
ITtlTimeProvider timeProvider) final ITtlTimeProvider timeProvider,
@Nullable final BulkOperationState operationState)
throws IOException { throws IOException {
dirMeta.setLastUpdated(timeProvider.getNow()); long now = timeProvider.getNow();
dirMeta.setLastUpdated(now);
dirMeta.getListing() dirMeta.getListing()
.forEach(pm -> pm.setLastUpdated(timeProvider.getNow())); .forEach(pm -> pm.setLastUpdated(now));
ms.put(dirMeta); ms.put(dirMeta, operationState);
} }
/**
* Put an entry, using the time provider to set its timestamp.
* @param ms metastore
* @param fileMeta entry to write
* @param timeProvider nullable time provider
* @param operationState nullable state for a bulk update
* @throws IOException failure.
*/
public static void putWithTtl(MetadataStore ms, PathMetadata fileMeta, public static void putWithTtl(MetadataStore ms, PathMetadata fileMeta,
@Nullable ITtlTimeProvider timeProvider) throws IOException { @Nullable ITtlTimeProvider timeProvider,
@Nullable final BulkOperationState operationState) throws IOException {
if (timeProvider != null) { if (timeProvider != null) {
fileMeta.setLastUpdated(timeProvider.getNow()); fileMeta.setLastUpdated(timeProvider.getNow());
} else { } else {
LOG.debug("timeProvider is null, put {} without setting last_updated", LOG.debug("timeProvider is null, put {} without setting last_updated",
fileMeta); fileMeta);
} }
ms.put(fileMeta); ms.put(fileMeta, operationState);
} }
/**
* Put entries, using the time provider to set their timestamp.
* @param ms metastore
* @param fileMetas file metadata entries.
* @param timeProvider nullable time provider
* @param operationState nullable state for a bulk update
* @throws IOException failure.
*/
public static void putWithTtl(MetadataStore ms, public static void putWithTtl(MetadataStore ms,
Collection<PathMetadata> fileMetas, Collection<? extends PathMetadata> fileMetas,
@Nullable ITtlTimeProvider timeProvider) @Nullable ITtlTimeProvider timeProvider,
@Nullable final BulkOperationState operationState)
throws IOException { throws IOException {
patchLastUpdated(fileMetas, timeProvider);
ms.put(fileMetas, operationState);
}
/**
* Patch any collection of metadata entries with the timestamp
* of a time provider.
* This <i>MUST</i> be used when creating new entries for directories.
* @param fileMetas file metadata entries.
* @param timeProvider nullable time provider
*/
static void patchLastUpdated(
final Collection<? extends PathMetadata> fileMetas,
@Nullable final ITtlTimeProvider timeProvider) {
if (timeProvider != null) { if (timeProvider != null) {
final long now = timeProvider.getNow(); final long now = timeProvider.getNow();
fileMetas.forEach(fileMeta -> fileMeta.setLastUpdated(now)); fileMetas.forEach(fileMeta -> fileMeta.setLastUpdated(now));
@ -601,9 +703,16 @@ public final class S3Guard {
LOG.debug("timeProvider is null, put {} without setting last_updated", LOG.debug("timeProvider is null, put {} without setting last_updated",
fileMetas); fileMetas);
} }
ms.put(fileMetas);
} }
/**
* Get a path entry provided it is not considered expired.
* @param ms metastore
* @param path path to look up.
* @param timeProvider nullable time provider
* @return the metadata or null if there as no entry.
* @throws IOException failure.
*/
public static PathMetadata getWithTtl(MetadataStore ms, Path path, public static PathMetadata getWithTtl(MetadataStore ms, Path path,
@Nullable ITtlTimeProvider timeProvider) throws IOException { @Nullable ITtlTimeProvider timeProvider) throws IOException {
final PathMetadata pathMetadata = ms.get(path); final PathMetadata pathMetadata = ms.get(path);
@ -616,11 +725,11 @@ public final class S3Guard {
long ttl = timeProvider.getMetadataTtl(); long ttl = timeProvider.getMetadataTtl();
if (pathMetadata != null) { if (pathMetadata != null) {
// Special case: the pathmetadata's last updated is 0. This can happen // Special case: the path metadata's last updated is 0. This can happen
// eg. with an old db using this implementation // eg. with an old db using this implementation
if (pathMetadata.getLastUpdated() == 0) { if (pathMetadata.getLastUpdated() == 0) {
LOG.debug("PathMetadata TTL for {} is 0, so it will be returned as " LOG.debug("PathMetadata TTL for {} is 0, so it will be returned as "
+ "not expired."); + "not expired.", path);
return pathMetadata; return pathMetadata;
} }
@ -636,6 +745,14 @@ public final class S3Guard {
return null; return null;
} }
/**
* List children; mark the result as non-auth if the TTL has expired.
* @param ms metastore
* @param path path to look up.
* @param timeProvider nullable time provider
* @return the listing of entries under a path, or null if there as no entry.
* @throws IOException failure.
*/
public static DirListingMetadata listChildrenWithTtl(MetadataStore ms, public static DirListingMetadata listChildrenWithTtl(MetadataStore ms,
Path path, @Nullable ITtlTimeProvider timeProvider) Path path, @Nullable ITtlTimeProvider timeProvider)
throws IOException { throws IOException {

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.fs.s3a.s3guard; package org.apache.hadoop.fs.s3a.s3guard;
import javax.annotation.Nullable;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.io.PrintStream; import java.io.PrintStream;
@ -63,6 +64,7 @@ import org.apache.hadoop.util.ToolRunner;
import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.Invoker.LOG_EVENT; import static org.apache.hadoop.fs.s3a.Invoker.LOG_EVENT;
import static org.apache.hadoop.fs.s3a.S3AUtils.clearBucketOption;
import static org.apache.hadoop.service.launcher.LauncherExitCodes.*; import static org.apache.hadoop.service.launcher.LauncherExitCodes.*;
/** /**
@ -650,7 +652,13 @@ public abstract class S3GuardTool extends Configured implements Tool {
Preconditions.checkState(getStore() != null, Preconditions.checkState(getStore() != null,
"Metadata Store is not initialized"); "Metadata Store is not initialized");
getStore().destroy(); try {
getStore().destroy();
} catch (TableDeleteTimeoutException e) {
LOG.warn("The table is been deleted but it is still (briefly)"
+ " listed as present in AWS");
LOG.debug("Timeout waiting for table disappearing", e);
}
println(out, "Metadata store is deleted."); println(out, "Metadata store is deleted.");
return SUCCESS; return SUCCESS;
} }
@ -696,9 +704,11 @@ public abstract class S3GuardTool extends Configured implements Tool {
* Put parents into MS and cache if the parents are not presented. * Put parents into MS and cache if the parents are not presented.
* *
* @param f the file or an empty directory. * @param f the file or an empty directory.
* @param operationState store's bulk update state.
* @throws IOException on I/O errors. * @throws IOException on I/O errors.
*/ */
private void putParentsIfNotPresent(FileStatus f) throws IOException { private void putParentsIfNotPresent(FileStatus f,
@Nullable BulkOperationState operationState) throws IOException {
Preconditions.checkNotNull(f); Preconditions.checkNotNull(f);
Path parent = f.getPath().getParent(); Path parent = f.getPath().getParent();
while (parent != null) { while (parent != null) {
@ -708,7 +718,8 @@ public abstract class S3GuardTool extends Configured implements Tool {
S3AFileStatus dir = DynamoDBMetadataStore.makeDirStatus(parent, S3AFileStatus dir = DynamoDBMetadataStore.makeDirStatus(parent,
f.getOwner()); f.getOwner());
S3Guard.putWithTtl(getStore(), new PathMetadata(dir), S3Guard.putWithTtl(getStore(), new PathMetadata(dir),
getFilesystem().getTtlTimeProvider()); getFilesystem().getTtlTimeProvider(),
operationState);
dirCache.add(parent); dirCache.add(parent);
parent = parent.getParent(); parent = parent.getParent();
} }
@ -721,6 +732,9 @@ public abstract class S3GuardTool extends Configured implements Tool {
*/ */
private long importDir(FileStatus status) throws IOException { private long importDir(FileStatus status) throws IOException {
Preconditions.checkArgument(status.isDirectory()); Preconditions.checkArgument(status.isDirectory());
BulkOperationState operationState = getStore().initiateBulkWrite(
BulkOperationState.OperationType.Put,
status.getPath());
RemoteIterator<S3ALocatedFileStatus> it = getFilesystem() RemoteIterator<S3ALocatedFileStatus> it = getFilesystem()
.listFilesAndEmptyDirectories(status.getPath(), true); .listFilesAndEmptyDirectories(status.getPath(), true);
long items = 0; long items = 0;
@ -741,9 +755,11 @@ public abstract class S3GuardTool extends Configured implements Tool {
located.getETag(), located.getETag(),
located.getVersionId()); located.getVersionId());
} }
putParentsIfNotPresent(child); putParentsIfNotPresent(child, operationState);
S3Guard.putWithTtl(getStore(), new PathMetadata(child), S3Guard.putWithTtl(getStore(),
getFilesystem().getTtlTimeProvider()); new PathMetadata(child),
getFilesystem().getTtlTimeProvider(),
operationState);
items++; items++;
} }
return items; return items;
@ -779,7 +795,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
long items = 1; long items = 1;
if (status.isFile()) { if (status.isFile()) {
PathMetadata meta = new PathMetadata(status); PathMetadata meta = new PathMetadata(status);
getStore().put(meta); getStore().put(meta, null);
} else { } else {
items = importDir(status); items = importDir(status);
} }
@ -1137,16 +1153,19 @@ public abstract class S3GuardTool extends Configured implements Tool {
} }
String s3Path = paths.get(0); String s3Path = paths.get(0);
CommandFormat commands = getCommandFormat(); CommandFormat commands = getCommandFormat();
URI fsURI = toUri(s3Path);
// check if UNGUARDED_FLAG is passed and use NullMetadataStore in // check if UNGUARDED_FLAG is passed and use NullMetadataStore in
// config to avoid side effects like creating the table if not exists // config to avoid side effects like creating the table if not exists
Configuration unguardedConf = getConf();
if (commands.getOpt(UNGUARDED_FLAG)) { if (commands.getOpt(UNGUARDED_FLAG)) {
LOG.debug("Unguarded flag is passed to command :" + this.getName()); LOG.debug("Unguarded flag is passed to command :" + this.getName());
getConf().set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL); clearBucketOption(unguardedConf, fsURI.getHost(), S3_METADATA_STORE_IMPL);
unguardedConf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL);
} }
S3AFileSystem fs = (S3AFileSystem) FileSystem.newInstance( S3AFileSystem fs = (S3AFileSystem) FileSystem.newInstance(
toUri(s3Path), getConf()); fsURI, unguardedConf);
setFilesystem(fs); setFilesystem(fs);
Configuration conf = fs.getConf(); Configuration conf = fs.getConf();
URI fsUri = fs.getUri(); URI fsUri = fs.getUri();

View File

@ -0,0 +1,34 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.s3guard;
import org.apache.hadoop.fs.PathIOException;
/**
* An exception raised when a table being deleted is still present after
* the wait time is exceeded.
*/
public class TableDeleteTimeoutException extends PathIOException {
TableDeleteTimeoutException(final String path,
final String error,
final Throwable cause) {
super(path, error, cause);
}
}

View File

@ -1474,6 +1474,18 @@ Caused by: java.lang.NullPointerException
... 1 more ... 1 more
``` ```
### Error `Attempt to change a resource which is still in use: Table is being deleted`
```
com.amazonaws.services.dynamodbv2.model.ResourceInUseException:
Attempt to change a resource which is still in use: Table is being deleted:
s3guard.test.testDynamoDBInitDestroy351245027
(Service: AmazonDynamoDBv2; Status Code: 400; Error Code: ResourceInUseException;)
```
You have attempted to call `hadoop s3guard destroy` on a table which is already
being destroyed.
## Other Topics ## Other Topics
For details on how to test S3Guard, see [Testing S3Guard](./testing.html#s3guard) For details on how to test S3Guard, see [Testing S3Guard](./testing.html#s3guard)

View File

@ -18,14 +18,23 @@
package org.apache.hadoop.fs.contract.s3a; package org.apache.hadoop.fs.contract.s3a;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.contract.AbstractContractRenameTest; import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
import org.apache.hadoop.fs.contract.AbstractFSContract; import org.apache.hadoop.fs.contract.AbstractFSContract;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.Statistic;
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
import static org.apache.hadoop.fs.s3a.S3ATestConstants.S3A_TEST_TIMEOUT;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard; import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
/** /**
@ -33,6 +42,15 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard;
*/ */
public class ITestS3AContractRename extends AbstractContractRenameTest { public class ITestS3AContractRename extends AbstractContractRenameTest {
public static final Logger LOG = LoggerFactory.getLogger(
ITestS3AContractRename.class);
@Override
protected int getTestTimeoutMillis() {
return S3A_TEST_TIMEOUT;
}
/** /**
* Create a configuration, possibly patching in S3Guard options. * Create a configuration, possibly patching in S3Guard options.
* @return a configuration * @return a configuration
@ -50,6 +68,12 @@ public class ITestS3AContractRename extends AbstractContractRenameTest {
return new S3AContract(conf); return new S3AContract(conf);
} }
@Override
public void teardown() throws Exception {
describe("\nTeardown\n");
super.teardown();
}
@Override @Override
public void testRenameDirIntoExistingDir() throws Throwable { public void testRenameDirIntoExistingDir() throws Throwable {
describe("Verify renaming a dir into an existing dir puts the files" describe("Verify renaming a dir into an existing dir puts the files"
@ -64,12 +88,55 @@ public class ITestS3AContractRename extends AbstractContractRenameTest {
Path destDir = path("dest"); Path destDir = path("dest");
Path destFilePath = new Path(destDir, "dest-512.txt"); Path destFilePath = new Path(destDir, "dest-512.txt");
byte[] destDateset = dataset(512, 'A', 'Z'); byte[] destDataset = dataset(512, 'A', 'Z');
writeDataset(fs, destFilePath, destDateset, destDateset.length, 1024, writeDataset(fs, destFilePath, destDataset, destDataset.length, 1024,
false); false);
assertIsFile(destFilePath); assertIsFile(destFilePath);
boolean rename = fs.rename(srcDir, destDir); boolean rename = fs.rename(srcDir, destDir);
assertFalse("s3a doesn't support rename to non-empty directory", rename); assertFalse("s3a doesn't support rename to non-empty directory", rename);
} }
/**
* Test that after renaming, the nested file is moved along with all its
* ancestors. It is similar to {@link #testRenamePopulatesDirectoryAncestors}.
*
* This is an extension testRenamePopulatesFileAncestors
* of the superclass version which does better
* logging of the state of the store before the assertions.
*/
@Test
public void testRenamePopulatesFileAncestors2() throws Exception {
final S3AFileSystem fs = (S3AFileSystem) getFileSystem();
Path base = path("testRenamePopulatesFileAncestors2");
final Path src = new Path(base, "src");
Path dest = new Path(base, "dest");
fs.mkdirs(src);
final String nestedFile = "/dir1/dir2/dir3/fileA";
// size of file to create
int filesize = 16 * 1024;
byte[] srcDataset = dataset(filesize, 'a', 'z');
Path srcFile = path(src + nestedFile);
Path destFile = path(dest + nestedFile);
writeDataset(fs, srcFile, srcDataset, srcDataset.length,
1024, false);
S3ATestUtils.MetricDiff fileCopyDiff = new S3ATestUtils.MetricDiff(fs,
Statistic.FILES_COPIED);
S3ATestUtils.MetricDiff fileCopyBytes = new S3ATestUtils.MetricDiff(fs,
Statistic.FILES_COPIED_BYTES);
fs.rename(src, dest);
describe("Rename has completed, examining data under " + base);
fileCopyDiff.assertDiffEquals("Number of files copied", 1);
fileCopyBytes.assertDiffEquals("Number of bytes copied", filesize);
// log everything in the base directory.
S3ATestUtils.lsR(fs, base, true);
// look at the data.
verifyFileContents(fs, destFile, srcDataset);
describe("validating results");
validateAncestorsMoved(src, dest, nestedFile);
}
} }

View File

@ -20,9 +20,14 @@ package org.apache.hadoop.fs.s3a;
import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.DeleteObjectsRequest;
import com.amazonaws.services.s3.model.MultiObjectDeleteException; import com.amazonaws.services.s3.model.MultiObjectDeleteException;
import com.google.common.collect.Lists;
import org.junit.Assume;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport;
import org.junit.Test; import org.junit.Test;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -31,13 +36,16 @@ import org.slf4j.LoggerFactory;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.nio.file.AccessDeniedException;
import static org.apache.hadoop.fs.contract.ContractTestUtils.*; import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.getLandsatCSVPath; import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.failIf;
import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.*;
import static org.apache.hadoop.fs.s3a.impl.TestPartialDeleteFailures.keysToDelete;
import static org.apache.hadoop.test.LambdaTestUtils.*; import static org.apache.hadoop.test.LambdaTestUtils.*;
/** /**
* Test S3A Failure translation. * ITest for failure handling, primarily multipart deletion.
*/ */
public class ITestS3AFailureHandling extends AbstractS3ATestBase { public class ITestS3AFailureHandling extends AbstractS3ATestBase {
private static final Logger LOG = private static final Logger LOG =
@ -69,12 +77,17 @@ public class ITestS3AFailureHandling extends AbstractS3ATestBase {
private void removeKeys(S3AFileSystem fileSystem, String... keys) private void removeKeys(S3AFileSystem fileSystem, String... keys)
throws IOException { throws IOException {
fileSystem.removeKeys(buildDeleteRequest(keys), false);
}
private List<DeleteObjectsRequest.KeyVersion> buildDeleteRequest(
final String[] keys) {
List<DeleteObjectsRequest.KeyVersion> request = new ArrayList<>( List<DeleteObjectsRequest.KeyVersion> request = new ArrayList<>(
keys.length); keys.length);
for (String key : keys) { for (String key : keys) {
request.add(new DeleteObjectsRequest.KeyVersion(key)); request.add(new DeleteObjectsRequest.KeyVersion(key));
} }
fileSystem.removeKeys(request, false, false); return request;
} }
@Test @Test
@ -87,12 +100,87 @@ public class ITestS3AFailureHandling extends AbstractS3ATestBase {
timer.end("removeKeys"); timer.end("removeKeys");
} }
private Path maybeGetCsvPath() {
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));
return new Path(csvFile);
}
/**
* Test low-level failure handling with low level delete request.
*/
@Test @Test
public void testMultiObjectDeleteNoPermissions() throws Throwable { public void testMultiObjectDeleteNoPermissions() throws Throwable {
Path testFile = getLandsatCSVPath(getConfiguration()); describe("Delete the landsat CSV file and expect it to fail");
S3AFileSystem fs = (S3AFileSystem)testFile.getFileSystem( Path csvPath = maybeGetCsvPath();
S3AFileSystem fs = (S3AFileSystem) csvPath.getFileSystem(
getConfiguration()); getConfiguration());
intercept(MultiObjectDeleteException.class, List<DeleteObjectsRequest.KeyVersion> keys
() -> removeKeys(fs, fs.pathToKey(testFile))); = buildDeleteRequest(
new String[]{
fs.pathToKey(csvPath),
"missing-key.csv"
});
MultiObjectDeleteException ex = intercept(
MultiObjectDeleteException.class,
() -> fs.removeKeys(keys, false));
final List<Path> undeleted
= extractUndeletedPaths(ex, fs::keyToQualifiedPath);
String undeletedFiles = join(undeleted);
failIf(undeleted.size() != 2,
"undeleted list size wrong: " + undeletedFiles,
ex);
assertTrue("no CSV in " +undeletedFiles, undeleted.contains(csvPath));
// and a full split, after adding a new key
String marker = "/marker";
Path markerPath = fs.keyToQualifiedPath(marker);
keys.add(new DeleteObjectsRequest.KeyVersion(marker));
Pair<List<Path>, List<Path>> pair =
new MultiObjectDeleteSupport(fs.createStoreContext())
.splitUndeletedKeys(ex, keys);
assertEquals(undeleted, pair.getLeft());
List<Path> right = pair.getRight();
assertEquals("Wrong size for " + join(right), 1, right.size());
assertEquals(markerPath, right.get(0));
}
/**
* See what happens when you delete two entries which do not exist.
* It must not raise an exception.
*/
@Test
public void testMultiObjectDeleteMissingEntriesSucceeds() throws Throwable {
describe("Delete keys which don't exist");
Path base = path("missing");
S3AFileSystem fs = getFileSystem();
List<DeleteObjectsRequest.KeyVersion> keys = keysToDelete(
Lists.newArrayList(new Path(base, "1"), new Path(base, "2")));
fs.removeKeys(keys, false);
}
private String join(final Iterable iterable) {
return "[" + StringUtils.join(iterable, ",") + "]";
}
/**
* Test low-level failure handling with a single-entry file.
* This is deleted as a single call, so isn't that useful.
*/
@Test
public void testSingleObjectDeleteNoPermissionsTranslated() throws Throwable {
describe("Delete the landsat CSV file and expect it to fail");
Path csvPath = maybeGetCsvPath();
S3AFileSystem fs = (S3AFileSystem) csvPath.getFileSystem(
getConfiguration());
AccessDeniedException aex = intercept(AccessDeniedException.class,
() -> fs.delete(csvPath, false));
Throwable cause = aex.getCause();
failIf(cause == null, "no nested exception", aex);
} }
} }

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff; import org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
import org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore; import org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore; import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore;
@ -129,8 +130,15 @@ public class ITestS3AMetadataPersistenceException extends AbstractS3ATestBase {
} }
@Override @Override
public void put(PathMetadata meta) throws IOException { public void put(PathMetadata meta,
final BulkOperationState operationState) throws IOException {
throw ioException; throw ioException;
} }
@Override
public void put(final PathMetadata meta) throws IOException {
put(meta, null);
}
} }
} }

View File

@ -47,6 +47,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile;
import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.FailureInjectionPolicy.*; import static org.apache.hadoop.fs.s3a.FailureInjectionPolicy.*;
import static org.apache.hadoop.fs.s3a.InconsistentAmazonS3Client.*; import static org.apache.hadoop.fs.s3a.InconsistentAmazonS3Client.*;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/** /**
* Test S3Guard list consistency feature by injecting delayed listObjects() * Test S3Guard list consistency feature by injecting delayed listObjects()
@ -253,13 +254,11 @@ public class ITestS3GuardListConsistency extends AbstractS3ATestBase {
assertFalse(list.contains(path("a3/b/dir3-" + assertFalse(list.contains(path("a3/b/dir3-" +
DEFAULT_DELAY_KEY_SUBSTRING))); DEFAULT_DELAY_KEY_SUBSTRING)));
try { intercept(FileNotFoundException.class, "",
RemoteIterator<S3ALocatedFileStatus> old = fs.listFilesAndEmptyDirectories( "Recently renamed dir should not be visible",
path("a"), true); () -> S3AUtils.mapLocatedFiles(
fail("Recently renamed dir should not be visible"); fs.listFilesAndEmptyDirectories(path("a"), true),
} catch(FileNotFoundException e) { FileStatus::getPath));
// expected
}
} }
@Test @Test

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.Progressable;
/** /**
@ -177,7 +178,8 @@ public class MockS3AFileSystem extends S3AFileSystem {
} }
@Override @Override
void finishedWrite(String key, long length, String eTag, String versionId) { void finishedWrite(String key, long length, String eTag, String versionId,
BulkOperationState operationState) {
} }

View File

@ -32,6 +32,7 @@ import org.junit.Test;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
import org.apache.hadoop.fs.s3a.impl.ChangeTracker; import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
@ -58,6 +59,8 @@ public class TestStreamChangeTracker extends HadoopTestBase {
public static final String URI = "s3a://" + BUCKET + "/" + OBJECT; public static final String URI = "s3a://" + BUCKET + "/" + OBJECT;
public static final Path PATH = new Path(URI);
@Test @Test
public void testVersionCheckingHandlingNoVersions() throws Throwable { public void testVersionCheckingHandlingNoVersions() throws Throwable {
LOG.info("If an endpoint doesn't return versions, that's OK"); LOG.info("If an endpoint doesn't return versions, that's OK");
@ -434,10 +437,12 @@ public class TestStreamChangeTracker extends HadoopTestBase {
private S3ObjectAttributes objectAttributes( private S3ObjectAttributes objectAttributes(
String etag, String versionId) { String etag, String versionId) {
return new S3ObjectAttributes(BUCKET, return new S3ObjectAttributes(BUCKET,
PATH,
OBJECT, OBJECT,
null, null,
null, null,
etag, etag,
versionId); versionId,
0);
} }
} }

View File

@ -23,9 +23,7 @@ import java.io.FileOutputStream;
import java.io.IOException; import java.io.IOException;
import java.net.URI; import java.net.URI;
import java.nio.file.AccessDeniedException; import java.nio.file.AccessDeniedException;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.IntStream; import java.util.stream.IntStream;
import com.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.AWSCredentials;
@ -53,7 +51,6 @@ import org.apache.hadoop.fs.s3a.commit.CommitOperations;
import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome;
import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
@ -524,78 +521,6 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
return path(getMethodName()); return path(getMethodName());
} }
@Test
public void testRestrictedRename() throws Throwable {
describe("rename with parent paths not writeable");
executeRestrictedRename(createAssumedRoleConfig());
}
@Test
public void testRestrictedSingleDeleteRename() throws Throwable {
describe("rename with parent paths not writeable"
+ " and multi-object delete disabled");
Configuration conf = createAssumedRoleConfig();
conf.setBoolean(ENABLE_MULTI_DELETE, false);
executeRestrictedRename(conf);
}
/**
* Execute a sequence of rename operations with access locked down.
* @param conf FS configuration
*/
public void executeRestrictedRename(final Configuration conf)
throws IOException {
Path basePath = methodPath();
Path restrictedDir = new Path(basePath, "renameSrc");
Path destPath = new Path(basePath, "renameDest");
Path child = new Path(restrictedDir, "child");
// the full FS
S3AFileSystem fs = getFileSystem();
fs.delete(basePath, true);
bindRolePolicyStatements(conf,
STATEMENT_S3GUARD_CLIENT,
STATEMENT_ALLOW_SSE_KMS_RW,
STATEMENT_ALL_BUCKET_READ_ACCESS,
new Statement(Effects.Allow)
.addActions(S3_PATH_RW_OPERATIONS)
.addResources(directory(restrictedDir))
.addResources(directory(destPath))
);
roleFS = (S3AFileSystem) restrictedDir.getFileSystem(conf);
roleFS.getFileStatus(ROOT);
roleFS.mkdirs(restrictedDir);
// you can create an adjacent child
touch(roleFS, child);
roleFS.delete(destPath, true);
// as dest doesn't exist, this will map child -> dest
assertRenameOutcome(roleFS, child, destPath, true);
assertIsFile(destPath);
assertIsDirectory(restrictedDir);
Path renamedDestPath = new Path(restrictedDir, destPath.getName());
assertRenameOutcome(roleFS, destPath, restrictedDir, true);
assertIsFile(renamedDestPath);
roleFS.delete(restrictedDir, true);
roleFS.delete(destPath, true);
}
@Test
public void testRestrictedRenameReadOnlyData() throws Throwable {
describe("rename with source read only, multidelete");
executeRenameReadOnlyData(createAssumedRoleConfig());
}
@Test
public void testRestrictedRenameReadOnlySingleDelete() throws Throwable {
describe("rename with source read only single delete");
Configuration conf = createAssumedRoleConfig();
conf.setBoolean(ENABLE_MULTI_DELETE, false);
executeRenameReadOnlyData(conf);
}
/** /**
* Without simulation of STS failures, and with STS overload likely to * Without simulation of STS failures, and with STS overload likely to
* be very rare, there'll be no implicit test coverage of * be very rare, there'll be no implicit test coverage of
@ -615,102 +540,6 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
} }
} }
/**
* Execute a sequence of rename operations where the source
* data is read only to the client calling rename().
* This will cause the inner delete() operations to fail, whose outcomes
* are explored.
* Multiple files are created (in parallel) for some renames, so exploring
* the outcome on bulk delete calls, including verifying that a
* MultiObjectDeleteException is translated to an AccessDeniedException.
* <ol>
* <li>The exception raised is AccessDeniedException,
* from single and multi DELETE calls.</li>
* <li>It happens after the COPY. Not ideal, but, well, we can't pretend
* it's a filesystem forever.</li>
* </ol>
* @param conf FS configuration
*/
public void executeRenameReadOnlyData(final Configuration conf)
throws Exception {
assume("Does not work with S3Guard", !getFileSystem().hasMetadataStore());
Path basePath = methodPath();
Path destDir = new Path(basePath, "renameDest");
Path readOnlyDir = new Path(basePath, "readonlyDir");
Path readOnlyFile = new Path(readOnlyDir, "readonlyChild");
// the full FS
S3AFileSystem fs = getFileSystem();
fs.delete(basePath, true);
// this file is readable by the roleFS, but cannot be deleted
touch(fs, readOnlyFile);
bindRolePolicyStatements(conf,
STATEMENT_S3GUARD_CLIENT,
STATEMENT_ALL_BUCKET_READ_ACCESS,
new Statement(Effects.Allow)
.addActions(S3_PATH_RW_OPERATIONS)
.addResources(directory(destDir))
);
roleFS = (S3AFileSystem) destDir.getFileSystem(conf);
roleFS.delete(destDir, true);
roleFS.mkdirs(destDir);
// rename will fail in the delete phase
forbidden(readOnlyFile.toString(),
() -> roleFS.rename(readOnlyFile, destDir));
// and the source file is still there
assertIsFile(readOnlyFile);
// but so is the copied version, because there's no attempt
// at rollback, or preflight checking on the delete permissions
Path renamedFile = new Path(destDir, readOnlyFile.getName());
assertIsFile(renamedFile);
ContractTestUtils.assertDeleted(roleFS, renamedFile, true);
assertFileCount("Empty Dest Dir", roleFS,
destDir, 0);
// create a set of files
// this is done in parallel as it is 10x faster on a long-haul test run.
int range = 10;
touchFiles(fs, readOnlyDir, range);
// don't forget about that original file!
final long createdFiles = range + 1;
// are they all there?
assertFileCount("files ready to rename", roleFS,
readOnlyDir, createdFiles);
// try to rename the directory
LOG.info("Renaming readonly files {} to {}", readOnlyDir, destDir);
AccessDeniedException ex = forbidden("",
() -> roleFS.rename(readOnlyDir, destDir));
LOG.info("Result of renaming read-only files is AccessDeniedException", ex);
assertFileCount("files copied to the destination", roleFS,
destDir, createdFiles);
assertFileCount("files in the source directory", roleFS,
readOnlyDir, createdFiles);
// and finally (so as to avoid the delay of POSTing some more objects,
// delete that r/o source
forbidden("", () -> roleFS.delete(readOnlyDir, true));
}
/**
* Parallel-touch a set of files in the destination directory.
* @param fs filesystem
* @param destDir destination
* @param range range 1..range inclusive of files to create.
*/
public void touchFiles(final S3AFileSystem fs,
final Path destDir,
final int range) {
IntStream.rangeClosed(1, range).parallel().forEach(
(i) -> eval(() -> touch(fs, new Path(destDir, "file-" + i))));
}
@Test @Test
public void testRestrictedCommitActions() throws Throwable { public void testRestrictedCommitActions() throws Throwable {
describe("Attempt commit operations against a path with restricted rights"); describe("Attempt commit operations against a path with restricted rights");
@ -779,12 +608,16 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
// all those commits must fail // all those commits must fail
List<SinglePendingCommit> commits = pendingCommits.getLeft().getCommits(); List<SinglePendingCommit> commits = pendingCommits.getLeft().getCommits();
assertEquals(range, commits.size()); assertEquals(range, commits.size());
commits.parallelStream().forEach( try(CommitOperations.CommitContext commitContext
(c) -> { = operations.initiateCommitOperation(uploadDest)) {
CommitOperations.MaybeIOE maybeIOE = operations.commit(c, "origin"); commits.parallelStream().forEach(
Path path = c.destinationPath(); (c) -> {
assertCommitAccessDenied(path, maybeIOE); CommitOperations.MaybeIOE maybeIOE =
}); commitContext.commit(c, "origin");
Path path = c.destinationPath();
assertCommitAccessDenied(path, maybeIOE);
});
}
// fail of all list and abort of .pending files. // fail of all list and abort of .pending files.
LOG.info("abortAllSinglePendingCommits({})", readOnlyDir); LOG.info("abortAllSinglePendingCommits({})", readOnlyDir);
@ -840,24 +673,25 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
@Test @Test
public void testPartialDelete() throws Throwable { public void testPartialDelete() throws Throwable {
describe("delete with part of the child tree read only; multidelete"); describe("delete with part of the child tree read only; multidelete");
executePartialDelete(createAssumedRoleConfig()); executePartialDelete(createAssumedRoleConfig(), false);
} }
@Test @Test
public void testPartialDeleteSingleDelete() throws Throwable { public void testPartialDeleteSingleDelete() throws Throwable {
describe("delete with part of the child tree read only"); describe("delete with part of the child tree read only");
Configuration conf = createAssumedRoleConfig(); executePartialDelete(createAssumedRoleConfig(), true);
conf.setBoolean(ENABLE_MULTI_DELETE, false);
executePartialDelete(conf);
} }
/** /**
* Have a directory with full R/W permissions, but then remove * Have a directory with full R/W permissions, but then remove
* write access underneath, and try to delete it. * write access underneath, and try to delete it.
* @param conf FS configuration * @param conf FS configuration
* @param singleDelete flag to indicate this is a single delete operation
*/ */
public void executePartialDelete(final Configuration conf) public void executePartialDelete(final Configuration conf,
final boolean singleDelete)
throws Exception { throws Exception {
conf.setBoolean(ENABLE_MULTI_DELETE, !singleDelete);
Path destDir = methodPath(); Path destDir = methodPath();
Path readOnlyDir = new Path(destDir, "readonlyDir"); Path readOnlyDir = new Path(destDir, "readonlyDir");
@ -888,25 +722,4 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
roleFS.delete(pathWhichDoesntExist, true)); roleFS.delete(pathWhichDoesntExist, true));
} }
/**
* Assert that the number of files in a destination matches that expected.
* @param text text to use in the message
* @param fs filesystem
* @param path path to list (recursively)
* @param expected expected count
* @throws IOException IO problem
*/
private static void assertFileCount(String text, FileSystem fs,
Path path, long expected)
throws IOException {
List<String> files = new ArrayList<>();
applyLocatedFiles(fs.listFiles(path, true),
(status) -> files.add(status.getPath().toString()));
long actual = files.size();
if (actual != expected) {
String ls = files.stream().collect(Collectors.joining("\n"));
fail(text + ": expected " + expected + " files in " + path
+ " but got " + actual + "\n" + ls);
}
}
} }

View File

@ -18,8 +18,12 @@
package org.apache.hadoop.fs.s3a.auth; package org.apache.hadoop.fs.s3a.auth;
import java.io.IOException;
import java.nio.file.AccessDeniedException; import java.nio.file.AccessDeniedException;
import java.util.List;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.JsonProcessingException;
import org.junit.Assume; import org.junit.Assume;
@ -160,18 +164,35 @@ public final class RoleTestUtils {
/** /**
* Assert that an operation is forbidden. * Assert that an operation is forbidden.
* @param <T> type of closure
* @param contained contained text, may be null * @param contained contained text, may be null
* @param eval closure to evaluate * @param eval closure to evaluate
* @param <T> type of closure
* @return the access denied exception * @return the access denied exception
* @throws Exception any other exception * @throws Exception any other exception
*/ */
public static <T> AccessDeniedException forbidden( public static <T> AccessDeniedException forbidden(
String contained, final String contained,
Callable<T> eval) final Callable<T> eval)
throws Exception {
return forbidden("", contained, eval);
}
/**
* Assert that an operation is forbidden.
* @param <T> type of closure
* @param message error message
* @param contained contained text, may be null
* @param eval closure to evaluate
* @return the access denied exception
* @throws Exception any other exception
*/
public static <T> AccessDeniedException forbidden(
final String message,
final String contained,
final Callable<T> eval)
throws Exception { throws Exception {
return intercept(AccessDeniedException.class, return intercept(AccessDeniedException.class,
contained, eval); contained, message, eval);
} }
/** /**
@ -209,4 +230,23 @@ public final class RoleTestUtils {
actual.getSessionToken()); actual.getSessionToken());
} }
/**
* Parallel-touch a set of files in the destination directory.
* @param fs filesystem
* @param destDir destination
* @param range range 1..range inclusive of files to create.
* @return the list of paths created.
*/
public static List<Path> touchFiles(final FileSystem fs,
final Path destDir,
final int range) throws IOException {
List<Path> paths = IntStream.rangeClosed(1, range)
.mapToObj((i) -> new Path(destDir, "file-" + i))
.collect(Collectors.toList());
for (Path path : paths) {
touch(fs, path);
}
return paths;
}
} }

View File

@ -205,6 +205,7 @@ public abstract class AbstractCommitITest extends AbstractS3ATestBase {
*/ */
@Override @Override
public void teardown() throws Exception { public void teardown() throws Exception {
LOG.info("AbstractCommitITest::teardown");
waitForConsistency(); waitForConsistency();
// make sure there are no failures any more // make sure there are no failures any more
resetFailures(); resetFailures();
@ -495,6 +496,7 @@ public abstract class AbstractCommitITest extends AbstractS3ATestBase {
public static SuccessData loadSuccessFile(final S3AFileSystem fs, public static SuccessData loadSuccessFile(final S3AFileSystem fs,
final Path outputPath) throws IOException { final Path outputPath) throws IOException {
Path success = new Path(outputPath, _SUCCESS); Path success = new Path(outputPath, _SUCCESS);
ContractTestUtils.assertIsFile(fs, success);
FileStatus status = fs.getFileStatus(success); FileStatus status = fs.getFileStatus(success);
assertTrue("0 byte success file - not a s3guard committer " + success, assertTrue("0 byte success file - not a s3guard committer " + success,
status.getLen() > 0); status.getLen() > 0);

View File

@ -71,6 +71,9 @@ public abstract class AbstractITCommitMRJob extends AbstractYarnClusterITest {
S3AFileSystem fs = getFileSystem(); S3AFileSystem fs = getFileSystem();
// final dest is in S3A // final dest is in S3A
Path outputPath = path(getMethodName()); Path outputPath = path(getMethodName());
// create and delete to force in a tombstone marker -see HADOOP-16207
fs.mkdirs(outputPath);
fs.delete(outputPath, true);
String commitUUID = UUID.randomUUID().toString(); String commitUUID = UUID.randomUUID().toString();
String suffix = isUniqueFilenames() ? ("-" + commitUUID) : ""; String suffix = isUniqueFilenames() ? ("-" + commitUUID) : "";
@ -116,6 +119,7 @@ public abstract class AbstractITCommitMRJob extends AbstractYarnClusterITest {
String sysprops = String.format("-Xmx256m -Dlog4j.configuration=%s", String sysprops = String.format("-Xmx256m -Dlog4j.configuration=%s",
log4j); log4j);
jobConf.set(JobConf.MAPRED_MAP_TASK_JAVA_OPTS, sysprops); jobConf.set(JobConf.MAPRED_MAP_TASK_JAVA_OPTS, sysprops);
jobConf.set(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS, sysprops);
jobConf.set("yarn.app.mapreduce.am.command-opts", sysprops); jobConf.set("yarn.app.mapreduce.am.command-opts", sysprops);
} }

View File

@ -71,7 +71,7 @@ public abstract class AbstractYarnClusterITest extends AbstractCommitITest {
LoggerFactory.getLogger(AbstractYarnClusterITest.class); LoggerFactory.getLogger(AbstractYarnClusterITest.class);
private static final int TEST_FILE_COUNT = 2; private static final int TEST_FILE_COUNT = 2;
private static final int SCALE_TEST_FILE_COUNT = 20; private static final int SCALE_TEST_FILE_COUNT = 50;
public static final int SCALE_TEST_KEYS = 1000; public static final int SCALE_TEST_KEYS = 1000;
public static final int BASE_TEST_KEYS = 10; public static final int BASE_TEST_KEYS = 10;
@ -138,6 +138,12 @@ public abstract class AbstractYarnClusterITest extends AbstractCommitITest {
return new ClusterBinding(miniDFSClusterService, yarnCluster); return new ClusterBinding(miniDFSClusterService, yarnCluster);
} }
protected static void terminateCluster(ClusterBinding clusterBinding) {
if (clusterBinding != null) {
clusterBinding.terminate();
}
}
/** /**
* Get the cluster binding for this subclass * Get the cluster binding for this subclass
* @return * @return

View File

@ -21,9 +21,11 @@ package org.apache.hadoop.fs.s3a.commit;
import java.io.File; import java.io.File;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import com.amazonaws.services.s3.model.PartETag; import com.amazonaws.services.s3.model.PartETag;
import com.google.common.collect.Lists;
import org.junit.Assume; import org.junit.Assume;
import org.junit.Test; import org.junit.Test;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -36,6 +38,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.Statistic;
import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker; import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker;
import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter; import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter;
@ -268,12 +271,17 @@ public class ITestCommitOperations extends AbstractCommitITest {
public void testBaseRelativePath() throws Throwable { public void testBaseRelativePath() throws Throwable {
describe("Test creating file with a __base marker and verify that it ends" + describe("Test creating file with a __base marker and verify that it ends" +
" up in where expected"); " up in where expected");
S3AFileSystem fs = getFileSystem();
Path destDir = methodPath("testBaseRelativePath"); Path destDir = methodPath("testBaseRelativePath");
fs.delete(destDir, true);
Path pendingBaseDir = new Path(destDir, MAGIC + "/child/" + BASE); Path pendingBaseDir = new Path(destDir, MAGIC + "/child/" + BASE);
String child = "subdir/child.txt"; String child = "subdir/child.txt";
Path pendingChildPath = new Path(pendingBaseDir, child); Path pendingChildPath = new Path(pendingBaseDir, child);
Path expectedDestPath = new Path(destDir, child); Path expectedDestPath = new Path(destDir, child);
createFile(getFileSystem(), pendingChildPath, true, DATASET); assertPathDoesNotExist("dest file was found before upload",
expectedDestPath);
createFile(fs, pendingChildPath, true, DATASET);
commit("child.txt", pendingChildPath, expectedDestPath, 0, 0); commit("child.txt", pendingChildPath, expectedDestPath, 0, 0);
} }
@ -281,7 +289,9 @@ public class ITestCommitOperations extends AbstractCommitITest {
throws Exception { throws Exception {
S3AFileSystem fs = getFileSystem(); S3AFileSystem fs = getFileSystem();
Path destFile = methodPath(filename); Path destFile = methodPath(filename);
fs.delete(destFile.getParent(), true);
Path magicDest = makeMagic(destFile); Path magicDest = makeMagic(destFile);
assertPathDoesNotExist("Magic file should not exist", magicDest);
try(FSDataOutputStream stream = fs.create(magicDest, true)) { try(FSDataOutputStream stream = fs.create(magicDest, true)) {
assertTrue(stream.hasCapability(STREAM_CAPABILITY_MAGIC_OUTPUT)); assertTrue(stream.hasCapability(STREAM_CAPABILITY_MAGIC_OUTPUT));
if (data != null && data.length > 0) { if (data != null && data.length > 0) {
@ -332,13 +342,21 @@ public class ITestCommitOperations extends AbstractCommitITest {
validateIntermediateAndFinalPaths(magicFile, destFile); validateIntermediateAndFinalPaths(magicFile, destFile);
SinglePendingCommit commit = SinglePendingCommit.load(getFileSystem(), SinglePendingCommit commit = SinglePendingCommit.load(getFileSystem(),
validatePendingCommitData(filename, magicFile)); validatePendingCommitData(filename, magicFile));
CommitOperations actions = newCommitOperations();
setThrottling(throttle, failures); setThrottling(throttle, failures);
actions.commitOrFail(commit); commitOrFail(destFile, commit, newCommitOperations());
resetFailures(); resetFailures();
verifyCommitExists(commit); verifyCommitExists(commit);
} }
private void commitOrFail(final Path destFile,
final SinglePendingCommit commit, final CommitOperations actions)
throws IOException {
try (CommitOperations.CommitContext commitContext
= actions.initiateCommitOperation(destFile)) {
commitContext.commitOrFail(commit);
}
}
/** /**
* Perform any validation of paths. * Perform any validation of paths.
* @param magicFilePath path to magic file * @param magicFilePath path to magic file
@ -348,7 +366,7 @@ public class ITestCommitOperations extends AbstractCommitITest {
private void validateIntermediateAndFinalPaths(Path magicFilePath, private void validateIntermediateAndFinalPaths(Path magicFilePath,
Path destFile) Path destFile)
throws IOException { throws IOException {
assertPathDoesNotExist("dest file was created", destFile); assertPathDoesNotExist("dest file was found", destFile);
} }
/** /**
@ -439,7 +457,7 @@ public class ITestCommitOperations extends AbstractCommitITest {
resetFailures(); resetFailures();
assertPathDoesNotExist("pending commit", dest); assertPathDoesNotExist("pending commit", dest);
fullThrottle(); fullThrottle();
actions.commitOrFail(pendingCommit); commitOrFail(dest, pendingCommit, actions);
resetFailures(); resetFailures();
FileStatus status = verifyPathExists(fs, FileStatus status = verifyPathExists(fs,
"uploaded file commit", dest); "uploaded file commit", dest);
@ -454,15 +472,19 @@ public class ITestCommitOperations extends AbstractCommitITest {
CommitOperations actions = newCommitOperations(); CommitOperations actions = newCommitOperations();
Path dest = methodPath("testUploadSmallFile"); Path dest = methodPath("testUploadSmallFile");
S3AFileSystem fs = getFileSystem(); S3AFileSystem fs = getFileSystem();
fs.delete(dest, true);
fullThrottle(); fullThrottle();
assertPathDoesNotExist("test setup", dest);
SinglePendingCommit pendingCommit = SinglePendingCommit pendingCommit =
actions.uploadFileToPendingCommit(tempFile, actions.uploadFileToPendingCommit(tempFile,
dest, null, dest, null,
DEFAULT_MULTIPART_SIZE); DEFAULT_MULTIPART_SIZE);
resetFailures(); resetFailures();
LOG.debug("Precommit validation");
assertPathDoesNotExist("pending commit", dest); assertPathDoesNotExist("pending commit", dest);
fullThrottle(); fullThrottle();
actions.commitOrFail(pendingCommit); LOG.debug("Postcommit validation");
commitOrFail(dest, pendingCommit, actions);
resetFailures(); resetFailures();
String s = readUTF8(fs, dest, -1); String s = readUTF8(fs, dest, -1);
assertEquals(text, s); assertEquals(text, s);
@ -544,4 +566,97 @@ public class ITestCommitOperations extends AbstractCommitITest {
assertTrue("Empty marker file: " + status, status.getLen() > 0); assertTrue("Empty marker file: " + status, status.getLen() > 0);
} }
/**
* Creates a bulk commit and commits multiple files.
* If the DDB metastore is in use, use the instrumentation to
* verify that the write count is as expected.
* This is done without actually looking into the store -just monitoring
* changes in the filesystem's instrumentation counters.
* As changes to the store may be made during get/list calls,
* when the counters must be reset before each commit, this must be
* *after* all probes for the outcome of the previous operation.
*/
@Test
public void testBulkCommitFiles() throws Throwable {
describe("verify bulk commit including metastore update count");
File localFile = File.createTempFile("commit", ".txt");
CommitOperations actions = newCommitOperations();
Path destDir = methodPath("out");
S3AFileSystem fs = getFileSystem();
fs.delete(destDir, false);
fullThrottle();
Path destFile1 = new Path(destDir, "file1");
// this subdir will only be created in the commit of file 2
Path subdir = new Path(destDir, "subdir");
// file 2
Path destFile2 = new Path(subdir, "file2");
Path destFile3 = new Path(subdir, "file3");
List<Path> destinations = Lists.newArrayList(destFile1, destFile2,
destFile3);
List<SinglePendingCommit> commits = new ArrayList<>(3);
for (Path destination : destinations) {
SinglePendingCommit commit1 =
actions.uploadFileToPendingCommit(localFile,
destination, null,
DEFAULT_MULTIPART_SIZE);
commits.add(commit1);
}
resetFailures();
assertPathDoesNotExist("destination dir", destDir);
assertPathDoesNotExist("subdirectory", subdir);
LOG.info("Initiating commit operations");
try (CommitOperations.CommitContext commitContext
= actions.initiateCommitOperation(destDir)) {
// how many records have been written
MetricDiff writes = new MetricDiff(fs,
Statistic.S3GUARD_METADATASTORE_RECORD_WRITES);
LOG.info("Commit #1");
commitContext.commitOrFail(commits.get(0));
final String firstCommitContextString = commitContext.toString();
LOG.info("First Commit state {}", firstCommitContextString);
long writesOnFirstCommit = writes.diff();
assertPathExists("destFile1", destFile1);
assertPathExists("destination dir", destDir);
LOG.info("Commit #2");
writes.reset();
commitContext.commitOrFail(commits.get(1));
assertPathExists("subdirectory", subdir);
assertPathExists("destFile2", destFile2);
final String secondCommitContextString = commitContext.toString();
LOG.info("Second Commit state {}", secondCommitContextString);
if (writesOnFirstCommit != 0) {
LOG.info("DynamoDB Metastore is in use: checking write count");
// S3Guard is in use against DDB, so the metrics can be checked
// to see how many records were updated.
// there should only be two new entries: one for the file and
// one for the parent.
// we include the string values of the contexts because that includes
// the internals of the bulk operation state.
writes.assertDiffEquals("Number of records written after commit #2"
+ "; first commit had " + writesOnFirstCommit
+ "; first commit ancestors " + firstCommitContextString
+ "; second commit ancestors: " + secondCommitContextString,
2);
}
LOG.info("Commit #3");
writes.reset();
commitContext.commitOrFail(commits.get(2));
assertPathExists("destFile3", destFile3);
if (writesOnFirstCommit != 0) {
// this file is in the same dir as destFile2, so only its entry
// is added
writes.assertDiffEquals(
"Number of records written after third commit; "
+ "first commit had " + writesOnFirstCommit,
1);
}
}
resetFailures();
}
} }

View File

@ -18,17 +18,23 @@
package org.apache.hadoop.fs.s3a.commit.magic; package org.apache.hadoop.fs.s3a.commit.magic;
import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.commit.AbstractITCommitMRJob; import org.apache.hadoop.fs.s3a.commit.AbstractITCommitMRJob;
import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobConf;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.lsR;
import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/** /**
* Full integration test for the Magic Committer. * Full integration test for the Magic Committer.
@ -55,7 +61,7 @@ public final class ITestMagicCommitMRJob extends AbstractITCommitMRJob {
@AfterClass @AfterClass
public static void teardownClusters() throws IOException { public static void teardownClusters() throws IOException {
clusterBinding.terminate(); terminateCluster(clusterBinding);
} }
@Override @Override
@ -93,6 +99,22 @@ public final class ITestMagicCommitMRJob extends AbstractITCommitMRJob {
@Override @Override
protected void customPostExecutionValidation(Path destPath, protected void customPostExecutionValidation(Path destPath,
SuccessData successData) throws Exception { SuccessData successData) throws Exception {
assertPathDoesNotExist("No cleanup", new Path(destPath, MAGIC)); Path magicDir = new Path(destPath, MAGIC);
// if an FNFE isn't raised on getFileStatus, list out the directory
// tree
S3AFileSystem fs = getFileSystem();
// log the contents
lsR(fs, destPath, true);
intercept(FileNotFoundException.class, () -> {
final FileStatus st = fs.getFileStatus(magicDir);
StringBuilder result = new StringBuilder("Found magic dir which should"
+ " have been deleted at ").append(st).append('\n');
result.append("[");
applyLocatedFiles(fs.listFiles(magicDir, true),
(status) -> result.append(status.getPath()).append('\n'));
result.append("[");
return result.toString();
});
} }
} }

View File

@ -143,9 +143,12 @@ public class ITestS3AHugeMagicCommits extends AbstractSTestS3AHugeFiles {
assertNotNull("jobDir", jobDir); assertNotNull("jobDir", jobDir);
Pair<PendingSet, List<Pair<LocatedFileStatus, IOException>>> Pair<PendingSet, List<Pair<LocatedFileStatus, IOException>>>
results = operations.loadSinglePendingCommits(jobDir, false); results = operations.loadSinglePendingCommits(jobDir, false);
for (SinglePendingCommit singlePendingCommit : try(CommitOperations.CommitContext commitContext
results.getKey().getCommits()) { = operations.initiateCommitOperation(jobDir)) {
operations.commitOrFail(singlePendingCommit); for (SinglePendingCommit singlePendingCommit :
results.getKey().getCommits()) {
commitContext.commitOrFail(singlePendingCommit);
}
} }
timer.end("time to commit %s", pendingDataFile); timer.end("time to commit %s", pendingDataFile);
// upload is no longer pending // upload is no longer pending

View File

@ -46,7 +46,7 @@ public final class ITestDirectoryCommitMRJob extends AbstractITCommitMRJob {
@AfterClass @AfterClass
public static void teardownClusters() throws IOException { public static void teardownClusters() throws IOException {
clusterBinding.terminate(); terminateCluster(clusterBinding);
} }
@Override @Override

View File

@ -47,7 +47,7 @@ public final class ITestPartitionCommitMRJob extends AbstractITCommitMRJob {
@AfterClass @AfterClass
public static void teardownClusters() throws IOException { public static void teardownClusters() throws IOException {
clusterBinding.terminate(); terminateCluster(clusterBinding);
} }
@Override @Override

View File

@ -58,7 +58,7 @@ public final class ITestStagingCommitMRJob extends AbstractITCommitMRJob {
@AfterClass @AfterClass
public static void teardownClusters() throws IOException { public static void teardownClusters() throws IOException {
clusterBinding.terminate(); terminateCluster(clusterBinding);
} }
@Override @Override

View File

@ -53,7 +53,7 @@ public final class ITestStagingCommitMRJobBadDest extends AbstractITCommitMRJob
@AfterClass @AfterClass
public static void teardownClusters() throws IOException { public static void teardownClusters() throws IOException {
clusterBinding.terminate(); terminateCluster(clusterBinding);
} }
@Override @Override

View File

@ -238,4 +238,9 @@ public abstract class AbstractCommitTerasortIT extends
public void test_150_teracleanup() throws Throwable { public void test_150_teracleanup() throws Throwable {
terasortDuration = Optional.empty(); terasortDuration = Optional.empty();
} }
@Test
public void test_200_directory_deletion() throws Throwable {
getFileSystem().delete(terasortPath, true);
}
} }

View File

@ -0,0 +1,871 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.impl;
import java.io.IOException;
import java.nio.file.AccessDeniedException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
import com.google.common.base.Charsets;
import com.google.common.util.concurrent.ListeningExecutorService;
import org.assertj.core.api.Assertions;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
import org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation;
import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
import org.apache.hadoop.util.DurationInfo;
import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles;
import static org.apache.hadoop.fs.s3a.Statistic.FILES_DELETE_REJECTED;
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUESTS;
import static org.apache.hadoop.fs.s3a.auth.RoleModel.Effects;
import static org.apache.hadoop.fs.s3a.auth.RoleModel.Statement;
import static org.apache.hadoop.fs.s3a.auth.RoleModel.directory;
import static org.apache.hadoop.fs.s3a.auth.RoleModel.statement;
import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*;
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.bindRolePolicyStatements;
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden;
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig;
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion;
import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.extractUndeletedPaths;
import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.removeUndeletedPaths;
import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.assertFileCount;
import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.extractCause;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
import static org.apache.hadoop.test.LambdaTestUtils.eval;
/**
* Test partial failures of delete and rename operations, especially
* that the S3Guard tables are consistent with the state of
* the filesystem.
*
* All these test have a unique path for each run, with a roleFS having
* full RW access to part of it, and R/O access to a restricted subdirectory
*
* <ol>
* <li>
* The tests are parameterized to single/multi delete, which control which
* of the two delete mechanisms are used.
* </li>
* <li>
* In multi delete, in a scale test run, a significantly larger set of files
* is created and then deleted.
* </li>
* <li>
* This isn't done in the single delete as it is much slower and it is not
* the situation we are trying to create.
* </li>
* </ol>
*
* This test manages to create lots of load on the s3guard prune command
* when that is tested in a separate test suite;
* too many tombstone files for the test to complete.
* An attempt is made in {@link #deleteTestDirInTeardown()} to prune these test
* files.
*/
@SuppressWarnings("ThrowableNotThrown")
@RunWith(Parameterized.class)
public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
private static final Logger LOG =
LoggerFactory.getLogger(ITestPartialRenamesDeletes.class);
private static final Path ROOT = new Path("/");
private static final Statement STATEMENT_ALL_BUCKET_READ_ACCESS
= statement(true, S3_ALL_BUCKETS, S3_BUCKET_READ_OPERATIONS);
/** Many threads for scale performance: {@value}. */
public static final int EXECUTOR_THREAD_COUNT = 64;
/**
* For submitting work.
*/
private static final ListeningExecutorService EXECUTOR =
BlockingThreadPoolExecutorService.newInstance(
EXECUTOR_THREAD_COUNT,
EXECUTOR_THREAD_COUNT * 2,
30, TimeUnit.SECONDS,
"test-operations");
/**
* The number of files in a non-scaled test.
* <p>
* Value: {@value}.
*/
public static final int FILE_COUNT_NON_SCALED = 2;
/**
* The number of files for a scaled test. This is still
* less than half the amount which can be fitted into a delete
* request, so that even with this many R/W and R/O files,
* both can fit in the same request.
* Then, when a partial delete occurs, we can make assertions
* knowing that all R/W files should have been deleted and all
* R/O files rejected.
* <p>
* Value: {@value}.
*/
public static final int FILE_COUNT_SCALED = 10;
public static final int DIR_COUNT = 2;
public static final int DIR_COUNT_SCALED = 4;
public static final int DEPTH = 2;
public static final int DEPTH_SCALED = 2;
/**
* A role FS; if non-null it is closed in teardown.
*/
private S3AFileSystem roleFS;
/**
* Base path for this test run.
* This is generated uniquely for each test.
*/
private Path basePath;
/**
* A directory which restricted roles have full write access to.
*/
private Path writableDir;
/**
* A directory to which restricted roles have only read access.
*/
private Path readOnlyDir;
/**
* A file under {@link #readOnlyDir} which cannot be written or deleted.
*/
private Path readOnlyChild;
/**
* A directory to which restricted roles have no read access.
*/
private Path noReadDir;
/** delete policy: single or multi? */
private final boolean multiDelete;
/**
* Configuration for the assume role FS.
*/
private Configuration assumedRoleConfig;
private int fileCount;
private int dirCount;
private int dirDepth;
/**
* Was the -Dscale switch passed in to the test run?
*/
private boolean scaleTest;
/**
* Test array for parameterized test runs.
* <ul>
* <li>Run 0: single deletes</li>
* <li>Run 1: multi deletes</li>
* </ul>
*
* @return a list of parameter tuples.
*/
@Parameterized.Parameters(name = "bulk-delete={0}")
public static Collection<Object[]> params() {
return Arrays.asList(new Object[][]{
{false},
{true},
});
}
/**
* Constructor.
* @param multiDelete single vs multi delete in the role FS?
*/
public ITestPartialRenamesDeletes(final boolean multiDelete) {
this.multiDelete = multiDelete;
}
/**
* This sets up a unique path for every test run, so as to guarantee isolation
* from previous runs.
* It creates a role policy which has read access to everything except
* the contents of {@link #noReadDir}, and with write access to
* {@link #writableDir}.
* @throws Exception failure
*/
@Override
public void setup() throws Exception {
super.setup();
assumeRoleTests();
basePath = uniquePath();
readOnlyDir = new Path(basePath, "readonlyDir");
writableDir = new Path(basePath, "writableDir");
readOnlyChild = new Path(readOnlyDir, "child");
noReadDir = new Path(basePath, "noReadDir");
// the full FS
S3AFileSystem fs = getFileSystem();
fs.delete(basePath, true);
fs.mkdirs(writableDir);
// create the baseline assumed role
assumedRoleConfig = createAssumedRoleConfig();
bindRolePolicyStatements(assumedRoleConfig,
STATEMENT_S3GUARD_CLIENT,
STATEMENT_ALL_BUCKET_READ_ACCESS, // root: r-x
new Statement(Effects.Allow) // dest: rwx
.addActions(S3_PATH_RW_OPERATIONS)
.addResources(directory(writableDir)),
new Statement(Effects.Deny) // noReadDir: --x
.addActions(S3_ALL_GET)
.addActions(S3_ALL_PUT)
.addActions(S3_ALL_DELETE)
.addResources(directory(noReadDir)));
// the role configured to that set of restrictions
roleFS = (S3AFileSystem) readOnlyDir.getFileSystem(assumedRoleConfig);
// switch to the big set of files iff this is a multidelete run
// with -Dscale set.
// without that the DELETE calls become a key part of the bottleneck
scaleTest = multiDelete && getTestPropertyBool(
getConfiguration(),
KEY_SCALE_TESTS_ENABLED,
DEFAULT_SCALE_TESTS_ENABLED);
fileCount = scaleTest ? FILE_COUNT_SCALED : FILE_COUNT_NON_SCALED;
dirCount = scaleTest ? DIR_COUNT_SCALED : DIR_COUNT;
dirDepth = scaleTest ? DEPTH_SCALED : DEPTH;
}
@Override
public void teardown() throws Exception {
cleanupWithLogger(LOG, roleFS);
super.teardown();
}
/**
* Directory cleanup includes pruning everything under the path.
* This ensures that any in the tree from failed tests don't fill up
* the store with many, many, deleted entries.
* @throws IOException failure.
*/
@Override
protected void deleteTestDirInTeardown() throws IOException {
super.deleteTestDirInTeardown();
Path path = getContract().getTestPath();
try {
prune(path);
} catch (IOException e) {
LOG.warn("When pruning the test directory {}", path, e);
}
}
private void assumeRoleTests() {
assume("No ARN for role tests", !getAssumedRoleARN().isEmpty());
}
private String getAssumedRoleARN() {
return getContract().getConf().getTrimmed(ASSUMED_ROLE_ARN, "");
}
/**
* Create the assumed role configuration.
* @return a config bonded to the ARN of the assumed role
*/
public Configuration createAssumedRoleConfig() {
return createAssumedRoleConfig(getAssumedRoleARN());
}
/**
* Create a config for an assumed role; it also disables FS caching
* and sets the multi delete option to that of the current mode.
* @param roleARN ARN of role
* @return the new configuration
*/
private Configuration createAssumedRoleConfig(String roleARN) {
Configuration conf = newAssumedRoleConfig(getContract().getConf(),
roleARN);
String bucketName = getTestBucketName(conf);
removeBucketOverrides(bucketName, conf, ENABLE_MULTI_DELETE);
conf.setBoolean(ENABLE_MULTI_DELETE, multiDelete);
return conf;
}
@Override
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
String bucketName = getTestBucketName(conf);
// ramp up the number of connections we can have for maximum PUT
// performance
removeBucketOverrides(bucketName, conf,
MAX_THREADS,
MAXIMUM_CONNECTIONS,
S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY);
conf.setInt(MAX_THREADS, EXECUTOR_THREAD_COUNT);
conf.setInt(MAXIMUM_CONNECTIONS, EXECUTOR_THREAD_COUNT * 2);
// turn off prune delays, so as to stop scale tests creating
// so much cruft that future CLI prune commands take forever
conf.setInt(S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY, 0);
return conf;
}
/**
* Create a unique path, which includes method name,
* multidelete flag and a random UUID.
* @return a string to use for paths.
* @throws IOException path creation failure.
*/
private Path uniquePath() throws IOException {
return path(
String.format("%s-%s-%04d",
getMethodName(),
multiDelete ? "multi" : "single",
System.currentTimeMillis() % 10000));
}
/**
* This is here to verify role and path setup.
*/
@Test
public void testCannotTouchUnderRODir() throws Throwable {
forbidden("touching the empty child " + readOnlyChild,
"",
() -> {
touch(roleFS, readOnlyChild);
return readOnlyChild;
});
}
@Test
public void testCannotReadUnderNoReadDir() throws Throwable {
Path path = new Path(noReadDir, "unreadable.txt");
createFile(getFileSystem(), path, true, "readonly".getBytes());
forbidden("trying to read " + path,
"",
() -> readUTF8(roleFS, path, -1));
}
@Test
public void testMultiDeleteOptionPropagated() throws Throwable {
describe("Verify the test parameter propagates to the store context");
StoreContext ctx = roleFS.createStoreContext();
Assertions.assertThat(ctx.isMultiObjectDeleteEnabled())
.as(ctx.toString())
.isEqualTo(multiDelete);
}
/**
* Execute a sequence of rename operations with access locked down.
*/
@Test
public void testRenameParentPathNotWriteable() throws Throwable {
describe("rename with parent paths not writeable; multi=%s", multiDelete);
final Configuration conf = createAssumedRoleConfig();
bindRolePolicyStatements(conf,
STATEMENT_S3GUARD_CLIENT,
STATEMENT_ALLOW_SSE_KMS_RW,
STATEMENT_ALL_BUCKET_READ_ACCESS,
new Statement(Effects.Allow)
.addActions(S3_PATH_RW_OPERATIONS)
.addResources(directory(readOnlyDir))
.addResources(directory(writableDir)));
roleFS = (S3AFileSystem) readOnlyDir.getFileSystem(conf);
S3AFileSystem fs = getFileSystem();
roleFS.getFileStatus(ROOT);
fs.mkdirs(readOnlyDir);
// you can create an adjacent child
touch(fs, readOnlyChild);
fs.delete(writableDir, true);
// as dest doesn't exist, this will map child -> dest
assertRenameOutcome(roleFS, readOnlyChild, writableDir, true);
assertIsFile(writableDir);
assertIsDirectory(readOnlyDir);
Path renamedDestPath = new Path(readOnlyDir, writableDir.getName());
assertRenameOutcome(roleFS, writableDir, readOnlyDir, true);
assertIsFile(renamedDestPath);
roleFS.delete(readOnlyDir, true);
roleFS.delete(writableDir, true);
}
@Test
public void testRenameSingleFileFailsInDelete() throws Throwable {
describe("rename with source read only; multi=%s", multiDelete);
Path readOnlyFile = readOnlyChild;
// the full FS
S3AFileSystem fs = getFileSystem();
fs.delete(basePath, true);
// this file is readable by the roleFS, but cannot be deleted
touch(fs, readOnlyFile);
roleFS.delete(writableDir, true);
roleFS.mkdirs(writableDir);
// rename will fail in the delete phase
expectRenameForbidden(readOnlyFile, writableDir);
// and the source file is still there
assertIsFile(readOnlyFile);
// and so is the copied version, because there's no attempt
// at rollback, or preflight checking on the delete permissions
Path renamedFile = new Path(writableDir, readOnlyFile.getName());
assertIsFile(renamedFile);
ContractTestUtils.assertDeleted(roleFS, renamedFile, true);
assertFileCount("Empty Dest Dir", roleFS,
writableDir, 0);
}
/**
* Execute a sequence of rename operations where the source
* data is read only to the client calling rename().
* This will cause the inner delete() operations to fail, whose outcomes
* are explored.
* Multiple files are created (in parallel) for some renames, so the test
* explores the outcome on bulk delete calls, including verifying that a
* MultiObjectDeleteException is translated to an AccessDeniedException.
* <ol>
* <li>The exception raised is AccessDeniedException,
* from single and multi DELETE calls.</li>
* <li>It happens after the COPY. Not ideal, but, well, we can't pretend
* it's a filesystem forever.</li>
* </ol>
*/
@Test
public void testRenameDirFailsInDelete() throws Throwable {
describe("rename with source read only; multi=%s", multiDelete);
// the full FS
S3AFileSystem fs = getFileSystem();
roleFS.mkdirs(writableDir);
// create a set of files
// this is done in parallel as it is 10x faster on a long-haul test run.
List<Path> createdFiles = createFiles(fs, readOnlyDir, dirDepth, fileCount,
dirCount);
// are they all there?
int expectedFileCount = createdFiles.size();
assertFileCount("files ready to rename", roleFS,
readOnlyDir, expectedFileCount);
// try to rename the directory
LOG.info("Renaming readonly files {} to {}", readOnlyDir, writableDir);
AccessDeniedException deniedException = expectRenameForbidden(readOnlyDir,
writableDir);
if (multiDelete) {
// look in that exception for a multidelete
MultiObjectDeleteException mde = extractCause(
MultiObjectDeleteException.class, deniedException);
final List<Path> undeleted
= extractUndeletedPaths(mde, fs::keyToQualifiedPath);
Assertions.assertThat(undeleted)
.as("files which could not be deleted")
.hasSize(expectedFileCount)
.containsAll(createdFiles)
.containsExactlyInAnyOrderElementsOf(createdFiles);
}
LOG.info("Result of renaming read-only files is as expected",
deniedException);
assertFileCount("files in the source directory", roleFS,
readOnlyDir, expectedFileCount);
// now lets look at the destination.
// even with S3Guard on, we expect the destination to match that of our
// the remote state.
// the test will exist
describe("Verify destination directory exists");
FileStatus st = roleFS.getFileStatus(writableDir);
assertTrue("Not a directory: " + st,
st.isDirectory());
assertFileCount("files in the dest directory", roleFS,
writableDir, expectedFileCount);
}
@Test
public void testRenameFileFailsNoWrite() throws Throwable {
describe("Try to rename to a write-only destination fails with src"
+ " & dest unchanged.");
roleFS.mkdirs(writableDir);
S3AFileSystem fs = getFileSystem();
Path source = new Path(writableDir, "source");
touch(fs, source);
fs.mkdirs(readOnlyDir);
Path dest = new Path(readOnlyDir, "dest");
describe("Renaming files {} to {}", writableDir, dest);
// rename fails but doesn't raise an exception. Good or bad?
expectRenameForbidden(source, dest);
assertIsFile(source);
assertPathDoesNotExist("rename destination", dest);
}
@Test
public void testCopyDirFailsToReadOnlyDir() throws Throwable {
describe("Try to copy to a read-only destination");
roleFS.mkdirs(writableDir);
S3AFileSystem fs = getFileSystem();
List<Path> files = createFiles(fs, writableDir, dirDepth, fileCount,
dirCount);
fs.mkdirs(readOnlyDir);
Path dest = new Path(readOnlyDir, "dest");
expectRenameForbidden(writableDir, dest);
assertPathDoesNotExist("rename destination", dest);
assertFileCount("files in the source directory", roleFS,
writableDir, files.size());
}
@Test
public void testCopyFileFailsOnSourceRead() throws Throwable {
describe("The source file isn't readable, so the COPY fails");
Path source = new Path(noReadDir, "source");
S3AFileSystem fs = getFileSystem();
touch(fs, source);
fs.mkdirs(writableDir);
Path dest = new Path(writableDir, "dest");
expectRenameForbidden(source, dest);
assertIsFile(source);
assertPathDoesNotExist("rename destination", dest);
}
@Test
public void testCopyDirFailsOnSourceRead() throws Throwable {
describe("The source file isn't readable, so the COPY fails");
S3AFileSystem fs = getFileSystem();
List<Path> files = createFiles(fs, noReadDir, dirDepth, fileCount,
dirCount);
fs.mkdirs(writableDir);
Path dest = new Path(writableDir, "dest");
expectRenameForbidden(noReadDir, dest);
assertFileCount("files in the source directory", fs,
noReadDir, files.size());
}
/**
* Have a directory with full R/W permissions, but then remove
* write access underneath, and try to delete it.
* This verifies that failures in the delete fake dir stage.
* are not visible.
*/
@Test
public void testPartialEmptyDirDelete() throws Throwable {
describe("delete an empty directory with parent dir r/o"
+ " multidelete=%s", multiDelete);
// the full FS
final Path deletableChild = new Path(writableDir, "deletableChild");
// deletable child is created.
roleFS.mkdirs(deletableChild);
assertPathExists("parent dir after create", writableDir);
assertPathExists("grandparent dir after create", writableDir.getParent());
// now delete it.
roleFS.delete(deletableChild, true);
assertPathExists("parent dir after deletion", writableDir);
assertPathExists("grandparent dir after deletion", writableDir.getParent());
assertPathDoesNotExist("deletable dir after deletion", deletableChild);
}
/**
* Have a directory with full R/W permissions, but then remove
* write access underneath, and try to delete it.
*/
@Test
public void testPartialDirDelete() throws Throwable {
describe("delete with part of the child tree read only;"
+ " multidelete=%s", multiDelete);
// the full FS
S3AFileSystem fs = getFileSystem();
List<Path> readOnlyFiles = createFiles(fs, readOnlyDir,
dirDepth, fileCount, dirCount);
List<Path> deletableFiles = createFiles(fs,
writableDir, dirDepth, fileCount, dirCount);
// as a safety check, verify that one of the deletable files can be deleted
Path head = deletableFiles.remove(0);
assertTrue("delete " + head + " failed",
roleFS.delete(head, false));
List<Path> allFiles = Stream.concat(
readOnlyFiles.stream(),
deletableFiles.stream())
.collect(Collectors.toList());
// this set can be deleted by the role FS
MetricDiff rejectionCount = new MetricDiff(roleFS, FILES_DELETE_REJECTED);
MetricDiff deleteVerbCount = new MetricDiff(roleFS, OBJECT_DELETE_REQUESTS);
describe("Trying to delete read only directory");
AccessDeniedException ex = expectDeleteForbidden(readOnlyDir);
if (multiDelete) {
// multi-delete status checks
extractCause(MultiObjectDeleteException.class, ex);
rejectionCount.assertDiffEquals("Wrong rejection count",
readOnlyFiles.size());
deleteVerbCount.assertDiffEquals("Wrong delete count", 1);
reset(rejectionCount, deleteVerbCount);
}
// all the files are still there? (avoid in scale test due to cost)
if (!scaleTest) {
readOnlyFiles.forEach(this::pathMustExist);
}
describe("Trying to delete upper-level directory");
ex = expectDeleteForbidden(basePath);
if (multiDelete) {
// multi-delete status checks
extractCause(MultiObjectDeleteException.class, ex);
deleteVerbCount.assertDiffEquals("Wrong delete count", 1);
MultiObjectDeleteException mde = extractCause(
MultiObjectDeleteException.class, ex);
final List<Path> undeleted
= removeUndeletedPaths(mde, allFiles, fs::keyToQualifiedPath);
Assertions.assertThat(undeleted)
.as("files which could not be deleted")
.containsExactlyInAnyOrderElementsOf(readOnlyFiles);
Assertions.assertThat(allFiles)
.as("files which were deleted")
.containsExactlyInAnyOrderElementsOf(deletableFiles);
rejectionCount.assertDiffEquals("Wrong rejection count",
readOnlyFiles.size());
}
reset(rejectionCount, deleteVerbCount);
// build the set of all paths under the directory tree through
// a directory listing (i.e. not getFileStatus()).
// small risk of observed inconsistency here on unguarded stores.
final Set<Path> readOnlyListing = listFilesUnderPath(readOnlyDir, true);
String directoryList = readOnlyListing.stream()
.map(Path::toString)
.collect(Collectors.joining(", ", "[", "]"));
Assertions.assertThat(readOnlyListing)
.as("ReadOnly directory " + directoryList)
.containsAll(readOnlyFiles);
// do this prune in the test as well as teardown, so that the test
// reporting includes it in the runtime of a successful run.
prune(basePath);
}
/**
* Expect the delete() call to fail.
* @param path path to delete.
* @return the expected exception.
* @throws Exception any other failure.
*/
private AccessDeniedException expectDeleteForbidden(Path path)
throws Exception {
try (DurationInfo ignored =
new DurationInfo(LOG, true, "delete %s", path)) {
return forbidden("Expected an error deleting " + path,
"",
() -> {
boolean r = roleFS.delete(path, true);
return " delete=" + r + " " + ls(path.getParent());
});
}
}
/**
* Expect that a rename will fail with an exception using the roleFS.
* @param src source
* @param dest dest
* @return the exception raised.
* @throws Exception any other failure
*/
private AccessDeniedException expectRenameForbidden(Path src, Path dest)
throws Exception {
try (DurationInfo ignored =
new DurationInfo(LOG, true,
"rename(%s, %s)", src, dest)) {
return forbidden(
"Renaming " + src + " to " + dest,
"",
() -> {
boolean result = roleFS.rename(src, dest);
LOG.error("Rename should have been forbidden but returned {}",
result);
LOG.error("Source directory:\n{}",
ContractTestUtils.ls(getFileSystem(), src.getParent()));
LOG.error("Destination directory:\n{}",
ContractTestUtils.ls(getFileSystem(), src.getParent()));
return "Rename unexpectedly returned " + result;
});
}
}
/**
* Assert that a path must exist, map IOEs to RTEs for loops.
* @param p path.
*/
private void pathMustExist(Path p) {
eval(() -> assertPathExists("Missing path", p));
}
/**
* Prune the store for everything under the test path.
* @param path path.
* @throws IOException on failure.
*/
private void prune(Path path) throws IOException {
S3AFileSystem fs = getFileSystem();
if (fs.hasMetadataStore()) {
MetadataStore store = fs.getMetadataStore();
try (DurationInfo ignored =
new DurationInfo(LOG, true, "prune %s", path)) {
store.prune(
MetadataStore.PruneMode.ALL_BY_MODTIME,
System.currentTimeMillis(),
PathMetadataDynamoDBTranslation.pathToParentKey(fs.qualify(path)));
}
}
}
/**
* List all files under a path.
* @param path path to list
* @param recursive recursive listing?
* @return an unordered set of the paths.
* @throws IOException failure
*/
private Set<Path> listFilesUnderPath(Path path, boolean recursive)
throws IOException {
Set<Path> files = new TreeSet<>();
try (DurationInfo ignore =
new DurationInfo(LOG, "ls -R %s", path)) {
applyLocatedFiles(getFileSystem().listFiles(path, recursive),
(status) -> files.add(status.getPath()));
}
return files;
}
/**
* Write the text to a file asynchronously. Logs the operation duration.
* @param fs filesystem
* @param path path
* @return future to the patch created.
*/
private static CompletableFuture<Path> put(FileSystem fs,
Path path, String text) {
return submit(EXECUTOR, () -> {
try (DurationInfo ignore =
new DurationInfo(LOG, false, "Creating %s", path)) {
createFile(fs, path, true, text.getBytes(Charsets.UTF_8));
return path;
}
});
}
/**
* Parallel-touch a set of files in the destination directory.
* @param fs filesystem
* @param destDir destination
* @param depth file depth
* @param fileCount number of files to create.
* @param dirCount number of dirs to create at each level
* @return the list of files created.
*/
public static List<Path> createFiles(final FileSystem fs,
final Path destDir,
final int depth,
final int fileCount,
final int dirCount) throws IOException {
List<CompletableFuture<Path>> futures = new ArrayList<>(fileCount);
List<Path> paths = new ArrayList<>(fileCount);
List<Path> dirs = new ArrayList<>(fileCount);
buildPaths(paths, dirs, destDir, depth, fileCount, dirCount);
try (DurationInfo ignore =
new DurationInfo(LOG, "Creating %d files", fileCount)) {
for (Path path : paths) {
futures.add(put(fs, path, path.getName()));
}
waitForCompletion(futures);
return paths;
}
}
/**
* Recursive method to build up lists of files and directories.
* @param filePaths list of file paths to add entries to.
* @param dirPaths list of directory paths to add entries to.
* @param destDir destination directory.
* @param depth depth of directories
* @param fileCount number of files.
* @param dirCount number of directories.
*/
private static void buildPaths(
final List<Path> filePaths,
final List<Path> dirPaths,
final Path destDir,
final int depth,
final int fileCount,
final int dirCount) {
if (depth<=0) {
return;
}
// create the file paths
for (int i = 0; i < fileCount; i++) {
String name = "file-" + i;
Path p = new Path(destDir, name);
filePaths.add(p);
}
for (int i = 0; i < dirCount; i++) {
String name = "dir-" + i;
Path p = new Path(destDir, name);
dirPaths.add(p);
buildPaths(filePaths, dirPaths, p, depth - 1, fileCount, dirCount);
}
}
}

View File

@ -0,0 +1,393 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.impl;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
import com.google.common.collect.Lists;
import org.assertj.core.api.Assertions;
import org.junit.Before;
import org.junit.Test;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.commons.lang3.tuple.Triple;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.Constants;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3AInputPolicy;
import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
import org.apache.hadoop.fs.s3a.s3guard.RenameTracker;
import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
import org.apache.hadoop.io.retry.RetryPolicies;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.ACCESS_DENIED;
import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.removeUndeletedPaths;
import static org.junit.Assert.assertEquals;
/**
* Unit test suite covering translation of AWS SDK exceptions to S3A exceptions,
* and retry/recovery policies.
*/
public class TestPartialDeleteFailures {
private static final ContextAccessors CONTEXT_ACCESSORS
= new MinimalContextAccessor();
private StoreContext context;
private static Path qualifyKey(String k) {
return new Path("s3a://bucket/" + k);
}
@Before
public void setUp() throws Exception {
context = createMockStoreContext(true,
new OperationTrackingStore());
}
@Test
public void testDeleteExtraction() {
List<Path> src = pathList("a", "a/b", "a/c");
List<Path> rejected = pathList("a/b");
MultiObjectDeleteException ex = createDeleteException(ACCESS_DENIED,
rejected);
List<Path> undeleted = removeUndeletedPaths(ex, src,
TestPartialDeleteFailures::qualifyKey);
assertEquals("mismatch of rejected and undeleted entries",
rejected, undeleted);
}
@Test
public void testSplitKeysFromResults() throws Throwable {
List<Path> src = pathList("a", "a/b", "a/c");
List<Path> rejected = pathList("a/b");
List<DeleteObjectsRequest.KeyVersion> keys = keysToDelete(src);
MultiObjectDeleteException ex = createDeleteException(ACCESS_DENIED,
rejected);
Pair<List<Path>, List<Path>> pair =
new MultiObjectDeleteSupport(context)
.splitUndeletedKeys(ex, keys);
List<Path> undeleted = pair.getLeft();
List<Path> deleted = pair.getRight();
assertEquals(rejected, undeleted);
// now check the deleted list to verify that it is valid
src.remove(rejected.get(0));
assertEquals(src, deleted);
}
/**
* Build a list of qualified paths from vararg parameters.
* @param paths paths to qualify and then convert to a lst.
* @return same paths as a list.
*/
private List<Path> pathList(String... paths) {
return Arrays.stream(paths)
.map(TestPartialDeleteFailures::qualifyKey)
.collect(Collectors.toList());
}
/**
* Build a delete exception containing all the rejected paths.
* The list of successful entries is empty.
* @param rejected the rejected paths.
* @return a new exception
*/
private MultiObjectDeleteException createDeleteException(
final String code,
final List<Path> rejected) {
List<MultiObjectDeleteException.DeleteError> errors = rejected.stream()
.map((p) -> {
MultiObjectDeleteException.DeleteError e
= new MultiObjectDeleteException.DeleteError();
e.setKey(p.toUri().getPath());
e.setCode(code);
e.setMessage("forbidden");
return e;
}).collect(Collectors.toList());
return new MultiObjectDeleteException(errors, Collections.emptyList());
}
/**
* From a list of paths, build up the list of keys for a delete request.
* @param paths path list
* @return a key list suitable for a delete request.
*/
public static List<DeleteObjectsRequest.KeyVersion> keysToDelete(
List<Path> paths) {
return paths.stream()
.map((p) -> p.toUri().getPath())
.map(DeleteObjectsRequest.KeyVersion::new)
.collect(Collectors.toList());
}
/**
* Verify that on a partial delete, the S3Guard tables are updated
* with deleted items. And only them.
*/
@Test
public void testProcessDeleteFailure() throws Throwable {
Path pathA = qualifyKey("/a");
Path pathAB = qualifyKey("/a/b");
Path pathAC = qualifyKey("/a/c");
List<Path> src = Lists.newArrayList(pathA, pathAB, pathAC);
List<DeleteObjectsRequest.KeyVersion> keyList = keysToDelete(src);
List<Path> deleteForbidden = Lists.newArrayList(pathAB);
final List<Path> deleteAllowed = Lists.newArrayList(pathA, pathAC);
MultiObjectDeleteException ex = createDeleteException(ACCESS_DENIED,
deleteForbidden);
OperationTrackingStore store
= new OperationTrackingStore();
StoreContext storeContext = createMockStoreContext(true, store);
MultiObjectDeleteSupport deleteSupport
= new MultiObjectDeleteSupport(storeContext);
Triple<List<Path>, List<Path>, List<Pair<Path, IOException>>>
triple = deleteSupport.processDeleteFailure(ex, keyList);
Assertions.assertThat(triple.getRight())
.as("failure list")
.isEmpty();
List<Path> undeleted = triple.getLeft();
List<Path> deleted = triple.getMiddle();
Assertions.assertThat(deleted).
as("deleted files")
.containsAll(deleteAllowed)
.doesNotContainAnyElementsOf(deleteForbidden);
Assertions.assertThat(undeleted).
as("undeleted store entries")
.containsAll(deleteForbidden)
.doesNotContainAnyElementsOf(deleteAllowed);
}
private StoreContext createMockStoreContext(boolean multiDelete,
OperationTrackingStore store) throws URISyntaxException, IOException {
URI name = new URI("s3a://bucket");
Configuration conf = new Configuration();
return new StoreContext(
name,
"bucket",
conf,
"alice",
UserGroupInformation.getCurrentUser(),
BlockingThreadPoolExecutorService.newInstance(
4,
4,
10, TimeUnit.SECONDS,
"s3a-transfer-shared"),
Constants.DEFAULT_EXECUTOR_CAPACITY,
new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL, Invoker.LOG_EVENT),
new S3AInstrumentation(name),
new S3AStorageStatistics(),
S3AInputPolicy.Normal,
ChangeDetectionPolicy.createPolicy(ChangeDetectionPolicy.Mode.None,
ChangeDetectionPolicy.Source.ETag, false),
multiDelete,
store,
false,
CONTEXT_ACCESSORS,
new S3Guard.TtlTimeProvider(conf));
}
private static class MinimalContextAccessor implements ContextAccessors {
@Override
public Path keyToPath(final String key) {
return qualifyKey(key);
}
@Override
public String pathToKey(final Path path) {
return null;
}
@Override
public File createTempFile(final String prefix, final long size)
throws IOException {
throw new UnsupportedOperationException("unsppported");
}
@Override
public String getBucketLocation() throws IOException {
return null;
}
}
/**
* MetadataStore which tracks what is deleted and added.
*/
private static class OperationTrackingStore implements MetadataStore {
private final List<Path> deleted = new ArrayList<>();
private final List<Path> created = new ArrayList<>();
@Override
public void initialize(final FileSystem fs) {
}
@Override
public void initialize(final Configuration conf) {
}
@Override
public void forgetMetadata(final Path path) {
}
@Override
public PathMetadata get(final Path path) {
return null;
}
@Override
public PathMetadata get(final Path path,
final boolean wantEmptyDirectoryFlag) {
return null;
}
@Override
public DirListingMetadata listChildren(final Path path) {
return null;
}
@Override
public void put(final PathMetadata meta) {
put(meta, null);
}
@Override
public void put(final PathMetadata meta,
final BulkOperationState operationState) {
created.add(meta.getFileStatus().getPath());
}
@Override
public void put(final Collection<? extends PathMetadata> metas,
final BulkOperationState operationState) {
metas.stream().forEach(meta -> put(meta, null));
}
@Override
public void put(final DirListingMetadata meta,
final BulkOperationState operationState) {
created.add(meta.getPath());
}
@Override
public void destroy() {
}
@Override
public void delete(final Path path,
final ITtlTimeProvider ttlTimeProvider) {
deleted.add(path);
}
@Override
public void deleteSubtree(final Path path,
final ITtlTimeProvider ttlTimeProvider) {
}
@Override
public void move(@Nullable final Collection<Path> pathsToDelete,
@Nullable final Collection<PathMetadata> pathsToCreate,
final ITtlTimeProvider ttlTimeProvider,
@Nullable final BulkOperationState operationState) {
}
@Override
public void prune(final PruneMode pruneMode, final long cutoff) {
}
@Override
public void prune(final PruneMode pruneMode,
final long cutoff,
final String keyPrefix) {
}
@Override
public BulkOperationState initiateBulkWrite(
final BulkOperationState.OperationType operation,
final Path dest) {
return null;
}
@Override
public Map<String, String> getDiagnostics() {
return null;
}
@Override
public void updateParameters(final Map<String, String> parameters) {
}
@Override
public void close() {
}
public List<Path> getDeleted() {
return deleted;
}
public List<Path> getCreated() {
return created;
}
@Override
public RenameTracker initiateRenameOperation(
final StoreContext storeContext,
final Path source,
final S3AFileStatus sourceStatus,
final Path dest) {
throw new UnsupportedOperationException("unsupported");
}
@Override
public void addAncestors(final Path qualifiedPath,
final ITtlTimeProvider timeProvider,
@Nullable final BulkOperationState operationState) {
}
}
}

View File

@ -80,6 +80,16 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
private MetadataStore ms; private MetadataStore ms;
private S3AFileSystem rawFs; private S3AFileSystem rawFs;
/**
* The test timeout is increased in case previous tests have created
* many tombstone markers which now need to be purged.
* @return the test timeout.
*/
@Override
protected int getTestTimeoutMillis() {
return SCALE_TEST_TIMEOUT_SECONDS * 1000;
}
protected static void expectResult(int expected, protected static void expectResult(int expected,
String message, String message,
S3GuardTool tool, S3GuardTool tool,
@ -187,19 +197,24 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
fs.mkdirs(path); fs.mkdirs(path);
} else if (onMetadataStore) { } else if (onMetadataStore) {
S3AFileStatus status = new S3AFileStatus(true, path, OWNER); S3AFileStatus status = new S3AFileStatus(true, path, OWNER);
ms.put(new PathMetadata(status)); ms.put(new PathMetadata(status), null);
} }
} }
protected static void putFile(MetadataStore ms, S3AFileStatus f) protected static void putFile(MetadataStore ms, S3AFileStatus f)
throws IOException { throws IOException {
assertNotNull(f); assertNotNull(f);
ms.put(new PathMetadata(f)); try (BulkOperationState bulkWrite =
Path parent = f.getPath().getParent(); ms.initiateBulkWrite(
while (parent != null) { BulkOperationState.OperationType.Put,
S3AFileStatus dir = new S3AFileStatus(false, parent, f.getOwner()); f.getPath())) {
ms.put(new PathMetadata(dir)); ms.put(new PathMetadata(f), bulkWrite);
parent = parent.getParent(); Path parent = f.getPath().getParent();
while (parent != null) {
S3AFileStatus dir = new S3AFileStatus(false, parent, f.getOwner());
ms.put(new PathMetadata(dir), bulkWrite);
parent = parent.getParent();
}
} }
} }
@ -252,12 +267,13 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
String...args) throws Exception { String...args) throws Exception {
Path keepParent = path("prune-cli-keep"); Path keepParent = path("prune-cli-keep");
StopWatch timer = new StopWatch(); StopWatch timer = new StopWatch();
final S3AFileSystem fs = getFileSystem();
try { try {
S3GuardTool.Prune cmd = new S3GuardTool.Prune(cmdConf); S3GuardTool.Prune cmd = new S3GuardTool.Prune(cmdConf);
cmd.setMetadataStore(ms); cmd.setMetadataStore(ms);
getFileSystem().mkdirs(parent); fs.mkdirs(parent);
getFileSystem().mkdirs(keepParent); fs.mkdirs(keepParent);
createFile(new Path(parent, "stale"), true, true); createFile(new Path(parent, "stale"), true, true);
createFile(new Path(keepParent, "stale-to-keep"), true, true); createFile(new Path(keepParent, "stale-to-keep"), true, true);
@ -279,8 +295,14 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
assertMetastoreListingCount(keepParent, assertMetastoreListingCount(keepParent,
"This child should have been kept (prefix restriction).", 1); "This child should have been kept (prefix restriction).", 1);
} finally { } finally {
getFileSystem().delete(parent, true); fs.delete(parent, true);
ms.prune(MetadataStore.PruneMode.ALL_BY_MODTIME, Long.MAX_VALUE); fs.delete(keepParent, true);
ms.prune(MetadataStore.PruneMode.ALL_BY_MODTIME,
Long.MAX_VALUE,
fs.pathToKey(parent));
ms.prune(MetadataStore.PruneMode.ALL_BY_MODTIME,
Long.MAX_VALUE,
fs.pathToKey(keepParent));
} }
} }

View File

@ -35,19 +35,21 @@ import com.amazonaws.services.dynamodbv2.document.Table;
import com.amazonaws.services.dynamodbv2.model.ListTagsOfResourceRequest; import com.amazonaws.services.dynamodbv2.model.ListTagsOfResourceRequest;
import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException; import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
import com.amazonaws.services.dynamodbv2.model.TableDescription; import com.amazonaws.services.dynamodbv2.model.TableDescription;
import com.amazonaws.services.dynamodbv2.model.Tag; import com.amazonaws.services.dynamodbv2.model.Tag;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import org.assertj.core.api.Assertions;
import org.apache.commons.collections.CollectionUtils; import org.apache.commons.collections.CollectionUtils;
import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.contract.s3a.S3AContract; import org.apache.hadoop.fs.contract.s3a.S3AContract;
import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.Constants;
import org.apache.hadoop.fs.s3a.S3ATestConstants; import org.apache.hadoop.fs.s3a.S3ATestConstants;
import org.apache.hadoop.fs.s3a.Tristate; import org.apache.hadoop.fs.s3a.Tristate;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.DurationInfo;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Assume; import org.junit.Assume;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
@ -62,6 +64,7 @@ import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
import static org.apache.hadoop.fs.s3a.S3AUtils.clearBucketOption; import static org.apache.hadoop.fs.s3a.S3AUtils.clearBucketOption;
@ -159,7 +162,8 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
testDynamoDBTableName = conf.get( testDynamoDBTableName = conf.get(
S3ATestConstants.S3GUARD_DDB_TEST_TABLE_NAME_KEY); S3ATestConstants.S3GUARD_DDB_TEST_TABLE_NAME_KEY);
String dynamoDbTableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY); String dynamoDbTableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY);
Assume.assumeTrue("No DynamoDB table name configured", Assume.assumeTrue("No DynamoDB table name configured in "
+ S3GUARD_DDB_TABLE_NAME_KEY,
!StringUtils.isEmpty(dynamoDbTableName)); !StringUtils.isEmpty(dynamoDbTableName));
// We should assert that the table name is configured, so the test should // We should assert that the table name is configured, so the test should
@ -181,6 +185,19 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
// We can use that table in the test if these assertions are valid // We can use that table in the test if these assertions are valid
conf.set(S3GUARD_DDB_TABLE_NAME_KEY, testDynamoDBTableName); conf.set(S3GUARD_DDB_TABLE_NAME_KEY, testDynamoDBTableName);
// remove some prune delays
conf.setInt(S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY, 0);
// clear all table tagging config before this test
conf.getPropsWithPrefix(S3GUARD_DDB_TABLE_TAG).keySet().forEach(
propKey -> conf.unset(S3GUARD_DDB_TABLE_TAG + propKey)
);
// set the tags on the table so that it can be tested later.
Map<String, String> tagMap = createTagMap();
for (Map.Entry<String, String> tagEntry : tagMap.entrySet()) {
conf.set(S3GUARD_DDB_TABLE_TAG + tagEntry.getKey(), tagEntry.getValue());
}
LOG.debug("Creating static ddbms which will be shared between tests."); LOG.debug("Creating static ddbms which will be shared between tests.");
enableOnDemand(conf); enableOnDemand(conf);
@ -191,14 +208,23 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
@AfterClass @AfterClass
public static void afterClassTeardown() { public static void afterClassTeardown() {
LOG.debug("Destroying static DynamoDBMetadataStore."); LOG.debug("Destroying static DynamoDBMetadataStore.");
if (ddbmsStatic != null) { destroy(ddbmsStatic);
ddbmsStatic = null;
}
/**
* Destroy and then close() a metastore instance.
* Exceptions are caught and logged at debug.
* @param ddbms store -may be null.
*/
private static void destroy(final DynamoDBMetadataStore ddbms) {
if (ddbms != null) {
try { try {
ddbmsStatic.destroy(); ddbms.destroy();
} catch (Exception e) { IOUtils.closeStream(ddbms);
LOG.warn("Failed to destroy tables in teardown", e); } catch (IOException e) {
LOG.debug("On ddbms shutdown", e);
} }
IOUtils.closeStream(ddbmsStatic);
ddbmsStatic = null;
} }
} }
@ -208,26 +234,91 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
Constants.S3GUARD_METASTORE_DYNAMO)); Constants.S3GUARD_METASTORE_DYNAMO));
} }
/**
* This teardown does not call super.teardown() so as to avoid the DDMBS table
* from being destroyed.
* <p>
* This is potentially quite slow, depending on DDB IO Capacity and number
* of entries to forget.
*/
@Override @Override
public void tearDown() throws Exception { public void tearDown() throws Exception {
LOG.info("Removing data from ddbms table in teardown.");
Thread.currentThread().setName("Teardown");
// The following is a way to be sure the table will be cleared and there
// will be no leftovers after the test.
try { try {
if (ddbmsStatic != null) { deleteAllMetadata();
LOG.info("Removing data from ddbms table in teardown."); } finally {
// The following is a way to be sure the table will be cleared and there IOUtils.cleanupWithLogger(LOG, fileSystem);
// will be no leftovers after the test.
PathMetadata meta = ddbmsStatic.get(strToPath("/"));
if (meta != null){
for (DescendantsIterator desc =
new DescendantsIterator(ddbmsStatic, meta);
desc.hasNext();) {
ddbmsStatic.forgetMetadata(desc.next().getPath());
}
}
}
} catch (IOException ignored) {
} }
IOUtils.cleanupWithLogger(LOG, fileSystem); }
/**
* Forget all metadata in the store.
* This originally did an iterate and forget, but using prune() hands off the
* bulk IO into the metastore itself; the forgetting is used
* to purge anything which wasn't pruned.
*/
private void deleteAllMetadata() throws IOException {
// The following is a way to be sure the table will be cleared and there
// will be no leftovers after the test.
// only executed if there is a filesystem, as failure during test setup
// means that strToPath will NPE.
if (getContract() != null && getContract().getFileSystem() != null) {
deleteMetadataUnderPath(ddbmsStatic, strToPath("/"), true);
}
}
/**
* Delete all metadata under a path.
* Attempt to use prune first as it scales slightly better.
* @param ms store
* @param path path to prune under
* @param suppressErrors should errors be suppressed?
* @throws IOException if there is a failure and suppressErrors == false
*/
public static void deleteMetadataUnderPath(final DynamoDBMetadataStore ms,
final Path path, final boolean suppressErrors) throws IOException {
ThrottleTracker throttleTracker = new ThrottleTracker(ms);
try (DurationInfo ignored = new DurationInfo(LOG, true, "prune")) {
ms.prune(PruneMode.ALL_BY_MODTIME,
System.currentTimeMillis(),
PathMetadataDynamoDBTranslation.pathToParentKey(path));
LOG.info("Throttle statistics: {}", throttleTracker);
} catch (FileNotFoundException fnfe) {
// there is no table.
return;
} catch (IOException ioe) {
// prune failed. warn and then fall back to forget.
LOG.warn("Failed to prune {}", path, ioe);
if (!suppressErrors) {
throw ioe;
}
}
// and after the pruning, make sure all other metadata is gone
int forgotten = 0;
try (DurationInfo ignored = new DurationInfo(LOG, true, "forget")) {
PathMetadata meta = ms.get(path);
if (meta != null) {
for (DescendantsIterator desc = new DescendantsIterator(ms,
meta);
desc.hasNext();) {
forgotten++;
ms.forgetMetadata(desc.next().getPath());
}
LOG.info("Forgot {} entries", forgotten);
}
} catch (FileNotFoundException fnfe) {
// there is no table.
return;
} catch (IOException ioe) {
LOG.warn("Failed to forget entries under {}", path, ioe);
if (!suppressErrors) {
throw ioe;
}
}
LOG.info("Throttle statistics: {}", throttleTracker);
} }
@Override protected String getPathStringForPrune(String path) @Override protected String getPathStringForPrune(String path)
@ -270,7 +361,7 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
} }
@Override @Override
S3AFileStatus basicFileStatus(Path path, int size, boolean isDir) protected S3AFileStatus basicFileStatus(Path path, int size, boolean isDir)
throws IOException { throws IOException {
String owner = UserGroupInformation.getCurrentUser().getShortUserName(); String owner = UserGroupInformation.getCurrentUser().getShortUserName();
return isDir return isDir
@ -279,6 +370,15 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
null, null); null, null);
} }
/**
* Create a directory status entry.
* @param dir directory.
* @return the status
*/
private S3AFileStatus dirStatus(Path dir) throws IOException {
return basicFileStatus(dir, 0, true);
}
private DynamoDBMetadataStore getDynamoMetadataStore() throws IOException { private DynamoDBMetadataStore getDynamoMetadataStore() throws IOException {
return (DynamoDBMetadataStore) getContract().getMetadataStore(); return (DynamoDBMetadataStore) getContract().getMetadataStore();
} }
@ -335,8 +435,7 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
expectedRegion, expectedRegion,
ddbms.getRegion()); ddbms.getRegion());
} finally { } finally {
ddbms.destroy(); destroy(ddbms);
ddbms.close();
} }
} }
@ -377,31 +476,69 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
keySchema(), keySchema(),
ddbms.getTable().describe().getKeySchema()); ddbms.getTable().describe().getKeySchema());
} finally { } finally {
ddbms.destroy(); destroy(ddbms);
ddbms.close();
} }
} }
/**
* This should really drive a parameterized test run of 5^2 entries, but it
* would require a major refactoring to set things up.
* For now, each source test has its own entry, with the destination written
* to.
* This seems to be enough to stop DDB throttling from triggering test
* timeouts.
*/
private static final int[] NUM_METAS_TO_DELETE_OR_PUT = {
-1, // null
0, // empty collection
1, // one path
S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT, // exact limit of a batch request
S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT + 1 // limit + 1
};
@Test
public void testBatchWrite00() throws IOException {
doBatchWriteForOneSet(0);
}
@Test
public void testBatchWrite01() throws IOException {
doBatchWriteForOneSet(1);
}
@Test
public void testBatchWrite02() throws IOException {
doBatchWriteForOneSet(2);
}
@Test
public void testBatchWrite03() throws IOException {
doBatchWriteForOneSet(3);
}
@Test
public void testBatchWrite04() throws IOException {
doBatchWriteForOneSet(4);
}
/**
* Test that for a large batch write request, the limit is handled correctly.
* With cleanup afterwards.
*/
private void doBatchWriteForOneSet(int index) throws IOException {
for (int numNewMetas : NUM_METAS_TO_DELETE_OR_PUT) {
doTestBatchWrite(NUM_METAS_TO_DELETE_OR_PUT[index],
numNewMetas,
getDynamoMetadataStore());
}
// The following is a way to be sure the table will be cleared and there
// will be no leftovers after the test.
deleteMetadataUnderPath(ddbmsStatic, strToPath("/"), false);
}
/** /**
* Test that for a large batch write request, the limit is handled correctly. * Test that for a large batch write request, the limit is handled correctly.
*/ */
@Test
public void testBatchWrite() throws IOException {
final int[] numMetasToDeleteOrPut = {
-1, // null
0, // empty collection
1, // one path
S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT, // exact limit of a batch request
S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT + 1 // limit + 1
};
DynamoDBMetadataStore ms = getDynamoMetadataStore();
for (int numOldMetas : numMetasToDeleteOrPut) {
for (int numNewMetas : numMetasToDeleteOrPut) {
doTestBatchWrite(numOldMetas, numNewMetas, ms);
}
}
}
private void doTestBatchWrite(int numDelete, int numPut, private void doTestBatchWrite(int numDelete, int numPut,
DynamoDBMetadataStore ms) throws IOException { DynamoDBMetadataStore ms) throws IOException {
Path path = new Path( Path path = new Path(
@ -411,15 +548,20 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
final Path oldDir = new Path(root, "oldDir"); final Path oldDir = new Path(root, "oldDir");
final Path newDir = new Path(root, "newDir"); final Path newDir = new Path(root, "newDir");
LOG.info("doTestBatchWrite: oldDir={}, newDir={}", oldDir, newDir); LOG.info("doTestBatchWrite: oldDir={}, newDir={}", oldDir, newDir);
Thread.currentThread()
.setName(String.format("Bulk put=%d; delete=%d", numPut, numDelete));
ms.put(new PathMetadata(basicFileStatus(oldDir, 0, true))); AncestorState putState = checkNotNull(ms.initiateBulkWrite(
ms.put(new PathMetadata(basicFileStatus(newDir, 0, true))); BulkOperationState.OperationType.Put, newDir),
"No state from initiateBulkWrite()");
ms.put(new PathMetadata(dirStatus(oldDir)), putState);
ms.put(new PathMetadata(dirStatus(newDir)), putState);
final List<PathMetadata> oldMetas = numDelete < 0 ? null : final List<PathMetadata> oldMetas = numDelete < 0 ? null :
new ArrayList<>(numDelete); new ArrayList<>(numDelete);
for (int i = 0; i < numDelete; i++) { for (int i = 0; i < numDelete; i++) {
oldMetas.add(new PathMetadata( oldMetas.add(new PathMetadata(
basicFileStatus(new Path(oldDir, "child" + i), i, true))); basicFileStatus(new Path(oldDir, "child" + i), i, false)));
} }
final List<PathMetadata> newMetas = numPut < 0 ? null : final List<PathMetadata> newMetas = numPut < 0 ? null :
new ArrayList<>(numPut); new ArrayList<>(numPut);
@ -431,8 +573,13 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
Collection<Path> pathsToDelete = null; Collection<Path> pathsToDelete = null;
if (oldMetas != null) { if (oldMetas != null) {
// put all metadata of old paths and verify // put all metadata of old paths and verify
ms.put(new DirListingMetadata(oldDir, oldMetas, false)); ms.put(new DirListingMetadata(oldDir, oldMetas, false), putState);
assertEquals(0, ms.listChildren(newDir).withoutTombstones().numEntries()); assertEquals("Child count",
0, ms.listChildren(newDir).withoutTombstones().numEntries());
Assertions.assertThat(ms.listChildren(oldDir).getListing())
.describedAs("Old Directory listing")
.containsExactlyInAnyOrderElementsOf(oldMetas);
assertTrue(CollectionUtils assertTrue(CollectionUtils
.isEqualCollection(oldMetas, ms.listChildren(oldDir).getListing())); .isEqualCollection(oldMetas, ms.listChildren(oldDir).getListing()));
@ -443,11 +590,28 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
} }
// move the old paths to new paths and verify // move the old paths to new paths and verify
ms.move(pathsToDelete, newMetas, getTtlTimeProvider()); AncestorState state = checkNotNull(ms.initiateBulkWrite(
assertEquals(0, ms.listChildren(oldDir).withoutTombstones().numEntries()); BulkOperationState.OperationType.Put, newDir),
"No state from initiateBulkWrite()");
assertEquals("bulk write destination", newDir, state.getDest());
ThrottleTracker throttleTracker = new ThrottleTracker(ms);
try(DurationInfo ignored = new DurationInfo(LOG, true,
"Move")) {
ms.move(pathsToDelete, newMetas, getTtlTimeProvider(), state);
}
LOG.info("Throttle status {}", throttleTracker);
assertEquals("Number of children in source directory",
0, ms.listChildren(oldDir).withoutTombstones().numEntries());
if (newMetas != null) { if (newMetas != null) {
assertTrue(CollectionUtils Assertions.assertThat(ms.listChildren(newDir).getListing())
.isEqualCollection(newMetas, ms.listChildren(newDir).getListing())); .describedAs("Directory listing")
.containsAll(newMetas);
if (!newMetas.isEmpty()) {
Assertions.assertThat(state.size())
.describedAs("Size of ancestor state")
.isGreaterThan(newMetas.size());
}
} }
} }
@ -483,64 +647,67 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
} }
/** /**
* Test versioning handling.
* <ol>
* <li>Create the table.</li>
* <li>Verify tag propagation.</li>
* <li>Delete the version marker -verify failure.</li>
* <li>Reinstate a different version marker -verify failure</li>
* </ol>
* Delete the version marker and verify that table init fails. * Delete the version marker and verify that table init fails.
* This also includes the checks for tagging, which goes against all
* principles of unit tests.
* However, merging the routines saves
*/ */
@Test @Test
public void testTableVersionRequired() throws Exception { public void testTableVersioning() throws Exception {
String tableName = getTestTableName("testTableVersionRequired"); String tableName = getTestTableName("testTableVersionRequired");
Configuration conf = getTableCreationConfig(); Configuration conf = getTableCreationConfig();
int maxRetries = conf.getInt(S3GUARD_DDB_MAX_RETRIES, int maxRetries = conf.getInt(S3GUARD_DDB_MAX_RETRIES,
S3GUARD_DDB_MAX_RETRIES_DEFAULT); S3GUARD_DDB_MAX_RETRIES_DEFAULT);
conf.setInt(S3GUARD_DDB_MAX_RETRIES, 3); conf.setInt(S3GUARD_DDB_MAX_RETRIES, 3);
conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName); conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
tagConfiguration(conf);
DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore(); DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore();
try { try {
ddbms.initialize(conf); ddbms.initialize(conf);
Table table = verifyTableInitialized(tableName, ddbms.getDynamoDB()); Table table = verifyTableInitialized(tableName, ddbms.getDynamoDB());
// check the tagging too
verifyStoreTags(createTagMap(), ddbms);
Item originalVersionMarker = table.getItem(VERSION_MARKER_PRIMARY_KEY);
table.deleteItem(VERSION_MARKER_PRIMARY_KEY); table.deleteItem(VERSION_MARKER_PRIMARY_KEY);
// create existing table // create existing table
intercept(IOException.class, E_NO_VERSION_MARKER, intercept(IOException.class, E_NO_VERSION_MARKER,
() -> ddbms.initTable()); () -> ddbms.initTable());
conf.setInt(S3GUARD_DDB_MAX_RETRIES, maxRetries); // now add a different version marker
} finally { Item v200 = createVersionMarker(VERSION_MARKER, VERSION * 2, 0);
ddbms.destroy();
ddbms.close();
}
}
/**
* Set the version value to a different number and verify that
* table init fails.
*/
@Test
public void testTableVersionMismatch() throws Exception {
String tableName = getTestTableName("testTableVersionMismatch");
Configuration conf = getTableCreationConfig();
conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName);
DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore();
try {
ddbms.initialize(conf);
Table table = verifyTableInitialized(tableName, ddbms.getDynamoDB());
table.deleteItem(VERSION_MARKER_PRIMARY_KEY);
Item v200 = createVersionMarker(VERSION_MARKER, 200, 0);
table.putItem(v200); table.putItem(v200);
// create existing table // create existing table
intercept(IOException.class, E_INCOMPATIBLE_VERSION, intercept(IOException.class, E_INCOMPATIBLE_VERSION,
() -> ddbms.initTable()); () -> ddbms.initTable());
// create a marker with no version and expect failure
final Item invalidMarker = new Item().withPrimaryKey(
createVersionMarkerPrimaryKey(VERSION_MARKER))
.withLong(TABLE_CREATED, 0);
table.putItem(invalidMarker);
intercept(IOException.class, E_NOT_VERSION_MARKER,
() -> ddbms.initTable());
// reinstate the version marker
table.putItem(originalVersionMarker);
ddbms.initTable();
conf.setInt(S3GUARD_DDB_MAX_RETRIES, maxRetries);
} finally { } finally {
ddbms.destroy(); destroy(ddbms);
ddbms.close();
} }
} }
/** /**
* Test that initTable fails with IOException when table does not exist and * Test that initTable fails with IOException when table does not exist and
* table auto-creation is disabled. * table auto-creation is disabled.
@ -580,7 +747,8 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
ddbms.put(new PathMetadata(new S3AFileStatus(true, ddbms.put(new PathMetadata(new S3AFileStatus(true,
new Path(rootPath, "foo"), new Path(rootPath, "foo"),
UserGroupInformation.getCurrentUser().getShortUserName()))); UserGroupInformation.getCurrentUser().getShortUserName())),
null);
verifyRootDirectory(ddbms.get(rootPath), false); verifyRootDirectory(ddbms.get(rootPath), false);
} }
@ -631,9 +799,13 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
final String destRoot = testRoot + "/c/d/e/dest"; final String destRoot = testRoot + "/c/d/e/dest";
final Path nestedPath1 = strToPath(srcRoot + "/file1.txt"); final Path nestedPath1 = strToPath(srcRoot + "/file1.txt");
ddbms.put(new PathMetadata(basicFileStatus(nestedPath1, 1024, false))); AncestorState bulkWrite = ddbms.initiateBulkWrite(
BulkOperationState.OperationType.Put, nestedPath1);
ddbms.put(new PathMetadata(basicFileStatus(nestedPath1, 1024, false)),
bulkWrite);
final Path nestedPath2 = strToPath(srcRoot + "/dir1/dir2"); final Path nestedPath2 = strToPath(srcRoot + "/dir1/dir2");
ddbms.put(new PathMetadata(basicFileStatus(nestedPath2, 0, true))); ddbms.put(new PathMetadata(basicFileStatus(nestedPath2, 0, true)),
bulkWrite);
// We don't put the destRoot path here, since put() would create ancestor // We don't put the destRoot path here, since put() would create ancestor
// entries, and we want to ensure that move() does it, instead. // entries, and we want to ensure that move() does it, instead.
@ -643,8 +815,8 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
strToPath(srcRoot), strToPath(srcRoot),
strToPath(srcRoot + "/dir1"), strToPath(srcRoot + "/dir1"),
strToPath(srcRoot + "/dir1/dir2"), strToPath(srcRoot + "/dir1/dir2"),
strToPath(srcRoot + "/file1.txt") strToPath(srcRoot + "/file1.txt"));
); final String finalFile = destRoot + "/file1.txt";
final Collection<PathMetadata> pathsToCreate = Lists.newArrayList( final Collection<PathMetadata> pathsToCreate = Lists.newArrayList(
new PathMetadata(basicFileStatus(strToPath(destRoot), new PathMetadata(basicFileStatus(strToPath(destRoot),
0, true)), 0, true)),
@ -652,22 +824,101 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
0, true)), 0, true)),
new PathMetadata(basicFileStatus(strToPath(destRoot + "/dir1/dir2"), new PathMetadata(basicFileStatus(strToPath(destRoot + "/dir1/dir2"),
0, true)), 0, true)),
new PathMetadata(basicFileStatus(strToPath(destRoot + "/file1.txt"), new PathMetadata(basicFileStatus(strToPath(finalFile),
1024, false)) 1024, false))
); );
ddbms.move(fullSourcePaths, pathsToCreate, getTtlTimeProvider()); ddbms.move(fullSourcePaths, pathsToCreate, getTtlTimeProvider(),
bulkWrite);
bulkWrite.close();
// assert that all the ancestors should have been populated automatically // assert that all the ancestors should have been populated automatically
assertCached(testRoot + "/c"); List<String> paths = Lists.newArrayList(
assertCached(testRoot + "/c/d"); testRoot + "/c", testRoot + "/c/d", testRoot + "/c/d/e", destRoot,
assertCached(testRoot + "/c/d/e"); destRoot + "/dir1", destRoot + "/dir1/dir2");
assertCached(destRoot /* /c/d/e/dest */); for (String p : paths) {
assertCached(p);
verifyInAncestor(bulkWrite, p, true);
}
// Also check moved files while we're at it // Also check moved files while we're at it
assertCached(destRoot + "/dir1"); assertCached(finalFile);
assertCached(destRoot + "/dir1/dir2"); verifyInAncestor(bulkWrite, finalFile, false);
assertCached(destRoot + "/file1.txt"); }
@Test
public void testAncestorOverwriteConflict() throws Throwable {
final DynamoDBMetadataStore ddbms = getDynamoMetadataStore();
String testRoot = "/" + getMethodName();
String parent = testRoot + "/parent";
Path parentPath = strToPath(parent);
String child = parent + "/child";
Path childPath = strToPath(child);
String grandchild = child + "/grandchild";
Path grandchildPath = strToPath(grandchild);
String child2 = parent + "/child2";
String grandchild2 = child2 + "/grandchild2";
Path grandchild2Path = strToPath(grandchild2);
AncestorState bulkWrite = ddbms.initiateBulkWrite(
BulkOperationState.OperationType.Put, parentPath);
// writing a child creates ancestors
ddbms.put(
new PathMetadata(basicFileStatus(childPath, 1024, false)),
bulkWrite);
verifyInAncestor(bulkWrite, child, false);
verifyInAncestor(bulkWrite, parent, true);
// overwrite an ancestor with a file entry in the same operation
// is an error.
intercept(PathIOException.class, E_INCONSISTENT_UPDATE,
() -> ddbms.put(
new PathMetadata(basicFileStatus(parentPath, 1024, false)),
bulkWrite));
// now put a file under the child and expect the put operation
// to fail fast, because the ancestor state includes a file at a parent.
intercept(PathIOException.class, E_INCONSISTENT_UPDATE,
() -> ddbms.put(
new PathMetadata(basicFileStatus(grandchildPath, 1024, false)),
bulkWrite));
// and expect a failure for directory update under the child
DirListingMetadata grandchildListing = new DirListingMetadata(
grandchildPath,
new ArrayList<>(), false);
intercept(PathIOException.class, E_INCONSISTENT_UPDATE,
() -> ddbms.put(grandchildListing, bulkWrite));
// but a directory update under another path is fine
DirListingMetadata grandchild2Listing = new DirListingMetadata(
grandchild2Path,
new ArrayList<>(), false);
ddbms.put(grandchild2Listing, bulkWrite);
// and it creates a new entry for its parent
verifyInAncestor(bulkWrite, child2, true);
}
/**
* Assert that a path has an entry in the ancestor state.
* @param state ancestor state
* @param path path to look for
* @param isDirectory is it a directory
* @return the value
* @throws IOException IO failure
* @throws AssertionError assertion failure.
*/
private DDBPathMetadata verifyInAncestor(AncestorState state,
String path,
final boolean isDirectory)
throws IOException {
final Path p = strToPath(path);
assertTrue("Path " + p + " not found in ancestor state", state.contains(p));
final DDBPathMetadata md = state.get(p);
assertTrue("Ancestor value for "+ path,
isDirectory
? md.getFileStatus().isDirectory()
: md.getFileStatus().isFile());
return md;
} }
@Test @Test
@ -731,50 +982,48 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
intercept(IOException.class, "", intercept(IOException.class, "",
"Should have failed after the table is destroyed!", "Should have failed after the table is destroyed!",
() -> ddbms.listChildren(testPath)); () -> ddbms.listChildren(testPath));
} finally {
ddbms.destroy(); ddbms.destroy();
ddbms.close(); intercept(FileNotFoundException.class, "",
"Destroyed table should raise FileNotFoundException when pruned",
() -> ddbms.prune(PruneMode.ALL_BY_MODTIME, 0));
} finally {
destroy(ddbms);
} }
} }
@Test protected void verifyStoreTags(final Map<String, String> tagMap,
public void testTableTagging() throws IOException { final DynamoDBMetadataStore store) {
final Configuration conf = getTableCreationConfig(); List<Tag> tags = listTagsOfStore(store);
// clear all table tagging config before this test Map<String, String> actual = new HashMap<>();
conf.getPropsWithPrefix(S3GUARD_DDB_TABLE_TAG).keySet().forEach( tags.forEach(t -> actual.put(t.getKey(), t.getValue()));
propKey -> conf.unset(S3GUARD_DDB_TABLE_TAG + propKey) Assertions.assertThat(actual)
); .describedAs("Tags from DDB table")
.containsExactlyEntriesOf(tagMap);
assertEquals(tagMap.size(), tags.size());
}
String tableName = protected List<Tag> listTagsOfStore(final DynamoDBMetadataStore store) {
getTestTableName("testTableTagging-" + UUID.randomUUID()); ListTagsOfResourceRequest listTagsOfResourceRequest =
conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName); new ListTagsOfResourceRequest()
conf.set(S3GUARD_DDB_TABLE_CREATE_KEY, "true"); .withResourceArn(store.getTable().getDescription()
.getTableArn());
return store.getAmazonDynamoDB()
.listTagsOfResource(listTagsOfResourceRequest).getTags();
}
private static Map<String, String> createTagMap() {
Map<String, String> tagMap = new HashMap<>(); Map<String, String> tagMap = new HashMap<>();
tagMap.put("hello", "dynamo"); tagMap.put("hello", "dynamo");
tagMap.put("tag", "youre it"); tagMap.put("tag", "youre it");
return tagMap;
}
private static void tagConfiguration(Configuration conf) {
// set the tags on the table so that it can be tested later.
Map<String, String> tagMap = createTagMap();
for (Map.Entry<String, String> tagEntry : tagMap.entrySet()) { for (Map.Entry<String, String> tagEntry : tagMap.entrySet()) {
conf.set(S3GUARD_DDB_TABLE_TAG + tagEntry.getKey(), tagEntry.getValue()); conf.set(S3GUARD_DDB_TABLE_TAG + tagEntry.getKey(), tagEntry.getValue());
} }
DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore();
try {
ddbms.initialize(conf);
assertNotNull(ddbms.getTable());
assertEquals(tableName, ddbms.getTable().getTableName());
ListTagsOfResourceRequest listTagsOfResourceRequest =
new ListTagsOfResourceRequest()
.withResourceArn(ddbms.getTable().getDescription().getTableArn());
List<Tag> tags = ddbms.getAmazonDynamoDB()
.listTagsOfResource(listTagsOfResourceRequest).getTags();
assertEquals(tagMap.size(), tags.size());
for (Tag tag : tags) {
Assert.assertEquals(tagMap.get(tag.getKey()), tag.getValue());
}
} finally {
ddbms.destroy();
ddbms.close();
}
} }
@Test @Test
@ -793,7 +1042,7 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
throws IOException { throws IOException {
// setup // setup
final DynamoDBMetadataStore ms = getDynamoMetadataStore(); final DynamoDBMetadataStore ms = getDynamoMetadataStore();
String rootPath = "/testAuthoritativeEmptyDirFlag"+ UUID.randomUUID(); String rootPath = "/testAuthoritativeEmptyDirFlag-" + UUID.randomUUID();
String filePath = rootPath + "/file1"; String filePath = rootPath + "/file1";
final Path dirToPut = fileSystem.makeQualified(new Path(rootPath)); final Path dirToPut = fileSystem.makeQualified(new Path(rootPath));
final Path fileToPut = fileSystem.makeQualified(new Path(filePath)); final Path fileToPut = fileSystem.makeQualified(new Path(filePath));
@ -809,7 +1058,7 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
assertEquals(auth, dlm.isAuthoritative()); assertEquals(auth, dlm.isAuthoritative());
// Test with non-authoritative listing, empty dir // Test with non-authoritative listing, empty dir
ms.put(dlm); ms.put(dlm, null);
final PathMetadata pmdResultEmpty = ms.get(dirToPut, true); final PathMetadata pmdResultEmpty = ms.get(dirToPut, true);
if(auth){ if(auth){
assertEquals(Tristate.TRUE, pmdResultEmpty.isEmptyDirectory()); assertEquals(Tristate.TRUE, pmdResultEmpty.isEmptyDirectory());
@ -819,7 +1068,7 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
// Test with non-authoritative listing, non-empty dir // Test with non-authoritative listing, non-empty dir
dlm.put(basicFileStatus(fileToPut, 1, false)); dlm.put(basicFileStatus(fileToPut, 1, false));
ms.put(dlm); ms.put(dlm, null);
final PathMetadata pmdResultNotEmpty = ms.get(dirToPut, true); final PathMetadata pmdResultNotEmpty = ms.get(dirToPut, true);
assertEquals(Tristate.FALSE, pmdResultNotEmpty.isEmptyDirectory()); assertEquals(Tristate.FALSE, pmdResultNotEmpty.isEmptyDirectory());
} }
@ -852,4 +1101,120 @@ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase {
private String getTestTableName(String suffix) { private String getTestTableName(String suffix) {
return getTestDynamoTablePrefix(s3AContract.getConf()) + suffix; return getTestDynamoTablePrefix(s3AContract.getConf()) + suffix;
} }
@Test
public void testPruneAgainstInvalidTable() throws Throwable {
describe("Create an Invalid listing and prune it");
DynamoDBMetadataStore ms
= ITestDynamoDBMetadataStore.ddbmsStatic;
String base = "/testPruneAgainstInvalidTable";
String subdir = base + "/subdir";
Path subDirPath = strToPath(subdir);
createNewDirs(base, subdir);
String subFile = subdir + "/file1";
Path subFilePath = strToPath(subFile);
putListStatusFiles(subdir, true,
subFile);
final DDBPathMetadata subDirMetadataOrig = ms.get(subDirPath);
Assertions.assertThat(subDirMetadataOrig.isAuthoritativeDir())
.describedAs("Subdirectory %s", subDirMetadataOrig)
.isTrue();
// now let's corrupt the graph by putting a file
// over the subdirectory
long now = getTime();
long oldTime = now - 60_000;
putFile(subdir, oldTime, null);
final DDBPathMetadata subDirAsFile = ms.get(subDirPath);
Assertions.assertThat(subDirAsFile.getFileStatus().isFile())
.describedAs("Subdirectory entry %s is now file", subDirMetadataOrig)
.isTrue();
Path basePath = strToPath(base);
DirListingMetadata listing = ms.listChildren(basePath);
String childText = listing.prettyPrint();
LOG.info("Listing {}", childText);
Collection<PathMetadata> childList = listing.getListing();
Assertions.assertThat(childList)
.as("listing of %s with %s", basePath, childText)
.hasSize(1);
PathMetadata[] pm = new PathMetadata[0];
S3AFileStatus status = childList.toArray(pm)[0]
.getFileStatus();
Assertions.assertThat(status.isFile())
.as("Entry %s", (Object)pm)
.isTrue();
DDBPathMetadata subFilePm = checkNotNull(ms.get(subFilePath));
LOG.info("Pruning");
// now prune
ms.prune(PruneMode.ALL_BY_MODTIME,
now + 60_000, subdir);
DDBPathMetadata prunedFile = ms.get(subFilePath);
final PathMetadata subDirMetadataFinal = getNonNull(subdir);
Assertions.assertThat(subDirMetadataFinal.getFileStatus().isFile())
.describedAs("Subdirectory entry %s is still a file",
subDirMetadataFinal)
.isTrue();
}
@Test
public void testPutFileDirectlyUnderTombstone() throws Throwable {
describe("Put a file under a tombstone");
String base = "/testPutFileDirectlyUnderTombstone";
long now = getTime();
putTombstone(base, now, null);
PathMetadata baseMeta1 = get(base);
Assertions.assertThat(baseMeta1.isDeleted())
.as("Metadata %s", baseMeta1)
.isTrue();
String child = base + "/file";
putFile(child, now, null);
PathMetadata baseMeta2 = get(base);
Assertions.assertThat(baseMeta2.isDeleted())
.as("Metadata %s", baseMeta2)
.isFalse();
}
@Test
public void testPutFileDeepUnderTombstone() throws Throwable {
describe("Put a file two levels under a tombstone");
String base = "/testPutFileDeepUnderTombstone";
String subdir = base + "/subdir";
long now = getTime();
// creating a file MUST create its parents
String child = subdir + "/file";
Path childPath = strToPath(child);
putFile(child, now, null);
getFile(child);
getDirectory(subdir);
getDirectory(base);
// now put the tombstone
putTombstone(base, now, null);
PathMetadata baseMeta1 = getNonNull(base);
Assertions.assertThat(baseMeta1.isDeleted())
.as("Metadata %s", baseMeta1)
.isTrue();
// this is the same ordering as S3FileSystem.finishedWrite()
AncestorState ancestorState = getDynamoMetadataStore()
.initiateBulkWrite(BulkOperationState.OperationType.Put,
childPath);
S3Guard.addAncestors(getDynamoMetadataStore(),
childPath,
getTtlTimeProvider(),
ancestorState);
// now write the file again.
putFile(child, now, ancestorState);
// the ancestor will now exist.
getDirectory(base);
}
} }

View File

@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import com.amazonaws.services.dynamodbv2.document.DynamoDB; import com.amazonaws.services.dynamodbv2.document.DynamoDB;
import com.amazonaws.services.dynamodbv2.document.Table; import com.amazonaws.services.dynamodbv2.document.Table;
import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription; import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription;
import org.junit.Assume;
import org.junit.FixMethodOrder; import org.junit.FixMethodOrder;
import org.junit.Test; import org.junit.Test;
import org.junit.internal.AssumptionViolatedException; import org.junit.internal.AssumptionViolatedException;
@ -53,7 +54,9 @@ import org.apache.hadoop.fs.s3a.scale.AbstractITestS3AMetadataStoreScale;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.test.LambdaTestUtils;
import org.apache.hadoop.util.DurationInfo;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.s3guard.MetadataStoreTestBase.basicFileStatus; import static org.apache.hadoop.fs.s3a.s3guard.MetadataStoreTestBase.basicFileStatus;
import static org.junit.Assume.*; import static org.junit.Assume.*;
@ -64,7 +67,10 @@ import static org.junit.Assume.*;
* The throttle tests aren't quite trying to verify that throttling can * The throttle tests aren't quite trying to verify that throttling can
* be recovered from, because that makes for very slow tests: you have * be recovered from, because that makes for very slow tests: you have
* to overload the system and them have them back of until they finally complete. * to overload the system and them have them back of until they finally complete.
* Instead * <p>
* With DDB on demand, throttling is very unlikely.
* Here the tests simply run to completion, so act as regression tests of
* parallel invocations on the metastore APIs
*/ */
@FixMethodOrder(MethodSorters.NAME_ASCENDING) @FixMethodOrder(MethodSorters.NAME_ASCENDING)
public class ITestDynamoDBMetadataStoreScale public class ITestDynamoDBMetadataStoreScale
@ -99,6 +105,8 @@ public class ITestDynamoDBMetadataStoreScale
private static final int OPERATIONS_PER_THREAD = 50; private static final int OPERATIONS_PER_THREAD = 50;
private boolean isOnDemandTable;
/** /**
* Create the metadata store. The table and region are determined from * Create the metadata store. The table and region are determined from
* the attributes of the FS used in the tests. * the attributes of the FS used in the tests.
@ -117,8 +125,7 @@ public class ITestDynamoDBMetadataStoreScale
store instanceof DynamoDBMetadataStore); store instanceof DynamoDBMetadataStore);
DDBCapacities capacities = DDBCapacities.extractCapacities( DDBCapacities capacities = DDBCapacities.extractCapacities(
store.getDiagnostics()); store.getDiagnostics());
assumeTrue("DBB table is on-demand", isOnDemandTable = capacities.isOnDemandTable();
!capacities.isOnDemandTable());
DynamoDBMetadataStore fsStore = (DynamoDBMetadataStore) store; DynamoDBMetadataStore fsStore = (DynamoDBMetadataStore) store;
Configuration conf = new Configuration(fs.getConf()); Configuration conf = new Configuration(fs.getConf());
@ -161,8 +168,6 @@ public class ITestDynamoDBMetadataStoreScale
isOverProvisionedForTest = ( isOverProvisionedForTest = (
originalCapacity.getReadCapacityUnits() > MAXIMUM_READ_CAPACITY originalCapacity.getReadCapacityUnits() > MAXIMUM_READ_CAPACITY
|| originalCapacity.getWriteCapacityUnits() > MAXIMUM_WRITE_CAPACITY); || originalCapacity.getWriteCapacityUnits() > MAXIMUM_WRITE_CAPACITY);
assumeFalse("Table has too much capacity: " + originalCapacity.toString(),
isOverProvisionedForTest);
} }
@Override @Override
@ -171,18 +176,22 @@ public class ITestDynamoDBMetadataStoreScale
super.teardown(); super.teardown();
} }
private boolean expectThrottling() {
return !isOverProvisionedForTest && !isOnDemandTable;
}
/** /**
* The subclass expects the superclass to be throttled; sometimes it is. * The subclass expects the superclass to be throttled; sometimes it is.
*/ */
@Test @Test
@Override @Override
public void test_020_Moves() throws Throwable { public void test_020_Moves() throws Throwable {
ThrottleTracker tracker = new ThrottleTracker(); ThrottleTracker tracker = new ThrottleTracker(ddbms);
try { try {
// if this doesn't throttle, all is well. // if this doesn't throttle, all is well.
super.test_020_Moves(); super.test_020_Moves();
} catch (AWSServiceThrottledException ex) { } catch (AWSServiceThrottledException ex) {
// if the service was throttled, we ex;ect the exception text // if the service was throttled, we expect the exception text
GenericTestUtils.assertExceptionContains( GenericTestUtils.assertExceptionContains(
DynamoDBMetadataStore.HINT_DDB_IOPS_TOO_LOW, DynamoDBMetadataStore.HINT_DDB_IOPS_TOO_LOW,
ex, ex,
@ -221,15 +230,18 @@ public class ITestDynamoDBMetadataStoreScale
try { try {
describe("Running %d iterations of batched put, size %d", iterations, describe("Running %d iterations of batched put, size %d", iterations,
BATCH_SIZE); BATCH_SIZE);
Path base = path(getMethodName());
final String pathKey = base.toUri().getPath();
ThrottleTracker result = execute("prune", ThrottleTracker result = execute("prune",
1, 1,
true, expectThrottling(),
() -> { () -> {
ThrottleTracker tracker = new ThrottleTracker(); ThrottleTracker tracker = new ThrottleTracker(ddbms);
long pruneItems = 0; long pruneItems = 0;
for (long i = 0; i < iterations; i++) { for (long i = 0; i < iterations; i++) {
Path longPath = pathOfDepth(BATCH_SIZE, String.valueOf(i)); Path longPath = pathOfDepth(BATCH_SIZE,
pathKey, String.valueOf(i));
S3AFileStatus status = basicFileStatus(longPath, 0, false, S3AFileStatus status = basicFileStatus(longPath, 0, false,
12345); 12345);
PathMetadata pm = new PathMetadata(status); PathMetadata pm = new PathMetadata(status);
@ -237,14 +249,14 @@ public class ITestDynamoDBMetadataStoreScale
toCleanup.add(pm); toCleanup.add(pm);
} }
ddbms.put(pm); ddbms.put(pm, null);
pruneItems++; pruneItems++;
if (pruneItems == BATCH_SIZE) { if (pruneItems == BATCH_SIZE) {
describe("pruning files"); describe("pruning files");
ddbms.prune(MetadataStore.PruneMode.ALL_BY_MODTIME, ddbms.prune(MetadataStore.PruneMode.ALL_BY_MODTIME,
Long.MAX_VALUE /* all files */); Long.MAX_VALUE, pathKey);
pruneItems = 0; pruneItems = 0;
} }
if (tracker.probe()) { if (tracker.probe()) {
@ -253,8 +265,10 @@ public class ITestDynamoDBMetadataStoreScale
} }
} }
}); });
assertNotEquals("No batch retries in " + result, if (expectThrottling()) {
0, result.batchThrottles); assertNotEquals("No batch retries in " + result,
0, result.getBatchThrottles());
}
} finally { } finally {
describe("Cleaning up table %s", tableName); describe("Cleaning up table %s", tableName);
for (PathMetadata pm : toCleanup) { for (PathMetadata pm : toCleanup) {
@ -274,11 +288,11 @@ public class ITestDynamoDBMetadataStoreScale
Path path = new Path("s3a://example.org/get"); Path path = new Path("s3a://example.org/get");
S3AFileStatus status = new S3AFileStatus(true, path, "alice"); S3AFileStatus status = new S3AFileStatus(true, path, "alice");
PathMetadata metadata = new PathMetadata(status); PathMetadata metadata = new PathMetadata(status);
ddbms.put(metadata); ddbms.put(metadata, null);
try { try {
execute("get", execute("get",
OPERATIONS_PER_THREAD, OPERATIONS_PER_THREAD,
true, expectThrottling(),
() -> ddbms.get(path, true) () -> ddbms.get(path, true)
); );
} finally { } finally {
@ -293,7 +307,7 @@ public class ITestDynamoDBMetadataStoreScale
public void test_050_getVersionMarkerItem() throws Throwable { public void test_050_getVersionMarkerItem() throws Throwable {
execute("get", execute("get",
OPERATIONS_PER_THREAD * 2, OPERATIONS_PER_THREAD * 2,
true, expectThrottling(),
() -> ddbms.getVersionMarkerItem() () -> ddbms.getVersionMarkerItem()
); );
} }
@ -318,12 +332,12 @@ public class ITestDynamoDBMetadataStoreScale
Path path = new Path("s3a://example.org/list"); Path path = new Path("s3a://example.org/list");
S3AFileStatus status = new S3AFileStatus(true, path, "alice"); S3AFileStatus status = new S3AFileStatus(true, path, "alice");
PathMetadata metadata = new PathMetadata(status); PathMetadata metadata = new PathMetadata(status);
ddbms.put(metadata); ddbms.put(metadata, null);
try { try {
Path parent = path.getParent(); Path parent = path.getParent();
execute("list", execute("list",
OPERATIONS_PER_THREAD, OPERATIONS_PER_THREAD,
true, expectThrottling(),
() -> ddbms.listChildren(parent) () -> ddbms.listChildren(parent)
); );
} finally { } finally {
@ -337,14 +351,16 @@ public class ITestDynamoDBMetadataStoreScale
Path path = new Path("s3a://example.org/putDirMarker"); Path path = new Path("s3a://example.org/putDirMarker");
S3AFileStatus status = new S3AFileStatus(true, path, "alice"); S3AFileStatus status = new S3AFileStatus(true, path, "alice");
PathMetadata metadata = new PathMetadata(status); PathMetadata metadata = new PathMetadata(status);
ddbms.put(metadata); ddbms.put(metadata, null);
DirListingMetadata children = ddbms.listChildren(path.getParent()); DirListingMetadata children = ddbms.listChildren(path.getParent());
try { try (DynamoDBMetadataStore.AncestorState state =
ddbms.initiateBulkWrite(
BulkOperationState.OperationType.Put,
path)) {
execute("list", execute("list",
OPERATIONS_PER_THREAD, OPERATIONS_PER_THREAD,
true, expectThrottling(),
() -> ddbms.put(children) () -> ddbms.put(children, state));
);
} finally { } finally {
retryingDelete(path); retryingDelete(path);
} }
@ -356,27 +372,73 @@ public class ITestDynamoDBMetadataStoreScale
Path base = new Path("s3a://example.org/test_080_fullPathsToPut"); Path base = new Path("s3a://example.org/test_080_fullPathsToPut");
Path child = new Path(base, "child"); Path child = new Path(base, "child");
List<PathMetadata> pms = new ArrayList<>(); List<PathMetadata> pms = new ArrayList<>();
ddbms.put(new PathMetadata(makeDirStatus(base)));
ddbms.put(new PathMetadata(makeDirStatus(child)));
ddbms.getInvoker().retry("set up directory tree",
base.toString(),
true,
() -> ddbms.put(pms));
try { try {
DDBPathMetadata dirData = ddbms.get(child, true); try (BulkOperationState bulkUpdate
execute("list", = ddbms.initiateBulkWrite(
OPERATIONS_PER_THREAD, BulkOperationState.OperationType.Put, child)) {
true, ddbms.put(new PathMetadata(makeDirStatus(base)), bulkUpdate);
() -> ddbms.fullPathsToPut(dirData) ddbms.put(new PathMetadata(makeDirStatus(child)), bulkUpdate);
); ddbms.getInvoker().retry("set up directory tree",
base.toString(),
true,
() -> ddbms.put(pms, bulkUpdate));
}
try (BulkOperationState bulkUpdate
= ddbms.initiateBulkWrite(
BulkOperationState.OperationType.Put, child)) {
DDBPathMetadata dirData = ddbms.get(child, true);
execute("put",
OPERATIONS_PER_THREAD,
expectThrottling(),
() -> ddbms.fullPathsToPut(dirData, bulkUpdate)
);
}
} finally { } finally {
retryingDelete(base); ddbms.forgetMetadata(child);
ddbms.forgetMetadata(base);
}
}
/**
* Try many deletes in parallel; this will create tombstones.
*/
@Test
public void test_090_delete() throws Throwable {
Path path = new Path("s3a://example.org/delete");
S3AFileStatus status = new S3AFileStatus(true, path, "alice");
PathMetadata metadata = new PathMetadata(status);
ddbms.put(metadata, null);
ITtlTimeProvider time = checkNotNull(getTtlTimeProvider(), "time provider");
try (DurationInfo ignored = new DurationInfo(LOG, true, "delete")) {
execute("delete",
OPERATIONS_PER_THREAD,
expectThrottling(),
() -> {
ddbms.delete(path, time);
});
}
}
/**
* Forget Metadata: delete entries without tombstones.
*/
@Test
public void test_100_forgetMetadata() throws Throwable {
Path path = new Path("s3a://example.org/delete");
try (DurationInfo ignored = new DurationInfo(LOG, true, "delete")) {
execute("delete",
OPERATIONS_PER_THREAD,
expectThrottling(),
() -> ddbms.forgetMetadata(path)
);
} }
} }
@Test @Test
public void test_900_instrumentation() throws Throwable { public void test_900_instrumentation() throws Throwable {
describe("verify the owner FS gets updated after throttling events"); describe("verify the owner FS gets updated after throttling events");
Assume.assumeTrue("No throttling expected", expectThrottling());
// we rely on the FS being shared // we rely on the FS being shared
S3AFileSystem fs = getFileSystem(); S3AFileSystem fs = getFileSystem();
String fsSummary = fs.toString(); String fsSummary = fs.toString();
@ -411,7 +473,7 @@ public class ITestDynamoDBMetadataStoreScale
throws Exception { throws Exception {
final ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); final ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
final ThrottleTracker tracker = new ThrottleTracker(); final ThrottleTracker tracker = new ThrottleTracker(ddbms);
final ExecutorService executorService = Executors.newFixedThreadPool( final ExecutorService executorService = Executors.newFixedThreadPool(
THREADS); THREADS);
final List<Callable<ExecutionOutcome>> tasks = new ArrayList<>(THREADS); final List<Callable<ExecutionOutcome>> tasks = new ArrayList<>(THREADS);
@ -488,22 +550,21 @@ public class ITestDynamoDBMetadataStoreScale
* @param ms store * @param ms store
* @param pm path to clean up * @param pm path to clean up
*/ */
private void cleanupMetadata(MetadataStore ms, PathMetadata pm) { private void cleanupMetadata(DynamoDBMetadataStore ms, PathMetadata pm) {
Path path = pm.getFileStatus().getPath(); Path path = pm.getFileStatus().getPath();
try { try {
ddbms.getInvoker().retry("clean up", path.toString(), true, ITestDynamoDBMetadataStore.deleteMetadataUnderPath(ms, path, true);
() -> ms.forgetMetadata(path));
} catch (IOException ioe) { } catch (IOException ioe) {
// Ignore. // Ignore.
LOG.info("Ignoring error while cleaning up {} in database", path, ioe); LOG.info("Ignoring error while cleaning up {} in database", path, ioe);
} }
} }
private Path pathOfDepth(long n, @Nullable String fileSuffix) { private Path pathOfDepth(long n,
String name, @Nullable String fileSuffix) {
StringBuilder sb = new StringBuilder(); StringBuilder sb = new StringBuilder();
for (long i = 0; i < n; i++) { for (long i = 0; i < n; i++) {
sb.append(i == 0 ? "/" + this.getClass().getSimpleName() : "lvl"); sb.append(i == 0 ? "/" + name : String.format("level-%03d", i));
sb.append(i);
if (i == n - 1 && fileSuffix != null) { if (i == n - 1 && fileSuffix != null) {
sb.append(fileSuffix); sb.append(fileSuffix);
} }
@ -512,86 +573,6 @@ public class ITestDynamoDBMetadataStoreScale
return new Path(getFileSystem().getUri().toString(), sb.toString()); return new Path(getFileSystem().getUri().toString(), sb.toString());
} }
/**
* Something to track throttles.
* The constructor sets the counters to the current count in the
* DDB table; a call to {@link #reset()} will set it to the latest values.
* The {@link #probe()} will pick up the latest values to compare them with
* the original counts.
*/
private class ThrottleTracker {
private long writeThrottleEventOrig = ddbms.getWriteThrottleEventCount();
private long readThrottleEventOrig = ddbms.getReadThrottleEventCount();
private long batchWriteThrottleCountOrig =
ddbms.getBatchWriteCapacityExceededCount();
private long readThrottles;
private long writeThrottles;
private long batchThrottles;
ThrottleTracker() {
reset();
}
/**
* Reset the counters.
*/
private synchronized void reset() {
writeThrottleEventOrig
= ddbms.getWriteThrottleEventCount();
readThrottleEventOrig
= ddbms.getReadThrottleEventCount();
batchWriteThrottleCountOrig
= ddbms.getBatchWriteCapacityExceededCount();
}
/**
* Update the latest throttle count; synchronized.
* @return true if throttling has been detected.
*/
private synchronized boolean probe() {
readThrottles = ddbms.getReadThrottleEventCount() - readThrottleEventOrig;
writeThrottles = ddbms.getWriteThrottleEventCount()
- writeThrottleEventOrig;
batchThrottles = ddbms.getBatchWriteCapacityExceededCount()
- batchWriteThrottleCountOrig;
return isThrottlingDetected();
}
@Override
public String toString() {
return String.format(
"Tracker with read throttle events = %d;"
+ " write events = %d;"
+ " batch throttles = %d",
readThrottles, writeThrottles, batchThrottles);
}
/**
* Assert that throttling has been detected.
*/
void assertThrottlingDetected() {
assertTrue("No throttling detected in " + this +
" against " + ddbms.toString(),
isThrottlingDetected());
}
/**
* Has there been any throttling on an operation?
* @return true iff read, write or batch operations were throttled.
*/
private boolean isThrottlingDetected() {
return readThrottles > 0 || writeThrottles > 0 || batchThrottles > 0;
}
}
/** /**
* Outcome of a thread's execution operation. * Outcome of a thread's execution operation.
*/ */

View File

@ -29,6 +29,7 @@ import java.util.concurrent.Callable;
import com.amazonaws.services.dynamodbv2.document.DynamoDB; import com.amazonaws.services.dynamodbv2.document.DynamoDB;
import com.amazonaws.services.dynamodbv2.document.Table; import com.amazonaws.services.dynamodbv2.document.Table;
import com.amazonaws.services.dynamodbv2.model.ListTagsOfResourceRequest; import com.amazonaws.services.dynamodbv2.model.ListTagsOfResourceRequest;
import com.amazonaws.services.dynamodbv2.model.ResourceInUseException;
import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException; import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
import com.amazonaws.services.dynamodbv2.model.Tag; import com.amazonaws.services.dynamodbv2.model.Tag;
import org.junit.Assert; import org.junit.Assert;
@ -251,7 +252,9 @@ public class ITestS3GuardToolDynamoDB extends AbstractS3GuardToolTestBase {
try { try {
table.delete(); table.delete();
table.waitForDelete(); table.waitForDelete();
} catch (ResourceNotFoundException e) { /* Ignore */ } } catch (ResourceNotFoundException | ResourceInUseException e) {
/* Ignore */
}
} }
} }
} }

View File

@ -417,7 +417,8 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
String[] fields = line.split("\\s"); String[] fields = line.split("\\s");
if (fields.length == 4 && fields[0].equals(Uploads.TOTAL)) { if (fields.length == 4 && fields[0].equals(Uploads.TOTAL)) {
int parsedUploads = Integer.valueOf(fields[1]); int parsedUploads = Integer.valueOf(fields[1]);
LOG.debug("Matched CLI output: {} {} {} {}", fields); LOG.debug("Matched CLI output: {} {} {} {}",
fields[0], fields[1], fields[2], fields[3]);
assertEquals("Unexpected number of uploads", numUploads, assertEquals("Unexpected number of uploads", numUploads,
parsedUploads); parsedUploads);
return; return;

View File

@ -26,7 +26,7 @@ import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import com.google.common.collect.Sets; import org.assertj.core.api.Assertions;
import org.junit.After; import org.junit.After;
import org.junit.Assume; import org.junit.Assume;
import org.junit.Before; import org.junit.Before;
@ -45,6 +45,7 @@ import org.apache.hadoop.fs.s3a.Tristate;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.test.HadoopTestBase; import org.apache.hadoop.test.HadoopTestBase;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.metadataStorePersistsAuthoritativeBit; import static org.apache.hadoop.fs.s3a.S3ATestUtils.metadataStorePersistsAuthoritativeBit;
/** /**
@ -143,6 +144,17 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
} }
} }
/**
* Describe a test in the logs.
* @param text text to print
* @param args arguments to format in the printing
*/
protected void describe(String text, Object... args) {
LOG.info("\n\n{}: {}\n",
getMethodName(),
String.format(text, args));
}
/** /**
* Helper function for verifying DescendantsIterator and * Helper function for verifying DescendantsIterator and
* MetadataStoreListFilesIterator behavior. * MetadataStoreListFilesIterator behavior.
@ -157,7 +169,7 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
final S3AFileStatus status = pathStr.contains("file") final S3AFileStatus status = pathStr.contains("file")
? basicFileStatus(strToPath(pathStr), 100, false) ? basicFileStatus(strToPath(pathStr), 100, false)
: basicFileStatus(strToPath(pathStr), 0, true); : basicFileStatus(strToPath(pathStr), 0, true);
ms.put(new PathMetadata(status)); ms.put(new PathMetadata(status), null);
} }
final PathMetadata rootMeta = new PathMetadata(makeDirStatus("/")); final PathMetadata rootMeta = new PathMetadata(makeDirStatus("/"));
@ -178,7 +190,9 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
LOG.info("We got {} by iterating DescendantsIterator", actual); LOG.info("We got {} by iterating DescendantsIterator", actual);
if (!allowMissing()) { if (!allowMissing()) {
assertEquals(Sets.newHashSet(checkNodes), actual); Assertions.assertThat(actual)
.as("files listed through DescendantsIterator")
.containsExactlyInAnyOrder(checkNodes);
} }
} }
@ -244,7 +258,7 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
* containing directory. We only track direct children of the directory. * containing directory. We only track direct children of the directory.
* Thus this will not affect entry for /da1. * Thus this will not affect entry for /da1.
*/ */
ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100))); ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100)), null);
assertEmptyDirs("/da2", "/da3"); assertEmptyDirs("/da2", "/da3");
assertDirectorySize("/da1/db1", 1); assertDirectorySize("/da1/db1", 1);
@ -256,7 +270,7 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
} }
/* This already exists, and should silently replace it. */ /* This already exists, and should silently replace it. */
ms.put(new PathMetadata(makeDirStatus("/da1/db1"))); ms.put(new PathMetadata(makeDirStatus("/da1/db1")), null);
/* If we had putNew(), and used it above, this would be empty again. */ /* If we had putNew(), and used it above, this would be empty again. */
assertDirectorySize("/da1", 1); assertDirectorySize("/da1", 1);
@ -264,8 +278,8 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
assertEmptyDirs("/da2", "/da3"); assertEmptyDirs("/da2", "/da3");
/* Ensure new files update correct parent dirs. */ /* Ensure new files update correct parent dirs. */
ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100))); ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100)), null);
ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc2", 200))); ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc2", 200)), null);
assertDirectorySize("/da1", 1); assertDirectorySize("/da1", 1);
assertDirectorySize("/da1/db1", 2); assertDirectorySize("/da1/db1", 2);
assertEmptyDirs("/da2", "/da3"); assertEmptyDirs("/da2", "/da3");
@ -280,14 +294,15 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
public void testPutOverwrite() throws Exception { public void testPutOverwrite() throws Exception {
final String filePath = "/a1/b1/c1/some_file"; final String filePath = "/a1/b1/c1/some_file";
final String dirPath = "/a1/b1/c1/d1"; final String dirPath = "/a1/b1/c1/d1";
ms.put(new PathMetadata(makeFileStatus(filePath, 100))); ms.put(new PathMetadata(makeFileStatus(filePath, 100)), null);
ms.put(new PathMetadata(makeDirStatus(dirPath))); ms.put(new PathMetadata(makeDirStatus(dirPath)), null);
PathMetadata meta = ms.get(strToPath(filePath)); PathMetadata meta = ms.get(strToPath(filePath));
if (!allowMissing() || meta != null) { if (!allowMissing() || meta != null) {
verifyFileStatus(meta.getFileStatus(), 100); verifyFileStatus(meta.getFileStatus(), 100);
} }
ms.put(new PathMetadata(basicFileStatus(strToPath(filePath), 9999, false))); ms.put(new PathMetadata(basicFileStatus(strToPath(filePath), 9999, false)),
null);
meta = ms.get(strToPath(filePath)); meta = ms.get(strToPath(filePath));
if (!allowMissing() || meta != null) { if (!allowMissing() || meta != null) {
verifyFileStatus(meta.getFileStatus(), 9999); verifyFileStatus(meta.getFileStatus(), 9999);
@ -298,15 +313,17 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
public void testRootDirPutNew() throws Exception { public void testRootDirPutNew() throws Exception {
Path rootPath = strToPath("/"); Path rootPath = strToPath("/");
ms.put(new PathMetadata(makeFileStatus("/file1", 100))); ms.put(new PathMetadata(makeFileStatus("/file1", 100)), null);
DirListingMetadata dir = ms.listChildren(rootPath); DirListingMetadata dir = ms.listChildren(rootPath);
if (!allowMissing() || dir != null) { if (!allowMissing() || dir != null) {
assertNotNull("Root dir cached", dir); assertNotNull("Root dir cached", dir);
assertFalse("Root not fully cached", dir.isAuthoritative()); assertFalse("Root not fully cached", dir.isAuthoritative());
assertNotNull("have root dir file listing", dir.getListing()); final Collection<PathMetadata> listing = dir.getListing();
assertEquals("One file in root dir", 1, dir.getListing().size()); Assertions.assertThat(listing)
assertEquals("file1 in root dir", strToPath("/file1"), .describedAs("Root dir listing")
dir.getListing().iterator().next().getFileStatus().getPath()); .isNotNull()
.extracting(p -> p.getFileStatus().getPath())
.containsExactly(strToPath("/file1"));
} }
} }
@ -338,7 +355,8 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
setUpDeleteTest(p); setUpDeleteTest(p);
createNewDirs(p + "/ADirectory1/db1/dc1", p + "/ADirectory1/db1/dc1/dd1"); createNewDirs(p + "/ADirectory1/db1/dc1", p + "/ADirectory1/db1/dc1/dd1");
ms.put(new PathMetadata( ms.put(new PathMetadata(
makeFileStatus(p + "/ADirectory1/db1/dc1/dd1/deepFile", 100))); makeFileStatus(p + "/ADirectory1/db1/dc1/dd1/deepFile", 100)),
null);
if (!allowMissing()) { if (!allowMissing()) {
assertCached(p + "/ADirectory1/db1"); assertCached(p + "/ADirectory1/db1");
} }
@ -388,9 +406,11 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
createNewDirs(prefix + "/ADirectory1", prefix + "/ADirectory2", createNewDirs(prefix + "/ADirectory1", prefix + "/ADirectory2",
prefix + "/ADirectory1/db1"); prefix + "/ADirectory1/db1");
ms.put(new PathMetadata(makeFileStatus(prefix + "/ADirectory1/db1/file1", ms.put(new PathMetadata(makeFileStatus(prefix + "/ADirectory1/db1/file1",
100))); 100)),
null);
ms.put(new PathMetadata(makeFileStatus(prefix + "/ADirectory1/db1/file2", ms.put(new PathMetadata(makeFileStatus(prefix + "/ADirectory1/db1/file2",
100))); 100)),
null);
PathMetadata meta = ms.get(strToPath(prefix + "/ADirectory1/db1/file2")); PathMetadata meta = ms.get(strToPath(prefix + "/ADirectory1/db1/file2"));
if (!allowMissing() || meta != null) { if (!allowMissing() || meta != null) {
@ -403,8 +423,8 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
public void testGet() throws Exception { public void testGet() throws Exception {
final String filePath = "/a1/b1/c1/some_file"; final String filePath = "/a1/b1/c1/some_file";
final String dirPath = "/a1/b1/c1/d1"; final String dirPath = "/a1/b1/c1/d1";
ms.put(new PathMetadata(makeFileStatus(filePath, 100))); ms.put(new PathMetadata(makeFileStatus(filePath, 100)), null);
ms.put(new PathMetadata(makeDirStatus(dirPath))); ms.put(new PathMetadata(makeDirStatus(dirPath)), null);
PathMetadata meta = ms.get(strToPath(filePath)); PathMetadata meta = ms.get(strToPath(filePath));
if (!allowMissing() || meta != null) { if (!allowMissing() || meta != null) {
assertNotNull("Get found file", meta); assertNotNull("Get found file", meta);
@ -532,7 +552,7 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
DirListingMetadata dirMeta = ms.listChildren(strToPath("/a1/b1")); DirListingMetadata dirMeta = ms.listChildren(strToPath("/a1/b1"));
dirMeta.setAuthoritative(true); dirMeta.setAuthoritative(true);
dirMeta.put(makeFileStatus("/a1/b1/file_new", 100)); dirMeta.put(makeFileStatus("/a1/b1/file_new", 100));
ms.put(dirMeta); ms.put(dirMeta, null);
dirMeta = ms.listChildren(strToPath("/a1/b1")); dirMeta = ms.listChildren(strToPath("/a1/b1"));
assertListingsEqual(dirMeta.getListing(), "/a1/b1/file1", "/a1/b1/file2", assertListingsEqual(dirMeta.getListing(), "/a1/b1/file1", "/a1/b1/file2",
@ -590,7 +610,7 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
destMetas.add(new PathMetadata(makeDirStatus("/b1"))); destMetas.add(new PathMetadata(makeDirStatus("/b1")));
destMetas.add(new PathMetadata(makeFileStatus("/b1/file1", 100))); destMetas.add(new PathMetadata(makeFileStatus("/b1/file1", 100)));
destMetas.add(new PathMetadata(makeFileStatus("/b1/file2", 100))); destMetas.add(new PathMetadata(makeFileStatus("/b1/file2", 100)));
ms.move(srcPaths, destMetas, ttlTimeProvider); ms.move(srcPaths, destMetas, ttlTimeProvider, null);
// Assert src is no longer there // Assert src is no longer there
dirMeta = ms.listChildren(strToPath("/a1")); dirMeta = ms.listChildren(strToPath("/a1"));
@ -634,7 +654,7 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
assertNull("Path2 should not be present yet.", meta); assertNull("Path2 should not be present yet.", meta);
// Put p1, assert p2 doesn't match // Put p1, assert p2 doesn't match
ms.put(new PathMetadata(makeFileStatus(p1, 100))); ms.put(new PathMetadata(makeFileStatus(p1, 100)), null);
meta = ms.get(new Path(p2)); meta = ms.get(new Path(p2));
assertNull("Path 2 should not match path 1.", meta); assertNull("Path 2 should not match path 1.", meta);
@ -653,7 +673,8 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
createNewDirs("/pruneFiles"); createNewDirs("/pruneFiles");
long oldTime = getTime(); long oldTime = getTime();
ms.put(new PathMetadata(makeFileStatus("/pruneFiles/old", 1, oldTime))); ms.put(new PathMetadata(makeFileStatus("/pruneFiles/old", 1, oldTime)),
null);
DirListingMetadata ls2 = ms.listChildren(strToPath("/pruneFiles")); DirListingMetadata ls2 = ms.listChildren(strToPath("/pruneFiles"));
if (!allowMissing()) { if (!allowMissing()) {
assertListingsEqual(ls2.getListing(), "/pruneFiles/old"); assertListingsEqual(ls2.getListing(), "/pruneFiles/old");
@ -664,7 +685,8 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
Thread.sleep(1); Thread.sleep(1);
long cutoff = System.currentTimeMillis(); long cutoff = System.currentTimeMillis();
long newTime = getTime(); long newTime = getTime();
ms.put(new PathMetadata(makeFileStatus("/pruneFiles/new", 1, newTime))); ms.put(new PathMetadata(makeFileStatus("/pruneFiles/new", 1, newTime)),
null);
DirListingMetadata ls; DirListingMetadata ls;
ls = ms.listChildren(strToPath("/pruneFiles")); ls = ms.listChildren(strToPath("/pruneFiles"));
@ -695,7 +717,7 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
long oldTime = getTime(); long oldTime = getTime();
ms.put(new PathMetadata(makeFileStatus("/pruneDirs/dir/file", ms.put(new PathMetadata(makeFileStatus("/pruneDirs/dir/file",
1, oldTime))); 1, oldTime)), null);
// It's possible for the Local implementation to get from the old // It's possible for the Local implementation to get from the old
// modification time to here in under 1ms, causing it to not get pruned // modification time to here in under 1ms, causing it to not get pruned
@ -720,16 +742,18 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
long time = System.currentTimeMillis(); long time = System.currentTimeMillis();
ms.put(new PathMetadata( ms.put(new PathMetadata(
basicFileStatus(0, false, 0, time - 1, strToPath(staleFile)), basicFileStatus(0, false, 0, time - 1, strToPath(staleFile)),
Tristate.FALSE, false)); Tristate.FALSE, false),
null);
ms.put(new PathMetadata( ms.put(new PathMetadata(
basicFileStatus(0, false, 0, time + 1, strToPath(freshFile)), basicFileStatus(0, false, 0, time + 1, strToPath(freshFile)),
Tristate.FALSE, false)); Tristate.FALSE, false),
null);
// set parent dir as authoritative // set parent dir as authoritative
if (!allowMissing()) { if (!allowMissing()) {
DirListingMetadata parentDirMd = ms.listChildren(strToPath(parentDir)); DirListingMetadata parentDirMd = ms.listChildren(strToPath(parentDir));
parentDirMd.setAuthoritative(true); parentDirMd.setAuthoritative(true);
ms.put(parentDirMd); ms.put(parentDirMd, null);
} }
ms.prune(MetadataStore.PruneMode.ALL_BY_MODTIME, time); ms.prune(MetadataStore.PruneMode.ALL_BY_MODTIME, time);
@ -757,16 +781,18 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
long time = System.currentTimeMillis(); long time = System.currentTimeMillis();
ms.put(new PathMetadata( ms.put(new PathMetadata(
basicFileStatus(0, false, 0, time + 1, strToPath(staleFile)), basicFileStatus(0, false, 0, time + 1, strToPath(staleFile)),
Tristate.FALSE, false)); Tristate.FALSE, false),
null);
ms.put(new PathMetadata( ms.put(new PathMetadata(
basicFileStatus(0, false, 0, time + 1, strToPath(freshFile)), basicFileStatus(0, false, 0, time + 1, strToPath(freshFile)),
Tristate.FALSE, false)); Tristate.FALSE, false),
null);
if (!allowMissing()) { if (!allowMissing()) {
// set parent dir as authoritative // set parent dir as authoritative
DirListingMetadata parentDirMd = ms.listChildren(strToPath(parentDir)); DirListingMetadata parentDirMd = ms.listChildren(strToPath(parentDir));
parentDirMd.setAuthoritative(true); parentDirMd.setAuthoritative(true);
ms.put(parentDirMd); ms.put(parentDirMd, null);
// prune the ms // prune the ms
ms.prune(MetadataStore.PruneMode.ALL_BY_MODTIME, time); ms.prune(MetadataStore.PruneMode.ALL_BY_MODTIME, time);
@ -798,7 +824,7 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
} }
DirListingMetadata dirMeta = DirListingMetadata dirMeta =
new DirListingMetadata(strToPath(dirPath), metas, authoritative); new DirListingMetadata(strToPath(dirPath), metas, authoritative);
ms.put(dirMeta); ms.put(dirMeta, null);
if (!allowMissing()) { if (!allowMissing()) {
assertDirectorySize(dirPath, filenames.length); assertDirectorySize(dirPath, filenames.length);
@ -818,7 +844,7 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
final S3AFileStatus fileStatus = basicFileStatus(path, 0, false); final S3AFileStatus fileStatus = basicFileStatus(path, 0, false);
PathMetadata pm = new PathMetadata(fileStatus); PathMetadata pm = new PathMetadata(fileStatus);
pm.setIsDeleted(true); pm.setIsDeleted(true);
ms.put(pm); ms.put(pm, null);
if(!allowMissing()) { if(!allowMissing()) {
final PathMetadata pathMetadata = final PathMetadata pathMetadata =
ms.listChildren(path.getParent()).get(path); ms.listChildren(path.getParent()).get(path);
@ -951,8 +977,8 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
private void setupListStatus() throws IOException { private void setupListStatus() throws IOException {
createNewDirs("/a1", "/a2", "/a1/b1", "/a1/b2", "/a1/b1/c1", createNewDirs("/a1", "/a2", "/a1/b1", "/a1/b2", "/a1/b1/c1",
"/a1/b1/c1/d1"); "/a1/b1/c1/d1");
ms.put(new PathMetadata(makeFileStatus("/a1/b1/file1", 100))); ms.put(new PathMetadata(makeFileStatus("/a1/b1/file1", 100)), null);
ms.put(new PathMetadata(makeFileStatus("/a1/b1/file2", 100))); ms.put(new PathMetadata(makeFileStatus("/a1/b1/file2", 100)), null);
} }
private void assertListingsEqual(Collection<PathMetadata> listing, private void assertListingsEqual(Collection<PathMetadata> listing,
@ -966,10 +992,12 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
for (String ps : pathStrs) { for (String ps : pathStrs) {
b.add(strToPath(ps)); b.add(strToPath(ps));
} }
assertEquals("Same set of files", b, a); Assertions.assertThat(a)
.as("Directory Listing")
.containsExactlyInAnyOrderElementsOf(b);
} }
private void putListStatusFiles(String dirPath, boolean authoritative, protected void putListStatusFiles(String dirPath, boolean authoritative,
String... filenames) throws IOException { String... filenames) throws IOException {
ArrayList<PathMetadata> metas = new ArrayList<>(filenames .length); ArrayList<PathMetadata> metas = new ArrayList<>(filenames .length);
for (String filename : filenames) { for (String filename : filenames) {
@ -977,13 +1005,13 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
} }
DirListingMetadata dirMeta = DirListingMetadata dirMeta =
new DirListingMetadata(strToPath(dirPath), metas, authoritative); new DirListingMetadata(strToPath(dirPath), metas, authoritative);
ms.put(dirMeta); ms.put(dirMeta, null);
} }
private void createNewDirs(String... dirs) protected void createNewDirs(String... dirs)
throws IOException { throws IOException {
for (String pathStr : dirs) { for (String pathStr : dirs) {
ms.put(new PathMetadata(makeDirStatus(pathStr))); ms.put(new PathMetadata(makeDirStatus(pathStr)), null);
} }
} }
@ -995,8 +1023,9 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
} }
if (!allowMissing() || dirMeta != null) { if (!allowMissing() || dirMeta != null) {
dirMeta = dirMeta.withoutTombstones(); dirMeta = dirMeta.withoutTombstones();
assertEquals("Number of entries in dir " + pathStr, size, Assertions.assertThat(nonDeleted(dirMeta.getListing()))
nonDeleted(dirMeta.getListing()).size()); .as("files in directory %s", pathStr)
.hasSize(size);
} }
} }
@ -1012,45 +1041,89 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
return currentStatuses; return currentStatuses;
} }
private void assertDeleted(String pathStr) throws IOException { protected PathMetadata get(final String pathStr) throws IOException {
Path path = strToPath(pathStr); Path path = strToPath(pathStr);
PathMetadata meta = ms.get(path); return ms.get(path);
}
protected PathMetadata getNonNull(final String pathStr) throws IOException {
return checkNotNull(get(pathStr), "No metastore entry for %s", pathStr);
}
protected void assertDeleted(String pathStr) throws IOException {
PathMetadata meta = get(pathStr);
boolean cached = meta != null && !meta.isDeleted(); boolean cached = meta != null && !meta.isDeleted();
assertFalse(pathStr + " should not be cached.", cached); assertFalse(pathStr + " should not be cached: " + meta, cached);
} }
protected void assertCached(String pathStr) throws IOException { protected void assertCached(String pathStr) throws IOException {
Path path = strToPath(pathStr); verifyCached(pathStr);
PathMetadata meta = ms.get(path); }
boolean cached = meta != null && !meta.isDeleted();
assertTrue(pathStr + " should be cached.", cached); /**
* Get an entry which must exist and not be deleted.
* @param pathStr path
* @return the entry
* @throws IOException IO failure.
*/
protected PathMetadata verifyCached(final String pathStr) throws IOException {
PathMetadata meta = getNonNull(pathStr);
assertFalse(pathStr + " was found but marked deleted: "+ meta,
meta.isDeleted());
return meta;
}
/**
* Get an entry which must be a file.
* @param pathStr path
* @return the entry
* @throws IOException IO failure.
*/
protected PathMetadata getFile(final String pathStr) throws IOException {
PathMetadata meta = verifyCached(pathStr);
assertFalse(pathStr + " is not a file: " + meta,
meta.getFileStatus().isDirectory());
return meta;
}
/**
* Get an entry which must be a directory.
* @param pathStr path
* @return the entry
* @throws IOException IO failure.
*/
protected PathMetadata getDirectory(final String pathStr) throws IOException {
PathMetadata meta = verifyCached(pathStr);
assertTrue(pathStr + " is not a directory: " + meta,
meta.getFileStatus().isDirectory());
return meta;
} }
/** /**
* Convenience to create a fully qualified Path from string. * Convenience to create a fully qualified Path from string.
*/ */
Path strToPath(String p) throws IOException { protected Path strToPath(String p) throws IOException {
final Path path = new Path(p); final Path path = new Path(p);
assert path.isAbsolute(); assertTrue("Non-absolute path: " + path, path.isAbsolute());
return path.makeQualified(contract.getFileSystem().getUri(), null); return path.makeQualified(contract.getFileSystem().getUri(), null);
} }
private void assertEmptyDirectory(String pathStr) throws IOException { protected void assertEmptyDirectory(String pathStr) throws IOException {
assertDirectorySize(pathStr, 0); assertDirectorySize(pathStr, 0);
} }
private void assertEmptyDirs(String ...dirs) throws IOException { protected void assertEmptyDirs(String...dirs) throws IOException {
for (String pathStr : dirs) { for (String pathStr : dirs) {
assertEmptyDirectory(pathStr); assertEmptyDirectory(pathStr);
} }
} }
S3AFileStatus basicFileStatus(Path path, int size, boolean isDir) throws protected S3AFileStatus basicFileStatus(Path path, int size, boolean isDir)
IOException { throws IOException {
return basicFileStatus(path, size, isDir, modTime); return basicFileStatus(path, size, isDir, modTime);
} }
S3AFileStatus basicFileStatus(int size, boolean isDir, protected S3AFileStatus basicFileStatus(int size, boolean isDir,
long blockSize, long modificationTime, Path path) { long blockSize, long modificationTime, Path path) {
if (isDir) { if (isDir) {
return new S3AFileStatus(Tristate.UNKNOWN, path, null); return new S3AFileStatus(Tristate.UNKNOWN, path, null);
@ -1066,33 +1139,33 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
return new S3AFileStatus(Tristate.UNKNOWN, path, OWNER); return new S3AFileStatus(Tristate.UNKNOWN, path, OWNER);
} else { } else {
return new S3AFileStatus(size, newModTime, path, BLOCK_SIZE, OWNER, return new S3AFileStatus(size, newModTime, path, BLOCK_SIZE, OWNER,
null, null); "etag", "version");
} }
} }
private S3AFileStatus makeFileStatus(String pathStr, int size) throws protected S3AFileStatus makeFileStatus(String pathStr, int size) throws
IOException { IOException {
return makeFileStatus(pathStr, size, modTime); return makeFileStatus(pathStr, size, modTime);
} }
private S3AFileStatus makeFileStatus(String pathStr, int size, protected S3AFileStatus makeFileStatus(String pathStr, int size,
long newModTime) throws IOException { long newModTime) throws IOException {
return basicFileStatus(strToPath(pathStr), size, false, return basicFileStatus(strToPath(pathStr), size, false,
newModTime); newModTime);
} }
void verifyFileStatus(FileStatus status, long size) { protected void verifyFileStatus(FileStatus status, long size) {
S3ATestUtils.verifyFileStatus(status, size, BLOCK_SIZE, modTime); S3ATestUtils.verifyFileStatus(status, size, BLOCK_SIZE, modTime);
} }
private S3AFileStatus makeDirStatus(String pathStr) throws IOException { protected S3AFileStatus makeDirStatus(String pathStr) throws IOException {
return basicFileStatus(strToPath(pathStr), 0, true, modTime); return basicFileStatus(strToPath(pathStr), 0, true, modTime);
} }
/** /**
* Verify the directory file status. Subclass may verify additional fields. * Verify the directory file status. Subclass may verify additional fields.
*/ */
void verifyDirStatus(S3AFileStatus status) { protected void verifyDirStatus(S3AFileStatus status) {
assertTrue("Is a dir", status.isDirectory()); assertTrue("Is a dir", status.isDirectory());
assertEquals("zero length", 0, status.getLen()); assertEquals("zero length", 0, status.getLen());
} }
@ -1113,4 +1186,51 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
return ttlTimeProvider; return ttlTimeProvider;
} }
/**
* Put a file to the shared DDB table.
* @param key key
* @param time timestamp.
* @param operationState ongoing state
* @return the entry
* @throws IOException IO failure
*/
protected PathMetadata putFile(
final String key,
final long time,
BulkOperationState operationState) throws IOException {
PathMetadata meta = new PathMetadata(makeFileStatus(key, 1, time));
ms.put(meta,
operationState);
return meta;
}
/**
* Put a tombstone to the shared DDB table.
* @param key key
* @param time timestamp.
* @param operationState ongoing state
* @return the entry
* @throws IOException IO failure
*/
protected PathMetadata putTombstone(
final String key,
final long time,
BulkOperationState operationState) throws IOException {
PathMetadata meta = tombstone(strToPath(key), time);
ms.put(meta, operationState);
return meta;
}
/**
* Create a tombstone from the timestamp.
* @param path path to tombstone
* @param time timestamp.
* @return the entry.
*/
public static PathMetadata tombstone(Path path, long time) {
S3AFileStatus s3aStatus = new S3AFileStatus(0,
time, path, 0, null,
null, null);
return new PathMetadata(s3aStatus, Tristate.UNKNOWN, true);
}
} }

View File

@ -26,7 +26,9 @@ import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
import com.amazonaws.waiters.WaiterTimedOutException; import com.amazonaws.waiters.WaiterTimedOutException;
import org.junit.Test; import org.junit.Test;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.s3a.AWSClientIOException; import org.apache.hadoop.fs.s3a.AWSClientIOException;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.test.HadoopTestBase; import org.apache.hadoop.test.HadoopTestBase;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -109,4 +111,71 @@ public class TestDynamoDBMiscOperations extends HadoopTestBase {
ddbms.getDirListingMetadataFromDirMetaAndList(p, metas, dirPathMeta)); ddbms.getDirListingMetadataFromDirMetaAndList(p, metas, dirPathMeta));
} }
@Test
public void testAncestorStateForDir() throws Throwable {
final DynamoDBMetadataStore.AncestorState ancestorState
= new DynamoDBMetadataStore.AncestorState(
BulkOperationState.OperationType.Rename, null);
// path 1 is a directory
final Path path1 = new Path("s3a://bucket/1");
final S3AFileStatus status1 = new S3AFileStatus(true,
path1, "hadoop");
final DDBPathMetadata md1 = new DDBPathMetadata(
status1);
ancestorState.put(md1);
assertTrue("Status not found in ancestors",
ancestorState.contains(path1));
final DDBPathMetadata result = ancestorState.get(path1);
assertEquals(status1, result.getFileStatus());
assertTrue("Lookup failed",
ancestorState.findEntry(path1, true));
final Path path2 = new Path("s3a://bucket/2");
assertFalse("Lookup didn't fail",
ancestorState.findEntry(path2, true));
assertFalse("Lookup didn't fail",
ancestorState.contains(path2));
assertNull("Lookup didn't fail",
ancestorState.get(path2));
}
@Test
public void testAncestorStateForFile() throws Throwable {
final DynamoDBMetadataStore.AncestorState ancestorState
= new DynamoDBMetadataStore.AncestorState(
BulkOperationState.OperationType.Rename, null);
// path 1 is a file
final Path path1 = new Path("s3a://bucket/1");
final S3AFileStatus status1 = new S3AFileStatus(
1024_1024_1024L,
0,
path1,
32_000_000,
"hadoop",
"e4",
"f5");
final DDBPathMetadata md1 = new DDBPathMetadata(
status1);
ancestorState.put(md1);
assertTrue("Lookup failed",
ancestorState.findEntry(path1, false));
intercept(PathIOException.class,
DynamoDBMetadataStore.E_INCONSISTENT_UPDATE,
() -> ancestorState.findEntry(path1, true));
}
@Test
public void testNoBulkRenameThroughInitiateBulkWrite() throws Throwable {
intercept(IllegalArgumentException.class,
() -> S3Guard.initiateBulkWrite(null,
BulkOperationState.OperationType.Rename, null));
}
@Test
public void testInitiateBulkWrite() throws Throwable {
assertNull(
S3Guard.initiateBulkWrite(null,
BulkOperationState.OperationType.Put, null));
}
} }

View File

@ -0,0 +1,197 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.s3guard;
import java.util.Comparator;
import java.util.List;
import org.junit.Test;
import org.apache.hadoop.fs.Path;
import static com.google.common.collect.Lists.newArrayList;
import static org.apache.hadoop.fs.s3a.s3guard.PathOrderComparators.TOPMOST_PATH_FIRST;
import static org.apache.hadoop.fs.s3a.s3guard.PathOrderComparators.TOPMOST_PATH_LAST;
import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.Assert.assertEquals;
/**
* Test ordering of paths with the comparator matches requirements.
*/
public class TestPathOrderComparators {
private static final Path ROOT = new Path("s3a://bucket/");
public static final Path DIR_A = new Path(ROOT, "dirA");
public static final Path DIR_B = new Path(ROOT, "dirB");
public static final Path DIR_A_FILE_1 = new Path(DIR_A, "file1");
public static final Path DIR_A_FILE_2 = new Path(DIR_A, "file2");
public static final Path DIR_B_FILE_3 = new Path(DIR_B, "file3");
public static final Path DIR_B_FILE_4 = new Path(DIR_B, "file4");
@Test
public void testRootEqual() throws Throwable {
assertComparesEqual(ROOT, ROOT);
}
@Test
public void testRootFirst() throws Throwable {
assertComparesTopmost(ROOT, DIR_A_FILE_1);
}
@Test
public void testDirOrdering() throws Throwable {
assertComparesTopmost(DIR_A, DIR_B);
}
@Test
public void testFilesEqual() throws Throwable {
assertComparesEqual(DIR_A_FILE_1, DIR_A_FILE_1);
}
@Test
public void testFilesInSameDir() throws Throwable {
assertComparesTopmost(ROOT, DIR_A_FILE_1);
assertComparesTopmost(DIR_A, DIR_A_FILE_1);
assertComparesTopmost(DIR_A, DIR_A_FILE_2);
assertComparesTopmost(DIR_A_FILE_1, DIR_A_FILE_2);
}
@Test
public void testReversedFiles() throws Throwable {
assertReverseOrder(DIR_A_FILE_1, ROOT);
assertReverseOrder(DIR_A_FILE_1, DIR_A);
assertReverseOrder(DIR_A_FILE_2, DIR_A);
assertReverseOrder(DIR_A_FILE_2, DIR_A_FILE_1);
}
@Test
public void testFilesAndDifferentShallowDir() throws Throwable {
assertComparesTopmost(DIR_B, DIR_A_FILE_1);
assertComparesTopmost(DIR_A, DIR_B_FILE_3);
}
@Test
public void testOrderRoot() throws Throwable {
verifySorted(ROOT);
}
@Test
public void testOrderRootDirs() throws Throwable {
verifySorted(ROOT, DIR_A, DIR_B);
}
@Test
public void testOrderRootDirsAndFiles() throws Throwable {
verifySorted(ROOT, DIR_A, DIR_B, DIR_A_FILE_1, DIR_A_FILE_2);
}
@Test
public void testOrderRootDirsAndAllFiles() throws Throwable {
verifySorted(ROOT, DIR_A, DIR_B,
DIR_A_FILE_1, DIR_A_FILE_2,
DIR_B_FILE_3, DIR_B_FILE_4);
}
@Test
public void testSortOrderConstant() throws Throwable {
List<Path> sort1 = verifySorted(ROOT, DIR_A, DIR_B,
DIR_A_FILE_1, DIR_A_FILE_2,
DIR_B_FILE_3, DIR_B_FILE_4);
List<Path> sort2 = newArrayList(sort1);
assertSortsTo(sort2, sort1, true);
}
@Test
public void testSortReverse() throws Throwable {
List<Path> sort1 = newArrayList(
ROOT,
DIR_A,
DIR_B,
DIR_A_FILE_1,
DIR_A_FILE_2,
DIR_B_FILE_3,
DIR_B_FILE_4);
List<Path> expected = newArrayList(
DIR_B_FILE_4,
DIR_B_FILE_3,
DIR_A_FILE_2,
DIR_A_FILE_1,
DIR_B,
DIR_A,
ROOT);
assertSortsTo(expected, sort1, false);
}
private List<Path> verifySorted(Path... paths) {
List<Path> original = newArrayList(paths);
List<Path> sorted = newArrayList(paths);
assertSortsTo(original, sorted, true);
return sorted;
}
private void assertSortsTo(
final List<Path> original,
final List<Path> sorted,
boolean topmost) {
sorted.sort(topmost ? TOPMOST_PATH_FIRST : TOPMOST_PATH_LAST);
assertThat(sorted)
.as("Sorted paths")
.containsExactlyElementsOf(original);
}
private void assertComparesEqual(Path l, Path r) {
assertOrder(0, l, r);
}
private void assertComparesTopmost(Path l, Path r) {
assertOrder(-1, l, r);
assertOrder(1, r, l);
}
private void assertReverseOrder(Path l, Path r) {
assertComparesTo(-1, TOPMOST_PATH_LAST, l, r);
assertComparesTo(1, TOPMOST_PATH_LAST, r, l);
}
private void assertOrder(int res,
Path l, Path r) {
assertComparesTo(res, TOPMOST_PATH_FIRST, l, r);
}
private void assertComparesTo(final int expected,
final Comparator<Path> comparator,
final Path l, final Path r) {
int actual = comparator.compare(l, r);
if (actual < -1) {
actual = -1;
}
if (actual > 1) {
actual = 1;
}
assertEquals("Comparing " + l + " to " + r,
expected, actual);
}
}

View File

@ -91,12 +91,12 @@ public class TestS3Guard extends Assert {
when(timeProvider.getNow()).thenReturn(100L); when(timeProvider.getNow()).thenReturn(100L);
// act // act
S3Guard.putWithTtl(ms, dlm, timeProvider); S3Guard.putWithTtl(ms, dlm, timeProvider, null);
// assert // assert
assertEquals("last update in " + dlm, 100L, dlm.getLastUpdated()); assertEquals("last update in " + dlm, 100L, dlm.getLastUpdated());
verify(timeProvider, times(1)).getNow(); verify(timeProvider, times(1)).getNow();
verify(ms, times(1)).put(dlm); verify(ms, times(1)).put(dlm, null);
} }
@Test @Test
@ -111,12 +111,12 @@ public class TestS3Guard extends Assert {
when(timeProvider.getNow()).thenReturn(100L); when(timeProvider.getNow()).thenReturn(100L);
// act // act
S3Guard.putWithTtl(ms, pm, timeProvider); S3Guard.putWithTtl(ms, pm, timeProvider, null);
// assert // assert
assertEquals("last update in " + pm, 100L, pm.getLastUpdated()); assertEquals("last update in " + pm, 100L, pm.getLastUpdated());
verify(timeProvider, times(1)).getNow(); verify(timeProvider, times(1)).getNow();
verify(ms, times(1)).put(pm); verify(ms, times(1)).put(pm, null);
} }
@Test @Test
@ -134,14 +134,14 @@ public class TestS3Guard extends Assert {
when(timeProvider.getNow()).thenReturn(100L); when(timeProvider.getNow()).thenReturn(100L);
// act // act
S3Guard.putWithTtl(ms, pmCollection, timeProvider); S3Guard.putWithTtl(ms, pmCollection, timeProvider, null);
// assert // assert
pmCollection.forEach( pmCollection.forEach(
pm -> assertEquals(100L, pm.getLastUpdated()) pm -> assertEquals(100L, pm.getLastUpdated())
); );
verify(timeProvider, times(1)).getNow(); verify(timeProvider, times(1)).getNow();
verify(ms, times(1)).put(pmCollection); verify(ms, times(1)).put(pmCollection, null);
} }
@Test @Test

View File

@ -0,0 +1,134 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.s3guard;
import org.junit.Assert;
/**
* Something to track throttles in DynamoDB metastores.
* The constructor sets the counters to the current count in the
* DDB table; a call to {@link #reset()} will set it to the latest values.
* The {@link #probe()} will pick up the latest values to compare them with
* the original counts.
* <p>
* The toString value logs the state.
* <p>
* This class was originally part of ITestDynamoDBMetadataStoreScale;
* it was converted to a toplevel class for broader use.
*/
class ThrottleTracker {
private final DynamoDBMetadataStore ddbms;
private long writeThrottleEventOrig = 0;
private long readThrottleEventOrig = 0;
private long batchWriteThrottleCountOrig = 0;
private long readThrottles;
private long writeThrottles;
private long batchThrottles;
ThrottleTracker(final DynamoDBMetadataStore ddbms) {
this.ddbms = ddbms;
reset();
}
/**
* Reset the counters.
*/
public synchronized void reset() {
writeThrottleEventOrig
= ddbms.getWriteThrottleEventCount();
readThrottleEventOrig
= ddbms.getReadThrottleEventCount();
batchWriteThrottleCountOrig
= ddbms.getBatchWriteCapacityExceededCount();
}
/**
* Update the latest throttle count; synchronized.
* @return true if throttling has been detected.
*/
public synchronized boolean probe() {
setReadThrottles(
ddbms.getReadThrottleEventCount() - readThrottleEventOrig);
setWriteThrottles(ddbms.getWriteThrottleEventCount()
- writeThrottleEventOrig);
setBatchThrottles(ddbms.getBatchWriteCapacityExceededCount()
- batchWriteThrottleCountOrig);
return isThrottlingDetected();
}
@Override
public String toString() {
return String.format(
"Tracker with read throttle events = %d;"
+ " write events = %d;"
+ " batch throttles = %d",
getReadThrottles(), getWriteThrottles(), getBatchThrottles());
}
/**
* Assert that throttling has been detected.
*/
public void assertThrottlingDetected() {
Assert.assertTrue("No throttling detected in " + this +
" against " + ddbms.toString(),
isThrottlingDetected());
}
/**
* Has there been any throttling on an operation?
* @return true iff read, write or batch operations were throttled.
*/
public boolean isThrottlingDetected() {
return getReadThrottles() > 0 || getWriteThrottles()
> 0 || getBatchThrottles() > 0;
}
public long getReadThrottles() {
return readThrottles;
}
public void setReadThrottles(long readThrottles) {
this.readThrottles = readThrottles;
}
public long getWriteThrottles() {
return writeThrottles;
}
public void setWriteThrottles(long writeThrottles) {
this.writeThrottles = writeThrottles;
}
public long getBatchThrottles() {
return batchThrottles;
}
public void setBatchThrottles(long batchThrottles) {
this.batchThrottles = batchThrottles;
}
}

View File

@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider; import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
import org.apache.hadoop.fs.s3a.s3guard.PathMetadata; import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
import org.apache.hadoop.fs.s3a.s3guard.S3Guard; import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
@ -73,6 +74,10 @@ public abstract class AbstractITestS3AMetadataStoreScale extends
*/ */
public abstract MetadataStore createMetadataStore() throws IOException; public abstract MetadataStore createMetadataStore() throws IOException;
protected ITtlTimeProvider getTtlTimeProvider() {
return ttlTimeProvider;
}
@Test @Test
public void test_010_Put() throws Throwable { public void test_010_Put() throws Throwable {
describe("Test workload of put() operations"); describe("Test workload of put() operations");
@ -139,13 +144,15 @@ public abstract class AbstractITestS3AMetadataStoreScale extends
toDelete = movedPaths; toDelete = movedPaths;
toCreate = origMetas; toCreate = origMetas;
} }
ms.move(toDelete, toCreate, ttlTimeProvider); ms.move(toDelete, toCreate, ttlTimeProvider, null);
} }
moveTimer.end(); moveTimer.end();
printTiming(LOG, "move", moveTimer, operations); printTiming(LOG, "move", moveTimer, operations);
} finally { } finally {
// Cleanup // Cleanup
clearMetadataStore(ms, count); clearMetadataStore(ms, count);
ms.move(origPaths, null, ttlTimeProvider, null);
ms.move(movedPaths, null, ttlTimeProvider, null);
} }
} }
} }
@ -191,9 +198,13 @@ public abstract class AbstractITestS3AMetadataStoreScale extends
long count = 0; long count = 0;
NanoTimer putTimer = new NanoTimer(); NanoTimer putTimer = new NanoTimer();
describe("Inserting into MetadataStore"); describe("Inserting into MetadataStore");
for (PathMetadata p : paths) { try (BulkOperationState operationState =
ms.put(p); ms.initiateBulkWrite(BulkOperationState.OperationType.Put,
count++; BUCKET_ROOT)) {
for (PathMetadata p : paths) {
ms.put(p, operationState);
count++;
}
} }
putTimer.end(); putTimer.end();
printTiming(LOG, "put", putTimer, count); printTiming(LOG, "put", putTimer, count);

View File

@ -0,0 +1,138 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.s3a.test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.stream.Collectors;
import org.junit.Assert;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.util.DurationInfo;
import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles;
import static org.junit.Assert.assertTrue;
/**
* Some extra assertions for tests.
*/
@InterfaceAudience.Private
public final class ExtraAssertions {
private static final Logger LOG = LoggerFactory.getLogger(
ExtraAssertions.class);
private ExtraAssertions() {
}
/**
* Assert that the number of files in a destination matches that expected.
* @param message text to use in the message
* @param fs filesystem
* @param path path to list (recursively)
* @param expected expected count
* @throws IOException IO problem
*/
public static void assertFileCount(final String message,
final FileSystem fs,
final Path path,
final long expected)
throws IOException {
List<String> files = new ArrayList<>();
try (DurationInfo ignored = new DurationInfo(LOG, false,
"Counting files in %s", path)) {
applyLocatedFiles(fs.listFiles(path, true),
(status) -> files.add(status.getPath().toString()));
}
long actual = files.size();
if (actual != expected) {
String ls = files.stream().collect(Collectors.joining("\n"));
Assert.fail(message + ": expected " + expected + " files in " + path
+ " but got " + actual + "\n" + ls);
}
}
/**
* Assert that a string contains a piece of text.
* @param text text to can.
* @param contained text to look for.
*/
public static void assertTextContains(String text, String contained) {
assertTrue("string \"" + contained + "\" not found in \"" + text + "\"",
text != null && text.contains(contained));
}
/**
* If the condition is met, throw an AssertionError with the message
* and any nested exception.
* @param condition condition
* @param message text to use in the exception
* @param cause a (possibly null) throwable to init the cause with
* @throws AssertionError with the text and throwable if condition == true.
*/
public static void failIf(boolean condition,
String message,
Throwable cause) {
if (condition) {
ContractTestUtils.fail(message, cause);
}
}
/**
* If the condition is met, throw an AssertionError with the message
* and any nested exception.
* @param condition condition
* @param message text to use in the exception
* @param cause a (possibly null) throwable to init the cause with
* @throws AssertionError with the text and throwable if condition == true.
*/
public static void failUnless(boolean condition,
String message,
Throwable cause) {
failIf(!condition, message, cause);
}
/**
* Extract the inner cause of an exception.
* @param expected expected class of the cause
* @param thrown thrown exception.
* @param <T> type of the cause
* @return the extracted exception.
* @throws AssertionError with the text and throwable if the cause is not as
* expected
*/
public static <T extends Throwable> T extractCause(Class<T> expected,
Throwable thrown) {
Throwable cause = thrown.getCause();
failIf(cause == null,
"No inner cause",
thrown);
failUnless(cause.getClass().equals(expected),
"Inner cause is of wrong type : expected " + expected,
thrown);
return (T)cause;
}
}