HADOOP-13230. S3A to optionally retain directory markers.

This adds an option to disable "empty directory" marker deletion,
so avoid throttling and other scale problems.

This feature is *not* backwards compatible.
Consult the documentation and use with care.

Contributed by Steve Loughran.

Change-Id: I69a61e7584dc36e485d5e39ff25b1e3e559a1958
This commit is contained in:
Steve Loughran 2020-08-15 12:51:08 +01:00
parent 5ededc0cba
commit 49f8ae965e
No known key found for this signature in database
GPG Key ID: D22CF846DBB162A0
53 changed files with 7845 additions and 924 deletions

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.test;
import java.util.concurrent.Callable;
import org.assertj.core.description.Description;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Extra classes to work with AssertJ.
* These are kept separate from {@link LambdaTestUtils} so there's
* no requirement for AssertJ to be on the classpath in that broadly
* used class.
*/
public final class AssertExtensions {
private static final Logger LOG =
LoggerFactory.getLogger(AssertExtensions.class);
private AssertExtensions() {
}
/**
* A description for AssertJ "describedAs" clauses which evaluates the
* lambda-expression only on failure. That must return a string
* or null/"" to be skipped.
* @param eval lambda expression to invoke
* @return a description for AssertJ
*/
public static Description dynamicDescription(Callable<String> eval) {
return new DynamicDescription(eval);
}
private static final class DynamicDescription extends Description {
private final Callable<String> eval;
private DynamicDescription(final Callable<String> eval) {
this.eval = eval;
}
@Override
public String value() {
try {
return eval.call();
} catch (Exception e) {
LOG.warn("Failed to evaluate description: " + e);
LOG.debug("Evaluation failure", e);
// return null so that the description evaluation chain
// will skip this one
return null;
}
}
}
}

View File

@ -51,6 +51,10 @@
<!-- Set a longer timeout for integration test (in milliseconds) -->
<test.integration.timeout>200000</test.integration.timeout>
<!-- should directory marker retention be audited? -->
<fs.s3a.directory.marker.audit>false</fs.s3a.directory.marker.audit>
<!-- marker retention policy -->
<fs.s3a.directory.marker.retention></fs.s3a.directory.marker.retention>
</properties>
<profiles>
@ -122,6 +126,9 @@
<fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
<fs.s3a.scale.test.huge.huge.partitionsize>${fs.s3a.scale.test.huge.partitionsize}</fs.s3a.scale.test.huge.huge.partitionsize>
<fs.s3a.scale.test.timeout>${fs.s3a.scale.test.timeout}</fs.s3a.scale.test.timeout>
<!-- Markers-->
<fs.s3a.directory.marker.retention>${fs.s3a.directory.marker.retention}</fs.s3a.directory.marker.retention>
<fs.s3a.directory.marker.audit>${fs.s3a.directory.marker.audit}</fs.s3a.directory.marker.audit>
</systemPropertyVariables>
</configuration>
</plugin>
@ -162,6 +169,7 @@
<fs.s3a.s3guard.test.enabled>${fs.s3a.s3guard.test.enabled}</fs.s3a.s3guard.test.enabled>
<fs.s3a.s3guard.test.authoritative>${fs.s3a.s3guard.test.authoritative}</fs.s3a.s3guard.test.authoritative>
<fs.s3a.s3guard.test.implementation>${fs.s3a.s3guard.test.implementation}</fs.s3a.s3guard.test.implementation>
<fs.s3a.directory.marker.retention>${fs.s3a.directory.marker.retention}</fs.s3a.directory.marker.retention>
<test.default.timeout>${test.integration.timeout}</test.default.timeout>
</systemPropertyVariables>
@ -188,6 +196,8 @@
<exclude>**/ITestDynamoDBMetadataStoreScale.java</exclude>
<!-- Terasort MR jobs spawn enough processes that they use up all RAM -->
<exclude>**/ITestTerasort*.java</exclude>
<!-- Root marker tool tests -->
<exclude>**/ITestMarkerToolRootOperations.java</exclude>
<!-- operations across the metastore -->
<exclude>**/ITestS3GuardDDBRootOperations.java</exclude>
</excludes>
@ -214,6 +224,9 @@
<fs.s3a.s3guard.test.enabled>${fs.s3a.s3guard.test.enabled}</fs.s3a.s3guard.test.enabled>
<fs.s3a.s3guard.test.implementation>${fs.s3a.s3guard.test.implementation}</fs.s3a.s3guard.test.implementation>
<fs.s3a.s3guard.test.authoritative>${fs.s3a.s3guard.test.authoritative}</fs.s3a.s3guard.test.authoritative>
<!-- Markers-->
<fs.s3a.directory.marker.retention>${fs.s3a.directory.marker.retention}</fs.s3a.directory.marker.retention>
<fs.s3a.directory.marker.audit>${fs.s3a.directory.marker.audit}</fs.s3a.directory.marker.audit>
</systemPropertyVariables>
<!-- Do a sequential run for tests that cannot handle -->
<!-- parallel execution. -->
@ -229,6 +242,10 @@
<!-- the local FS. Running them sequentially guarantees isolation -->
<!-- and that they don't conflict with the other MR jobs for RAM -->
<include>**/ITestTerasort*.java</include>
<!-- Root marker tool tests -->
<!-- MUST be run before the other root ops so there's
more likelihood of files in the bucket -->
<include>**/ITestMarkerToolRootOperations.java</include>
<!-- operations across the metastore -->
<include>**/ITestS3AContractRootDir.java</include>
<include>**/ITestS3GuardDDBRootOperations.java</include>
@ -268,6 +285,9 @@
<fs.s3a.s3guard.test.enabled>${fs.s3a.s3guard.test.enabled}</fs.s3a.s3guard.test.enabled>
<fs.s3a.s3guard.test.implementation>${fs.s3a.s3guard.test.implementation}</fs.s3a.s3guard.test.implementation>
<fs.s3a.s3guard.test.authoritative>${fs.s3a.s3guard.test.authoritative}</fs.s3a.s3guard.test.authoritative>
<!-- Markers-->
<fs.s3a.directory.marker.retention>${fs.s3a.directory.marker.retention}</fs.s3a.directory.marker.retention>
<fs.s3a.directory.marker.audit>${fs.s3a.directory.marker.audit}</fs.s3a.directory.marker.audit>
</systemPropertyVariables>
<forkedProcessTimeoutInSeconds>${fs.s3a.scale.test.timeout}</forkedProcessTimeoutInSeconds>
</configuration>
@ -331,6 +351,44 @@
</properties>
</profile>
<!-- Directory marker retention options, all from the -Dmarkers value-->
<profile>
<id>keep-markers</id>
<activation>
<property>
<name>markers</name>
<value>keep</value>
</property>
</activation>
<properties >
<fs.s3a.directory.marker.retention>keep</fs.s3a.directory.marker.retention>
</properties>
</profile>
<profile>
<id>delete-markers</id>
<activation>
<property>
<name>markers</name>
<value>delete</value>
</property>
</activation>
<properties >
<fs.s3a.directory.marker.retention>delete</fs.s3a.directory.marker.retention>
</properties>
</profile>
<profile>
<id>auth-markers</id>
<activation>
<property>
<name>markers</name>
<value>authoritative</value>
</property>
</activation>
<properties >
<fs.s3a.directory.marker.retention>authoritative</fs.s3a.directory.marker.retention>
</properties>
</profile>
</profiles>
<build>

View File

@ -953,4 +953,92 @@ public final class Constants {
* Value: {@value} seconds.
*/
public static final int THREAD_POOL_SHUTDOWN_DELAY_SECONDS = 30;
/**
* Policy for directory markers.
* This is a new feature of HADOOP-13230 which addresses
* some scale, performance and permissions issues -but
* at the risk of backwards compatibility.
*/
public static final String DIRECTORY_MARKER_POLICY =
"fs.s3a.directory.marker.retention";
/**
* Delete directory markers. This is the backwards compatible option.
* Value: {@value}.
*/
public static final String DIRECTORY_MARKER_POLICY_DELETE =
"delete";
/**
* Retain directory markers.
* Value: {@value}.
*/
public static final String DIRECTORY_MARKER_POLICY_KEEP =
"keep";
/**
* Retain directory markers in authoritative directory trees only.
* Value: {@value}.
*/
public static final String DIRECTORY_MARKER_POLICY_AUTHORITATIVE =
"authoritative";
/**
* Default retention policy: {@value}.
*/
public static final String DEFAULT_DIRECTORY_MARKER_POLICY =
DIRECTORY_MARKER_POLICY_DELETE;
/**
* {@code PathCapabilities} probe to verify that an S3A Filesystem
* has the changes needed to safely work with buckets where
* directoy markers have not been deleted.
* Value: {@value}.
*/
public static final String STORE_CAPABILITY_DIRECTORY_MARKER_AWARE
= "fs.s3a.capability.directory.marker.aware";
/**
* {@code PathCapabilities} probe to indicate that the filesystem
* keeps directory markers.
* Value: {@value}.
*/
public static final String STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_KEEP
= "fs.s3a.capability.directory.marker.policy.keep";
/**
* {@code PathCapabilities} probe to indicate that the filesystem
* deletes directory markers.
* Value: {@value}.
*/
public static final String STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_DELETE
= "fs.s3a.capability.directory.marker.policy.delete";
/**
* {@code PathCapabilities} probe to indicate that the filesystem
* keeps directory markers in authoritative paths only.
* Value: {@value}.
*/
public static final String
STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_AUTHORITATIVE =
"fs.s3a.capability.directory.marker.policy.authoritative";
/**
* {@code PathCapabilities} probe to indicate that a path
* keeps directory markers.
* Value: {@value}.
*/
public static final String STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP
= "fs.s3a.capability.directory.marker.action.keep";
/**
* {@code PathCapabilities} probe to indicate that a path
* deletes directory markers.
* Value: {@value}.
*/
public static final String STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE
= "fs.s3a.capability.directory.marker.action.delete";
}

View File

@ -330,7 +330,9 @@ public class InconsistentAmazonS3Client extends AmazonS3Client {
} else {
Path actualParentPath = new Path(child).getParent();
Path expectedParentPath = new Path(parent);
return actualParentPath.equals(expectedParentPath);
// children which are directory markers are excluded here
return actualParentPath.equals(expectedParentPath)
&& !child.endsWith("/");
}
}

View File

@ -142,12 +142,27 @@ public class Listing extends AbstractStoreOperation {
Listing.FileStatusAcceptor acceptor,
RemoteIterator<S3AFileStatus> providedStatus) throws IOException {
return new FileStatusListingIterator(
new ObjectListingIterator(listPath, request),
createObjectListingIterator(listPath, request),
filter,
acceptor,
providedStatus);
}
/**
* Create an object listing iterator against a path, with a given
* list object request.
* @param listPath path of the listing
* @param request initial request to make
* @return the iterator
* @throws IOException IO Problems
*/
@Retries.RetryRaw
public ObjectListingIterator createObjectListingIterator(
final Path listPath,
final S3ListRequest request) throws IOException {
return new ObjectListingIterator(listPath, request);
}
/**
* Create a located status iterator over a file status iterator.
* @param statusIterator an iterator over the remote status entries
@ -194,8 +209,12 @@ public class Listing extends AbstractStoreOperation {
String key = maybeAddTrailingSlash(pathToKey(path));
String delimiter = recursive ? null : "/";
LOG.debug("Requesting all entries under {} with delimiter '{}'",
key, delimiter);
if (recursive) {
LOG.debug("Recursive list of all entries under {}", key);
} else {
LOG.debug("Requesting all entries under {} with delimiter '{}'",
key, delimiter);
}
final RemoteIterator<S3AFileStatus> cachedFilesIterator;
final Set<Path> tombstones;
boolean allowAuthoritative = listingOperationCallbacks

View File

@ -67,7 +67,7 @@ import com.amazonaws.services.s3.model.MultipartUpload;
import com.amazonaws.services.s3.model.ObjectMetadata;
import com.amazonaws.services.s3.model.PutObjectRequest;
import com.amazonaws.services.s3.model.PutObjectResult;
import com.amazonaws.services.s3.model.S3ObjectSummary;
import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams;
import com.amazonaws.services.s3.model.SSECustomerKey;
import com.amazonaws.services.s3.model.UploadPartRequest;
@ -104,6 +104,8 @@ import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
import org.apache.hadoop.fs.s3a.impl.ContextAccessors;
import org.apache.hadoop.fs.s3a.impl.CopyOutcome;
import org.apache.hadoop.fs.s3a.impl.DeleteOperation;
import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy;
import org.apache.hadoop.fs.s3a.impl.DirectoryPolicyImpl;
import org.apache.hadoop.fs.s3a.impl.InternalConstants;
import org.apache.hadoop.fs.s3a.impl.ListingOperationCallbacks;
import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport;
@ -116,6 +118,8 @@ import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder;
import org.apache.hadoop.fs.s3a.impl.statistics.S3AMultipartUploaderStatisticsImpl;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
import org.apache.hadoop.fs.s3a.select.InternalSelectConstants;
import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations;
import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.token.DelegationTokenIssuer;
@ -295,6 +299,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
private final ListingOperationCallbacks listingOperationCallbacks =
new ListingOperationCallbacksImpl();
/**
* Directory policy.
*/
private DirectoryPolicy directoryPolicy;
/**
* Context accessors for re-use.
*/
private final ContextAccessors contextAccessors = new ContextAccessorsImpl();
/** Add any deprecated keys. */
@SuppressWarnings("deprecation")
@ -452,6 +465,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
DEFAULT_S3GUARD_DISABLED_WARN_LEVEL);
S3Guard.logS3GuardDisabled(LOG, warnLevel, bucket);
}
// directory policy, which may look at authoritative paths
directoryPolicy = DirectoryPolicyImpl.getDirectoryPolicy(conf,
this::allowAuthoritative);
LOG.debug("Directory marker retention policy is {}", directoryPolicy);
initMultipartUploads(conf);
@ -1285,7 +1302,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
* is not a directory.
*/
@Override
public FSDataOutputStream createNonRecursive(Path path,
public FSDataOutputStream createNonRecursive(Path p,
FsPermission permission,
EnumSet<CreateFlag> flags,
int bufferSize,
@ -1293,10 +1310,22 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
long blockSize,
Progressable progress) throws IOException {
entryPoint(INVOCATION_CREATE_NON_RECURSIVE);
final Path path = makeQualified(p);
Path parent = path.getParent();
if (parent != null) {
// expect this to raise an exception if there is no parent
if (!getFileStatus(parent).isDirectory()) {
// expect this to raise an exception if there is no parent dir
if (parent != null && !parent.isRoot()) {
S3AFileStatus status;
try {
// optimize for the directory existing: Call list first
status = innerGetFileStatus(parent, false,
StatusProbeEnum.DIRECTORIES);
} catch (FileNotFoundException e) {
// no dir, fall back to looking for a file
// (failure condition if true)
status = innerGetFileStatus(parent, false,
StatusProbeEnum.HEAD_ONLY);
}
if (!status.isDirectory()) {
throw new FileAlreadyExistsException("Not a directory: " + parent);
}
}
@ -1431,10 +1460,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
LOG.debug("rename: destination path {} not found", dst);
// Parent must exist
Path parent = dst.getParent();
if (!pathToKey(parent).isEmpty()) {
if (!pathToKey(parent).isEmpty()
&& !parent.equals(src.getParent())) {
try {
S3AFileStatus dstParentStatus = innerGetFileStatus(dst.getParent(),
false, StatusProbeEnum.ALL);
// only look against S3 for directories; saves
// a HEAD request on all normal operations.
S3AFileStatus dstParentStatus = innerGetFileStatus(parent,
false, StatusProbeEnum.DIRECTORIES);
if (!dstParentStatus.isDirectory()) {
throw new RenameFailedException(src, dst,
"destination parent is not a directory");
@ -1535,7 +1567,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
final boolean isFile,
final BulkOperationState operationState)
throws IOException {
once("delete", key, () ->
once("delete", path.toString(), () ->
S3AFileSystem.this.deleteObjectAtPath(path, key, isFile,
operationState));
}
@ -1585,7 +1617,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
Path destParent = destCreated.getParent();
if (!sourceRenamed.getParent().equals(destParent)) {
LOG.debug("source & dest parents are different; fix up dir markers");
deleteUnnecessaryFakeDirectories(destParent);
if (!keepDirectoryMarkers(destParent)) {
deleteUnnecessaryFakeDirectories(destParent, null);
}
maybeCreateFakeParentDirectory(sourceRenamed);
}
}
@ -1940,6 +1974,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
protected S3ListResult listObjects(S3ListRequest request) throws IOException {
incrementReadOperations();
incrementStatistic(OBJECT_LIST_REQUESTS);
LOG.debug("LIST {}", request);
validateListArguments(request);
try(DurationInfo ignored =
new DurationInfo(LOG, false, "LIST")) {
@ -2381,6 +2416,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
boolean quiet)
throws MultiObjectDeleteException, AmazonClientException,
IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("Initiating delete operation for {} objects",
keysToDelete.size());
for (DeleteObjectsRequest.KeyVersion key : keysToDelete) {
LOG.debug(" {} {}", key.getKey(),
key.getVersion() != null ? key.getVersion() : "");
}
}
DeleteObjectsResult result = null;
if (keysToDelete.isEmpty()) {
// exit fast if there are no keys to delete
@ -2490,7 +2533,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
final boolean quiet)
throws MultiObjectDeleteException, AmazonClientException, IOException {
undeletedObjectsOnFailure.clear();
try (DurationInfo ignored = new DurationInfo(LOG, false, "Deleting")) {
try (DurationInfo ignored = new DurationInfo(LOG, false,
"Deleting %d keys", keysToDelete.size())) {
return removeKeysS3(keysToDelete, deleteFakeDir, quiet);
} catch (MultiObjectDeleteException ex) {
LOG.debug("Partial delete failure");
@ -2573,7 +2617,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
// we only make the LIST call; the codepaths to get here should not
// be reached if there is an empty dir marker -and if they do, it
// is mostly harmless to create a new one.
if (!key.isEmpty() && !s3Exists(f, EnumSet.of(StatusProbeEnum.List))) {
if (!key.isEmpty() && !s3Exists(f, StatusProbeEnum.DIRECTORIES)) {
LOG.debug("Creating new fake directory at {}", f);
createFakeDirectory(key);
}
@ -2589,7 +2633,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
void maybeCreateFakeParentDirectory(Path path)
throws IOException, AmazonClientException {
Path parent = path.getParent();
if (parent != null) {
if (parent != null && !parent.isRoot()) {
createFakeDirectoryIfNecessary(parent);
}
}
@ -2618,7 +2662,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
* @throws IOException due to an IO problem.
* @throws AmazonClientException on failures inside the AWS SDK
*/
public FileStatus[] innerListStatus(Path f) throws FileNotFoundException,
private S3AFileStatus[] innerListStatus(Path f) throws FileNotFoundException,
IOException, AmazonClientException {
Path path = qualify(f);
String key = pathToKey(path);
@ -2626,7 +2670,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
entryPoint(INVOCATION_LIST_STATUS);
List<S3AFileStatus> result;
final FileStatus fileStatus = getFileStatus(path);
final S3AFileStatus fileStatus = innerGetFileStatus(path, false,
StatusProbeEnum.ALL);
if (fileStatus.isDirectory()) {
if (!key.isEmpty()) {
@ -2658,7 +2703,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
allowAuthoritative, ttlTimeProvider);
} else {
LOG.debug("Adding: rd (not a dir): {}", path);
FileStatus[] stats = new FileStatus[1];
S3AFileStatus[] stats = new S3AFileStatus[1];
stats[0]= fileStatus;
return stats;
}
@ -2769,9 +2814,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
public boolean mkdirs(Path path, FsPermission permission) throws IOException,
FileAlreadyExistsException {
try {
entryPoint(INVOCATION_MKDIRS);
return innerMkdirs(path, permission);
} catch (AmazonClientException e) {
throw translateException("innerMkdirs", path, e);
throw translateException("mkdirs", path, e);
}
}
@ -2791,11 +2837,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
throws IOException, FileAlreadyExistsException, AmazonClientException {
Path f = qualify(p);
LOG.debug("Making directory: {}", f);
entryPoint(INVOCATION_MKDIRS);
if (p.isRoot()) {
// fast exit for root.
return true;
}
FileStatus fileStatus;
try {
fileStatus = getFileStatus(f);
fileStatus = innerGetFileStatus(f, false,
StatusProbeEnum.ALL);
if (fileStatus.isDirectory()) {
return true;
@ -2805,7 +2855,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
} catch (FileNotFoundException e) {
// Walk path to root, ensuring closest ancestor is a directory, not file
Path fPart = f.getParent();
while (fPart != null) {
while (fPart != null && !fPart.isRoot()) {
try {
fileStatus = getFileStatus(fPart);
if (fileStatus.isDirectory()) {
@ -2866,7 +2916,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
final Set<StatusProbeEnum> probes) throws IOException {
final Path path = qualify(f);
String key = pathToKey(path);
LOG.debug("Getting path status for {} ({})", path, key);
LOG.debug("Getting path status for {} ({}); needEmptyDirectory={}",
path, key, needEmptyDirectoryFlag);
boolean allowAuthoritative = allowAuthoritative(path);
// Check MetadataStore, if any.
@ -2877,9 +2928,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
}
Set<Path> tombstones = Collections.emptySet();
if (pm != null) {
S3AFileStatus msStatus = pm.getFileStatus();
if (pm.isDeleted()) {
OffsetDateTime deletedAt = OffsetDateTime.ofInstant(
Instant.ofEpochMilli(pm.getFileStatus().getModificationTime()),
Instant.ofEpochMilli(msStatus.getModificationTime()),
ZoneOffset.UTC);
throw new FileNotFoundException("Path " + path + " is recorded as " +
"deleted by S3Guard at " + deletedAt);
@ -2890,72 +2942,114 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
// Skip going to s3 if the file checked is a directory. Because if the
// dest is also a directory, there's no difference.
if (!pm.getFileStatus().isDirectory() &&
if (!msStatus.isDirectory() &&
!allowAuthoritative &&
probes.contains(StatusProbeEnum.Head)) {
// a file has been found in a non-auth path and the caller has not said
// they only care about directories
LOG.debug("Metadata for {} found in the non-auth metastore.", path);
final long msModTime = pm.getFileStatus().getModificationTime();
// If the timestamp of the pm is close to "now", we don't need to
// bother with a check of S3. that means:
// one of : status modtime is close to now,
// or pm.getLastUpdated() == now
S3AFileStatus s3AFileStatus;
try {
s3AFileStatus = s3GetFileStatus(path, key, probes, tombstones);
} catch (FileNotFoundException fne) {
s3AFileStatus = null;
}
if (s3AFileStatus == null) {
LOG.warn("Failed to find file {}. Either it is not yet visible, or "
+ "it has been deleted.", path);
} else {
final long s3ModTime = s3AFileStatus.getModificationTime();
// get the time in which a status modtime is considered valid
// in a non-auth metastore
long validTime =
ttlTimeProvider.getNow() - ttlTimeProvider.getMetadataTtl();
final long msModTime = msStatus.getModificationTime();
if(s3ModTime > msModTime) {
LOG.debug("S3Guard metadata for {} is outdated;"
+ " s3modtime={}; msModTime={} updating metastore",
path, s3ModTime, msModTime);
return S3Guard.putAndReturn(metadataStore, s3AFileStatus,
ttlTimeProvider);
if (msModTime < validTime) {
LOG.debug("Metastore entry of {} is out of date, probing S3", path);
try {
S3AFileStatus s3AFileStatus = s3GetFileStatus(path,
key,
probes,
tombstones,
needEmptyDirectoryFlag);
// if the new status is more current than that in the metastore,
// it means S3 has changed and the store needs updating
final long s3ModTime = s3AFileStatus.getModificationTime();
if (s3ModTime > msModTime) {
// there's new data in S3
LOG.debug("S3Guard metadata for {} is outdated;"
+ " s3modtime={}; msModTime={} updating metastore",
path, s3ModTime, msModTime);
// add to S3Guard
S3Guard.putAndReturn(metadataStore, s3AFileStatus,
ttlTimeProvider);
} else {
// the modtime of the data is the same as/older than the s3guard
// value either an old object has been found, or the existing one
// was retrieved in both cases -refresh the S3Guard entry so the
// record's TTL is updated.
S3Guard.refreshEntry(metadataStore, pm, s3AFileStatus,
ttlTimeProvider);
}
// return the value
// note that the checks for empty dir status below can be skipped
// because the call to s3GetFileStatus include the checks there
return s3AFileStatus;
} catch (FileNotFoundException fne) {
// the attempt to refresh the record failed because there was
// no entry. Either it is a new file not visible, or it
// has been deleted (and therefore S3Guard is out of sync with S3)
LOG.warn("Failed to find file {}. Either it is not yet visible, or "
+ "it has been deleted.", path);
}
}
}
S3AFileStatus msStatus = pm.getFileStatus();
if (needEmptyDirectoryFlag && msStatus.isDirectory()) {
// the caller needs to know if a directory is empty,
// and that this is a directory.
if (pm.isEmptyDirectory() != Tristate.UNKNOWN) {
// We have a definitive true / false from MetadataStore, we are done.
return msStatus;
} else {
// execute a S3Guard listChildren command to list tombstones under the
// path.
// This list will be used in the forthcoming s3GetFileStatus call.
DirListingMetadata children =
S3Guard.listChildrenWithTtl(metadataStore, path, ttlTimeProvider,
allowAuthoritative);
if (children != null) {
tombstones = children.listTombstones();
}
LOG.debug("MetadataStore doesn't know if dir is empty, using S3.");
LOG.debug("MetadataStore doesn't know if {} is empty, using S3.",
path);
}
} else {
// Either this is not a directory, or we don't care if it is empty
return msStatus;
}
// If the metadata store has no children for it and it's not listed in
// S3 yet, we'll assume the empty directory is true;
S3AFileStatus s3FileStatus;
// now issue the S3 getFileStatus call.
try {
s3FileStatus = s3GetFileStatus(path, key, probes, tombstones);
S3AFileStatus s3FileStatus = s3GetFileStatus(path,
key,
probes,
tombstones,
true);
// entry was found, so save in S3Guard and return the final value.
return S3Guard.putAndReturn(metadataStore, s3FileStatus,
ttlTimeProvider);
} catch (FileNotFoundException e) {
// If the metadata store has no children for it and it's not listed in
// S3 yet, we'll conclude that it is an empty directory
return S3AFileStatus.fromFileStatus(msStatus, Tristate.TRUE,
null, null);
}
// entry was found, save in S3Guard
return S3Guard.putAndReturn(metadataStore, s3FileStatus,
ttlTimeProvider);
} else {
// there was no entry in S3Guard
// retrieve the data and update the metadata store in the process.
return S3Guard.putAndReturn(metadataStore,
s3GetFileStatus(path, key, probes, tombstones),
s3GetFileStatus(path,
key,
probes,
tombstones,
needEmptyDirectoryFlag),
ttlTimeProvider);
}
}
@ -3010,6 +3104,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
* @param key Key string for the path
* @param probes probes to make
* @param tombstones tombstones to filter
* @param needEmptyDirectoryFlag if true, implementation will calculate
* a TRUE or FALSE value for {@link S3AFileStatus#isEmptyDirectory()}
* @return Status
* @throws FileNotFoundException the supplied probes failed.
* @throws IOException on other problems.
@ -3019,88 +3115,88 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
S3AFileStatus s3GetFileStatus(final Path path,
final String key,
final Set<StatusProbeEnum> probes,
@Nullable Set<Path> tombstones) throws IOException {
if (!key.isEmpty()) {
if (probes.contains(StatusProbeEnum.Head) && !key.endsWith("/")) {
try {
// look for the simple file
ObjectMetadata meta = getObjectMetadata(key);
LOG.debug("Found exact file: normal file {}", key);
return new S3AFileStatus(meta.getContentLength(),
dateToLong(meta.getLastModified()),
path,
getDefaultBlockSize(path),
username,
meta.getETag(),
meta.getVersionId());
} catch (AmazonServiceException e) {
// if the response is a 404 error, it just means that there is
// no file at that path...the remaining checks will be needed.
if (e.getStatusCode() != SC_404 || isUnknownBucket(e)) {
throw translateException("getFileStatus", path, e);
}
} catch (AmazonClientException e) {
@Nullable final Set<Path> tombstones,
final boolean needEmptyDirectoryFlag) throws IOException {
LOG.debug("S3GetFileStatus {}", path);
// either you aren't looking for the directory flag, or you are,
// and if you are, the probe list must contain list.
Preconditions.checkArgument(!needEmptyDirectoryFlag
|| probes.contains(StatusProbeEnum.List),
"s3GetFileStatus(%s) wants to know if a directory is empty but"
+ " does not request a list probe", path);
if (!key.isEmpty() && !key.endsWith("/")
&& probes.contains(StatusProbeEnum.Head)) {
try {
// look for the simple file
ObjectMetadata meta = getObjectMetadata(key);
LOG.debug("Found exact file: normal file {}", key);
return new S3AFileStatus(meta.getContentLength(),
dateToLong(meta.getLastModified()),
path,
getDefaultBlockSize(path),
username,
meta.getETag(),
meta.getVersionId());
} catch (AmazonServiceException e) {
// if the response is a 404 error, it just means that there is
// no file at that path...the remaining checks will be needed.
if (e.getStatusCode() != SC_404 || isUnknownBucket(e)) {
throw translateException("getFileStatus", path, e);
}
}
// Either a normal file was not found or the probe was skipped.
// because the key ended in "/" or it was not in the set of probes.
// Look for the dir marker
if (probes.contains(StatusProbeEnum.DirMarker)) {
String newKey = maybeAddTrailingSlash(key);
try {
ObjectMetadata meta = getObjectMetadata(newKey);
if (objectRepresentsDirectory(newKey, meta.getContentLength())) {
LOG.debug("Found file (with /): fake directory");
return new S3AFileStatus(Tristate.TRUE, path, username);
} else {
LOG.warn("Found file (with /): real file? should not happen: {}",
key);
return new S3AFileStatus(meta.getContentLength(),
dateToLong(meta.getLastModified()),
path,
getDefaultBlockSize(path),
username,
meta.getETag(),
meta.getVersionId());
}
} catch (AmazonServiceException e) {
if (e.getStatusCode() != SC_404 || isUnknownBucket(e)) {
throw translateException("getFileStatus", newKey, e);
}
} catch (AmazonClientException e) {
throw translateException("getFileStatus", newKey, e);
}
} catch (AmazonClientException e) {
throw translateException("getFileStatus", path, e);
}
}
// execute the list
if (probes.contains(StatusProbeEnum.List)) {
try {
// this will find a marker dir / as well as an entry.
// When making a simple "is this a dir check" all is good.
// but when looking for an empty dir, we need to verify there are no
// children, so ask for two entries, so as to find
// a child
String dirKey = maybeAddTrailingSlash(key);
S3ListRequest request = createListObjectsRequest(dirKey, "/", 1);
// list size is dir marker + at least one non-tombstone entry
// there's a corner case: more tombstones than you have in a
// single page list. We assume that if you have been deleting
// that many files, then the AWS listing will have purged some
// by the time of listing so that the response includes some
// which have not.
S3ListResult objects = listObjects(request);
int listSize;
if (tombstones == null) {
// no tombstones so look for a marker and at least one child.
listSize = 2;
} else {
// build a listing > tombstones. If the caller has many thousands
// of tombstones this won't work properly, which is why pruning
// of expired tombstones matters.
listSize = Math.min(2 + tombstones.size(), Math.max(2, maxKeys));
}
S3ListRequest request = createListObjectsRequest(dirKey, "/",
listSize);
// execute the request
S3ListResult listResult = listObjects(request);
Collection<String> prefixes = objects.getCommonPrefixes();
Collection<S3ObjectSummary> summaries = objects.getObjectSummaries();
if (!isEmptyOfKeys(prefixes, tombstones) ||
!isEmptyOfObjects(summaries, tombstones)) {
if (listResult.hasPrefixesOrObjects(contextAccessors, tombstones)) {
if (LOG.isDebugEnabled()) {
LOG.debug("Found path as directory (with /): {}/{}",
prefixes.size(), summaries.size());
for (S3ObjectSummary summary : summaries) {
LOG.debug("Summary: {} {}", summary.getKey(), summary.getSize());
}
for (String prefix : prefixes) {
LOG.debug("Prefix: {}", prefix);
}
LOG.debug("Found path as directory (with /)");
listResult.logAtDebug(LOG);
}
// At least one entry has been found.
// If looking for an empty directory, the marker must exist but no
// children.
// So the listing must contain the marker entry only.
if (needEmptyDirectoryFlag
&& listResult.representsEmptyDirectory(
contextAccessors, dirKey, tombstones)) {
return new S3AFileStatus(Tristate.TRUE, path, username);
}
// either an empty directory is not needed, or the
// listing does not meet the requirements.
return new S3AFileStatus(Tristate.FALSE, path, username);
} else if (key.isEmpty()) {
LOG.debug("Found root directory");
@ -3119,48 +3215,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
throw new FileNotFoundException("No such file or directory: " + path);
}
/**
* Helper function to determine if a collection of paths is empty
* after accounting for tombstone markers (if provided).
* @param keys Collection of path (prefixes / directories or keys).
* @param tombstones Set of tombstone markers, or null if not applicable.
* @return false if summaries contains objects not accounted for by
* tombstones.
*/
private boolean isEmptyOfKeys(Collection<String> keys, Set<Path>
tombstones) {
if (tombstones == null) {
return keys.isEmpty();
}
for (String key : keys) {
Path qualified = keyToQualifiedPath(key);
if (!tombstones.contains(qualified)) {
return false;
}
}
return true;
}
/**
* Helper function to determine if a collection of object summaries is empty
* after accounting for tombstone markers (if provided).
* @param summaries Collection of objects as returned by listObjects.
* @param tombstones Set of tombstone markers, or null if not applicable.
* @return false if summaries contains objects not accounted for by
* tombstones.
*/
private boolean isEmptyOfObjects(Collection<S3ObjectSummary> summaries,
Set<Path> tombstones) {
if (tombstones == null) {
return summaries.isEmpty();
}
Collection<String> stringCollection = new ArrayList<>(summaries.size());
for (S3ObjectSummary summary : summaries) {
stringCollection.add(summary.getKey());
}
return isEmptyOfKeys(stringCollection, tombstones);
}
/**
* Raw version of {@link FileSystem#exists(Path)} which uses S3 only:
* S3Guard MetadataStore, if any, will be skipped.
@ -3175,7 +3229,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
throws IOException {
String key = pathToKey(path);
try {
s3GetFileStatus(path, key, probes, null);
s3GetFileStatus(path, key, probes, null, false);
return true;
} catch (FileNotFoundException e) {
return false;
@ -3578,6 +3632,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
copyObjectRequest.setNewObjectMetadata(dstom);
Optional.ofNullable(srcom.getStorageClass())
.ifPresent(copyObjectRequest::setStorageClass);
incrementStatistic(OBJECT_COPY_REQUESTS);
Copy copy = transfers.copy(copyObjectRequest);
copy.addProgressListener(progressListener);
CopyOutcome copyOutcome = CopyOutcome.waitForCopy(copy);
@ -3711,16 +3766,21 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
/**
* Perform post-write actions.
* Calls {@link #deleteUnnecessaryFakeDirectories(Path)} and then
* updates any metastore.
* <p></p>
* This operation MUST be called after any PUT/multipart PUT completes
* successfully.
*
* The operations actions include
* <p></p>
* The actions include:
* <ol>
* <li>Calling {@link #deleteUnnecessaryFakeDirectories(Path)}</li>
* <li>Updating any metadata store with details on the newly created
* object.</li>
* <li>
* Calling
* {@link #deleteUnnecessaryFakeDirectories(Path, BulkOperationState)}
* if directory markers are not being retained.
* </li>
* <li>
* Updating any metadata store with details on the newly created
* object.
* </li>
* </ol>
* @param key key written to
* @param length total length of file written
@ -3743,12 +3803,19 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
Preconditions.checkArgument(length >= 0, "content length is negative");
final boolean isDir = objectRepresentsDirectory(key, length);
// kick off an async delete
final CompletableFuture<?> deletion = submit(
unboundedThreadPool,
() -> {
deleteUnnecessaryFakeDirectories(p.getParent());
return null;
});
CompletableFuture<?> deletion;
if (!keepDirectoryMarkers(p)) {
deletion = submit(
unboundedThreadPool,
() -> {
deleteUnnecessaryFakeDirectories(
p.getParent(),
operationState);
return null;
});
} else {
deletion = null;
}
// this is only set if there is a metastore to update and the
// operationState parameter passed in was null.
BulkOperationState stateToClose = null;
@ -3807,13 +3874,26 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
}
}
/**
* Should we keep directory markers under the path being created
* by mkdir/file creation/rename?
* @param path path to probe
* @return true if the markers MAY be retained,
* false if they MUST be deleted
*/
private boolean keepDirectoryMarkers(Path path) {
return directoryPolicy.keepDirectoryMarkers(path);
}
/**
* Delete mock parent directories which are no longer needed.
* Retry policy: retrying; exceptions swallowed.
* @param path path
* @param operationState (nullable) operational state for a bulk update
*/
@Retries.RetryExceptionsSwallowed
private void deleteUnnecessaryFakeDirectories(Path path) {
private void deleteUnnecessaryFakeDirectories(Path path,
final BulkOperationState operationState) {
List<DeleteObjectsRequest.KeyVersion> keysToRemove = new ArrayList<>();
while (!path.isRoot()) {
String key = pathToKey(path);
@ -3823,7 +3903,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
path = path.getParent();
}
try {
removeKeys(keysToRemove, true, null);
removeKeys(keysToRemove, true, operationState);
} catch(AmazonClientException | IOException e) {
instrumentation.errorIgnored();
if (LOG.isDebugEnabled()) {
@ -3952,6 +4032,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
return getConf().getLongBytes(FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE);
}
/**
* Get the directory marker policy of this filesystem.
* @return the marker policy.
*/
public DirectoryPolicy getDirectoryMarkerPolicy() {
return directoryPolicy;
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder(
@ -3990,6 +4078,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
sb.append(", credentials=").append(credentials);
sb.append(", delegation tokens=")
.append(delegationTokens.map(Objects::toString).orElse("disabled"));
sb.append(", ").append(directoryPolicy);
sb.append(", statistics {")
.append(statistics)
.append("}");
@ -4086,25 +4175,41 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
}
/**
* Override superclass so as to add statistic collection.
* Optimized probe for a path referencing a dir.
* Even though it is optimized to a single HEAD, applications
* should not over-use this method...it is all too common.
* {@inheritDoc}
*/
@Override
@SuppressWarnings("deprecation")
public boolean isDirectory(Path f) throws IOException {
entryPoint(INVOCATION_IS_DIRECTORY);
return super.isDirectory(f);
try {
return innerGetFileStatus(f, false, StatusProbeEnum.DIRECTORIES)
.isDirectory();
} catch (FileNotFoundException e) {
// not found or it is a file.
return false;
}
}
/**
* Override superclass so as to add statistic collection.
* Optimized probe for a path referencing a file.
* Even though it is optimized to a single HEAD, applications
* should not over-use this method...it is all too common.
* {@inheritDoc}
*/
@Override
@SuppressWarnings("deprecation")
public boolean isFile(Path f) throws IOException {
entryPoint(INVOCATION_IS_FILE);
return super.isFile(f);
try {
return innerGetFileStatus(f, false, StatusProbeEnum.HEAD_ONLY)
.isFile();
} catch (FileNotFoundException e) {
// not found or it is a dir.
return false;
}
}
/**
@ -4511,7 +4616,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
public boolean hasPathCapability(final Path path, final String capability)
throws IOException {
final Path p = makeQualified(path);
switch (validatePathCapabilityArgs(p, capability)) {
String cap = validatePathCapabilityArgs(p, capability);
switch (cap) {
case CommitConstants.STORE_CAPABILITY_MAGIC_COMMITTER:
case CommitConstants.STORE_CAPABILITY_MAGIC_COMMITTER_OLD:
@ -4530,8 +4636,24 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
case CommonPathCapabilities.FS_MULTIPART_UPLOADER:
return true;
// this client is safe to use with buckets
// containing directory markers anywhere in
// the hierarchy
case STORE_CAPABILITY_DIRECTORY_MARKER_AWARE:
return true;
/*
* Marker policy capabilities are handed off.
*/
case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_KEEP:
case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_DELETE:
case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_AUTHORITATIVE:
case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP:
case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE:
return getDirectoryMarkerPolicy().hasPathCapability(path, cap);
default:
return super.hasPathCapability(p, capability);
return super.hasPathCapability(p, cap);
}
}
@ -4546,7 +4668,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
@Override
public boolean hasCapability(String capability) {
try {
return hasPathCapability(workingDir, capability);
return hasPathCapability(new Path("/"), capability);
} catch (IOException ex) {
// should never happen, so log and downgrade.
LOG.debug("Ignoring exception on hasCapability({}})", capability, ex);
@ -4800,6 +4922,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
.build();
}
/**
* Create a marker tools operations binding for this store.
* @return callbacks for operations.
*/
@InterfaceAudience.Private
public MarkerToolOperations createMarkerToolOperations() {
return new MarkerToolOperationsImpl(operationCallbacks);
}
/**
* The implementation of context accessors.
*/

View File

@ -24,11 +24,18 @@ import com.amazonaws.services.s3.model.ListObjectsV2Request;
/**
* API version-independent container for S3 List requests.
*/
public class S3ListRequest {
private ListObjectsRequest v1Request;
private ListObjectsV2Request v2Request;
public final class S3ListRequest {
protected S3ListRequest(ListObjectsRequest v1, ListObjectsV2Request v2) {
/**
* Format for the toString() method: {@value}.
*/
private static final String DESCRIPTION
= "List %s:/%s delimiter=%s keys=%d requester pays=%s";
private final ListObjectsRequest v1Request;
private final ListObjectsV2Request v2Request;
private S3ListRequest(ListObjectsRequest v1, ListObjectsV2Request v2) {
v1Request = v1;
v2Request = v2;
}
@ -70,11 +77,15 @@ public class S3ListRequest {
@Override
public String toString() {
if (isV1()) {
return String.format("List %s:/%s",
v1Request.getBucketName(), v1Request.getPrefix());
return String.format(DESCRIPTION,
v1Request.getBucketName(), v1Request.getPrefix(),
v1Request.getDelimiter(), v1Request.getMaxKeys(),
v1Request.isRequesterPays());
} else {
return String.format("List %s:/%s",
v2Request.getBucketName(), v2Request.getPrefix());
return String.format(DESCRIPTION,
v2Request.getBucketName(), v2Request.getPrefix(),
v2Request.getDelimiter(), v2Request.getMaxKeys(),
v2Request.isRequesterPays());
}
}
}

View File

@ -18,11 +18,18 @@
package org.apache.hadoop.fs.s3a;
import java.util.Collection;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import com.amazonaws.services.s3.model.ListObjectsV2Result;
import com.amazonaws.services.s3.model.ObjectListing;
import com.amazonaws.services.s3.model.S3ObjectSummary;
import org.slf4j.Logger;
import java.util.List;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.impl.ContextAccessors;
/**
* API version-independent container for S3 List responses.
@ -92,6 +99,110 @@ public class S3ListResult {
} else {
return v2Result.getCommonPrefixes();
}
}
/**
* Is the list of object summaries empty
* after accounting for tombstone markers (if provided)?
* @param accessors callback for key to path mapping.
* @param tombstones Set of tombstone markers, or null if not applicable.
* @return false if summaries contains objects not accounted for by
* tombstones.
*/
public boolean isEmptyOfObjects(
final ContextAccessors accessors,
final Set<Path> tombstones) {
if (tombstones == null) {
return getObjectSummaries().isEmpty();
}
return isEmptyOfKeys(accessors,
objectSummaryKeys(),
tombstones);
}
/**
* Get the list of keys in the object summary.
* @return a possibly empty list
*/
private List<String> objectSummaryKeys() {
return getObjectSummaries().stream()
.map(S3ObjectSummary::getKey)
.collect(Collectors.toList());
}
/**
* Does this listing have prefixes or objects after entries with
* tombstones have been stripped?
* @param accessors callback for key to path mapping.
* @param tombstones Set of tombstone markers, or null if not applicable.
* @return true if the reconciled list is non-empty
*/
public boolean hasPrefixesOrObjects(
final ContextAccessors accessors,
final Set<Path> tombstones) {
return !isEmptyOfKeys(accessors, getCommonPrefixes(), tombstones)
|| !isEmptyOfObjects(accessors, tombstones);
}
/**
* Helper function to determine if a collection of keys is empty
* after accounting for tombstone markers (if provided).
* @param accessors callback for key to path mapping.
* @param keys Collection of path (prefixes / directories or keys).
* @param tombstones Set of tombstone markers, or null if not applicable.
* @return true if the list is considered empty.
*/
public boolean isEmptyOfKeys(
final ContextAccessors accessors,
final Collection<String> keys,
final Set<Path> tombstones) {
if (tombstones == null) {
return keys.isEmpty();
}
for (String key : keys) {
Path qualified = accessors.keyToPath(key);
if (!tombstones.contains(qualified)) {
return false;
}
}
return true;
}
/**
* Does this listing represent an empty directory?
* @param contextAccessors callback for key to path mapping.
* @param dirKey directory key
* @param tombstones Set of tombstone markers, or null if not applicable.
* @return true if the list is considered empty.
*/
public boolean representsEmptyDirectory(
final ContextAccessors contextAccessors,
final String dirKey,
final Set<Path> tombstones) {
// If looking for an empty directory, the marker must exist but
// no children.
// So the listing must contain the marker entry only as an object,
// and prefixes is null
List<String> keys = objectSummaryKeys();
return keys.size() == 1 && keys.contains(dirKey)
&& getCommonPrefixes().isEmpty();
}
/**
* Dmp the result at debug level.
* @param log log to use
*/
public void logAtDebug(Logger log) {
Collection<String> prefixes = getCommonPrefixes();
Collection<S3ObjectSummary> summaries = getObjectSummaries();
log.debug("Prefix count = {}; object count={}",
prefixes.size(), summaries.size());
for (S3ObjectSummary summary : summaries) {
log.debug("Summary: {} {}", summary.getKey(), summary.getSize());
}
for (String prefix : prefixes) {
log.debug("Prefix: {}", prefix);
}
}
}

View File

@ -0,0 +1,352 @@
/*
* 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.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;
/**
* Tracks directory markers which have been reported in object listings.
* This is needed for auditing and cleanup, including during rename
* operations.
* <p></p>
* Designed to be used while scanning through the results of listObject
* calls, where are we assume the results come in alphanumeric sort order
* and parent entries before children.
* <p></p>
* This lets as assume that we can identify all leaf markers as those
* markers which were added to set of leaf markers and not subsequently
* removed as a child entries were discovered.
* <p></p>
* To avoid scanning datastructures excessively, the path of the parent
* directory of the last file added is cached. This allows for a
* quick bailout when many children of the same directory are
* returned in a listing.
* <p></p>
* Consult the directory_markers document for details on this feature,
* including terminology.
*/
public class DirMarkerTracker {
private static final Logger LOG =
LoggerFactory.getLogger(DirMarkerTracker.class);
/**
* all leaf markers.
*/
private final Map<Path, Marker> leafMarkers
= new TreeMap<>();
/**
* all surplus markers.
*/
private final Map<Path, Marker> surplusMarkers
= new TreeMap<>();
/**
* Base path of the tracking operation.
*/
private final Path basePath;
/**
* Should surplus markers be recorded in
* the {@link #surplusMarkers} map?
*/
private final boolean recordSurplusMarkers;
/**
* last parent directory checked.
*/
private Path lastDirChecked;
/**
* Count of scans; used for test assertions.
*/
private int scanCount;
/**
* How many files were found.
*/
private int filesFound;
/**
* How many markers were found.
*/
private int markersFound;
/**
* How many objects of any kind were found?
*/
private int objectsFound;
/**
* Construct.
* <p></p>
* The base path is currently only used for information rather than
* validating paths supplied in other methods.
* @param basePath base path of track
* @param recordSurplusMarkers save surplus markers to a map?
*/
public DirMarkerTracker(final Path basePath,
boolean recordSurplusMarkers) {
this.basePath = basePath;
this.recordSurplusMarkers = recordSurplusMarkers;
}
/**
* Get the base path of the tracker.
* @return the path
*/
public Path getBasePath() {
return basePath;
}
/**
* A marker has been found; this may or may not be a leaf.
* <p></p>
* Trigger a move of all markers above it into the surplus map.
* @param path marker path
* @param key object key
* @param source listing source
* @return the surplus markers found.
*/
public List<Marker> markerFound(Path path,
final String key,
final S3ALocatedFileStatus source) {
markersFound++;
leafMarkers.put(path, new Marker(path, key, source));
return pathFound(path, key, source);
}
/**
* A file has been found. Trigger a move of all
* markers above it into the surplus map.
* @param path marker path
* @param key object key
* @param source listing source
* @return the surplus markers found.
*/
public List<Marker> fileFound(Path path,
final String key,
final S3ALocatedFileStatus source) {
filesFound++;
return pathFound(path, key, source);
}
/**
* A path has been found.
* <p></p>
* Declare all markers above it as surplus
* @param path marker path
* @param key object key
* @param source listing source
* @return the surplus markers found.
*/
private List<Marker> pathFound(Path path,
final String key,
final S3ALocatedFileStatus source) {
objectsFound++;
List<Marker> removed = new ArrayList<>();
// all parent entries are superfluous
final Path parent = path.getParent();
if (parent == null || parent.equals(lastDirChecked)) {
// short cut exit
return removed;
}
removeParentMarkers(parent, removed);
lastDirChecked = parent;
return removed;
}
/**
* Remove all markers from the path and its parents from the
* {@link #leafMarkers} map.
* <p></p>
* if {@link #recordSurplusMarkers} is true, the marker is
* moved to the surplus map. Not doing this is simply an
* optimisation designed to reduce risk of excess memory consumption
* when renaming (hypothetically) large directory trees.
* @param path path to start at
* @param removed list of markers removed; is built up during the
* recursive operation.
*/
private void removeParentMarkers(final Path path,
List<Marker> removed) {
if (path == null || path.isRoot()) {
return;
}
scanCount++;
removeParentMarkers(path.getParent(), removed);
final Marker value = leafMarkers.remove(path);
if (value != null) {
// marker is surplus
removed.add(value);
if (recordSurplusMarkers) {
surplusMarkers.put(path, value);
}
}
}
/**
* Get the map of leaf markers.
* @return all leaf markers.
*/
public Map<Path, Marker> getLeafMarkers() {
return leafMarkers;
}
/**
* Get the map of surplus markers.
* <p></p>
* Empty if they were not being recorded.
* @return all surplus markers.
*/
public Map<Path, Marker> getSurplusMarkers() {
return surplusMarkers;
}
public Path getLastDirChecked() {
return lastDirChecked;
}
/**
* How many objects were found.
* @return count
*/
public int getObjectsFound() {
return objectsFound;
}
public int getScanCount() {
return scanCount;
}
public int getFilesFound() {
return filesFound;
}
public int getMarkersFound() {
return markersFound;
}
@Override
public String toString() {
return "DirMarkerTracker{" +
"leafMarkers=" + leafMarkers.size() +
", surplusMarkers=" + surplusMarkers.size() +
", lastDirChecked=" + lastDirChecked +
", filesFound=" + filesFound +
", scanCount=" + scanCount +
'}';
}
/**
* Scan the surplus marker list and remove from it all where the directory
* policy says "keep". This is useful when auditing
* @param policy policy to use when auditing markers for
* inclusion/exclusion.
* @return list of markers stripped
*/
public List<Path> removeAllowedMarkers(DirectoryPolicy policy) {
List<Path> removed = new ArrayList<>();
Iterator<Map.Entry<Path, Marker>> entries =
surplusMarkers.entrySet().iterator();
while (entries.hasNext()) {
Map.Entry<Path, Marker> entry = entries.next();
Path path = entry.getKey();
if (policy.keepDirectoryMarkers(path)) {
// there's a match
// remove it from the map.
entries.remove();
LOG.debug("Removing {}", entry.getValue());
removed.add(path);
}
}
return removed;
}
/**
* This is a marker entry stored in the map and
* returned as markers are deleted.
*/
public static final class Marker {
/** Path of the marker. */
private final Path path;
/**
* Key in the store.
*/
private final String key;
/**
* The file status of the marker.
*/
private final S3ALocatedFileStatus status;
private Marker(final Path path,
final String key,
final S3ALocatedFileStatus status) {
this.path = path;
this.key = key;
this.status = status;
}
public Path getPath() {
return path;
}
public String getKey() {
return key;
}
public S3ALocatedFileStatus getStatus() {
return status;
}
/**
* Get the version ID of the status object; may be null.
* @return a version ID, if known.
*/
public String getVersionId() {
return status.getVersionId();
}
@Override
public String toString() {
return "Marker{" +
"path=" + path +
", key='" + key + '\'' +
", status=" + status +
'}';
}
}
}

View File

@ -0,0 +1,110 @@
/*
* 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 org.apache.hadoop.fs.Path;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_AUTHORITATIVE;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP;
/**
* Interface for Directory Marker policies to implement.
*/
public interface DirectoryPolicy {
/**
* Should a directory marker be retained?
* @param path path a file/directory is being created with.
* @return true if the marker MAY be kept, false if it MUST be deleted.
*/
boolean keepDirectoryMarkers(Path path);
/**
* Get the marker policy.
* @return policy.
*/
MarkerPolicy getMarkerPolicy();
/**
* Describe the policy for marker tools and logs.
* @return description of the current policy.
*/
String describe();
/**
* Does a specific path have the relevant option.
* This is to be forwarded from the S3AFileSystem.hasPathCapability
* But only for those capabilities related to markers*
* @param path path
* @param capability capability
* @return true if the capability is supported, false if not
* @throws IllegalArgumentException if the capability is unknown.
*/
boolean hasPathCapability(Path path, String capability);
/**
* Supported retention policies.
*/
enum MarkerPolicy {
/**
* Delete markers.
* <p></p>
* This is the classic S3A policy,
*/
Delete(DIRECTORY_MARKER_POLICY_DELETE),
/**
* Keep markers.
* <p></p>
* This is <i>Not backwards compatible</i>.
*/
Keep(DIRECTORY_MARKER_POLICY_KEEP),
/**
* Keep markers in authoritative paths only.
* <p></p>
* This is <i>Not backwards compatible</i> within the
* auth paths, but is outside these.
*/
Authoritative(DIRECTORY_MARKER_POLICY_AUTHORITATIVE);
/**
* The name of the option as allowed in configuration files
* and marker-aware tooling.
*/
private final String optionName;
MarkerPolicy(final String optionName) {
this.optionName = optionName;
}
/**
* Get the option name.
* @return name of the option
*/
public String getOptionName() {
return optionName;
}
}
}

View File

@ -0,0 +1,212 @@
/*
* 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.util.EnumSet;
import java.util.Locale;
import java.util.Set;
import java.util.function.Predicate;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_DIRECTORY_MARKER_POLICY;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_AUTHORITATIVE;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP;
import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_AWARE;
import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE;
import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP;
import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_AUTHORITATIVE;
import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_DELETE;
import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_KEEP;
/**
* Implementation of directory policy.
*/
public final class DirectoryPolicyImpl
implements DirectoryPolicy {
private static final Logger LOG = LoggerFactory.getLogger(
DirectoryPolicyImpl.class);
/**
* Error string when unable to parse the marker policy option.
*/
public static final String UNKNOWN_MARKER_POLICY =
"Unknown policy in "
+ DIRECTORY_MARKER_POLICY + ": ";
/**
* All available policies.
*/
private static final Set<MarkerPolicy> AVAILABLE_POLICIES =
EnumSet.allOf(MarkerPolicy.class);
/**
* Keep all markers.
*/
public static final DirectoryPolicy KEEP = new DirectoryPolicyImpl(
MarkerPolicy.Keep, (p) -> false);
/**
* Delete all markers.
*/
public static final DirectoryPolicy DELETE = new DirectoryPolicyImpl(
MarkerPolicy.Delete, (p) -> false);
/**
* Chosen marker policy.
*/
private final MarkerPolicy markerPolicy;
/**
* Callback to evaluate authoritativeness of a
* path.
*/
private final Predicate<Path> authoritativeness;
/**
* Constructor.
* @param markerPolicy marker policy
* @param authoritativeness function for authoritativeness
*/
public DirectoryPolicyImpl(final MarkerPolicy markerPolicy,
final Predicate<Path> authoritativeness) {
this.markerPolicy = markerPolicy;
this.authoritativeness = authoritativeness;
}
@Override
public boolean keepDirectoryMarkers(final Path path) {
switch (markerPolicy) {
case Keep:
return true;
case Authoritative:
return authoritativeness.test(path);
case Delete:
default: // which cannot happen
return false;
}
}
@Override
public MarkerPolicy getMarkerPolicy() {
return markerPolicy;
}
@Override
public String describe() {
return markerPolicy.getOptionName();
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder(
"DirectoryMarkerRetention{");
sb.append("policy='").append(markerPolicy.getOptionName()).append('\'');
sb.append('}');
return sb.toString();
}
/**
* Return path policy for store and paths.
* @param path path
* @param capability capability
* @return true if a capability is active
*/
@Override
public boolean hasPathCapability(final Path path, final String capability) {
switch (capability) {
/*
* Marker policy is dynamically determined for the given path.
*/
case STORE_CAPABILITY_DIRECTORY_MARKER_AWARE:
return true;
case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_KEEP:
return markerPolicy == MarkerPolicy.Keep;
case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_DELETE:
return markerPolicy == MarkerPolicy.Delete;
case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_AUTHORITATIVE:
return markerPolicy == MarkerPolicy.Authoritative;
case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP:
return keepDirectoryMarkers(path);
case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE:
return !keepDirectoryMarkers(path);
default:
throw new IllegalArgumentException("Unknown capability " + capability);
}
}
/**
* Create/Get the policy for this configuration.
* @param conf config
* @param authoritativeness Callback to evaluate authoritativeness of a
* path.
* @return a policy
*/
public static DirectoryPolicy getDirectoryPolicy(
final Configuration conf,
final Predicate<Path> authoritativeness) {
DirectoryPolicy policy;
String option = conf.getTrimmed(DIRECTORY_MARKER_POLICY,
DEFAULT_DIRECTORY_MARKER_POLICY);
switch (option.toLowerCase(Locale.ENGLISH)) {
case DIRECTORY_MARKER_POLICY_DELETE:
// backwards compatible.
LOG.debug("Directory markers will be deleted");
policy = DELETE;
break;
case DIRECTORY_MARKER_POLICY_KEEP:
LOG.info("Directory markers will be kept");
policy = KEEP;
break;
case DIRECTORY_MARKER_POLICY_AUTHORITATIVE:
LOG.info("Directory markers will be kept on authoritative"
+ " paths");
policy = new DirectoryPolicyImpl(MarkerPolicy.Authoritative,
authoritativeness);
break;
default:
throw new IllegalArgumentException(UNKNOWN_MARKER_POLICY + option);
}
return policy;
}
/**
* Enumerate all available policies.
* @return set of the policies.
*/
public static Set<MarkerPolicy> availablePolicies() {
return AVAILABLE_POLICIES;
}
}

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a.impl;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicLong;
@ -43,6 +44,7 @@ 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 org.apache.hadoop.util.OperationDuration;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_BLOCK_SIZE;
@ -55,19 +57,31 @@ import static org.apache.hadoop.fs.s3a.impl.InternalConstants.RENAME_PARALLEL_LI
/**
* A parallelized rename operation which updates the metastore in the
* process, through whichever {@link RenameTracker} the store provides.
* <p></p>
* 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.
* <p></p>
* 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.
*
* <p></p>
* The RenameTracker has the task of keeping the metastore up to date
* as the rename proceeds.
*
* <p></p>
* Directory Markers which have child entries are never copied; only those
* which represent empty directories are copied in the rename.
* The {@link DirMarkerTracker} tracks which markers must be copied, and
* which can simply be deleted from the source.
* As a result: rename always purges all non-leaf directory markers from
* the copied tree. This is to ensure that even if a directory tree
* is copied from an authoritative path to a non-authoritative one
* there is never any contamination of the non-auth path with markers.
* <p></p>
* The rename operation implements the classic HDFS rename policy of
* rename(file, dir) renames the file under the directory.
* <p></p>
*
* There is <i>no</i> validation of input and output paths.
* Callers are required to themselves verify that destination is not under
@ -183,12 +197,59 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
/**
* Queue an object for deletion.
* <p></p>
* This object will be deleted when the next page of objects to delete
* is posted to S3. Therefore, the COPY must have finished
* before that deletion operation takes place.
* This is managed by:
* <ol>
* <li>
* The delete operation only being executed once all active
* copies have completed.
* </li>
* <li>
* Only queuing objects here whose copy operation has
* been submitted and so is in that thread pool.
* </li>
* </ol>
* This method must only be called from the primary thread.
* @param path path to the object
* @param key key of the object.
* @param version object version.
*/
private void queueToDelete(Path path, String key) {
private void queueToDelete(Path path, String key, String version) {
LOG.debug("Queueing to delete {}", path);
pathsToDelete.add(path);
keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key));
keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key, version));
}
/**
* Queue a list of markers for deletion.
* <p></p>
* no-op if the list is empty.
* <p></p>
* See {@link #queueToDelete(Path, String, String)} for
* details on safe use of this method.
*
* @param markersToDelete markers
*/
private void queueToDelete(
List<DirMarkerTracker.Marker> markersToDelete) {
markersToDelete.forEach(m ->
queueToDelete(m));
}
/**
* Queue a single marker for deletion.
* <p></p>
* See {@link #queueToDelete(Path, String, String)} for
* details on safe use of this method.
*
* @param marker markers
*/
private void queueToDelete(final DirMarkerTracker.Marker marker) {
queueToDelete(marker.getPath(), marker.getKey(),
marker.getStatus().getVersionId());
}
/**
@ -225,11 +286,19 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
storeContext,
sourcePath, sourceStatus, destPath);
// The path to whichever file or directory is created by the
// rename. When deleting markers all parents of
// this path will need their markers pruned.
Path destCreated = destPath;
// Ok! Time to start
try {
if (sourceStatus.isFile()) {
renameFileToDest();
// rename the file. The destination path will be different
// from that passed in if the destination is a directory;
// the final value is needed to completely delete parent markers
// when they are not being retained.
destCreated = renameFileToDest();
} else {
recursiveDirectoryRename();
}
@ -254,15 +323,17 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
// Tell the metastore this fact and let it complete its changes
renameTracker.completeRename();
callbacks.finishRename(sourcePath, destPath);
callbacks.finishRename(sourcePath, destCreated);
return bytesCopied.get();
}
/**
* The source is a file: rename it to the destination.
* The source is a file: rename it to the destination, which
* will be under the current destination path if that is a directory.
* @return the path of the object created.
* @throws IOException failure
*/
protected void renameFileToDest() throws IOException {
protected Path renameFileToDest() throws IOException {
final StoreContext storeContext = getStoreContext();
// the source is a file.
Path copyDestinationPath = destPath;
@ -295,12 +366,14 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
callbacks.deleteObjectAtPath(sourcePath, sourceKey, true, null);
// and update the tracker
renameTracker.sourceObjectsDeleted(Lists.newArrayList(sourcePath));
return copyDestinationPath;
}
/**
* Execute a full recursive rename.
* The source is a file: rename it to the destination.
* @throws IOException failure
* There is a special handling of directly markers here -only leaf markers
* are copied. This reduces incompatibility "regions" across versions.
Are * @throws IOException failure
*/
protected void recursiveDirectoryRename() throws IOException {
final StoreContext storeContext = getStoreContext();
@ -325,10 +398,18 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
// marker.
LOG.debug("Deleting fake directory marker at destination {}",
destStatus.getPath());
// Although the dir marker policy doesn't always need to do this,
// it's simplest just to be consistent here.
callbacks.deleteObjectAtPath(destStatus.getPath(), dstKey, false, null);
}
Path parentPath = storeContext.keyToPath(srcKey);
// Track directory markers so that we know which leaf directories need to be
// recreated
DirMarkerTracker dirMarkerTracker = new DirMarkerTracker(parentPath,
false);
final RemoteIterator<S3ALocatedFileStatus> iterator =
callbacks.listFilesAndEmptyDirectories(parentPath,
sourceStatus,
@ -347,36 +428,45 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
// the source object to copy as a path.
Path childSourcePath = storeContext.keyToPath(key);
// mark for deletion on a successful copy.
queueToDelete(childSourcePath, key);
List<DirMarkerTracker.Marker> markersToDelete;
// 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);
boolean isMarker = key.endsWith("/");
if (isMarker) {
// add the marker to the tracker.
// it will not be deleted _yet_ but it may find a list of parent
// markers which may now be deleted.
markersToDelete = dirMarkerTracker.markerFound(
childSourcePath, key, child);
} else {
// it is a file.
// note that it has been found -this may find a list of parent
// markers which may now be deleted.
markersToDelete = dirMarkerTracker.fileFound(
childSourcePath, key, child);
// 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() == pageSize) {
// 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");
// mark the source file for deletion on a successful copy.
queueToDelete(childSourcePath, key, child.getVersionId());
// now begin the single copy
CompletableFuture<Path> copy = initiateCopy(child, key,
childSourcePath, newDestKey, childDestPath);
activeCopies.add(copy);
bytesCopied.addAndGet(sourceStatus.getLen());
}
// add any markers to delete to the operation so they get cleaned
// incrementally
queueToDelete(markersToDelete);
// and trigger any end of loop operations
endOfLoopActions();
} // end of iteration through the list
// finally process remaining directory markers
copyEmptyDirectoryMarkers(srcKey, dstKey, dirMarkerTracker);
// await the final set of copies and their deletion
// This will notify the renameTracker that these objects
// have been deleted.
@ -387,6 +477,93 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
renameTracker.moveSourceDirectory();
}
/**
* Operations to perform at the end of every loop iteration.
* <p></p>
* This may block the thread waiting for copies to complete
* and/or delete a page of data.
*/
private void endOfLoopActions() throws IOException {
if (keysToDelete.size() == pageSize) {
// finish ongoing copies then delete all queued keys.
completeActiveCopiesAndDeleteSources("paged delete");
} else {
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");
}
}
}
/**
* Process all directory markers at the end of the rename.
* All leaf markers are queued to be copied in the store;
* this updates the metastore tracker as it does so.
* <p></p>
* Why not simply create new markers? All the metadata
* gets copied too, so if there was anything relevant then
* it would be preserved.
* <p></p>
* At the same time: markers aren't valued much and may
* be deleted without any safety checks -so if there was relevant
* data it is at risk of destruction at any point.
* If there are lots of empty directory rename operations taking place,
* the decision to copy the source may need revisiting.
* Be advised though: the costs of the copy not withstanding,
* it is a lot easier to have one single type of scheduled copy operation
* than have copy and touch calls being scheduled.
* <p></p>
* The duration returned is the time to initiate all copy/delete operations,
* including any blocking waits for active copies and paged deletes
* to execute. There may still be outstanding operations
* queued by this method -the duration may be an underestimate
* of the time this operation actually takes.
*
* @param srcKey source key with trailing /
* @param dstKey dest key with trailing /
* @param dirMarkerTracker tracker of markers
* @return how long it took.
*/
private OperationDuration copyEmptyDirectoryMarkers(
final String srcKey,
final String dstKey,
final DirMarkerTracker dirMarkerTracker) throws IOException {
// directory marker work.
LOG.debug("Copying markers from {}", dirMarkerTracker);
final StoreContext storeContext = getStoreContext();
Map<Path, DirMarkerTracker.Marker> leafMarkers =
dirMarkerTracker.getLeafMarkers();
Map<Path, DirMarkerTracker.Marker> surplus =
dirMarkerTracker.getSurplusMarkers();
// for all leaf markers: copy the original
DurationInfo duration = new DurationInfo(LOG, false,
"copying %d leaf markers with %d surplus not copied",
leafMarkers.size(), surplus.size());
for (DirMarkerTracker.Marker entry: leafMarkers.values()) {
Path source = entry.getPath();
String key = entry.getKey();
String newDestKey =
dstKey + key.substring(srcKey.length());
Path childDestPath = storeContext.keyToPath(newDestKey);
LOG.debug("copying dir marker from {} to {}", key, newDestKey);
activeCopies.add(
initiateCopy(
entry.getStatus(),
key,
source,
newDestKey,
childDestPath));
queueToDelete(entry);
// end of loop
endOfLoopActions();
}
duration.close();
return duration;
}
/**
* Initiate a copy operation in the executor.
* @param source status of the source object.
@ -487,6 +664,16 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
List<Path> undeletedObjects = new ArrayList<>();
try {
// remove the keys
// list what is being deleted for the interest of anyone
// who is trying to debug why objects are no longer there.
if (LOG.isDebugEnabled()) {
LOG.debug("Initiating delete operation for {} objects", keys.size());
for (DeleteObjectsRequest.KeyVersion key : keys) {
LOG.debug(" {} {}", key.getKey(),
key.getVersion() != null ? key.getVersion() : "");
}
}
// this will update the metastore on a failure, but on
// a successful operation leaves the store as is.
callbacks.removeKeys(
@ -498,7 +685,7 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
// and clear the list.
} catch (AmazonClientException | IOException e) {
// Failed.
// Notify the rename operation.
// Notify the rename tracker.
// 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

View File

@ -21,9 +21,12 @@ package org.apache.hadoop.fs.s3a.impl;
import java.util.EnumSet;
import java.util.Set;
import org.apache.hadoop.classification.InterfaceAudience;
/**
* Enum of probes which can be made of S3.
*/
@InterfaceAudience.Private
public enum StatusProbeEnum {
/** The actual path. */
@ -33,28 +36,23 @@ public enum StatusProbeEnum {
/** LIST under the path. */
List;
/** All probes. */
public static final Set<StatusProbeEnum> ALL = EnumSet.allOf(
StatusProbeEnum.class);
/** Skip the HEAD and only look for directories. */
public static final Set<StatusProbeEnum> DIRECTORIES =
EnumSet.of(DirMarker, List);
/** We only want the HEAD or dir marker. */
public static final Set<StatusProbeEnum> HEAD_OR_DIR_MARKER =
EnumSet.of(Head, DirMarker);
/** Look for files and directories. */
public static final Set<StatusProbeEnum> ALL =
EnumSet.of(Head, List);
/** We only want the HEAD. */
public static final Set<StatusProbeEnum> HEAD_ONLY =
EnumSet.of(Head);
/** We only want the dir marker. */
public static final Set<StatusProbeEnum> DIR_MARKER_ONLY =
EnumSet.of(DirMarker);
/** We only want the dir marker. */
/** List operation only. */
public static final Set<StatusProbeEnum> LIST_ONLY =
EnumSet.of(List);
/** Look for files and directories. */
public static final Set<StatusProbeEnum> FILE =
HEAD_ONLY;
/** Skip the HEAD and only look for directories. */
public static final Set<StatusProbeEnum> DIRECTORIES =
LIST_ONLY;
}

View File

@ -25,6 +25,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
@ -118,6 +119,10 @@ public class DirListingMetadata extends ExpirableMetadata {
return Collections.unmodifiableCollection(listMap.values());
}
/**
* List all tombstones.
* @return all tombstones in the listing.
*/
public Set<Path> listTombstones() {
Set<Path> tombstones = new HashSet<>();
for (PathMetadata meta : listMap.values()) {
@ -128,6 +133,12 @@ public class DirListingMetadata extends ExpirableMetadata {
return tombstones;
}
/**
* Get the directory listing excluding tombstones.
* Returns a new DirListingMetadata instances, without the tombstones -the
* lastUpdated field is copied from this instance.
* @return a new DirListingMetadata without the tombstones.
*/
public DirListingMetadata withoutTombstones() {
Collection<PathMetadata> filteredList = new ArrayList<>();
for (PathMetadata meta : listMap.values()) {
@ -143,6 +154,7 @@ public class DirListingMetadata extends ExpirableMetadata {
* @return number of entries tracked. This is not the same as the number
* of entries in the actual directory unless {@link #isAuthoritative()} is
* true.
* It will also include any tombstones.
*/
public int numEntries() {
return listMap.size();
@ -251,19 +263,24 @@ public class DirListingMetadata extends ExpirableMetadata {
* Remove expired entries from the listing based on TTL.
* @param ttl the ttl time
* @param now the current time
* @return the expired values.
*/
public synchronized void removeExpiredEntriesFromListing(long ttl,
long now) {
public synchronized List<PathMetadata> removeExpiredEntriesFromListing(
long ttl, long now) {
List<PathMetadata> expired = new ArrayList<>();
final Iterator<Map.Entry<Path, PathMetadata>> iterator =
listMap.entrySet().iterator();
while (iterator.hasNext()) {
final Map.Entry<Path, PathMetadata> entry = iterator.next();
// we filter iff the lastupdated is not 0 and the entry is expired
if (entry.getValue().getLastUpdated() != 0
&& (entry.getValue().getLastUpdated() + ttl) <= now) {
PathMetadata metadata = entry.getValue();
if (metadata.getLastUpdated() != 0
&& (metadata.getLastUpdated() + ttl) <= now) {
expired.add(metadata);
iterator.remove();
}
}
return expired;
}
/**

View File

@ -29,6 +29,19 @@ package org.apache.hadoop.fs.s3a.s3guard;
* Time is measured in milliseconds,
*/
public interface ITtlTimeProvider {
/**
* The current time in milliseconds.
* Assuming this calls System.currentTimeMillis(), this is a native iO call
* and so should be invoked sparingly (i.e. evaluate before any loop, rather
* than inside).
* @return the current time.
*/
long getNow();
/**
* The TTL of the metadata.
* @return time in millis after which metadata is considered out of date.
*/
long getMetadataTtl();
}

View File

@ -172,8 +172,10 @@ public class NullMetadataStore implements MetadataStore {
private NullRenameTracker(
final StoreContext storeContext,
final Path source,
final Path dest, MetadataStore metadataStore) {
super("null tracker", storeContext, metadataStore, source, dest, null);
final Path dest,
MetadataStore metadataStore) {
super("NullRenameTracker", storeContext, metadataStore, source, dest,
null);
}
@Override

View File

@ -159,6 +159,54 @@ public final class S3Guard {
}
/**
* We update the metastore for the specific case of S3 value == S3Guard value
* so as to place a more recent modtime in the store.
* because if not, we will continue to probe S3 whenever we look for this
* object, even we only do this if confident the S3 status is the same
* as the one in the store (i.e. it is not an older version)
* @param metadataStore MetadataStore to {@code put()} into.
* @param pm current data
* @param s3AFileStatus status to store
* @param timeProvider Time provider to use when writing entries
* @return true if the entry was updated.
* @throws IOException if metadata store update failed
*/
@RetryTranslated
public static boolean refreshEntry(
MetadataStore metadataStore,
PathMetadata pm,
S3AFileStatus s3AFileStatus,
ITtlTimeProvider timeProvider) throws IOException {
// the modtime of the data is the same as/older than the s3guard value
// either an old object has been found, or the existing one was retrieved
// in both cases -return s3guard value
S3AFileStatus msStatus = pm.getFileStatus();
// first check: size
boolean sizeMatch = msStatus.getLen() == s3AFileStatus.getLen();
// etags are expected on all objects, but handle the situation
// that a third party store doesn't serve them.
String s3Etag = s3AFileStatus.getETag();
String pmEtag = msStatus.getETag();
boolean etagsMatch = s3Etag != null && s3Etag.equals(pmEtag);
// version ID: only in some stores, and will be missing in the metastore
// if the entry was created through a list operation.
String s3VersionId = s3AFileStatus.getVersionId();
String pmVersionId = msStatus.getVersionId();
boolean versionsMatchOrMissingInMetastore =
pmVersionId == null || pmVersionId.equals(s3VersionId);
if (sizeMatch && etagsMatch && versionsMatchOrMissingInMetastore) {
// update the store, return the new value
LOG.debug("Refreshing the metastore entry/timestamp");
putAndReturn(metadataStore, s3AFileStatus, timeProvider);
return true;
}
return false;
}
/**
* Helper function which puts a given S3AFileStatus into the MetadataStore and
* returns the same S3AFileStatus. Instrumentation monitors the put operation.
@ -314,14 +362,14 @@ public final class S3Guard {
* @return Final result of directory listing.
* @throws IOException if metadata store update failed
*/
public static FileStatus[] dirListingUnion(MetadataStore ms, Path path,
public static S3AFileStatus[] dirListingUnion(MetadataStore ms, Path path,
List<S3AFileStatus> backingStatuses, DirListingMetadata dirMeta,
boolean isAuthoritative, ITtlTimeProvider timeProvider)
throws IOException {
// Fast-path for NullMetadataStore
if (isNullMetadataStore(ms)) {
return backingStatuses.toArray(new FileStatus[backingStatuses.size()]);
return backingStatuses.toArray(new S3AFileStatus[backingStatuses.size()]);
}
assertQualified(path);
@ -927,8 +975,10 @@ public final class S3Guard {
if (!pathMetadata.isExpired(ttl, timeProvider.getNow())) {
return pathMetadata;
} else {
LOG.debug("PathMetadata TTl for {} is expired in metadata store.",
path);
LOG.debug("PathMetadata TTl for {} is expired in metadata store"
+ " -removing entry", path);
// delete the tombstone
ms.forgetMetadata(path);
return null;
}
}
@ -940,6 +990,8 @@ public final class S3Guard {
* List children; mark the result as non-auth if the TTL has expired.
* If the allowAuthoritative flag is true, return without filtering or
* checking for TTL expiry.
* If false: the expiry scan takes place and the
* TODO: should we always purge tombstones? Even in auth?
* @param ms metastore
* @param path path to look up.
* @param timeProvider nullable time provider
@ -968,9 +1020,15 @@ public final class S3Guard {
// filter expired entries
if (dlm != null) {
dlm.removeExpiredEntriesFromListing(
List<PathMetadata> expired = dlm.removeExpiredEntriesFromListing(
timeProvider.getMetadataTtl(),
timeProvider.getNow());
// now purge the tombstones
for (PathMetadata metadata : expired) {
if (metadata.isDeleted()) {
ms.forgetMetadata(metadata.getFileStatus().getPath());
}
}
}
return dlm;

View File

@ -30,12 +30,14 @@ import java.util.Collection;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Scanner;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import com.amazonaws.services.s3.model.MultipartUpload;
import com.google.common.annotations.VisibleForTesting;
@ -44,12 +46,15 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FilterFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.StorageStatistics;
import org.apache.hadoop.fs.s3a.MultipartUtils;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
@ -58,7 +63,10 @@ import org.apache.hadoop.fs.s3a.auth.RolePolicies;
import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens;
import org.apache.hadoop.fs.s3a.commit.CommitConstants;
import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants;
import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy;
import org.apache.hadoop.fs.s3a.impl.DirectoryPolicyImpl;
import org.apache.hadoop.fs.s3a.select.SelectTool;
import org.apache.hadoop.fs.s3a.tools.MarkerTool;
import org.apache.hadoop.fs.shell.CommandFormat;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.security.UserGroupInformation;
@ -79,7 +87,11 @@ import static org.apache.hadoop.service.launcher.LauncherExitCodes.*;
/**
* CLI to manage S3Guard Metadata Store.
* <p></p>
* Some management tools invoke this class directly.
*/
@InterfaceAudience.LimitedPrivate("management tools")
@InterfaceStability.Evolving
public abstract class S3GuardTool extends Configured implements Tool,
Closeable {
private static final Logger LOG = LoggerFactory.getLogger(S3GuardTool.class);
@ -98,15 +110,17 @@ public abstract class S3GuardTool extends Configured implements Tool,
"Commands: \n" +
"\t" + Init.NAME + " - " + Init.PURPOSE + "\n" +
"\t" + Destroy.NAME + " - " + Destroy.PURPOSE + "\n" +
"\t" + Import.NAME + " - " + Import.PURPOSE + "\n" +
"\t" + Authoritative.NAME + " - " + Authoritative.PURPOSE + "\n" +
"\t" + BucketInfo.NAME + " - " + BucketInfo.PURPOSE + "\n" +
"\t" + Uploads.NAME + " - " + Uploads.PURPOSE + "\n" +
"\t" + Diff.NAME + " - " + Diff.PURPOSE + "\n" +
"\t" + Fsck.NAME + " - " + Fsck.PURPOSE + "\n" +
"\t" + Import.NAME + " - " + Import.PURPOSE + "\n" +
"\t" + MarkerTool.MARKERS + " - " + MarkerTool.PURPOSE + "\n" +
"\t" + Prune.NAME + " - " + Prune.PURPOSE + "\n" +
"\t" + SetCapacity.NAME + " - " + SetCapacity.PURPOSE + "\n" +
"\t" + SelectTool.NAME + " - " + SelectTool.PURPOSE + "\n" +
"\t" + Fsck.NAME + " - " + Fsck.PURPOSE + "\n" +
"\t" + Authoritative.NAME + " - " + Authoritative.PURPOSE + "\n";
"\t" + Uploads.NAME + " - " + Uploads.PURPOSE + "\n";
private static final String DATA_IN_S3_IS_PRESERVED
= "(all data in S3 is preserved)";
@ -116,6 +130,7 @@ public abstract class S3GuardTool extends Configured implements Tool,
static final int SUCCESS = EXIT_SUCCESS;
static final int INVALID_ARGUMENT = EXIT_COMMAND_ARGUMENT_ERROR;
static final int E_USAGE = EXIT_USAGE;
static final int ERROR = EXIT_FAIL;
static final int E_BAD_STATE = EXIT_NOT_ACCEPTABLE;
static final int E_NOT_FOUND = EXIT_NOT_FOUND;
@ -472,6 +487,14 @@ public abstract class S3GuardTool extends Configured implements Tool,
this.store = store;
}
/**
* Reset the store and filesystem bindings.
*/
protected void resetBindings() {
store = null;
filesystem = null;
}
protected CommandFormat getCommandFormat() {
return commandFormat;
}
@ -497,6 +520,30 @@ public abstract class S3GuardTool extends Configured implements Tool,
public abstract int run(String[] args, PrintStream out) throws Exception,
ExitUtil.ExitException;
/**
* Dump the filesystem Storage Statistics if the FS is not null.
* Only non-zero statistics are printed.
* @param stream output stream
*/
protected void dumpFileSystemStatistics(PrintStream stream) {
FileSystem fs = getFilesystem();
if (fs == null) {
return;
}
println(stream, "%nStorage Statistics for %s%n", fs.getUri());
StorageStatistics st = fs.getStorageStatistics();
Iterator<StorageStatistics.LongStatistic> it
= st.getLongStatistics();
while (it.hasNext()) {
StorageStatistics.LongStatistic next = it.next();
long value = next.getValue();
if (value != 0) {
println(stream, "%s\t%s", next.getName(), value);
}
}
println(stream, "");
}
/**
* Create the metadata store.
*/
@ -1167,16 +1214,20 @@ public abstract class S3GuardTool extends Configured implements Tool,
* Get info about a bucket and its S3Guard integration status.
*/
public static class BucketInfo extends S3GuardTool {
public static final String NAME = "bucket-info";
public static final String BUCKET_INFO = "bucket-info";
public static final String NAME = BUCKET_INFO;
public static final String GUARDED_FLAG = "guarded";
public static final String UNGUARDED_FLAG = "unguarded";
public static final String AUTH_FLAG = "auth";
public static final String NONAUTH_FLAG = "nonauth";
public static final String ENCRYPTION_FLAG = "encryption";
public static final String MAGIC_FLAG = "magic";
public static final String MARKERS_FLAG = "markers";
public static final String MARKERS_AWARE = "aware";
public static final String PURPOSE = "provide/check S3Guard information"
+ " about a specific bucket";
private static final String USAGE = NAME + " [OPTIONS] s3a://BUCKET\n"
+ "\t" + PURPOSE + "\n\n"
+ "Common options:\n"
@ -1186,7 +1237,9 @@ public abstract class S3GuardTool extends Configured implements Tool,
+ " -" + NONAUTH_FLAG + " - Require the S3Guard mode to be \"non-authoritative\"\n"
+ " -" + MAGIC_FLAG + " - Require the S3 filesystem to be support the \"magic\" committer\n"
+ " -" + ENCRYPTION_FLAG
+ " -require {none, sse-s3, sse-kms} - Require encryption policy";
+ " (none, sse-s3, sse-kms) - Require encryption policy\n"
+ " -" + MARKERS_FLAG
+ " (aware, keep, delete, authoritative) - directory markers policy\n";
/**
* Output when the client cannot get the location of a bucket.
@ -1196,10 +1249,17 @@ public abstract class S3GuardTool extends Configured implements Tool,
"Location unknown -caller lacks "
+ RolePolicies.S3_GET_BUCKET_LOCATION + " permission";
@VisibleForTesting
public static final String IS_MARKER_AWARE =
"The S3A connector is compatible with buckets where"
+ " directory markers are not deleted";
public BucketInfo(Configuration conf) {
super(conf, GUARDED_FLAG, UNGUARDED_FLAG, AUTH_FLAG, NONAUTH_FLAG, MAGIC_FLAG);
CommandFormat format = getCommandFormat();
format.addOptionWithValue(ENCRYPTION_FLAG);
format.addOptionWithValue(MARKERS_FLAG);
}
@Override
@ -1384,10 +1444,57 @@ public abstract class S3GuardTool extends Configured implements Tool,
fsUri, desiredEncryption, encryption);
}
// directory markers
processMarkerOption(out, fs,
getCommandFormat().getOptValue(MARKERS_FLAG));
// and finally flush the output and report a success.
out.flush();
return SUCCESS;
}
/**
* Validate the marker options.
* @param out output stream
* @param fs filesystem
* @param path test path
* @param marker desired marker option -may be null.
*/
private void processMarkerOption(final PrintStream out,
final S3AFileSystem fs,
final String marker) {
DirectoryPolicy markerPolicy = fs.getDirectoryMarkerPolicy();
String desc = markerPolicy.describe();
println(out, "%nThe directory marker policy is \"%s\"%n", desc);
DirectoryPolicy.MarkerPolicy mp = markerPolicy.getMarkerPolicy();
String desiredMarker = marker == null
? ""
: marker.trim();
final String optionName = mp.getOptionName();
if (!desiredMarker.isEmpty()) {
if (MARKERS_AWARE.equalsIgnoreCase(desiredMarker)) {
// simple awareness test -provides a way to validate compatibility
// on the command line
println(out, IS_MARKER_AWARE);
String pols = DirectoryPolicyImpl.availablePolicies()
.stream()
.map(DirectoryPolicy.MarkerPolicy::getOptionName)
.collect(Collectors.joining(", "));
println(out, "Available Policies: %s", pols);
} else {
// compare with current policy
if (!optionName.equalsIgnoreCase(desiredMarker)) {
throw badState("Bucket %s: required marker policy is \"%s\""
+ " but actual policy is \"%s\"",
fs.getUri(), desiredMarker, optionName);
}
}
}
}
private String printOption(PrintStream out,
String description, String key, String defVal) {
String t = getFilesystem().getConf().getTrimmed(key, defVal);
@ -1991,6 +2098,9 @@ public abstract class S3GuardTool extends Configured implements Tool,
case Diff.NAME:
command = new Diff(conf);
break;
case MarkerTool.MARKERS:
command = new MarkerTool(conf);
break;
case Prune.NAME:
command = new Prune(conf);
break;

View File

@ -0,0 +1,723 @@
/*
* 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.tools;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStreamWriter;
import java.io.PrintStream;
import java.io.Writer;
import java.net.URI;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import com.amazonaws.AmazonClientException;
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.io.IOUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;
import org.apache.hadoop.fs.s3a.UnknownStoreException;
import org.apache.hadoop.fs.s3a.impl.DirMarkerTracker;
import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool;
import org.apache.hadoop.fs.shell.CommandFormat;
import org.apache.hadoop.util.DurationInfo;
import org.apache.hadoop.util.ExitUtil;
import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH;
import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE;
import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE_DEFAULT;
import static org.apache.hadoop.fs.s3a.Invoker.once;
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_INTERRUPTED;
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE;
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_FOUND;
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_SUCCESS;
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE;
/**
* Audit and S3 bucket for directory markers.
* <p></p>
* This tool does not go anywhere near S3Guard; its scan bypasses any
* metastore as we are explicitly looking for marker objects.
*/
@InterfaceAudience.LimitedPrivate("management tools")
@InterfaceStability.Unstable
public final class MarkerTool extends S3GuardTool {
private static final Logger LOG = LoggerFactory.getLogger(MarkerTool.class);
/**
* Name of this tool: {@value}.
*/
public static final String MARKERS = "markers";
/**
* Purpose of this tool: {@value}.
*/
public static final String PURPOSE =
"View and manipulate S3 directory markers";
/**
* Audit sub-command: {@value}.
*/
public static final String OPT_AUDIT = "audit";
/**
* Clean Sub-command: {@value}.
*/
public static final String OPT_CLEAN = "clean";
/**
* Audit sub-command: {@value}.
*/
public static final String AUDIT = "-" + OPT_AUDIT;
/**
* Clean Sub-command: {@value}.
*/
public static final String CLEAN = "-" + OPT_CLEAN;
/**
* Expected number of markers to find: {@value}.
*/
public static final String OPT_EXPECTED = "expected";
/**
* Name of a file to save the list of markers to: {@value}.
*/
public static final String OPT_OUT = "out";
/**
* Limit of objects to scan: {@value}.
*/
public static final String OPT_LIMIT = "limit";
/**
* Only consider markers found in non-authoritative paths
* as failures: {@value}.
*/
public static final String OPT_NONAUTH = "nonauth";
/**
* Error text when too few arguments are found.
*/
@VisibleForTesting
static final String E_ARGUMENTS = "Wrong number of arguments: %d";
/**
* Constant to use when there is no limit on the number of
* objects listed: {@value}.
* <p></p>
* The value is 0 and not -1 because it allows for the limit to be
* set on the command line {@code -limit 0}.
* The command line parser rejects {@code -limit -1} as the -1
* is interpreted as the (unknown) option "-1".
*/
public static final int UNLIMITED_LISTING = 0;
/**
* Usage string: {@value}.
*/
private static final String USAGE = MARKERS
+ " (-" + OPT_AUDIT
+ " | -" + OPT_CLEAN + ")"
+ " [-" + OPT_EXPECTED + " <count>]"
+ " [-" + OPT_OUT + " <filename>]"
+ " [-" + OPT_LIMIT + " <limit>]"
+ " [-" + OPT_NONAUTH + "]"
+ " [-" + VERBOSE + "]"
+ " <PATH>\n"
+ "\t" + PURPOSE + "\n\n";
/** Will be overridden in run(), but during tests needs to avoid NPEs. */
private PrintStream out = System.out;
/**
* Verbosity flag.
*/
private boolean verbose;
/**
* Store context.
*/
private StoreContext storeContext;
/**
* Operations during the scan.
*/
private MarkerToolOperations operations;
/**
* Constructor.
* @param conf configuration
*/
public MarkerTool(final Configuration conf) {
super(conf,
OPT_AUDIT,
OPT_CLEAN,
VERBOSE,
OPT_NONAUTH);
CommandFormat format = getCommandFormat();
format.addOptionWithValue(OPT_EXPECTED);
format.addOptionWithValue(OPT_LIMIT);
format.addOptionWithValue(OPT_OUT);
}
@Override
public String getUsage() {
return USAGE;
}
@Override
public String getName() {
return MARKERS;
}
@Override
public void resetBindings() {
super.resetBindings();
storeContext = null;
operations = null;
}
@Override
public int run(final String[] args, final PrintStream stream)
throws ExitUtil.ExitException, Exception {
this.out = stream;
final List<String> parsedArgs;
try {
parsedArgs = parseArgs(args);
} catch (CommandFormat.UnknownOptionException e) {
errorln(getUsage());
throw new ExitUtil.ExitException(EXIT_USAGE, e.getMessage(), e);
}
if (parsedArgs.size() != 1) {
errorln(getUsage());
println(out, "Supplied arguments: ["
+ parsedArgs.stream()
.collect(Collectors.joining(", "))
+ "]");
throw new ExitUtil.ExitException(EXIT_USAGE,
String.format(E_ARGUMENTS, parsedArgs.size()));
}
// read arguments
CommandFormat command = getCommandFormat();
verbose = command.getOpt(VERBOSE);
// How many markers are expected?
int expected = 0;
String value = command.getOptValue(OPT_EXPECTED);
if (value != null && !value.isEmpty()) {
expected = Integer.parseInt(value);
}
// determine the action
boolean audit = command.getOpt(OPT_AUDIT);
boolean clean = command.getOpt(OPT_CLEAN);
if (audit == clean) {
// either both are set or neither are set
// this is equivalent to (not audit xor clean)
errorln(getUsage());
throw new ExitUtil.ExitException(EXIT_USAGE,
"Exactly one of " + AUDIT + " and " + CLEAN);
}
int limit = UNLIMITED_LISTING;
value = command.getOptValue(OPT_LIMIT);
if (value != null && !value.isEmpty()) {
limit = Integer.parseInt(value);
}
final String dir = parsedArgs.get(0);
Path path = new Path(dir);
URI uri = path.toUri();
if (uri.getPath().isEmpty()) {
// fix up empty URI for better CLI experience
path = new Path(path, "/");
}
FileSystem fs = path.getFileSystem(getConf());
ScanResult result = execute(
fs,
path,
clean,
expected,
limit,
command.getOpt(OPT_NONAUTH));
if (verbose) {
dumpFileSystemStatistics(out);
}
// and finally see if the output should be saved to a file
String saveFile = command.getOptValue(OPT_OUT);
if (saveFile != null && !saveFile.isEmpty()) {
println(out, "Saving result to %s", saveFile);
try (Writer writer =
new OutputStreamWriter(
new FileOutputStream(saveFile),
StandardCharsets.UTF_8)) {
final List<String> surplus = result.getTracker()
.getSurplusMarkers()
.keySet()
.stream()
.map(p-> p.toString() + "/")
.sorted()
.collect(Collectors.toList());
IOUtils.writeLines(surplus, "\n", writer);
}
}
return result.exitCode;
}
/**
* Execute the scan/purge.
* @param sourceFS source FS; must be or wrap an S3A FS.
* @param path path to scan.
* @param doPurge purge?
* @param expectedMarkerCount expected marker count
* @param limit limit of files to scan; -1 for 'unlimited'
* @param nonAuth consider only markers in nonauth paths as errors
* @return scan+purge result.
* @throws IOException failure
*/
@VisibleForTesting
ScanResult execute(
final FileSystem sourceFS,
final Path path,
final boolean doPurge,
final int expectedMarkerCount,
final int limit,
final boolean nonAuth)
throws IOException {
S3AFileSystem fs = bindFilesystem(sourceFS);
// extract the callbacks needed for the rest of the work
storeContext = fs.createStoreContext();
operations = fs.createMarkerToolOperations();
// filesystem policy.
// if the -nonauth option is set, this is used to filter
// out surplus markers from the results.
DirectoryPolicy activePolicy = fs.getDirectoryMarkerPolicy();
DirectoryPolicy.MarkerPolicy policy = activePolicy
.getMarkerPolicy();
println(out, "The directory marker policy of %s is \"%s\"",
storeContext.getFsURI(),
policy);
String authPath = storeContext.getConfiguration()
.getTrimmed(AUTHORITATIVE_PATH, "");
if (policy == DirectoryPolicy.MarkerPolicy.Authoritative) {
// in auth mode, note the auth paths.
println(out, "Authoritative path list is \"%s\"", authPath);
}
// qualify the path
Path target = path.makeQualified(fs.getUri(), new Path("/"));
// initial safety check: does the path exist?
try {
getFilesystem().getFileStatus(target);
} catch (UnknownStoreException ex) {
// bucket doesn't exist.
// replace the stack trace with an error code.
throw new ExitUtil.ExitException(EXIT_NOT_FOUND,
ex.toString(), ex);
} catch (FileNotFoundException ex) {
throw new ExitUtil.ExitException(EXIT_NOT_FOUND,
"Not found: " + target, ex);
}
// the default filter policy is that all entries should be deleted
DirectoryPolicy filterPolicy = nonAuth
? activePolicy
: null;
ScanResult result = scan(target, doPurge, expectedMarkerCount, limit,
filterPolicy);
return result;
}
/**
* Result of the scan operation.
*/
public static final class ScanResult {
/**
* Exit code to return if an exception was not raised.
*/
private int exitCode;
/**
* The tracker.
*/
private DirMarkerTracker tracker;
/**
* Scan summary.
*/
private MarkerPurgeSummary purgeSummary;
private ScanResult() {
}
@Override
public String toString() {
return "ScanResult{" +
"exitCode=" + exitCode +
", tracker=" + tracker +
", purgeSummary=" + purgeSummary +
'}';
}
/** Exit code to report. */
public int getExitCode() {
return exitCode;
}
/** Tracker which did the scan. */
public DirMarkerTracker getTracker() {
return tracker;
}
/** Summary of purge. Null if none took place. */
public MarkerPurgeSummary getPurgeSummary() {
return purgeSummary;
}
}
/**
* Do the scan/purge.
* @param path path to scan.
* @param clean purge?
* @param expectedMarkerCount expected marker count
* @param limit limit of files to scan; 0 for 'unlimited'
* @param filterPolicy filter policy on a nonauth scan; may be null
* @return result.
* @throws IOException IO failure
* @throws ExitUtil.ExitException explicitly raised failure
*/
@Retries.RetryTranslated
private ScanResult scan(
final Path path,
final boolean clean,
final int expectedMarkerCount,
final int limit,
final DirectoryPolicy filterPolicy)
throws IOException, ExitUtil.ExitException {
ScanResult result = new ScanResult();
// Mission Accomplished
result.exitCode = EXIT_SUCCESS;
// Now do the work.
DirMarkerTracker tracker = new DirMarkerTracker(path, true);
result.tracker = tracker;
boolean completed;
try (DurationInfo ignored =
new DurationInfo(LOG, "marker scan %s", path)) {
completed = scanDirectoryTree(path, tracker, limit);
}
int objectsFound = tracker.getObjectsFound();
println(out, "Listed %d object%s under %s%n",
objectsFound,
suffix(objectsFound),
path);
// scan done. what have we got?
Map<Path, DirMarkerTracker.Marker> surplusMarkers
= tracker.getSurplusMarkers();
Map<Path, DirMarkerTracker.Marker> leafMarkers
= tracker.getLeafMarkers();
int surplus = surplusMarkers.size();
if (surplus == 0) {
println(out, "No surplus directory markers were found under %s", path);
} else {
println(out, "Found %d surplus directory marker%s under %s",
surplus,
suffix(surplus),
path);
for (Path markers : surplusMarkers.keySet()) {
println(out, " %s/", markers);
}
}
if (!leafMarkers.isEmpty()) {
println(out, "Found %d empty directory 'leaf' marker%s under %s",
leafMarkers.size(),
suffix(leafMarkers.size()),
path);
for (Path markers : leafMarkers.keySet()) {
println(out, " %s/", markers);
}
println(out, "These are required to indicate empty directories");
}
if (clean) {
// clean: remove the markers, do not worry about their
// presence when reporting success/failiure
int deletePageSize = storeContext.getConfiguration()
.getInt(BULK_DELETE_PAGE_SIZE,
BULK_DELETE_PAGE_SIZE_DEFAULT);
result.purgeSummary = purgeMarkers(tracker, deletePageSize);
} else {
// this is an audit, so validate the marker count
if (filterPolicy != null) {
// if a filter policy is supplied, filter out all markers
// under the auth path
List<Path> allowed = tracker.removeAllowedMarkers(filterPolicy);
int allowedMarkers = allowed.size();
println(out, "%nIgnoring %d marker%s in authoritative paths",
allowedMarkers, suffix(allowedMarkers));
if (verbose) {
allowed.forEach(p -> println(out, p.toString()));
}
// recalculate the marker size
surplus = surplusMarkers.size();
}
if (surplus > expectedMarkerCount) {
// failure
if (expectedMarkerCount > 0) {
println(out, "Expected %d marker%s", expectedMarkerCount,
suffix(surplus));
}
println(out, "Surplus markers were found -failing audit");
result.exitCode = EXIT_NOT_ACCEPTABLE;
}
}
// now one little check for whether a limit was reached.
if (!completed) {
println(out, "Listing limit reached before completing the scan");
result.exitCode = EXIT_INTERRUPTED;
}
return result;
}
/**
* Suffix for plurals.
* @param size size to generate a suffix for
* @return "" or "s", depending on size
*/
private String suffix(final int size) {
return size == 1 ? "" : "s";
}
/**
* Scan a directory tree.
* @param path path to scan
* @param tracker tracker to update
* @param limit limit of files to scan; -1 for 'unlimited'
* @return true if the scan completedly scanned the entire tree
* @throws IOException IO failure
*/
@Retries.RetryTranslated
private boolean scanDirectoryTree(
final Path path,
final DirMarkerTracker tracker,
final int limit) throws IOException {
int count = 0;
RemoteIterator<S3AFileStatus> listing = operations
.listObjects(path, storeContext.pathToKey(path));
while (listing.hasNext()) {
count++;
S3AFileStatus status = listing.next();
Path statusPath = status.getPath();
S3ALocatedFileStatus locatedStatus = new S3ALocatedFileStatus(
status, null);
String key = storeContext.pathToKey(statusPath);
if (status.isDirectory()) {
if (verbose) {
println(out, " Directory Marker %s/", key);
}
LOG.debug("{}", key);
tracker.markerFound(statusPath,
key + "/",
locatedStatus);
} else {
tracker.fileFound(statusPath,
key,
locatedStatus);
}
if ((count % 1000) == 0) {
println(out, "Scanned %,d objects", count);
}
if (limit > 0 && count >= limit) {
println(out, "Limit of scan reached - %,d object%s",
limit, suffix(limit));
return false;
}
}
return true;
}
/**
* Result of a call of {@link #purgeMarkers(DirMarkerTracker, int)};
* included in {@link ScanResult} so must share visibility.
*/
static final class MarkerPurgeSummary {
/** Number of markers deleted. */
private int markersDeleted;
/** Number of delete requests issued. */
private int deleteRequests;
/**
* Total duration of delete requests.
* If this is ever parallelized, this will
* be greater than the elapsed time of the
* operation.
*/
private long totalDeleteRequestDuration;
@Override
public String toString() {
return "MarkerPurgeSummary{" +
"markersDeleted=" + markersDeleted +
", deleteRequests=" + deleteRequests +
", totalDeleteRequestDuration=" + totalDeleteRequestDuration +
'}';
}
int getMarkersDeleted() {
return markersDeleted;
}
int getDeleteRequests() {
return deleteRequests;
}
long getTotalDeleteRequestDuration() {
return totalDeleteRequestDuration;
}
}
/**
* Purge the markers.
* @param tracker tracker with the details
* @param deletePageSize page size of deletes
* @return summary
* @throws IOException IO failure
*/
@Retries.RetryTranslated
private MarkerPurgeSummary purgeMarkers(
final DirMarkerTracker tracker,
final int deletePageSize)
throws MultiObjectDeleteException, AmazonClientException, IOException {
MarkerPurgeSummary summary = new MarkerPurgeSummary();
// we get a map of surplus markers to delete.
Map<Path, DirMarkerTracker.Marker> markers
= tracker.getSurplusMarkers();
int size = markers.size();
// build a list from the strings in the map
List<DeleteObjectsRequest.KeyVersion> collect =
markers.values().stream()
.map(p -> new DeleteObjectsRequest.KeyVersion(p.getKey()))
.collect(Collectors.toList());
// build an array list for ease of creating the lists of
// keys in each page through the subList() method.
List<DeleteObjectsRequest.KeyVersion> markerKeys =
new ArrayList<>(collect);
// now randomize. Why so? if the list spans multiple S3 partitions,
// it should reduce the IO load on each part.
Collections.shuffle(markerKeys);
int pages = size / deletePageSize;
if (size % deletePageSize > 0) {
pages += 1;
}
if (verbose) {
println(out, "%n%d marker%s to delete in %d page%s of %d keys/page",
size, suffix(size),
pages, suffix(pages),
deletePageSize);
}
DurationInfo durationInfo = new DurationInfo(LOG, "Deleting markers");
int start = 0;
while (start < size) {
// end is one past the end of the page
int end = Math.min(start + deletePageSize, size);
List<DeleteObjectsRequest.KeyVersion> page = markerKeys.subList(start,
end);
List<Path> undeleted = new ArrayList<>();
once("Remove S3 Keys",
tracker.getBasePath().toString(), () ->
operations.removeKeys(page, true, undeleted, null, false));
summary.deleteRequests++;
// and move to the start of the next page
start = end;
}
durationInfo.close();
summary.totalDeleteRequestDuration = durationInfo.value();
summary.markersDeleted = size;
return summary;
}
public boolean isVerbose() {
return verbose;
}
public void setVerbose(final boolean verbose) {
this.verbose = verbose;
}
/**
* Execute the marker tool, with no checks on return codes.
*
* @param sourceFS filesystem to use
* @param path path to scan
* @param doPurge should markers be purged
* @param expectedMarkers number of markers expected
* @param limit limit of files to scan; -1 for 'unlimited'
* @param nonAuth only use nonauth path count for failure rules
* @return the result
*/
@SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
public static MarkerTool.ScanResult execMarkerTool(
final FileSystem sourceFS,
final Path path,
final boolean doPurge,
final int expectedMarkers,
final int limit, boolean nonAuth) throws IOException {
MarkerTool tool = new MarkerTool(sourceFS.getConf());
tool.setVerbose(LOG.isDebugEnabled());
return tool.execute(sourceFS, path, doPurge,
expectedMarkers, limit, nonAuth);
}
}

View File

@ -0,0 +1,91 @@
/*
* 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.tools;
import java.io.IOException;
import java.util.List;
import com.amazonaws.AmazonClientException;
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
import com.amazonaws.services.s3.model.DeleteObjectsResult;
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
import org.apache.hadoop.fs.InvalidRequestException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
/**
* Operations which must be offered by the store for {@link MarkerTool}.
* These are a proper subset of {@code OperationCallbacks}; this interface
* strips down those provided to the tool.
*/
public interface MarkerToolOperations {
/**
* Create an iterator over objects in S3 only; S3Guard
* is not involved.
* The listing includes the key itself, if found.
* @param path path of the listing.
* @param key object key
* @return iterator with the first listing completed.
* @throws IOException failure.
*/
@Retries.RetryTranslated
RemoteIterator<S3AFileStatus> listObjects(
Path path,
String key)
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.
* @param operationState bulk operation state
* @param quiet should a bulk query be quiet, or should its result list
* all deleted keys
* @return the deletion result if a multi object delete was invoked
* and it returned without a failure, else null.
* @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
DeleteObjectsResult removeKeys(
List<DeleteObjectsRequest.KeyVersion> keysToDelete,
boolean deleteFakeDir,
List<Path> undeletedObjectsOnFailure,
BulkOperationState operationState,
boolean quiet)
throws MultiObjectDeleteException, AmazonClientException,
IOException;
}

View File

@ -0,0 +1,70 @@
/*
* 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.tools;
import java.io.IOException;
import java.util.List;
import com.amazonaws.AmazonClientException;
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
import com.amazonaws.services.s3.model.DeleteObjectsResult;
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.impl.OperationCallbacks;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
/**
* Implement the marker tool operations by forwarding to the
* {@link OperationCallbacks} instance provided in the constructor.
*/
public class MarkerToolOperationsImpl implements MarkerToolOperations {
private final OperationCallbacks operationCallbacks;
/**
* Constructor.
* @param operations implementation of the operations
*/
public MarkerToolOperationsImpl(final OperationCallbacks operations) {
this.operationCallbacks = operations;
}
@Override
public RemoteIterator<S3AFileStatus> listObjects(final Path path,
final String key)
throws IOException {
return operationCallbacks.listObjects(path, key);
}
@Override
public DeleteObjectsResult removeKeys(
final List<DeleteObjectsRequest.KeyVersion> keysToDelete,
final boolean deleteFakeDir,
final List<Path> undeletedObjectsOnFailure,
final BulkOperationState operationState,
final boolean quiet)
throws MultiObjectDeleteException, AmazonClientException, IOException {
return operationCallbacks.removeKeys(keysToDelete, deleteFakeDir,
undeletedObjectsOnFailure, operationState, quiet);
}
}

View File

@ -0,0 +1,27 @@
/*
* 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.
*/
/**
* S3A Command line tools independent of S3Guard.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
package org.apache.hadoop.fs.s3a.tools;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -0,0 +1,694 @@
<!---
Licensed 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. See accompanying LICENSE file.
-->
# Controlling the S3A Directory Marker Behavior
## <a name="compatibility"></a> Critical: this is not backwards compatible!
This document shows how the performance of S3 I/O, especially applications
creating many files (for example Apache Hive) or working with versioned S3 buckets can
increase performance by changing the S3A directory marker retention policy.
Changing the policy from the default value, `"delete"` _is not backwards compatible_.
Versions of Hadoop which are incompatible with other marker retention policies,
as of August 2020.
-------------------------------------------------------
| Branch | Compatible Since | Future Fix Planned? |
|------------|------------------|---------------------|
| Hadoop 2.x | | NO |
| Hadoop 3.0 | | NO |
| Hadoop 3.1 | check | Yes |
| Hadoop 3.2 | check | Yes |
| Hadoop 3.3 | 3.3.1 | Done |
-------------------------------------------------------
The `s3guard bucket-info` tool [can be used to verify support](#bucket-info).
This allows for a command line check of compatibility, including
in scripts.
External Hadoop-based applications should also be assumed to be incompatible
unless otherwise stated/known.
It is only safe change the directory marker policy if the following
conditions are met:
1. You know exactly which applications are writing to and reading from
(including backing up) an S3 bucket.
2. You know all applications which read data from the bucket are compatible.
### <a name="backups"></a> Applications backing up data.
It is not enough to have a version of Apache Hadoop which is compatible, any
application which backs up an S3 bucket or copies elsewhere must have an S3
connector which is compatible. For the Hadoop codebase, that means that if
distcp is used, it _must_ be from a compatible hadoop version.
### <a name="fallure-mode"></a> How will incompatible applications/versions fail?
Applications using an incompatible version of the S3A connector will mistake
directories containing data for empty directories. This means that:
* Listing directories/directory trees may exclude files which exist.
* Queries across the data will miss data files.
* Renaming a directory to a new location may exclude files underneath.
The failures are silent: there is no error message, stack trace or
other warning that files may have been missed. They simply aren't
found.
### <a name="recovery"></a> If an application has updated a directory tree incompatibly-- what can be done?
There's a tool on the hadoop command line, [marker tool](#marker-tool) which can audit
a bucket/path for markers, and clean up any markers which were found.
It can be used to make a bucket compatible with older applications.
Now that this is all clear, let's explain the problem.
## <a name="background"></a> Background: Directory Markers: what and why?
Amazon S3 is not a filesystem, it is an object store.
The S3A connector not only provides a hadoop-compatible API to interact with
data in S3, it tries to maintain the filesystem metaphor.
One key aspect of the metaphor of a file system is "directories"
#### The directory concept
In normal Unix-style filesystems, the "filesystem" is really a "directory and
file tree" in which files are always stored in "directories"
* A directory may contain zero or more files.
* A directory may contain zero or more directories "subdirectories"
* At the base of a filesystem is the "root directory"
* All files MUST be in a directory "the parent directory"
* All directories other than the root directory must be in another directory.
* If a directory contains no files or directories, it is "empty"
* When a directory is _listed_, all files and directories in it are enumerated
and returned to the caller
The S3A connector emulates this metaphor by grouping all objects which have
the same prefix as if they are in the same directory tree.
If there are two objects `a/b/file1` and `a/b/file2` then S3A pretends that there is a
directory `/a/b` containing two files `file1` and `file2`.
The directory itself does not exist.
There's a bit of a complication here.
#### What does `mkdirs()` do?
1. In HDFS and other "real" filesystems, when `mkdirs()` is invoked on a path
whose parents are all directories, then an _empty directory_ is created.
1. This directory can be probed for "it exists" and listed (an empty list is
returned)
1. Files and other directories can be created in it.
Lots of code contains a big assumption here: after you create a directory it
exists. They also assume that after files in a directory are deleted, the
directory still exists.
Given the S3A connector mimics directories just by aggregating objects which share a
prefix, how can you have empty directories?
The original Hadoop `s3n://` connector created a Directory Marker -any path ending
in `_$folder$` was considered to be a sign that a directory existed. A call to
`mkdir(s3n://bucket/a/b)` would create a new marker object `a/b_$folder$` .
The S3A also has directory markers, but it just appends a "/" to the directory
name, so `mkdir(s3a://bucket/a/b)` will create a new marker object `a/b/` .
When a file is created under a path, the directory marker is deleted. And when a
file is deleted, if it was the last file in the directory, the marker is
recreated.
And, historically, When a path is listed, if a marker to that path is found, *it
has been interpreted as an empty directory.*
It is that little detail which is the cause of the incompatibility issues.
## <a name="problem"></a> The Problem with Directory Markers
Creating, deleting and the listing directory markers adds overhead and can slow
down applications.
Whenever a file is created we have to delete any marker which could exist in
parent directory _or any parent paths_. Rather than do a sequence of probes for
parent markers existing, the connector issues a single request to S3 to delete
all parents. For example, if a file `/a/b/file1` is created, a multi-object
`DELETE` request containing the keys `/a/` and `/a/b/` is issued.
If no markers exists, this is harmless.
When a file is deleted, a check for the parent directory continuing to exist
(i.e. are there sibling files/directories?), and if not a marker is created.
This all works well and has worked well for many years.
However, it turns out to have some scale problems, especially from the delete
call made whenever a file is created.
1. The number of the objects listed in each request is that of the number of
parent directories: deeper trees create longer requests.
2. Every single object listed in the delete request is considered to be a write
operation.
3. In versioned S3 buckets, tombstone markers are added to the S3 indices even
if no object was deleted.
4. There's also the overhead of actually issuing the request and awaiting the
response.
Issue #2 has turned out to cause significant problems on some interactions with
large hive tables:
Because each object listed in a DELETE call is treated as one operation, and
there is (as of summer 2020) a limit of 3500 write requests/second in a directory
tree.
When writing many files to a deep directory tree, it is the delete calls which
create throttling problems.
The tombstone markers have follow-on consequences -it makes listings against
S3 versioned buckets slower.
This can have adverse effects on those large directories, again.
## <a name="solutions"></a> Strategies to avoid marker-related problems.
### Presto: every path is a directory
In the Presto [S3 connector](https://prestodb.io/docs/current/connector/hive.html#amazon-s3-configuration),
`mkdirs()` is a no-op.
Whenever it lists any path which isn't an object or a prefix of one more more objects, it returns an
empty listing. That is:; by default, every path is an empty directory.
Provided no code probes for a directory existing and fails if it is there, this
is very efficient. That's a big requirement however, -one Presto can pull off
because they know how their file uses data in S3.
### Hadoop 3.3.1+: marker deletion is now optional
From Hadoop 3.3.1 onwards, the S3A client can be configured to skip deleting
directory markers when creating files under paths. This removes all scalability
problems caused by deleting these markers -however, it is achieved at the expense
of backwards compatibility.
## <a name="marker-retention"></a> Controlling marker retention with `fs.s3a.directory.marker.retention`
There is now an option `fs.s3a.directory.marker.retention` which controls how
markers are managed when new files are created
*Default* `delete`: a request is issued to delete any parental directory markers
whenever a file or directory is created.
*New* `keep`: No delete request is issued.
Any directory markers which exist are not deleted.
This is *not* backwards compatible
*New* `authoritative`: directory markers are deleted _except for files created
in "authoritative" directories_.
This is backwards compatible _outside authoritative directories_.
Until now, the notion of an "authoritative"
directory has only been used as a performance optimization for deployments
where it is known that all Applications are using the same S3Guard metastore
when writing and reading data.
In such a deployment, if it is also known that all applications are using a
compatible version of the s3a connector, then they
can switch to the higher-performance mode for those specific directories.
Only the default setting, `fs.s3a.directory.marker.retention = delete` is compatible with
every shipping Hadoop releases.
## <a name="authoritative"></a> Directory Markers and S3Guard
Applications which interact with S3A in S3A clients with S3Guard enabled still
create and delete markers. There's no attempt to skip operations, such as by having
`mkdirs() `create entries in the DynamoDB table but not the store.
Having the client always update S3 ensures that other applications and clients
do (eventually) see the changes made by the "guarded" application.
When S3Guard is configured to treat some directories as [Authoritative](s3guard.html#authoritative)
then an S3A connector with a retention policy of `fs.s3a.directory.marker.retention` of
`authoritative` will omit deleting markers in authoritative directories.
*Note* there may be further changes in directory semantics in "authoritative mode";
only use in managed applications where all clients are using the same version of
hadoop, and configured consistently.
## <a name="bucket-info"></a> Verifying marker policy with `s3guard bucket-info`
The `bucket-info` command has been enhanced to support verification from the command
line of bucket policies via the `-marker` option
| option | verifies |
|--------|--------|
| `-markers aware` | the hadoop release is "aware" of directory markers |
| `-markers delete` | directory markers are deleted |
| `-markers keep` | directory markers are kept (not backwards compatible) |
| `-markers authoritative` | directory markers are kept in authoritative paths |
All releases of Hadoop which have been updated to be marker aware will support the `-markers aware` option.
1. Updated releases which do not support switching marker retention policy will also support the
`-markers delete` option.
Example: `s3guard bucket-info -markers aware` on a compatible release.
```
> hadoop s3guard bucket-info -markers aware s3a://landsat-pds/
Filesystem s3a://landsat-pds
Location: us-west-2
Filesystem s3a://landsat-pds is not using S3Guard
...
Security
Delegation token support is disabled
The directory marker policy is "delete"
The S3A connector is compatible with buckets where directory markers are not deleted
Available Policies: delete, keep, authoritative
```
The same command will fail on older releases, because the `-markers` option
is unknown
```
> hadoop s3guard bucket-info -markers aware s3a://landsat-pds/
Illegal option -markers
Usage: hadoop bucket-info [OPTIONS] s3a://BUCKET
provide/check S3Guard information about a specific bucket
Common options:
-guarded - Require S3Guard
-unguarded - Force S3Guard to be disabled
-auth - Require the S3Guard mode to be "authoritative"
-nonauth - Require the S3Guard mode to be "non-authoritative"
-magic - Require the S3 filesystem to be support the "magic" committer
-encryption -require {none, sse-s3, sse-kms} - Require encryption policy
When possible and not overridden by more specific options, metadata
repository information will be inferred from the S3A URL (if provided)
Generic options supported are:
-conf <config file> - specify an application configuration file
-D <property=value> - define a value for a given property
2020-08-12 16:47:16,579 [main] INFO util.ExitUtil (ExitUtil.java:terminate(210)) - Exiting with status 42: Illegal option -markers
````
A specific policy check verifies that the connector is configured as desired
```
> hadoop s3guard bucket-info -markers delete s3a://landsat-pds/
Filesystem s3a://landsat-pds
Location: us-west-2
Filesystem s3a://landsat-pds is not using S3Guard
...
The directory marker policy is "delete"
```
When probing for a specific policy, the error code "46" is returned if the active policy
does not match that requested:
```
> hadoop s3guard bucket-info -markers keep s3a://landsat-pds/
Filesystem s3a://landsat-pds
Location: us-west-2
Filesystem s3a://landsat-pds is not using S3Guard
...
Security
Delegation token support is disabled
The directory marker policy is "delete"
2020-08-12 17:14:30,563 [main] INFO util.ExitUtil (ExitUtil.java:terminate(210)) - Exiting with status 46: 46: Bucket s3a://landsat-pds: required marker policy is "keep" but actual policy is "delete"
```
## <a name="marker-tool"></a> The marker tool:`hadoop s3guard markers`
The marker tool aims to help migration by scanning/auditing directory trees
for surplus markers, and for optionally deleting them.
Leaf-node markers for empty directories are not considered surplus and
will be retained.
Syntax
```
> hadoop s3guard markers -verbose -nonauth
markers (-audit | -clean) [-expected <count>] [-out <filename>] [-limit <limit>] [-nonauth] [-verbose] <PATH>
View and manipulate S3 directory markers
```
*Options*
| Option | Meaning |
|-------------------------|-------------------------|
| `-audit` | Audit the path for surplus markers |
| `-clean` | Clean all surplus markers under a path |
| `-expected <count>]` | Expected number of markers to find (primarily for testing) |
| `-limit <count>]` | Limit the number of objects to scan |
| `-nonauth` | Only consider markers in non-authoritative paths as errors |
| `-out <filename>` | Save a list of all markers found to the nominated file |
| `-verbose` | Verbose output |
*Exit Codes*
| Code | Meaning |
|-------|---------|
| 0 | Success |
| 3 | interrupted -the value of `-limit` was reached |
| 42 | Usage |
| 46 | Markers were found (see HTTP "406", "unacceptable") |
All other non-zero status code also indicate errors of some form or other.
### <a name="marker-tool-report"></a>`markers -audit`
Audit the path and fail if any markers were found.
```
> hadoop s3guard markers -limit 8000 -audit s3a://landsat-pds/
The directory marker policy of s3a://landsat-pds is "Delete"
2020-08-05 13:42:56,079 [main] INFO tools.MarkerTool (DurationInfo.java:<init>(77)) - Starting: marker scan s3a://landsat-pds/
Scanned 1,000 objects
Scanned 2,000 objects
Scanned 3,000 objects
Scanned 4,000 objects
Scanned 5,000 objects
Scanned 6,000 objects
Scanned 7,000 objects
Scanned 8,000 objects
Limit of scan reached - 8,000 objects
2020-08-05 13:43:01,184 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - marker scan s3a://landsat-pds/: duration 0:05.107s
No surplus directory markers were found under s3a://landsat-pds/
Listing limit reached before completing the scan
2020-08-05 13:43:01,187 [main] INFO util.ExitUtil (ExitUtil.java:terminate(210)) - Exiting with status 3:
```
Here the scan reached its object limit before completing the audit; the exit code of 3, "interrupted" indicates this.
Example: a verbose audit of a bucket whose policy if authoritative -it is not an error if markers
are found under the path `/tables`.
```
> bin/hadoop s3guard markers -audit s3a://london/
2020-08-05 18:29:16,473 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) - Directory markers will be kept on authoritative paths
The directory marker policy of s3a://london is "Authoritative"
Authoritative path list is "/tables"
2020-08-05 18:29:19,186 [main] INFO tools.MarkerTool (DurationInfo.java:<init>(77)) - Starting: marker scan s3a://london/
2020-08-05 18:29:21,610 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - marker scan s3a://london/: duration 0:02.425s
Listed 8 objects under s3a://london/
Found 3 surplus directory markers under s3a://london/
s3a://london/tables
s3a://london/tables/tables-4
s3a://london/tables/tables-4/tables-5
Found 5 empty directory 'leaf' markers under s3a://london/
s3a://london/tables/tables-2
s3a://london/tables/tables-3
s3a://london/tables/tables-4/tables-5/06
s3a://london/tables2
s3a://london/tables3
These are required to indicate empty directories
Surplus markers were found -failing audit
2020-08-05 18:29:21,614 [main] INFO util.ExitUtil (ExitUtil.java:terminate(210)) - Exiting with status 46:
```
This fails because surplus markers were found. This S3A bucket would *NOT* be safe for older Hadoop versions
to use.
The `-nonauth` option does not treat markers under authoritative paths as errors:
```
bin/hadoop s3guard markers -nonauth -audit s3a://london/
2020-08-05 18:31:16,255 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) - Directory markers will be kept on authoritative paths
The directory marker policy of s3a://london is "Authoritative"
Authoritative path list is "/tables"
2020-08-05 18:31:19,210 [main] INFO tools.MarkerTool (DurationInfo.java:<init>(77)) - Starting: marker scan s3a://london/
2020-08-05 18:31:22,240 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - marker scan s3a://london/: duration 0:03.031s
Listed 8 objects under s3a://london/
Found 3 surplus directory markers under s3a://london/
s3a://london/tables/
s3a://london/tables/tables-4/
s3a://london/tables/tables-4/tables-5/
Found 5 empty directory 'leaf' markers under s3a://london/
s3a://london/tables/tables-2/
s3a://london/tables/tables-3/
s3a://london/tables/tables-4/tables-5/06/
s3a://london/tables2/
s3a://london/tables3/
These are required to indicate empty directories
Ignoring 3 markers in authoritative paths
```
All of this S3A bucket _other_ than the authoritative path `/tables` will be safe for
incompatible Hadoop releases to to use.
### <a name="marker-tool-clean"></a>`markers clean`
The `markers clean` command will clean the directory tree of all surplus markers.
The `-verbose` option prints more detail on the operation as well as some IO statistics
```
> hadoop s3guard markers -verbose -clean s3a://london/
2020-08-05 18:33:25,303 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) - Directory markers will be kept on authoritative paths
The directory marker policy of s3a://london is "Authoritative"
Authoritative path list is "/tables"
2020-08-05 18:33:28,511 [main] INFO tools.MarkerTool (DurationInfo.java:<init>(77)) - Starting: marker scan s3a://london/
Directory Marker tables
Directory Marker tables/tables-2/
Directory Marker tables/tables-3/
Directory Marker tables/tables-4/
Directory Marker tables/tables-4/tables-5/
Directory Marker tables/tables-4/tables-5/06/
Directory Marker tables2/
Directory Marker tables3/
2020-08-05 18:33:31,685 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - marker scan s3a://london/: duration 0:03.175s
Listed 8 objects under s3a://london/
Found 3 surplus directory markers under s3a://london/
s3a://london/tables/
s3a://london/tables/tables-4/
s3a://london/tables/tables-4/tables-5/
Found 5 empty directory 'leaf' markers under s3a://london/
s3a://london/tables/tables-2/
s3a://london/tables/tables-3/
s3a://london/tables/tables-4/tables-5/06/
s3a://london/tables2/
s3a://london/tables3/
These are required to indicate empty directories
3 markers to delete in 1 page of 250 keys/page
2020-08-05 18:33:31,688 [main] INFO tools.MarkerTool (DurationInfo.java:<init>(77)) - Starting: Deleting markers
2020-08-05 18:33:31,812 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - Deleting markers: duration 0:00.124s
Storage Statistics for s3a://london
op_get_file_status 1
object_delete_requests 1
object_list_requests 2
```
The `markers -clean` command _does not_ delete markers above empty directories -only those which have
files underneath. If invoked on a path, it will clean up the directory tree into a state
where it is safe for older versions of Hadoop to interact with.
Note that if invoked with a `-limit` value, surplus markers found during the scan will be removed,
even though the scan will be considered a failure due to the limit being reached.
## <a name="advanced-topics"></a> Advanced Topics
### <a name="pathcapabilities"></a> Probing for retention via `PathCapabilities` and `StreamCapabilities`
An instance of the filesystem can be probed for its directory marker retention ability/
policy can be probed for through the `org.apache.hadoop.fs.PathCapabilities` interface,
which all FileSystem classes have supported since Hadoop 3.3.
| Probe | Meaning |
|-------------------------|-------------------------|
| `fs.s3a.capability.directory.marker.aware` | Does the filesystem support surplus directory markers? |
| `fs.s3a.capability.directory.marker.policy.delete` | Is the bucket policy "delete"? |
| `fs.s3a.capability.directory.marker.policy.keep` | Is the bucket policy "keep"? |
| `fs.s3a.capability.directory.marker.policy.authoritative` | Is the bucket policy "authoritative"? |
| `fs.s3a.capability.directory.marker.action.delete` | If a file was created at this path, would directory markers be deleted? |
| `fs.s3a.capability.directory.marker.action.keep` | If a file was created at this path, would directory markers be retained? |
The probe `fs.s3a.capability.directory.marker.aware` allows for a filesystem to be
probed to determine if its file listing policy is "aware" of directory marker retention
-that is: can this s3a client safely work with S3 buckets where markers have not been deleted.
The `fs.s3a.capability.directory.marker.policy.` probes return the active policy for the bucket.
The two `fs.s3a.capability.directory.marker.action.` probes dynamically query the marker
retention behavior of a specific path.
That is: if a file was created at that location, would ancestor directory markers
be kept or deleted?
The `S3AFileSystem` class also implements the `org.apache.hadoop.fs.StreamCapabilities` interface, which
can be used to probe for marker awareness via the `fs.s3a.capability.directory.marker.aware` capability.
Again, this will be true if-and-only-if the S3A connector is safe to work with S3A buckets/paths where
directories are retained.
*If an S3A instance, probed by `PathCapabilities` or `StreamCapabilities` for the capability
`fs.s3a.capability.directory.marker.aware` and it returns false, *it is not safe to be used with
S3A paths where markers have been retained*.
This is programmatic probe -however it can be accessed on the command line via the
external [`cloudstore`](https://github.com/steveloughran/cloudstore) tool:
```
> hadoop jar cloudstore-1.0.jar pathcapability fs.s3a.capability.directory.marker.aware s3a://london/
Probing s3a://london/ for capability fs.s3a.capability.directory.marker.aware
Using filesystem s3a://london
Path s3a://london/ has capability fs.s3a.capability.directory.marker.aware
```
If the exit code of the command is `0`, then the S3A is safe to work with buckets
where markers have not been deleted.
The same tool can be used to dynamically probe for the policy.
Take a bucket with a retention policy of "authoritative" -only paths under `/tables` will have markers retained.
```xml
<property>
<name>fs.s3a.bucket.london.directory.marker.retention</name>
<value>authoritative</value>
</property>
<property>
<name>fs.s3a.bucket.london.authoritative.path</name>
<value>/tables</value>
</property>```
```
With this policy the path capability `fs.s3a.capability.directory.marker.action.keep` will hold under
the path `s3a://london/tables`
```
bin/hadoop jar cloudstore-1.0.jar pathcapability fs.s3a.capability.directory.marker.action.keep s3a://london/tables
Probing s3a://london/tables for capability fs.s3a.capability.directory.marker.action.keep
2020-08-11 22:03:31,658 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143))
- Directory markers will be kept on authoritative paths
Using filesystem s3a://london
Path s3a://london/tables has capability fs.s3a.capability.directory.marker.action.keep
```
However it will not hold for other paths, so indicating that older Hadoop versions will be safe
to work with data written there by this S3A client.
```
bin/hadoop jar cloudstore-1.0.jar pathcapability fs.s3a.capability.directory.marker.action.keep s3a://london/tempdir
Probing s3a://london/tempdir for capability fs.s3a.capability.directory.marker.action.keep
2020-08-11 22:06:56,300 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143))
- Directory markers will be kept on authoritative paths
Using filesystem s3a://london
Path s3a://london/tempdir lacks capability fs.s3a.capability.directory.marker.action.keep
2020-08-11 22:06:56,308 [main] INFO util.ExitUtil (ExitUtil.java:terminate(210)) - Exiting with status -1:
```
## <a name="glossary"></a> Glossary
#### Directory Marker
An object in an S3 bucket with a trailing "/", used to indicate that there is a directory at that location.
These are necessary to maintain expectations about directories in an object store:
1. After `mkdirs(path)`, `exists(path)` holds.
1. After `rm(path/*)`, `exists(path)` holds.
In previous releases of Hadoop, the marker created by a `mkdirs()` operation was deleted after a file was created.
Rather than make a slow HEAD probe + optional marker DELETE of every parent path element, HADOOP-13164 switched
to enumerating all parent paths and issuing a single bulk DELETE request.
This is faster under light load, but
as each row in the delete consumes one write operation on the allocated IOPs of that bucket partition, creates
load issues when many worker threads/processes are writing to files.
This problem is bad on Apache Hive as:
* The hive partition structure places all files within the same S3 partition.
* As they are deep structures, there are many parent entries to include in the bulk delete calls.
* It's creating a lot temporary files, and still uses rename to commit output.
Apache Spark has less of an issue when an S3A committer is used -although the partition structure
is the same, the delayed manifestation of output files reduces load.
#### Leaf Marker
A directory marker which has not files or directory marker objects underneath.
It genuinely represents an empty directory.
#### Surplus Marker
A directory marker which is above one or more files, and so is superfluous.
These are the markers which were traditionally deleted; now it is optional.
Older versions of Hadoop mistake such surplus markers as Leaf Markers.
#### Versioned Bucket
An S3 Bucket which has Object Versioning enabled.
This provides a backup and recovery mechanism for data within the same
bucket: older objects can be listed and restored through the AWS S3 console
and some applications.
## References
<!-- if extending, keep JIRAs separate, have them in numerical order; the rest in lexical.` -->
* [HADOOP-13164](https://issues.apache.org/jira/browse/HADOOP-13164). _Optimize S3AFileSystem::deleteUnnecessaryFakeDirectories._
* [HADOOP-13230](https://issues.apache.org/jira/browse/HADOOP-13230). _S3A to optionally retain directory markers_
* [HADOOP-16090](https://issues.apache.org/jira/browse/HADOOP-16090). _S3A Client to add explicit support for versioned stores._
* [HADOOP-16823](https://issues.apache.org/jira/browse/HADOOP-16823). _Large DeleteObject requests are their own Thundering Herd_
* [Object Versioning](https://docs.aws.amazon.com/AmazonS3/latest/dev/Versioning.html). _Using versioning_
* [Optimizing Performance](https://docs.aws.amazon.com/AmazonS3/latest/dev/optimizing-performance.html). _Best Practices Design Patterns: Optimizing Amazon S3 Performance_

View File

@ -16,18 +16,29 @@
<!-- MACRO{toc|fromDepth=0|toDepth=2} -->
**NOTE: Hadoop's `s3:` and `s3n:` connectors have been removed.
Please use `s3a:` as the connector to data hosted in S3 with Apache Hadoop.**
**Consult the [s3n documentation](./s3n.html) for migration instructions.**
See also:
## <a name="compatibility"></a> Compatibility
### <a name="directory-marker-compatibility"></a> Directory Marker Compatibility
1. This release can safely list/index/read S3 buckets where "empty directory"
markers are retained.
1. This release can be configured to retain these directory makers at the
expense of being backwards incompatible.
Consult [Controlling the S3A Directory Marker Behavior](directory_markers.html) for
full details.
## <a name="documents"></a> Documents
* [Encryption](./encryption.html)
* [Performance](./performance.html)
* [S3Guard](./s3guard.html)
* [Troubleshooting](./troubleshooting_s3a.html)
* [Controlling the S3A Directory Marker Behavior](directory_markers.html).
* [Committing work to S3 with the "S3A Committers"](./committers.html)
* [S3A Committers Architecture](./committer_architecture.html)
* [Working with IAM Assumed Roles](./assumed_roles.html)

View File

@ -113,7 +113,19 @@ Currently the only Metadata Store-independent setting, besides the
implementation class above, are the *allow authoritative* and *fail-on-error*
flags.
#### Allow Authoritative
#### <a name="authoritative"></a> Authoritative S3Guard
Authoritative S3Guard is a complicated configuration which delivers performance
at the expense of being unsafe for other applications to use the same directory
tree/bucket unless configured consistently.
It can also be used to support [directory marker retention](directory_markers.html)
in higher-performance but non-backwards-compatible modes.
Most deployments do not use this setting -it is ony used in deployments where
specific parts of a bucket (e.g. Apache Hive managed tables) are known to
have exclusive access by a single application (Hive) and other tools/applications
from exactly the same Hadoop release.
The _authoritative_ expression in S3Guard is present in two different layers, for
two different reasons:
@ -178,7 +190,7 @@ recommended that you leave the default setting here:
<value>false</value>
</property>
```
.
Note that a MetadataStore MAY persist this bit in the directory listings. (Not
MUST).

View File

@ -324,6 +324,49 @@ Once a bucket is converted to being versioned, it cannot be converted back
to being unversioned.
## <a name="marker"></a> Testing Different Marker Retention Policy
Hadoop supports [different policies for directory marker retention](directory_markers.html)
-essentially the classic "delete" and the higher-performance "keep" options; "authoritative"
is just "keep" restricted to a part of the bucket.
Example: test with `markers=delete`
```
mvn verify -Dparallel-tests -DtestsThreadCount=4 -Dmarkers=delete
```
Example: test with `markers=keep`
```
mvn verify -Dparallel-tests -DtestsThreadCount=4 -Dmarkers=keep
```
Example: test with `markers=authoritative`
```
mvn verify -Dparallel-tests -DtestsThreadCount=4 -Dmarkers=authoritative
```
This final option is of limited use unless paths in the bucket have actually been configured to be
of mixed status; unless anything is set up then the outcome should equal that of "delete"
### Enabling auditing of markers
To enable an audit of the output directory of every test suite,
enable the option `fs.s3a.directory.marker.audit`
```
-Dfs.s3a.directory.marker.audit=true
```
When set, if the marker policy is to delete markers under the test output directory, then
the marker tool audit command will be run. This will fail if a marker was found.
This adds extra overhead to every operation, but helps verify that the connector is
not keeping markers where it needs to be deleting them -and hence backwards compatibility
is maintained.
## <a name="scale"></a> Scale Tests
There are a set of tests designed to measure the scalability and performance

View File

@ -25,16 +25,20 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.contract.s3a.S3AContract;
import org.apache.hadoop.fs.s3a.tools.MarkerTool;
import org.apache.hadoop.io.IOUtils;
import org.junit.Before;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.FileNotFoundException;
import java.io.IOException;
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestDynamoTablePrefix;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool;
import static org.apache.hadoop.fs.s3a.S3AUtils.E_FS_CLOSED;
import static org.apache.hadoop.fs.s3a.tools.MarkerTool.UNLIMITED_LISTING;
/**
* An extension of the contract test base set up for S3A tests.
@ -62,18 +66,43 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
@Override
public void teardown() throws Exception {
Thread.currentThread().setName("teardown");
maybeAuditTestPath();
super.teardown();
describe("closing file system");
IOUtils.closeStream(getFileSystem());
}
@Before
public void nameThread() {
Thread.currentThread().setName("JUnit-" + getMethodName());
}
protected String getMethodName() {
return methodName.getMethodName();
/**
* Audit the FS under {@link #methodPath()} if
* the test option {@link #DIRECTORY_MARKER_AUDIT} is
* true.
*/
public void maybeAuditTestPath() {
final S3AFileSystem fs = getFileSystem();
if (fs != null) {
try {
boolean audit = getTestPropertyBool(fs.getConf(),
DIRECTORY_MARKER_AUDIT, false);
Path methodPath = methodPath();
if (audit
&& !fs.getDirectoryMarkerPolicy()
.keepDirectoryMarkers(methodPath)
&& fs.isDirectory(methodPath)) {
MarkerTool.ScanResult result = MarkerTool.execMarkerTool(fs,
methodPath, true, 0, UNLIMITED_LISTING, false);
assertEquals("Audit of " + methodPath + " failed: " + result,
0, result.getExitCode());
}
} catch (FileNotFoundException ignored) {
} catch (Exception e) {
// If is this is not due to the FS being closed: log.
if (!e.toString().contains(E_FS_CLOSED)) {
LOG.warn("Marker Tool Failure", e);
}
}
}
}
@Override

View File

@ -109,7 +109,8 @@ public class ITestAuthoritativePath extends AbstractS3ATestBase {
URI uri = testFS.getUri();
removeBaseAndBucketOverrides(uri.getHost(), config,
METADATASTORE_AUTHORITATIVE);
METADATASTORE_AUTHORITATIVE,
AUTHORITATIVE_PATH);
config.setBoolean(METADATASTORE_AUTHORITATIVE, true);
final S3AFileSystem newFS = createFS(uri, config);
// set back the same metadata store instance
@ -124,7 +125,8 @@ public class ITestAuthoritativePath extends AbstractS3ATestBase {
URI uri = testFS.getUri();
removeBaseAndBucketOverrides(uri.getHost(), config,
METADATASTORE_AUTHORITATIVE);
METADATASTORE_AUTHORITATIVE,
AUTHORITATIVE_PATH);
config.set(AUTHORITATIVE_PATH, authPath.toString());
final S3AFileSystem newFS = createFS(uri, config);
// set back the same metadata store instance
@ -139,7 +141,8 @@ public class ITestAuthoritativePath extends AbstractS3ATestBase {
URI uri = testFS.getUri();
removeBaseAndBucketOverrides(uri.getHost(), config,
METADATASTORE_AUTHORITATIVE);
METADATASTORE_AUTHORITATIVE,
AUTHORITATIVE_PATH);
config.set(AUTHORITATIVE_PATH, first + "," + middle + "," + last);
final S3AFileSystem newFS = createFS(uri, config);
// set back the same metadata store instance
@ -155,7 +158,8 @@ public class ITestAuthoritativePath extends AbstractS3ATestBase {
removeBaseAndBucketOverrides(uri.getHost(), config,
S3_METADATA_STORE_IMPL);
removeBaseAndBucketOverrides(uri.getHost(), config,
METADATASTORE_AUTHORITATIVE);
METADATASTORE_AUTHORITATIVE,
AUTHORITATIVE_PATH);
return createFS(uri, config);
}

View File

@ -75,7 +75,11 @@ public class ITestS3ABucketExistence extends AbstractS3ATestBase {
// the exception must not be caught and marked down to an FNFE
expectUnknownStore(() -> fs.exists(src));
expectUnknownStore(() -> fs.isFile(src));
// now that isFile() only does a HEAD, it will get a 404 without
// the no-such-bucket error.
assertFalse("isFile(" + src + ")"
+ " was expected to complete by returning false",
fs.isFile(src));
expectUnknownStore(() -> fs.isDirectory(src));
expectUnknownStore(() -> fs.mkdirs(src));
expectUnknownStore(() -> fs.delete(src));

View File

@ -20,8 +20,13 @@ package org.apache.hadoop.fs.s3a;
import java.io.IOException;
import java.nio.file.AccessDeniedException;
import java.util.Arrays;
import java.util.Collection;
import org.assertj.core.api.Assertions;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@ -31,13 +36,26 @@ import org.apache.hadoop.fs.contract.s3a.S3AContract;
import org.apache.hadoop.io.IOUtils;
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP;
import static org.apache.hadoop.fs.s3a.Constants.ETAG_CHECKSUM_ENABLED;
import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM;
import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_KEY;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
* Concrete class that extends {@link AbstractTestS3AEncryption}
* and tests SSE-C encryption.
* HEAD requests against SSE-C-encrypted data will fail if the wrong key
* is presented, so the tests are very brittle to S3Guard being on vs. off.
* Equally "vexing" has been the optimizations of getFileStatus(), wherein
* LIST comes before HEAD path + /
*/
@RunWith(Parameterized.class)
public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
private static final String SERVICE_AMAZON_S3_STATUS_CODE_403
@ -52,18 +70,67 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
= "msdo3VvvZznp66Gth58a91Hxe/UpExMkwU9BHkIjfW8=";
private static final int TEST_FILE_LEN = 2048;
/**
* Parameterization.
*/
@Parameterized.Parameters(name = "{0}")
public static Collection<Object[]> params() {
return Arrays.asList(new Object[][]{
{"raw-keep-markers", false, true},
{"raw-delete-markers", false, false},
{"guarded-keep-markers", true, true},
{"guarded-delete-markers", true, false}
});
}
/**
* Parameter: should the stores be guarded?
*/
private final boolean s3guard;
/**
* Parameter: should directory markers be retained?
*/
private final boolean keepMarkers;
/**
* Filesystem created with a different key.
*/
private FileSystem fsKeyB;
private S3AFileSystem fsKeyB;
public ITestS3AEncryptionSSEC(final String name,
final boolean s3guard,
final boolean keepMarkers) {
this.s3guard = s3guard;
this.keepMarkers = keepMarkers;
}
@Override
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
disableFilesystemCaching(conf);
conf.set(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM,
String bucketName = getTestBucketName(conf);
removeBucketOverrides(bucketName, conf,
S3_METADATA_STORE_IMPL);
if (!s3guard) {
// in a raw run remove all s3guard settings
removeBaseAndBucketOverrides(bucketName, conf,
S3_METADATA_STORE_IMPL);
}
// directory marker options
removeBaseAndBucketOverrides(bucketName, conf,
DIRECTORY_MARKER_POLICY,
ETAG_CHECKSUM_ENABLED,
SERVER_SIDE_ENCRYPTION_ALGORITHM,
SERVER_SIDE_ENCRYPTION_KEY);
conf.set(DIRECTORY_MARKER_POLICY,
keepMarkers
? DIRECTORY_MARKER_POLICY_KEEP
: DIRECTORY_MARKER_POLICY_DELETE);
conf.set(SERVER_SIDE_ENCRYPTION_ALGORITHM,
getSSEAlgorithm().getMethod());
conf.set(Constants.SERVER_SIDE_ENCRYPTION_KEY, KEY_1);
conf.set(SERVER_SIDE_ENCRYPTION_KEY, KEY_1);
conf.setBoolean(ETAG_CHECKSUM_ENABLED, true);
return conf;
}
@ -109,31 +176,19 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
}
/**
* While each object has its own key and should be distinct, this verifies
* that hadoop treats object keys as a filesystem path. So if a top level
* dir is encrypted with keyA, a sublevel dir cannot be accessed with a
* different keyB.
*
* This is expected AWS S3 SSE-C behavior.
*
* You can use a different key under a sub directory, even if you
* do not have permissions to read the marker.
* @throws Exception
*/
@Test
public void testCreateSubdirWithDifferentKey() throws Exception {
requireUnguardedFilesystem();
intercept(AccessDeniedException.class,
SERVICE_AMAZON_S3_STATUS_CODE_403,
() -> {
Path base = path("testCreateSubdirWithDifferentKey");
Path nestedDirectory = new Path(base, "nestedDir");
fsKeyB = createNewFileSystemWithSSECKey(
KEY_2);
getFileSystem().mkdirs(base);
fsKeyB.mkdirs(nestedDirectory);
// expected to fail
return fsKeyB.getFileStatus(nestedDirectory);
});
Path base = path("testCreateSubdirWithDifferentKey");
Path nestedDirectory = new Path(base, "nestedDir");
fsKeyB = createNewFileSystemWithSSECKey(
KEY_2);
getFileSystem().mkdirs(base);
fsKeyB.mkdirs(nestedDirectory);
}
/**
@ -176,14 +231,11 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
}
/**
* It is possible to list the contents of a directory up to the actual
* end of the nested directories. This is due to how S3A mocks the
* directories and how prefixes work in S3.
* Directory listings always work.
* @throws Exception
*/
@Test
public void testListEncryptedDir() throws Exception {
requireUnguardedFilesystem();
Path pathABC = path("testListEncryptedDir/a/b/c/");
Path pathAB = pathABC.getParent();
@ -196,17 +248,11 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
fsKeyB.listFiles(pathA, true);
fsKeyB.listFiles(pathAB, true);
//Until this point, no exception is thrown about access
intercept(AccessDeniedException.class,
SERVICE_AMAZON_S3_STATUS_CODE_403,
() -> {
fsKeyB.listFiles(pathABC, false);
});
fsKeyB.listFiles(pathABC, false);
Configuration conf = this.createConfiguration();
conf.unset(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM);
conf.unset(Constants.SERVER_SIDE_ENCRYPTION_KEY);
conf.unset(SERVER_SIDE_ENCRYPTION_ALGORITHM);
conf.unset(SERVER_SIDE_ENCRYPTION_KEY);
S3AContract contract = (S3AContract) createContract(conf);
contract.init();
@ -215,20 +261,14 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
//unencrypted can access until the final directory
unencryptedFileSystem.listFiles(pathA, true);
unencryptedFileSystem.listFiles(pathAB, true);
AWSBadRequestException ex = intercept(AWSBadRequestException.class,
() -> {
unencryptedFileSystem.listFiles(pathABC, false);
});
unencryptedFileSystem.listFiles(pathABC, false);
}
/**
* Much like the above list encrypted directory test, you cannot get the
* metadata of an object without the correct encryption key.
* @throws Exception
* listStatus also works with encrypted directories and key mismatch.
*/
@Test
public void testListStatusEncryptedDir() throws Exception {
requireUnguardedFilesystem();
Path pathABC = path("testListStatusEncryptedDir/a/b/c/");
Path pathAB = pathABC.getParent();
@ -240,17 +280,14 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
fsKeyB.listStatus(pathA);
fsKeyB.listStatus(pathAB);
//Until this point, no exception is thrown about access
intercept(AccessDeniedException.class,
SERVICE_AMAZON_S3_STATUS_CODE_403,
() -> {
fsKeyB.listStatus(pathABC);
});
// this used to raise 403, but with LIST before HEAD,
// no longer true.
fsKeyB.listStatus(pathABC);
//Now try it with an unencrypted filesystem.
Configuration conf = createConfiguration();
conf.unset(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM);
conf.unset(Constants.SERVER_SIDE_ENCRYPTION_KEY);
conf.unset(SERVER_SIDE_ENCRYPTION_ALGORITHM);
conf.unset(SERVER_SIDE_ENCRYPTION_KEY);
S3AContract contract = (S3AContract) createContract(conf);
contract.init();
@ -259,21 +296,15 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
//unencrypted can access until the final directory
unencryptedFileSystem.listStatus(pathA);
unencryptedFileSystem.listStatus(pathAB);
intercept(AWSBadRequestException.class,
() -> {
unencryptedFileSystem.listStatus(pathABC);
});
unencryptedFileSystem.listStatus(pathABC);
}
/**
* Much like trying to access a encrypted directory, an encrypted file cannot
* have its metadata read, since both are technically an object.
* An encrypted file cannot have its metadata read.
* @throws Exception
*/
@Test
public void testListStatusEncryptedFile() throws Exception {
requireUnguardedFilesystem();
Path pathABC = path("testListStatusEncryptedFile/a/b/c/");
assertTrue("mkdirs failed", getFileSystem().mkdirs(pathABC));
@ -283,23 +314,15 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
fsKeyB = createNewFileSystemWithSSECKey(KEY_4);
//Until this point, no exception is thrown about access
intercept(AccessDeniedException.class,
SERVICE_AMAZON_S3_STATUS_CODE_403,
() -> {
fsKeyB.listStatus(fileToStat);
});
if (!fsKeyB.hasMetadataStore()) {
intercept(AccessDeniedException.class,
SERVICE_AMAZON_S3_STATUS_CODE_403,
() -> fsKeyB.listStatus(fileToStat));
} else {
fsKeyB.listStatus(fileToStat);
}
}
/**
* Skip the test case if S3Guard is enabled; generally this is because
* list and GetFileStatus calls can succeed even with different keys.
*/
protected void requireUnguardedFilesystem() {
assume("Filesystem has a metastore",
!getFileSystem().hasMetadataStore());
}
/**
* It is possible to delete directories without the proper encryption key and
* the hierarchy above it.
@ -308,7 +331,7 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
*/
@Test
public void testDeleteEncryptedObjectWithDifferentKey() throws Exception {
requireUnguardedFilesystem();
//requireUnguardedFilesystem();
Path pathABC = path("testDeleteEncryptedObjectWithDifferentKey/a/b/c/");
Path pathAB = pathABC.getParent();
@ -317,12 +340,13 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
Path fileToDelete = new Path(pathABC, "filetobedeleted.txt");
writeThenReadFile(fileToDelete, TEST_FILE_LEN);
fsKeyB = createNewFileSystemWithSSECKey(KEY_4);
intercept(AccessDeniedException.class,
SERVICE_AMAZON_S3_STATUS_CODE_403,
() -> {
fsKeyB.delete(fileToDelete, false);
});
if (!fsKeyB.hasMetadataStore()) {
intercept(AccessDeniedException.class,
SERVICE_AMAZON_S3_STATUS_CODE_403,
() -> fsKeyB.delete(fileToDelete, false));
} else {
fsKeyB.delete(fileToDelete, false);
}
//This is possible
fsKeyB.delete(pathABC, true);
fsKeyB.delete(pathAB, true);
@ -330,15 +354,33 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
assertPathDoesNotExist("expected recursive delete", fileToDelete);
}
private FileSystem createNewFileSystemWithSSECKey(String sseCKey) throws
/**
* getFileChecksum always goes to S3, so when
* the caller lacks permissions, it fails irrespective
* of guard.
*/
@Test
public void testChecksumRequiresReadAccess() throws Throwable {
Path path = path("tagged-file");
S3AFileSystem fs = getFileSystem();
touch(fs, path);
Assertions.assertThat(fs.getFileChecksum(path))
.isNotNull();
fsKeyB = createNewFileSystemWithSSECKey(KEY_4);
intercept(AccessDeniedException.class,
SERVICE_AMAZON_S3_STATUS_CODE_403,
() -> fsKeyB.getFileChecksum(path));
}
private S3AFileSystem createNewFileSystemWithSSECKey(String sseCKey) throws
IOException {
Configuration conf = this.createConfiguration();
conf.set(Constants.SERVER_SIDE_ENCRYPTION_KEY, sseCKey);
conf.set(SERVER_SIDE_ENCRYPTION_KEY, sseCKey);
S3AContract contract = (S3AContract) createContract(conf);
contract.init();
FileSystem fileSystem = contract.getTestFileSystem();
return fileSystem;
return (S3AFileSystem) fileSystem;
}
@Override

View File

@ -18,15 +18,14 @@
package org.apache.hadoop.fs.s3a;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileAlreadyExistsException;
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.impl.StatusProbeEnum;
import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
import org.assertj.core.api.Assertions;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
@ -39,26 +38,21 @@ import java.net.URI;
import java.util.Arrays;
import java.util.Collection;
import java.util.EnumSet;
import java.util.UUID;
import java.util.concurrent.Callable;
import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
import static org.apache.hadoop.fs.s3a.Statistic.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
import static org.apache.hadoop.fs.s3a.performance.OperationCost.*;
import static org.apache.hadoop.test.GenericTestUtils.getTestDir;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
* Use metrics to assert about the cost of file status queries.
* {@link S3AFileSystem#getFileStatus(Path)}.
* Parameterized on guarded vs raw.
* Use metrics to assert about the cost of file API calls.
* Parameterized on guarded vs raw. and directory marker keep vs delete
*/
@RunWith(Parameterized.class)
public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
private MetricDiff metadataRequests;
private MetricDiff listRequests;
public class ITestS3AFileOperationCost extends AbstractS3ACostTest {
private static final Logger LOG =
LoggerFactory.getLogger(ITestS3AFileOperationCost.class);
@ -69,103 +63,62 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
@Parameterized.Parameters(name = "{0}")
public static Collection<Object[]> params() {
return Arrays.asList(new Object[][]{
{"raw", false},
{"guarded", true}
{"raw-keep-markers", false, true, false},
{"raw-delete-markers", false, false, false},
{"nonauth-keep-markers", true, true, false},
{"auth-delete-markers", true, false, true}
});
}
private final String name;
private final boolean s3guard;
public ITestS3AFileOperationCost(final String name, final boolean s3guard) {
this.name = name;
this.s3guard = s3guard;
}
@Override
public Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
String bucketName = getTestBucketName(conf);
removeBucketOverrides(bucketName, conf,
S3_METADATA_STORE_IMPL);
if (!s3guard) {
// in a raw run remove all s3guard settings
removeBaseAndBucketOverrides(bucketName, conf,
S3_METADATA_STORE_IMPL);
}
disableFilesystemCaching(conf);
return conf;
}
@Override
public void setup() throws Exception {
super.setup();
if (s3guard) {
// s3guard is required for those test runs where any of the
// guard options are set
assumeS3GuardState(true, getConfiguration());
}
S3AFileSystem fs = getFileSystem();
metadataRequests = new MetricDiff(fs, OBJECT_METADATA_REQUESTS);
listRequests = new MetricDiff(fs, OBJECT_LIST_REQUESTS);
skipDuringFaultInjection(fs);
public ITestS3AFileOperationCost(final String name,
final boolean s3guard,
final boolean keepMarkers,
final boolean authoritative) {
super(s3guard, keepMarkers, authoritative);
}
/**
* Test the cost of {@code listLocatedStatus(file)}.
* There's a minor inefficiency in that calling this on
* a file in S3Guard still executes a LIST call, even
* though the file record is in the store.
*/
@Test
public void testCostOfLocatedFileStatusOnFile() throws Throwable {
describe("performing listLocatedStatus on a file");
Path file = path(getMethodName() + ".txt");
Path file = file(methodPath());
S3AFileSystem fs = getFileSystem();
touch(fs, file);
resetMetricDiffs();
fs.listLocatedStatus(file);
if (!fs.hasMetadataStore()) {
// Unguarded FS.
metadataRequests.assertDiffEquals(1);
}
listRequests.assertDiffEquals(1);
verifyMetrics(() -> fs.listLocatedStatus(file),
whenRaw(FILE_STATUS_FILE_PROBE
.plus(LIST_LOCATED_STATUS_LIST_OP)),
whenAuthoritative(LIST_LOCATED_STATUS_LIST_OP),
whenNonauth(LIST_LOCATED_STATUS_LIST_OP));
}
@Test
public void testCostOfListLocatedStatusOnEmptyDir() throws Throwable {
describe("performing listLocatedStatus on an empty dir");
Path dir = path(getMethodName());
Path dir = dir(methodPath());
S3AFileSystem fs = getFileSystem();
fs.mkdirs(dir);
resetMetricDiffs();
fs.listLocatedStatus(dir);
if (!fs.hasMetadataStore()) {
// Unguarded FS.
verifyOperationCount(2, 1);
} else {
if (fs.allowAuthoritative(dir)) {
verifyOperationCount(0, 0);
} else {
verifyOperationCount(0, 1);
}
}
verifyMetrics(() ->
fs.listLocatedStatus(dir),
whenRaw(LIST_LOCATED_STATUS_LIST_OP
.plus(GET_FILE_STATUS_ON_EMPTY_DIR)),
whenAuthoritative(NO_IO),
whenNonauth(LIST_LOCATED_STATUS_LIST_OP));
}
@Test
public void testCostOfListLocatedStatusOnNonEmptyDir() throws Throwable {
describe("performing listLocatedStatus on a non empty dir");
Path dir = path(getMethodName() + "dir");
Path dir = dir(methodPath());
S3AFileSystem fs = getFileSystem();
fs.mkdirs(dir);
Path file = new Path(dir, "file.txt");
touch(fs, file);
resetMetricDiffs();
fs.listLocatedStatus(dir);
if (!fs.hasMetadataStore()) {
// Unguarded FS.
verifyOperationCount(0, 1);
} else {
if(fs.allowAuthoritative(dir)) {
verifyOperationCount(0, 0);
} else {
verifyOperationCount(0, 1);
}
}
Path file = file(new Path(dir, "file.txt"));
verifyMetrics(() ->
fs.listLocatedStatus(dir),
whenRaw(LIST_LOCATED_STATUS_LIST_OP),
whenAuthoritative(NO_IO),
whenNonauth(LIST_LOCATED_STATUS_LIST_OP));
}
@Test
@ -174,36 +127,27 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
Path file = path(getMethodName() + ".txt");
S3AFileSystem fs = getFileSystem();
touch(fs, file);
resetMetricDiffs();
fs.listFiles(file, true);
if (!fs.hasMetadataStore()) {
metadataRequests.assertDiffEquals(1);
} else {
if (fs.allowAuthoritative(file)) {
listRequests.assertDiffEquals(0);
} else {
listRequests.assertDiffEquals(1);
}
}
verifyMetrics(() ->
fs.listFiles(file, true),
whenRaw(LIST_LOCATED_STATUS_LIST_OP
.plus(GET_FILE_STATUS_ON_FILE)),
whenAuthoritative(NO_IO),
whenNonauth(LIST_LOCATED_STATUS_LIST_OP));
}
@Test
public void testCostOfListFilesOnEmptyDir() throws Throwable {
describe("Performing listFiles() on an empty dir");
describe("Perpforming listFiles() on an empty dir with marker");
// this attem
Path dir = path(getMethodName());
S3AFileSystem fs = getFileSystem();
fs.mkdirs(dir);
resetMetricDiffs();
fs.listFiles(dir, true);
if (!fs.hasMetadataStore()) {
verifyOperationCount(2, 1);
} else {
if (fs.allowAuthoritative(dir)) {
verifyOperationCount(0, 0);
} else {
verifyOperationCount(0, 1);
}
}
verifyMetrics(() ->
fs.listFiles(dir, true),
whenRaw(LIST_FILES_LIST_OP
.plus(GET_FILE_STATUS_ON_EMPTY_DIR)),
whenAuthoritative(NO_IO),
whenNonauth(LIST_FILES_LIST_OP));
}
@Test
@ -214,17 +158,11 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
fs.mkdirs(dir);
Path file = new Path(dir, "file.txt");
touch(fs, file);
resetMetricDiffs();
fs.listFiles(dir, true);
if (!fs.hasMetadataStore()) {
verifyOperationCount(0, 1);
} else {
if (fs.allowAuthoritative(dir)) {
verifyOperationCount(0, 0);
} else {
verifyOperationCount(0, 1);
}
}
verifyMetrics(() ->
fs.listFiles(dir, true),
whenRaw(LIST_FILES_LIST_OP),
whenAuthoritative(NO_IO),
whenNonauth(LIST_FILES_LIST_OP));
}
@Test
@ -232,118 +170,70 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
describe("Performing listFiles() on a non existing dir");
Path dir = path(getMethodName());
S3AFileSystem fs = getFileSystem();
resetMetricDiffs();
intercept(FileNotFoundException.class,
() -> fs.listFiles(dir, true));
verifyOperationCount(2, 2);
verifyMetricsIntercepting(FileNotFoundException.class, "",
() -> fs.listFiles(dir, true),
whenRaw(LIST_FILES_LIST_OP
.plus(GET_FILE_STATUS_FNFE)));
}
@Test
public void testCostOfGetFileStatusOnFile() throws Throwable {
describe("performing getFileStatus on a file");
Path simpleFile = path("simple.txt");
S3AFileSystem fs = getFileSystem();
touch(fs, simpleFile);
resetMetricDiffs();
FileStatus status = fs.getFileStatus(simpleFile);
Path simpleFile = file(methodPath());
S3AFileStatus status = verifyRawInnerGetFileStatus(simpleFile, true,
StatusProbeEnum.ALL,
GET_FILE_STATUS_ON_FILE);
assertTrue("not a file: " + status, status.isFile());
if (!fs.hasMetadataStore()) {
metadataRequests.assertDiffEquals(1);
}
listRequests.assertDiffEquals(0);
}
private void resetMetricDiffs() {
reset(metadataRequests, listRequests);
}
/**
* Verify that the head and list calls match expectations,
* then reset the counters ready for the next operation.
* @param head expected HEAD count
* @param list expected LIST count
*/
private void verifyOperationCount(int head, int list) {
metadataRequests.assertDiffEquals(head);
listRequests.assertDiffEquals(list);
metadataRequests.reset();
listRequests.reset();
}
@Test
public void testCostOfGetFileStatusOnEmptyDir() throws Throwable {
describe("performing getFileStatus on an empty directory");
S3AFileSystem fs = getFileSystem();
Path dir = path("empty");
fs.mkdirs(dir);
resetMetricDiffs();
S3AFileStatus status = fs.innerGetFileStatus(dir, true,
StatusProbeEnum.ALL);
Path dir = dir(methodPath());
S3AFileStatus status = verifyRawInnerGetFileStatus(dir, true,
StatusProbeEnum.ALL,
GET_FILE_STATUS_ON_DIR_MARKER);
assertSame("not empty: " + status, Tristate.TRUE,
status.isEmptyDirectory());
if (!fs.hasMetadataStore()) {
metadataRequests.assertDiffEquals(2);
}
listRequests.assertDiffEquals(0);
// but now only ask for the directories and the file check is skipped.
resetMetricDiffs();
fs.innerGetFileStatus(dir, false,
StatusProbeEnum.DIRECTORIES);
if (!fs.hasMetadataStore()) {
metadataRequests.assertDiffEquals(1);
}
verifyRawInnerGetFileStatus(dir, false,
StatusProbeEnum.DIRECTORIES,
FILE_STATUS_DIR_PROBE);
// now look at isFile/isDir against the same entry
isDir(dir, true, FILE_STATUS_DIR_PROBE);
isFile(dir, false, FILE_STATUS_FILE_PROBE);
}
@Test
public void testCostOfGetFileStatusOnMissingFile() throws Throwable {
describe("performing getFileStatus on a missing file");
S3AFileSystem fs = getFileSystem();
Path path = path("missing");
resetMetricDiffs();
intercept(FileNotFoundException.class,
() -> fs.getFileStatus(path));
metadataRequests.assertDiffEquals(2);
listRequests.assertDiffEquals(1);
interceptRawGetFileStatusFNFE(methodPath(), false,
StatusProbeEnum.ALL,
GET_FILE_STATUS_FNFE);
}
@Test
public void testCostOfGetFileStatusOnMissingSubPath() throws Throwable {
describe("performing getFileStatus on a missing file");
S3AFileSystem fs = getFileSystem();
Path path = path("missingdir/missingpath");
resetMetricDiffs();
intercept(FileNotFoundException.class,
() -> fs.getFileStatus(path));
metadataRequests.assertDiffEquals(2);
listRequests.assertDiffEquals(1);
public void testIsDirIsFileMissingPath() throws Throwable {
describe("performing isDir and isFile on a missing file");
Path path = methodPath();
// now look at isFile/isDir against the same entry
isDir(path, false,
FILE_STATUS_DIR_PROBE);
isFile(path, false,
FILE_STATUS_FILE_PROBE);
}
@Test
public void testCostOfGetFileStatusOnNonEmptyDir() throws Throwable {
describe("performing getFileStatus on a non-empty directory");
S3AFileSystem fs = getFileSystem();
Path dir = path("empty");
fs.mkdirs(dir);
Path simpleFile = new Path(dir, "simple.txt");
touch(fs, simpleFile);
resetMetricDiffs();
S3AFileStatus status = fs.innerGetFileStatus(dir, true,
StatusProbeEnum.ALL);
if (status.isEmptyDirectory() == Tristate.TRUE) {
// erroneous state
String fsState = fs.toString();
fail("FileStatus says directory isempty: " + status
+ "\n" + ContractTestUtils.ls(fs, dir)
+ "\n" + fsState);
}
if (!fs.hasMetadataStore()) {
metadataRequests.assertDiffEquals(2);
listRequests.assertDiffEquals(1);
}
Path dir = dir(methodPath());
file(new Path(dir, "simple.txt"));
S3AFileStatus status = verifyRawInnerGetFileStatus(dir, true,
StatusProbeEnum.ALL,
GET_FILE_STATUS_ON_DIR);
assertEmptyDirStatus(status, Tristate.FALSE);
}
@Test
public void testCostOfCopyFromLocalFile() throws Throwable {
describe("testCostOfCopyFromLocalFile");
@ -361,19 +251,18 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
byte[] data = dataset(len, 'A', 'Z');
writeDataset(localFS, localPath, data, len, 1024, true);
S3AFileSystem s3a = getFileSystem();
MetricDiff copyLocalOps = new MetricDiff(s3a,
INVOCATION_COPY_FROM_LOCAL_FILE);
MetricDiff putRequests = new MetricDiff(s3a,
OBJECT_PUT_REQUESTS);
MetricDiff putBytes = new MetricDiff(s3a,
OBJECT_PUT_BYTES);
Path remotePath = path("copied");
s3a.copyFromLocalFile(false, true, localPath, remotePath);
Path remotePath = methodPath();
verifyMetrics(() -> {
s3a.copyFromLocalFile(false, true, localPath, remotePath);
return "copy";
},
with(INVOCATION_COPY_FROM_LOCAL_FILE, 1),
with(OBJECT_PUT_REQUESTS, 1),
with(OBJECT_PUT_BYTES, len));
verifyFileContents(s3a, remotePath, data);
copyLocalOps.assertDiffEquals(1);
putRequests.assertDiffEquals(1);
putBytes.assertDiffEquals(len);
// print final stats
LOG.info("Filesystem {}", s3a);
} finally {
@ -381,268 +270,123 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
}
}
private boolean reset(MetricDiff... diffs) {
for (MetricDiff diff : diffs) {
diff.reset();
}
return true;
}
@Test
public void testFakeDirectoryDeletion() throws Throwable {
describe("Verify whether create file works after renaming a file. "
+ "In S3, rename deletes any fake directories as a part of "
+ "clean up activity");
S3AFileSystem fs = getFileSystem();
Path srcBaseDir = path("src");
mkdirs(srcBaseDir);
MetricDiff deleteRequests =
new MetricDiff(fs, Statistic.OBJECT_DELETE_REQUESTS);
MetricDiff directoriesDeleted =
new MetricDiff(fs, Statistic.DIRECTORIES_DELETED);
MetricDiff fakeDirectoriesDeleted =
new MetricDiff(fs, Statistic.FAKE_DIRECTORIES_DELETED);
MetricDiff directoriesCreated =
new MetricDiff(fs, Statistic.DIRECTORIES_CREATED);
// when you call toString() on this, you get the stats
// so it gets auto-evaluated in log calls.
Object summary = new Object() {
@Override
public String toString() {
return String.format("[%s, %s, %s, %s]",
directoriesCreated, directoriesDeleted,
deleteRequests, fakeDirectoriesDeleted);
}
};
// reset operation to invoke
Callable<Boolean> reset = () ->
reset(deleteRequests, directoriesCreated, directoriesDeleted,
fakeDirectoriesDeleted);
Path srcDir = new Path(srcBaseDir, "1/2/3/4/5/6");
int srcDirDepth = directoriesInPath(srcDir);
// one dir created, one removed
mkdirs(srcDir);
String state = "after mkdir(srcDir) " + summary;
directoriesCreated.assertDiffEquals(state, 1);
deleteRequests.assertDiffEquals(state, 1);
directoriesDeleted.assertDiffEquals(state, 0);
// HADOOP-14255 deletes unnecessary fake directory objects in mkdirs()
fakeDirectoriesDeleted.assertDiffEquals(state, srcDirDepth - 1);
reset.call();
// creating a file should trigger demise of the src dir
final Path srcFilePath = new Path(srcDir, "source.txt");
touch(fs, srcFilePath);
state = "after touch(fs, srcFilePath) " + summary;
deleteRequests.assertDiffEquals(state, 1);
directoriesCreated.assertDiffEquals(state, 0);
directoriesDeleted.assertDiffEquals(state, 0);
fakeDirectoriesDeleted.assertDiffEquals(state, srcDirDepth);
reset.call();
// create a directory tree, expect the dir to be created and
// a request to delete all parent directories made.
Path destBaseDir = path("dest");
Path destDir = new Path(destBaseDir, "1/2/3/4/5/6");
Path destFilePath = new Path(destDir, "dest.txt");
mkdirs(destDir);
state = "after mkdir(destDir) " + summary;
int destDirDepth = directoriesInPath(destDir);
directoriesCreated.assertDiffEquals(state, 1);
deleteRequests.assertDiffEquals(state, 1);
directoriesDeleted.assertDiffEquals(state, 0);
fakeDirectoriesDeleted.assertDiffEquals(state, destDirDepth - 1);
// create a new source file.
// Explicitly use a new path object to guarantee that the parent paths
// are different object instances
final Path srcFile2 = new Path(srcDir.toUri() + "/source2.txt");
touch(fs, srcFile2);
reset.call();
// rename the source file to the destination file.
// this tests the file rename path, not the dir rename path
// as srcFile2 exists, the parent dir of srcFilePath must not be created.
fs.rename(srcFilePath, destFilePath);
state = String.format("after rename(srcFilePath, destFilePath)"
+ " %s dest dir depth=%d",
summary,
destDirDepth);
directoriesCreated.assertDiffEquals(state, 0);
// one for the renamed file, one for the parent of the dest dir
deleteRequests.assertDiffEquals(state, 2);
directoriesDeleted.assertDiffEquals(state, 0);
fakeDirectoriesDeleted.assertDiffEquals(state, destDirDepth);
// these asserts come after the checks on iop counts, so they don't
// interfere
assertIsFile(destFilePath);
assertIsDirectory(srcDir);
assertPathDoesNotExist("should have gone in the rename", srcFilePath);
reset.call();
// rename the source file2 to the (no longer existing
// this tests the file rename path, not the dir rename path
// as srcFile2 exists, the parent dir of srcFilePath must not be created.
fs.rename(srcFile2, srcFilePath);
state = String.format("after rename(%s, %s) %s dest dir depth=%d",
srcFile2, srcFilePath,
summary,
destDirDepth);
// here we expect there to be no fake directories
directoriesCreated.assertDiffEquals(state, 0);
// one for the renamed file only
deleteRequests.assertDiffEquals(state, 1);
directoriesDeleted.assertDiffEquals(state, 0);
fakeDirectoriesDeleted.assertDiffEquals(state, 0);
}
private int directoriesInPath(Path path) {
return path.isRoot() ? 0 : 1 + directoriesInPath(path.getParent());
}
@Test
public void testCostOfRootRename() throws Throwable {
describe("assert that a root directory rename doesn't"
+ " do much in terms of parent dir operations");
S3AFileSystem fs = getFileSystem();
// unique name, so that even when run in parallel tests, there's no conflict
String uuid = UUID.randomUUID().toString();
Path src = new Path("/src-" + uuid);
Path dest = new Path("/dest-" + uuid);
try {
MetricDiff deleteRequests =
new MetricDiff(fs, Statistic.OBJECT_DELETE_REQUESTS);
MetricDiff directoriesDeleted =
new MetricDiff(fs, Statistic.DIRECTORIES_DELETED);
MetricDiff fakeDirectoriesDeleted =
new MetricDiff(fs, Statistic.FAKE_DIRECTORIES_DELETED);
MetricDiff directoriesCreated =
new MetricDiff(fs, Statistic.DIRECTORIES_CREATED);
touch(fs, src);
fs.rename(src, dest);
Object summary = new Object() {
@Override
public String toString() {
return String.format("[%s, %s, %s, %s]",
directoriesCreated, directoriesDeleted,
deleteRequests, fakeDirectoriesDeleted);
}
};
String state = String.format("after touch(%s) %s",
src, summary);
touch(fs, src);
fs.rename(src, dest);
directoriesCreated.assertDiffEquals(state, 0);
state = String.format("after rename(%s, %s) %s",
src, dest, summary);
// here we expect there to be no fake directories
directoriesCreated.assertDiffEquals(state, 0);
// one for the renamed file only
deleteRequests.assertDiffEquals(state, 1);
directoriesDeleted.assertDiffEquals(state, 0);
fakeDirectoriesDeleted.assertDiffEquals(state, 0);
// delete that destination file, assert only the file delete was issued
reset(deleteRequests, directoriesCreated, directoriesDeleted,
fakeDirectoriesDeleted);
fs.delete(dest, false);
// here we expect there to be no fake directories
directoriesCreated.assertDiffEquals(state, 0);
// one for the deleted file
deleteRequests.assertDiffEquals(state, 1);
directoriesDeleted.assertDiffEquals(state, 0);
fakeDirectoriesDeleted.assertDiffEquals(state, 0);
} finally {
fs.delete(src, false);
fs.delete(dest, false);
}
}
@Test
public void testDirProbes() throws Throwable {
describe("Test directory probe cost -raw only");
describe("Test directory probe cost");
assumeUnguarded();
S3AFileSystem fs = getFileSystem();
assume("Unguarded FS only", !fs.hasMetadataStore());
String dir = "testEmptyDirHeadProbe";
Path emptydir = path(dir);
// Create the empty directory.
fs.mkdirs(emptydir);
Path emptydir = dir(methodPath());
// metrics and assertions.
resetMetricDiffs();
// head probe fails
interceptRawGetFileStatusFNFE(emptydir, false,
StatusProbeEnum.HEAD_ONLY,
FILE_STATUS_FILE_PROBE);
intercept(FileNotFoundException.class, () ->
fs.innerGetFileStatus(emptydir, false,
StatusProbeEnum.HEAD_ONLY));
verifyOperationCount(1, 0);
// a LIST will find it and declare as empty
S3AFileStatus status = verifyRawInnerGetFileStatus(emptydir, true,
StatusProbeEnum.LIST_ONLY,
FILE_STATUS_DIR_PROBE);
assertEmptyDirStatus(status, Tristate.TRUE);
// a LIST will find it -but it doesn't consider it an empty dir.
S3AFileStatus status = fs.innerGetFileStatus(emptydir, true,
StatusProbeEnum.LIST_ONLY);
verifyOperationCount(0, 1);
Assertions.assertThat(status)
.describedAs("LIST output is not considered empty")
.matches(s -> !s.isEmptyDirectory().equals(Tristate.TRUE), "is empty");
// finally, skip all probes and expect no operations toThere are
// take place
intercept(FileNotFoundException.class, () ->
fs.innerGetFileStatus(emptydir, false,
EnumSet.noneOf(StatusProbeEnum.class)));
verifyOperationCount(0, 0);
// skip all probes and expect no operations to take place
interceptRawGetFileStatusFNFE(emptydir, false,
EnumSet.noneOf(StatusProbeEnum.class),
NO_IO);
// now add a trailing slash to the key and use the
// deep internal s3GetFileStatus method call.
String emptyDirTrailingSlash = fs.pathToKey(emptydir.getParent())
+ "/" + dir + "/";
+ "/" + emptydir.getName() + "/";
// A HEAD request does not probe for keys with a trailing /
intercept(FileNotFoundException.class, () ->
interceptRaw(FileNotFoundException.class, "",
NO_IO, () ->
fs.s3GetFileStatus(emptydir, emptyDirTrailingSlash,
StatusProbeEnum.HEAD_ONLY, null));
verifyOperationCount(0, 0);
StatusProbeEnum.HEAD_ONLY, null, false));
// but ask for a directory marker and you get the entry
status = fs.s3GetFileStatus(emptydir,
emptyDirTrailingSlash,
StatusProbeEnum.DIR_MARKER_ONLY, null);
verifyOperationCount(1, 0);
status = verifyRaw(FILE_STATUS_DIR_PROBE, () ->
fs.s3GetFileStatus(emptydir,
emptyDirTrailingSlash,
StatusProbeEnum.LIST_ONLY,
null,
true));
assertEquals(emptydir, status.getPath());
assertEmptyDirStatus(status, Tristate.TRUE);
}
@Test
public void testNeedEmptyDirectoryProbeRequiresList() throws Throwable {
S3AFileSystem fs = getFileSystem();
intercept(IllegalArgumentException.class, "", () ->
fs.s3GetFileStatus(new Path("/something"), "/something",
StatusProbeEnum.HEAD_ONLY, null, true));
}
@Test
public void testCreateCost() throws Throwable {
describe("Test file creation cost -raw only");
S3AFileSystem fs = getFileSystem();
assume("Unguarded FS only", !fs.hasMetadataStore());
resetMetricDiffs();
Path testFile = path("testCreateCost");
assumeUnguarded();
Path testFile = methodPath();
// when overwrite is false, the path is checked for existence.
try (FSDataOutputStream out = fs.create(testFile, false)) {
verifyOperationCount(2, 1);
}
create(testFile, false,
CREATE_FILE_NO_OVERWRITE);
// but when true: only the directory checks take place.
try (FSDataOutputStream out = fs.create(testFile, true)) {
verifyOperationCount(1, 1);
}
create(testFile, true, CREATE_FILE_OVERWRITE);
}
@Test
public void testCreateCostFileExists() throws Throwable {
describe("Test cost of create file failing with existing file");
assumeUnguarded();
Path testFile = file(methodPath());
// now there is a file there, an attempt with overwrite == false will
// fail on the first HEAD.
interceptRaw(FileAlreadyExistsException.class, "",
FILE_STATUS_FILE_PROBE,
() -> file(testFile, false));
}
@Test
public void testCreateCostDirExists() throws Throwable {
describe("Test cost of create file failing with existing dir");
assumeUnguarded();
Path testFile = dir(methodPath());
// now there is a file there, an attempt with overwrite == false will
// fail on the first HEAD.
interceptRaw(FileAlreadyExistsException.class, "",
GET_FILE_STATUS_ON_DIR_MARKER,
() -> file(testFile, false));
}
/**
* Use the builder API.
* This always looks for a parent unless the caller says otherwise.
*/
@Test
public void testCreateBuilder() throws Throwable {
describe("Test builder file creation cost -raw only");
assumeUnguarded();
Path testFile = methodPath();
dir(testFile.getParent());
// builder defaults to looking for parent existence (non-recursive)
buildFile(testFile, false, false,
GET_FILE_STATUS_FNFE // destination file
.plus(FILE_STATUS_DIR_PROBE)); // parent dir
// recursive = false and overwrite=true:
// only make sure the dest path isn't a directory.
buildFile(testFile, true, true,
FILE_STATUS_DIR_PROBE);
// now there is a file there, an attempt with overwrite == false will
// fail on the first HEAD.
interceptRaw(FileAlreadyExistsException.class, "",
GET_FILE_STATUS_ON_FILE,
() -> buildFile(testFile, false, true,
GET_FILE_STATUS_ON_FILE));
}
@Test
@ -656,15 +400,15 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
// create a bunch of files
int filesToCreate = 10;
for (int i = 0; i < filesToCreate; i++) {
try (FSDataOutputStream out = fs.create(basePath.suffix("/" + i))) {
verifyOperationCount(1, 1);
}
create(basePath.suffix("/" + i));
}
fs.globStatus(basePath.suffix("/*"));
// 2 head + 1 list from getFileStatus on path,
// plus 1 list to match the glob pattern
verifyOperationCount(2, 2);
verifyRaw(GET_FILE_STATUS_ON_DIR
.plus(LIST_OPERATION),
() -> fs.globStatus(basePath.suffix("/*")));
}
@Test
@ -678,14 +422,14 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase {
// create a single file, globStatus returning a single file on a pattern
// triggers attempts at symlinks resolution if configured
String fileName = "/notASymlinkDOntResolveMeLikeOne";
try (FSDataOutputStream out = fs.create(basePath.suffix(fileName))) {
verifyOperationCount(1, 1);
}
fs.globStatus(basePath.suffix("/*"));
create(basePath.suffix(fileName));
// unguarded: 2 head + 1 list from getFileStatus on path,
// plus 1 list to match the glob pattern
// no additional operations from symlink resolution
verifyOperationCount(2, 2);
verifyRaw(GET_FILE_STATUS_ON_DIR
.plus(LIST_OPERATION),
() -> fs.globStatus(basePath.suffix("/*")));
}
}

View File

@ -326,7 +326,7 @@ public class ITestS3ARemoteFileChanged extends AbstractS3ATestBase {
* @return a number >= 0.
*/
private int getFileStatusHeadCount() {
return authMode ? 0 : 1;
return authMode ? 0 : 0;
}
/**

View File

@ -26,6 +26,7 @@ import java.util.stream.Stream;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.model.ListObjectsV2Request;
import com.amazonaws.services.s3.model.ListObjectsV2Result;
import com.amazonaws.services.s3.model.ObjectMetadata;
import com.amazonaws.services.s3.model.PutObjectRequest;
import com.amazonaws.services.s3.model.S3ObjectSummary;
import org.assertj.core.api.Assertions;
@ -57,6 +58,10 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.getStatusWithEmptyDirFlag;
*/
public class ITestS3GuardEmptyDirs extends AbstractS3ATestBase {
/**
* Rename an empty directory, verify that the empty dir
* marker moves in both S3Guard and in the S3A FS.
*/
@Test
public void testRenameEmptyDir() throws Throwable {
S3AFileSystem fs = getFileSystem();
@ -67,7 +72,7 @@ public class ITestS3GuardEmptyDirs extends AbstractS3ATestBase {
String destDirMarker = fs.pathToKey(destDir) + "/";
// set things up.
mkdirs(sourceDir);
// there'a source directory marker
// there's source directory marker
fs.getObjectMetadata(sourceDirMarker);
S3AFileStatus srcStatus = getEmptyDirStatus(sourceDir);
assertEquals("Must be an empty dir: " + srcStatus, Tristate.TRUE,
@ -82,8 +87,12 @@ public class ITestS3GuardEmptyDirs extends AbstractS3ATestBase {
() -> getEmptyDirStatus(sourceDir));
// and verify that there's no dir marker hidden under a tombstone
intercept(FileNotFoundException.class,
() -> Invoker.once("HEAD", sourceDirMarker,
() -> fs.getObjectMetadata(sourceDirMarker)));
() -> Invoker.once("HEAD", sourceDirMarker, () -> {
ObjectMetadata md = fs.getObjectMetadata(sourceDirMarker);
return String.format("Object %s of length %d",
sourceDirMarker, md.getInstanceLength());
}));
// the parent dir mustn't be confused
S3AFileStatus baseStatus = getEmptyDirStatus(basePath);
assertEquals("Must not be an empty dir: " + baseStatus, Tristate.FALSE,

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.contract.s3a.S3AContract;
import com.amazonaws.services.s3.model.S3ObjectSummary;
import com.google.common.collect.Lists;
import org.assertj.core.api.Assertions;
import org.junit.Assume;
@ -560,24 +561,23 @@ public class ITestS3GuardListConsistency extends AbstractS3ATestBase {
+ " paths");
ListObjectsV2Result postDeleteDelimited = listObjectsV2(fs, key, "/");
assertListSizeEqual(
boolean stripTombstones = false;
assertObjectSummariesEqual(
"InconsistentAmazonS3Client added back objects incorrectly " +
"in a non-recursive listing",
preDeleteDelimited.getObjectSummaries(),
postDeleteDelimited.getObjectSummaries());
preDeleteDelimited, postDeleteDelimited,
stripTombstones);
assertListSizeEqual("InconsistentAmazonS3Client added back prefixes incorrectly " +
"in a non-recursive listing",
preDeleteDelimited.getCommonPrefixes(),
postDeleteDelimited.getCommonPrefixes()
);
postDeleteDelimited.getCommonPrefixes());
LOG.info("Executing Deep listing");
ListObjectsV2Result postDeleteUndelimited = listObjectsV2(fs, key, null);
assertListSizeEqual("InconsistentAmazonS3Client added back objects incorrectly " +
"in a recursive listing",
preDeleteUndelimited.getObjectSummaries(),
postDeleteUndelimited.getObjectSummaries()
);
assertObjectSummariesEqual("InconsistentAmazonS3Client added back objects"
+ " incorrectly in a recursive listing",
preDeleteUndelimited, postDeleteUndelimited,
stripTombstones);
assertListSizeEqual("InconsistentAmazonS3Client added back prefixes incorrectly " +
"in a recursive listing",
@ -586,6 +586,24 @@ public class ITestS3GuardListConsistency extends AbstractS3ATestBase {
);
}
private void assertObjectSummariesEqual(final String message,
final ListObjectsV2Result expected,
final ListObjectsV2Result actual,
final boolean stripTombstones) {
assertCollectionsEqual(
message,
stringify(expected.getObjectSummaries(), stripTombstones),
stringify(actual.getObjectSummaries(), stripTombstones));
}
List<String> stringify(List<S3ObjectSummary> objects,
boolean stripTombstones) {
return objects.stream()
.filter(s -> !stripTombstones || !(s.getKey().endsWith("/")))
.map(s -> s.getKey())
.collect(Collectors.toList());
}
/**
* Require the v2 S3 list API.
*/
@ -682,6 +700,22 @@ public class ITestS3GuardListConsistency extends AbstractS3ATestBase {
versionId, locatedFileStatus.getVersionId());
}
/**
* Assert that the two collections match using
* object equality of the elements within.
* @param message text for the assertion
* @param expected expected list
* @param actual actual list
* @param <T> type of list
*/
private <T> void assertCollectionsEqual(String message,
Collection<T> expected,
Collection<T> actual) {
Assertions.assertThat(actual)
.describedAs(message)
.containsExactlyInAnyOrderElementsOf(expected);
}
/**
* Assert that the two list sizes match; failure message includes the lists.
* @param message text for the assertion

View File

@ -56,7 +56,8 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.toChar;
import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile;
import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH;
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_METADATASTORE_METADATA_TTL;
import static org.apache.hadoop.fs.s3a.Constants.CHANGE_DETECT_MODE;
import static org.apache.hadoop.fs.s3a.Constants.CHANGE_DETECT_MODE_NONE;
import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE;
import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_METADATA_TTL;
import static org.apache.hadoop.fs.s3a.Constants.RETRY_INTERVAL;
@ -169,12 +170,16 @@ public class ITestS3GuardOutOfBandOperations extends AbstractS3ATestBase {
RETRY_LIMIT,
RETRY_INTERVAL,
S3GUARD_CONSISTENCY_RETRY_INTERVAL,
S3GUARD_CONSISTENCY_RETRY_LIMIT);
S3GUARD_CONSISTENCY_RETRY_LIMIT,
CHANGE_DETECT_MODE,
METADATASTORE_METADATA_TTL);
conf.setInt(RETRY_LIMIT, 3);
conf.setInt(S3GUARD_CONSISTENCY_RETRY_LIMIT, 3);
conf.set(CHANGE_DETECT_MODE, CHANGE_DETECT_MODE_NONE);
final String delay = "10ms";
conf.set(RETRY_INTERVAL, delay);
conf.set(S3GUARD_CONSISTENCY_RETRY_INTERVAL, delay);
conf.set(METADATASTORE_METADATA_TTL, delay);
return conf;
}
@ -232,12 +237,13 @@ public class ITestS3GuardOutOfBandOperations extends AbstractS3ATestBase {
URI uri = testFS.getUri();
removeBaseAndBucketOverrides(uri.getHost(), config,
CHANGE_DETECT_MODE,
METADATASTORE_AUTHORITATIVE,
METADATASTORE_METADATA_TTL,
AUTHORITATIVE_PATH);
config.setBoolean(METADATASTORE_AUTHORITATIVE, authoritativeMode);
config.setLong(METADATASTORE_METADATA_TTL,
DEFAULT_METADATASTORE_METADATA_TTL);
5_000);
final S3AFileSystem gFs = createFS(uri, config);
// set back the same metadata store instance
gFs.setMetadataStore(realMs);
@ -857,7 +863,7 @@ public class ITestS3GuardOutOfBandOperations extends AbstractS3ATestBase {
expectedLength, guardedLength);
} else {
assertEquals(
"File length in authoritative table with " + stats,
"File length in non-authoritative table with " + stats,
expectedLength, guardedLength);
}
}

View File

@ -87,10 +87,15 @@ public interface S3ATestConstants {
*/
String KEY_CSVTEST_FILE = S3A_SCALE_TEST + "csvfile";
/**
* The landsat bucket: {@value}.
*/
String LANDSAT_BUCKET = "s3a://landsat-pds/";
/**
* Default path for the multi MB test file: {@value}.
*/
String DEFAULT_CSVTEST_FILE = "s3a://landsat-pds/scene_list.gz";
String DEFAULT_CSVTEST_FILE = LANDSAT_BUCKET + "scene_list.gz";
/**
* Name of the property to define the timeout for scale tests: {@value}.
@ -218,4 +223,10 @@ public interface S3ATestConstants {
*/
String S3GUARD_DDB_TEST_TABLE_NAME_KEY =
"fs.s3a.s3guard.ddb.test.table";
/**
* Test option to enable audits of the method path after
* every test case.
*/
String DIRECTORY_MARKER_AUDIT = "fs.s3a.directory.marker.audit";
}

View File

@ -618,6 +618,14 @@ public final class S3ATestUtils {
// add this so that even on tests where the FS is shared,
// the FS is always "magic"
conf.setBoolean(MAGIC_COMMITTER_ENABLED, true);
// directory marker policy
String directoryRetention = getTestProperty(
conf,
DIRECTORY_MARKER_POLICY,
DEFAULT_DIRECTORY_MARKER_POLICY);
conf.set(DIRECTORY_MARKER_POLICY, directoryRetention);
return conf;
}
@ -882,7 +890,8 @@ public final class S3ATestUtils {
public static S3AFileStatus getStatusWithEmptyDirFlag(
final S3AFileSystem fs,
final Path dir) throws IOException {
return fs.innerGetFileStatus(dir, true, StatusProbeEnum.ALL);
return fs.innerGetFileStatus(dir, true,
StatusProbeEnum.ALL);
}
/**
@ -1441,4 +1450,26 @@ public final class S3ATestUtils {
.collect(Collectors.toCollection(TreeSet::new));
return threads;
}
/**
* Call the package-private {@code innerGetFileStatus()} method
* on the passed in FS.
* @param fs filesystem
* @param path path
* @param needEmptyDirectoryFlag look for empty directory
* @param probes file status probes to perform
* @return the status
* @throws IOException
*/
public static S3AFileStatus innerGetFileStatus(
S3AFileSystem fs,
Path path,
boolean needEmptyDirectoryFlag,
Set<StatusProbeEnum> probes) throws IOException {
return fs.innerGetFileStatus(
path,
needEmptyDirectoryFlag,
probes);
}
}

View File

@ -76,11 +76,15 @@ public class TestS3AGetFileStatus extends AbstractS3AMockTest {
String key = path.toUri().getPath().substring(1);
when(s3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key))))
.thenThrow(NOT_FOUND);
ObjectMetadata meta = new ObjectMetadata();
meta.setContentLength(0L);
when(s3.getObjectMetadata(argThat(
correctGetMetadataRequest(BUCKET, key + "/"))
)).thenReturn(meta);
String keyDir = key + "/";
ListObjectsV2Result listResult = new ListObjectsV2Result();
S3ObjectSummary objectSummary = new S3ObjectSummary();
objectSummary.setKey(keyDir);
objectSummary.setSize(0L);
listResult.getObjectSummaries().add(objectSummary);
when(s3.listObjectsV2(argThat(
matchListV2Request(BUCKET, keyDir))
)).thenReturn(listResult);
FileStatus stat = fs.getFileStatus(path);
assertNotNull(stat);
assertEquals(fs.makeQualified(path), stat.getPath());
@ -161,4 +165,14 @@ public class TestS3AGetFileStatus extends AbstractS3AMockTest {
&& request.getBucketName().equals(bucket)
&& request.getKey().equals(key);
}
private ArgumentMatcher<ListObjectsV2Request> matchListV2Request(
String bucket, String key) {
return (ListObjectsV2Request request) -> {
return request != null
&& request.getBucketName().equals(bucket)
&& request.getPrefix().equals(key);
};
}
}

View File

@ -410,8 +410,7 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
// this is HEAD + "/" on S3; get on S3Guard auth when the path exists,
accessDeniedIf(!s3guard, () ->
readonlyFS.listStatus(emptyDir));
readonlyFS.listStatus(emptyDir);
// a recursive list of the no-read-directory works because
// there is no directory marker, it becomes a LIST call.
@ -421,14 +420,9 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
// and so working.
readonlyFS.getFileStatus(noReadDir);
// empty dir checks work when guarded because even in non-auth mode
// there are no checks for directories being out of date
// without S3, the HEAD path + "/" is blocked
accessDeniedIf(!s3guard, () ->
readonlyFS.getFileStatus(emptyDir));
readonlyFS.getFileStatus(emptyDir);
// now look at a file; the outcome depends on the mode.
accessDeniedIf(!guardedInAuthMode, () ->
accessDeniedIf(!s3guard, () ->
readonlyFS.getFileStatus(subdirFile));
// irrespective of mode, the attempt to read the data will fail.
@ -443,7 +437,7 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
// This means that permissions on the file do not get checked.
// See: HADOOP-16464.
Optional<FSDataInputStream> optIn = accessDeniedIf(
!guardedInAuthMode, () -> readonlyFS.open(emptyFile));
!s3guard, () -> readonlyFS.open(emptyFile));
if (optIn.isPresent()) {
try (FSDataInputStream is = optIn.get()) {
Assertions.assertThat(is.read())
@ -461,17 +455,17 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
describe("Glob Status operations");
// baseline: the real filesystem on a subdir
globFS(getFileSystem(), subdirFile, null, false, 1);
// a file fails if not in auth mode
globFS(readonlyFS, subdirFile, null, !guardedInAuthMode, 1);
// a file fails if not guarded
globFS(readonlyFS, subdirFile, null, !s3guard, 1);
// empty directories don't fail.
FileStatus[] st = globFS(readonlyFS, emptyDir, null, !s3guard, 1);
FileStatus[] st = globFS(readonlyFS, emptyDir, null, false, 1);
if (s3guard) {
assertStatusPathEquals(emptyDir, st);
}
st = globFS(readonlyFS,
noReadWildcard,
null, !s3guard, 2);
null, false, 2);
if (s3guard) {
Assertions.assertThat(st)
.extracting(FileStatus::getPath)
@ -481,12 +475,12 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
// there is precisely one .docx file (subdir2File2.docx)
globFS(readonlyFS,
new Path(noReadDir, "*/*.docx"),
null, !s3guard, 1);
null, false, 1);
// there are no .doc files.
globFS(readonlyFS,
new Path(noReadDir, "*/*.doc"),
null, !s3guard, 0);
null, false, 0);
globFS(readonlyFS, noReadDir,
EVERYTHING, false, 1);
// and a filter without any wildcarded pattern only finds
@ -513,17 +507,14 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
true,
HIDDEN_FILE_FILTER,
true);
accessDeniedIf(!s3guard,
() -> fetcher.getFileStatuses())
.ifPresent(stats -> {
Assertions.assertThat(stats)
.describedAs("result of located scan").flatExtracting(FileStatus::getPath)
.containsExactlyInAnyOrder(
emptyFile,
subdirFile,
subdir2File1,
subdir2File2);
});
Assertions.assertThat(fetcher.getFileStatuses())
.describedAs("result of located scan")
.flatExtracting(FileStatus::getPath)
.containsExactlyInAnyOrder(
emptyFile,
subdirFile,
subdir2File1,
subdir2File2);
}
/**
@ -542,15 +533,11 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
true,
EVERYTHING,
true);
accessDeniedIf(!s3guard,
() -> fetcher.getFileStatuses())
.ifPresent(stats -> {
Assertions.assertThat(stats)
.describedAs("result of located scan")
.isNotNull()
.flatExtracting(FileStatus::getPath)
.containsExactlyInAnyOrder(subdirFile, subdir2File1);
});
Assertions.assertThat(fetcher.getFileStatuses())
.describedAs("result of located scan")
.isNotNull()
.flatExtracting(FileStatus::getPath)
.containsExactlyInAnyOrder(subdirFile, subdir2File1);
}
/**
@ -567,7 +554,7 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
true,
TEXT_FILE,
true);
accessDeniedIf(!guardedInAuthMode,
accessDeniedIf(!s3guard,
() -> fetcher.getFileStatuses())
.ifPresent(stats -> {
Assertions.assertThat(stats)
@ -631,19 +618,16 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
*/
public void checkDeleteOperations() throws Throwable {
describe("Testing delete operations");
if (!authMode) {
// unguarded or non-auth S3Guard to fail on HEAD + /
accessDenied(() -> readonlyFS.delete(emptyDir, true));
readonlyFS.delete(emptyDir, true);
if (!s3guard) {
// to fail on HEAD
accessDenied(() -> readonlyFS.delete(emptyFile, true));
} else {
// auth mode checks DDB for status and then issues the DELETE
readonlyFS.delete(emptyDir, true);
// checks DDB for status and then issues the DELETE
readonlyFS.delete(emptyFile, true);
}
// this will succeed for both as there is no subdir marker.
// this will succeed for both
readonlyFS.delete(subDir, true);
// after which it is not there
fileNotFound(() -> readonlyFS.getFileStatus(subDir));

View File

@ -0,0 +1,163 @@
/*
* 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.util.Arrays;
import java.util.Collection;
import java.util.function.Predicate;
import org.assertj.core.api.Assertions;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE;
import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP;
/**
* Unit tests for directory marker policies.
*/
@RunWith(Parameterized.class)
public class TestDirectoryMarkerPolicy extends AbstractHadoopTestBase {
@Parameterized.Parameters(name = "{0}")
public static Collection<Object[]> data() {
return Arrays.asList(new Object[][]{
{
DirectoryPolicy.MarkerPolicy.Delete,
FAIL_IF_INVOKED,
false, false
},
{
DirectoryPolicy.MarkerPolicy.Keep,
FAIL_IF_INVOKED,
true, true
},
{
DirectoryPolicy.MarkerPolicy.Authoritative,
AUTH_PATH_ONLY,
false, true
}
});
}
private final DirectoryPolicy directoryPolicy;
private final boolean expectNonAuthDelete;
private final boolean expectAuthDelete;
public TestDirectoryMarkerPolicy(
final DirectoryPolicy.MarkerPolicy markerPolicy,
final Predicate<Path> authoritativeness,
final boolean expectNonAuthDelete,
final boolean expectAuthDelete) {
this.directoryPolicy = newPolicy(markerPolicy, authoritativeness);
this.expectNonAuthDelete = expectNonAuthDelete;
this.expectAuthDelete = expectAuthDelete;
}
/**
* Create a new retention policy.
* @param markerPolicy policy option
* @param authoritativeness predicate for determining if
* a path is authoritative.
* @return the retention policy.
*/
private DirectoryPolicy newPolicy(
DirectoryPolicy.MarkerPolicy markerPolicy,
Predicate<Path> authoritativeness) {
return new DirectoryPolicyImpl(markerPolicy, authoritativeness);
}
private static final Predicate<Path> AUTH_PATH_ONLY =
(p) -> p.toUri().getPath().startsWith("/auth/");
private static final Predicate<Path> FAIL_IF_INVOKED = (p) -> {
throw new RuntimeException("failed");
};
private final Path nonAuthPath = new Path("s3a://bucket/nonauth/data");
private final Path authPath = new Path("s3a://bucket/auth/data1");
private final Path deepAuth = new Path("s3a://bucket/auth/d1/d2/data2");
/**
* Assert that a path has a retention outcome.
* @param path path
* @param retain should the marker be retained
*/
private void assertMarkerRetention(Path path, boolean retain) {
Assertions.assertThat(directoryPolicy.keepDirectoryMarkers(path))
.describedAs("Retention of path %s by %s", path, directoryPolicy)
.isEqualTo(retain);
}
/**
* Assert that a path has a capability.
*/
private void assertPathCapability(Path path,
String capability,
boolean outcome) {
Assertions.assertThat(directoryPolicy)
.describedAs("%s support for capability %s by path %s"
+ " expected as %s",
directoryPolicy, capability, path, outcome)
.matches(p -> p.hasPathCapability(path, capability) == outcome,
"pathCapability");
}
@Test
public void testNonAuthPath() throws Throwable {
assertMarkerRetention(nonAuthPath, expectNonAuthDelete);
assertPathCapability(nonAuthPath,
STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE,
!expectNonAuthDelete);
assertPathCapability(nonAuthPath,
STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP,
expectNonAuthDelete);
}
@Test
public void testAuthPath() throws Throwable {
assertMarkerRetention(authPath, expectAuthDelete);
assertPathCapability(authPath,
STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE,
!expectAuthDelete);
assertPathCapability(authPath,
STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP,
expectAuthDelete);
}
@Test
public void testDeepAuthPath() throws Throwable {
assertMarkerRetention(deepAuth, expectAuthDelete);
assertPathCapability(deepAuth,
STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE,
!expectAuthDelete);
assertPathCapability(deepAuth,
STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP,
expectAuthDelete);
}
}

View File

@ -0,0 +1,637 @@
/*
* 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.performance;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.Set;
import java.util.concurrent.Callable;
import org.assertj.core.api.Assertions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FSDataOutputStreamBuilder;
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.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.Statistic;
import org.apache.hadoop.fs.s3a.Tristate;
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
import static org.apache.hadoop.fs.s3a.Statistic.*;
import static org.apache.hadoop.fs.s3a.performance.OperationCost.*;
import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.expect;
import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.probe;
import static org.apache.hadoop.test.AssertExtensions.dynamicDescription;
/**
* Abstract class for tests which make assertions about cost.
* <p></p>
* Factored out from {@code ITestS3AFileOperationCost}
*/
public class AbstractS3ACostTest extends AbstractS3ATestBase {
/**
* Parameter: should the stores be guarded?
*/
private final boolean s3guard;
/**
* Parameter: should directory markers be retained?
*/
private final boolean keepMarkers;
/**
* Is this an auth mode test run?
*/
private final boolean authoritative;
/** probe states calculated from the configuration options. */
private boolean isGuarded;
private boolean isRaw;
private boolean isAuthoritative;
private boolean isNonAuth;
private boolean isKeeping;
private boolean isDeleting;
private OperationCostValidator costValidator;
public AbstractS3ACostTest(
final boolean s3guard,
final boolean keepMarkers,
final boolean authoritative) {
this.s3guard = s3guard;
this.keepMarkers = keepMarkers;
this.authoritative = authoritative;
}
@Override
public Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
String bucketName = getTestBucketName(conf);
removeBucketOverrides(bucketName, conf,
S3_METADATA_STORE_IMPL);
if (!isGuarded()) {
// in a raw run remove all s3guard settings
removeBaseAndBucketOverrides(bucketName, conf,
S3_METADATA_STORE_IMPL);
}
removeBaseAndBucketOverrides(bucketName, conf,
DIRECTORY_MARKER_POLICY,
METADATASTORE_AUTHORITATIVE,
AUTHORITATIVE_PATH);
// directory marker options
conf.set(DIRECTORY_MARKER_POLICY,
keepMarkers
? DIRECTORY_MARKER_POLICY_KEEP
: DIRECTORY_MARKER_POLICY_DELETE);
conf.setBoolean(METADATASTORE_AUTHORITATIVE, authoritative);
disableFilesystemCaching(conf);
return conf;
}
@Override
public void setup() throws Exception {
super.setup();
if (isGuarded()) {
// s3guard is required for those test runs where any of the
// guard options are set
assumeS3GuardState(true, getConfiguration());
}
S3AFileSystem fs = getFileSystem();
skipDuringFaultInjection(fs);
// build up the states
isGuarded = isGuarded();
isRaw = !isGuarded;
isAuthoritative = isGuarded && authoritative;
isNonAuth = isGuarded && !authoritative;
isKeeping = isKeepingMarkers();
isDeleting = !isKeeping;
// insert new metrics so as to keep the list sorted
costValidator = OperationCostValidator.builder(getFileSystem())
.withMetrics(
DIRECTORIES_CREATED,
DIRECTORIES_DELETED,
FAKE_DIRECTORIES_DELETED,
FILES_DELETED,
INVOCATION_COPY_FROM_LOCAL_FILE,
OBJECT_COPY_REQUESTS,
OBJECT_DELETE_REQUESTS,
OBJECT_LIST_REQUESTS,
OBJECT_METADATA_REQUESTS,
OBJECT_PUT_BYTES,
OBJECT_PUT_REQUESTS)
.build();
}
public void assumeUnguarded() {
assume("Unguarded FS only", !isGuarded());
}
/**
* Is the store guarded authoritatively on the test path?
* @return true if the condition is met on this test run.
*/
public boolean isAuthoritative() {
return authoritative;
}
/**
* Is the store guarded?
* @return true if the condition is met on this test run.
*/
public boolean isGuarded() {
return s3guard;
}
/**
* Is the store raw?
* @return true if the condition is met on this test run.
*/
public boolean isRaw() {
return isRaw;
}
/**
* Is the store guarded non-authoritatively on the test path?
* @return true if the condition is met on this test run.
*/
public boolean isNonAuth() {
return isNonAuth;
}
public boolean isDeleting() {
return isDeleting;
}
public boolean isKeepingMarkers() {
return keepMarkers;
}
/**
* A special object whose toString() value is the current
* state of the metrics.
*/
protected Object getMetricSummary() {
return costValidator;
}
/**
* Create then close the file through the builder API.
* @param path path
* @param overwrite overwrite flag
* @param recursive true == skip parent existence check
* @param cost expected cost
* @return path to new object.
*/
protected Path buildFile(Path path,
boolean overwrite,
boolean recursive,
OperationCost cost) throws Exception {
resetStatistics();
verifyRaw(cost, () -> {
FSDataOutputStreamBuilder builder = getFileSystem().createFile(path)
.overwrite(overwrite);
if (recursive) {
builder.recursive();
}
FSDataOutputStream stream = builder.build();
stream.close();
return stream.toString();
});
return path;
}
/**
* Create a directory, returning its path.
* @param p path to dir.
* @return path of new dir
*/
protected Path dir(Path p) throws IOException {
mkdirs(p);
return p;
}
/**
* Create a file, returning its path.
* @param p path to file.
* @return path of new file
*/
protected Path file(Path p) throws IOException {
return file(p, true);
}
/**
* Create a file, returning its path.
* @param path path to file.
* @param overwrite overwrite flag
* @return path of new file
*/
protected Path file(Path path, final boolean overwrite)
throws IOException {
getFileSystem().create(path, overwrite).close();
return path;
}
/**
* Touch a file, overwriting.
* @param path path
* @return path to new object.
*/
protected Path create(Path path) throws Exception {
return create(path, true, CREATE_FILE_OVERWRITE);
}
/**
* Create then close the file.
* @param path path
* @param overwrite overwrite flag
* @param cost expected cost
* @return path to new object.
*/
protected Path create(Path path, boolean overwrite,
OperationCost cost) throws Exception {
return verifyRaw(cost, () ->
file(path, overwrite));
}
/**
* Execute rename, returning the current metrics.
* For use in l-expressions.
* @param source source path.
* @param dest dest path
* @return a string for exceptions.
*/
public String execRename(final Path source,
final Path dest) throws IOException {
getFileSystem().rename(source, dest);
return String.format("rename(%s, %s): %s",
dest, source, getMetricSummary());
}
/**
* How many directories are in a path?
* @param path path to probe.
* @return the number of entries below root this path is
*/
protected int directoriesInPath(Path path) {
return path.isRoot() ? 0 : 1 + directoriesInPath(path.getParent());
}
/**
* Reset all the metrics being tracked.
*/
private void resetStatistics() {
costValidator.resetMetricDiffs();
}
/**
* Execute a closure and verify the metrics.
* @param eval closure to evaluate
* @param expected varargs list of expected diffs
* @param <T> return type.
* @return the result of the evaluation
*/
protected <T> T verifyMetrics(
Callable<T> eval,
OperationCostValidator.ExpectedProbe... expected) throws Exception {
return costValidator.exec(eval, expected);
}
/**
* Execute a closure, expecting an exception.
* Verify the metrics after the exception has been caught and
* validated.
* @param clazz type of exception
* @param text text to look for in exception (optional)
* @param eval closure to evaluate
* @param expected varargs list of expected diffs
* @param <T> return type of closure
* @param <E> exception type
* @return the exception caught.
* @throws Exception any other exception
*/
protected <T, E extends Throwable> E verifyMetricsIntercepting(
Class<E> clazz,
String text,
Callable<T> eval,
OperationCostValidator.ExpectedProbe... expected) throws Exception {
return costValidator.intercepting(clazz, text, eval, expected);
}
/**
* Execute a closure expecting an exception.
* @param clazz type of exception
* @param text text to look for in exception (optional)
* @param head expected head request count.
* @param list expected list request count.
* @param eval closure to evaluate
* @param <T> return type of closure
* @param <E> exception type
* @return the exception caught.
* @throws Exception any other exception
*/
protected <T, E extends Throwable> E interceptRaw(
Class<E> clazz,
String text,
OperationCost cost,
Callable<T> eval) throws Exception {
return verifyMetricsIntercepting(clazz, text, eval, whenRaw(cost));
}
/**
* Declare the expected cost on any FS.
* @param cost costs to expect
* @return a probe.
*/
protected OperationCostValidator.ExpectedProbe always(
OperationCost cost) {
return expect(true, cost);
}
/**
* Declare the expected cost on a raw FS.
* @param cost costs to expect
* @return a probe.
*/
protected OperationCostValidator.ExpectedProbe whenRaw(
OperationCost cost) {
return expect(isRaw(), cost);
}
/**
* Declare the expected cost on a guarded FS.
* @param cost costs to expect
* @return a probe.
*/
protected OperationCostValidator.ExpectedProbe whenGuarded(
OperationCost cost) {
return expect(isGuarded(), cost);
}
/**
* Declare the expected cost on a guarded auth FS.
* @param cost costs to expect
* @return a probe.
*/
protected OperationCostValidator.ExpectedProbe whenAuthoritative(
OperationCost cost) {
return expect(isAuthoritative(), cost);
}
/**
* Declare the expected cost on a guarded nonauth FS.
* @param cost costs to expect
* @return a probe.
*/
protected OperationCostValidator.ExpectedProbe whenNonauth(
OperationCost cost) {
return expect(isNonAuth(), cost);
}
/**
* A metric diff which must hold when the fs is keeping markers.
* @param cost expected cost
* @return the diff.
*/
protected OperationCostValidator.ExpectedProbe whenKeeping(
OperationCost cost) {
return expect(isKeepingMarkers(), cost);
}
/**
* A metric diff which must hold when the fs is keeping markers.
* @param cost expected cost
* @return the diff.
*/
protected OperationCostValidator.ExpectedProbe whenDeleting(
OperationCost cost) {
return expect(isDeleting(), cost);
}
/**
* Execute a closure expecting a specific number of HEAD/LIST calls
* on <i>raw</i> S3 stores only.
* @param cost expected cost
* @param eval closure to evaluate
* @param <T> return type of closure
* @return the result of the evaluation
*/
protected <T> T verifyRaw(
OperationCost cost,
Callable<T> eval) throws Exception {
return verifyMetrics(eval, whenRaw(cost));
}
/**
* Execute {@code S3AFileSystem#innerGetFileStatus(Path, boolean, Set)}
* for the given probes.
* expect the specific HEAD/LIST count with a raw FS.
* @param path path
* @param needEmptyDirectoryFlag look for empty directory
* @param probes file status probes to perform
* @param cost expected cost
* @return the status
*/
public S3AFileStatus verifyRawInnerGetFileStatus(
Path path,
boolean needEmptyDirectoryFlag,
Set<StatusProbeEnum> probes,
OperationCost cost) throws Exception {
return verifyRaw(cost, () ->
innerGetFileStatus(getFileSystem(),
path,
needEmptyDirectoryFlag,
probes));
}
/**
* Execute {@code S3AFileSystem#innerGetFileStatus(Path, boolean, Set)}
* for the given probes -expect a FileNotFoundException,
* and the specific HEAD/LIST count with a raw FS.
* @param path path
* @param needEmptyDirectoryFlag look for empty directory
* @param probes file status probes to perform
* @param cost expected cost
*/
public void interceptRawGetFileStatusFNFE(
Path path,
boolean needEmptyDirectoryFlag,
Set<StatusProbeEnum> probes,
OperationCost cost) throws Exception {
interceptRaw(FileNotFoundException.class, "",
cost, () ->
innerGetFileStatus(getFileSystem(),
path,
needEmptyDirectoryFlag,
probes));
}
/**
* Probe for a path being a directory.
* Metrics are only checked on unguarded stores.
* @param path path
* @param expected expected outcome
* @param cost expected cost on a Raw FS.
*/
protected void isDir(Path path,
boolean expected,
OperationCost cost) throws Exception {
boolean b = verifyRaw(cost, () ->
getFileSystem().isDirectory(path));
Assertions.assertThat(b)
.describedAs("isDirectory(%s)", path)
.isEqualTo(expected);
}
/**
* Probe for a path being a file.
* Metrics are only checked on unguarded stores.
* @param path path
* @param expected expected outcome
* @param cost expected cost on a Raw FS.
*/
protected void isFile(Path path,
boolean expected,
OperationCost cost) throws Exception {
boolean b = verifyRaw(cost, () ->
getFileSystem().isFile(path));
Assertions.assertThat(b)
.describedAs("isFile(%s)", path)
.isEqualTo(expected);
}
/**
* A metric diff which must always hold.
* @param stat metric source
* @param expected expected value.
* @return the diff.
*/
protected OperationCostValidator.ExpectedProbe with(
final Statistic stat, final int expected) {
return probe(stat, expected);
}
/**
* A metric diff which must hold when the fs is unguarded.
* @param stat metric source
* @param expected expected value.
* @return the diff.
*/
protected OperationCostValidator.ExpectedProbe withWhenRaw(
final Statistic stat, final int expected) {
return probe(isRaw(), stat, expected);
}
/**
* A metric diff which must hold when the fs is guarded.
* @param stat metric source
* @param expected expected value.
* @return the diff.
*/
protected OperationCostValidator.ExpectedProbe withWhenGuarded(
final Statistic stat,
final int expected) {
return probe(isGuarded(), stat, expected);
}
/**
* A metric diff which must hold when the fs is guarded + authoritative.
* @param stat metric source
* @param expected expected value.
* @return the diff.
*/
protected OperationCostValidator.ExpectedProbe withWhenAuthoritative(
final Statistic stat,
final int expected) {
return probe(isAuthoritative(), stat, expected);
}
/**
* A metric diff which must hold when the fs is guarded + authoritative.
* @param stat metric source
* @param expected expected value.
* @return the diff.
*/
protected OperationCostValidator.ExpectedProbe withWhenNonauth(
final Statistic stat,
final int expected) {
return probe(isNonAuth(), stat, expected);
}
/**
* A metric diff which must hold when the fs is keeping markers.
* @param stat metric source
* @param expected expected value.
* @return the diff.
*/
protected OperationCostValidator.ExpectedProbe withWhenKeeping(
final Statistic stat,
final int expected) {
return probe(isKeepingMarkers(), stat, expected);
}
/**
* A metric diff which must hold when the fs is keeping markers.
* @param stat metric source
* @param expected expected value.
* @return the diff.
*/
protected OperationCostValidator.ExpectedProbe withWhenDeleting(
final Statistic stat,
final int expected) {
return probe(isDeleting(), stat, expected);
}
/**
* Assert the empty directory status of a file is as expected.
* The raised assertion message includes a list of the path.
* @param status status to probe.
* @param expected expected value
*/
protected void assertEmptyDirStatus(final S3AFileStatus status,
final Tristate expected) {
Assertions.assertThat(status.isEmptyDirectory())
.describedAs(dynamicDescription(() ->
"FileStatus says directory is not empty: " + status
+ "\n" + ContractTestUtils.ls(
getFileSystem(), status.getPath())))
.isEqualTo(expected);
}
}

View File

@ -0,0 +1,824 @@
/*
* 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.performance;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.stream.Collectors;
import com.amazonaws.AmazonClientException;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.model.ObjectMetadata;
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.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
import org.apache.hadoop.fs.RemoteIterator;
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.S3AUtils;
import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP;
import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE;
import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
* This is a test suite designed to verify that directory markers do
* not get misconstrued as empty directories during operations
* which explicitly or implicitly list directory trees.
* <p></p>
* It is also intended it to be backported to all releases
* which are enhanced to read directory trees where markers have
* been retained.
* Hence: it does not use any of the new helper classes to
* measure the cost of operations or attempt to create markers
* through the FS APIs.
* <p></p>
* Instead, the directory structure to test is created through
* low-level S3 SDK API calls.
* We also skip any probes to measure/assert metrics.
* We're testing the semantics here, not the cost of the operations.
* Doing that makes it a lot easier to backport.
*
* <p></p>
* Similarly: JUnit assertions over AssertJ.
* <p></p>
* The tests work with unguarded buckets only -the bucket settings are changed
* appropriately.
*/
@RunWith(Parameterized.class)
public class ITestDirectoryMarkerListing extends AbstractS3ATestBase {
private static final Logger LOG =
LoggerFactory.getLogger(ITestDirectoryMarkerListing.class);
private static final String FILENAME = "fileUnderMarker";
private static final String HELLO = "hello";
private static final String MARKER = "marker";
private static final String MARKER_PEER = "markerpeer";
/**
* Parameterization.
*/
@Parameterized.Parameters(name = "{0}")
public static Collection<Object[]> params() {
return Arrays.asList(new Object[][]{
{"keep-markers", true},
{"delete-markers", false},
});
}
/**
* Does rename copy markers?
* Value: {@value}
* <p></p>
* Older releases: yes.
* <p></p>
* The full marker-optimized releases: no.
*/
private static final boolean RENAME_COPIES_MARKERS = false;
/**
* Test configuration name.
*/
private final String name;
/**
* Does this test configuration keep markers?
*/
private final boolean keepMarkers;
/**
* Is this FS deleting markers?
*/
private final boolean isDeletingMarkers;
/**
* Path to a directory which has a marker.
*/
private Path markerDir;
/**
* Key to the object representing {@link #markerDir}.
*/
private String markerKey;
/**
* Key to the object representing {@link #markerDir} with
* a trailing / added. This references the actual object
* which has been created.
*/
private String markerKeySlash;
/**
* bucket of tests.
*/
private String bucket;
/**
* S3 Client of the FS.
*/
private AmazonS3 s3client;
/**
* Path to a file under the marker.
*/
private Path filePathUnderMarker;
/**
* Key to a file under the marker.
*/
private String fileKeyUnderMarker;
/**
* base path for the test files; the marker dir goes under this.
*/
private Path basePath;
/**
* Path to a file a peer of markerDir.
*/
private Path markerPeer;
/**
* Key to a file a peer of markerDir.
*/
private String markerPeerKey;
public ITestDirectoryMarkerListing(final String name,
final boolean keepMarkers) {
this.name = name;
this.keepMarkers = keepMarkers;
this.isDeletingMarkers = !keepMarkers;
}
@Override
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
String bucketName = getTestBucketName(conf);
// Turn off S3Guard
removeBaseAndBucketOverrides(bucketName, conf,
S3_METADATA_STORE_IMPL,
METADATASTORE_AUTHORITATIVE,
AUTHORITATIVE_PATH);
// directory marker options
removeBaseAndBucketOverrides(bucketName, conf,
DIRECTORY_MARKER_POLICY);
conf.set(DIRECTORY_MARKER_POLICY,
keepMarkers
? DIRECTORY_MARKER_POLICY_KEEP
: DIRECTORY_MARKER_POLICY_DELETE);
return conf;
}
/**
* The setup phase includes creating the test objects.
*/
@Override
public void setup() throws Exception {
super.setup();
S3AFileSystem fs = getFileSystem();
assume("unguarded FS only",
!fs.hasMetadataStore());
s3client = fs.getAmazonS3ClientForTesting("markers");
bucket = fs.getBucket();
Path base = new Path(methodPath(), "base");
createTestObjects(base);
}
/**
* Teardown deletes the objects created before
* the superclass does the directory cleanup.
*/
@Override
public void teardown() throws Exception {
if (s3client != null) {
deleteObject(markerKey);
deleteObject(markerKeySlash);
deleteObject(markerPeerKey);
deleteObject(fileKeyUnderMarker);
}
// do this ourselves to avoid audits teardown failing
// when surplus markers are found
deleteTestDirInTeardown();
super.teardown();
}
/**
* Create the test objects under the given path, setting
* various fields in the process.
* @param path parent path of everything
*/
private void createTestObjects(final Path path) throws Exception {
S3AFileSystem fs = getFileSystem();
basePath = path;
markerDir = new Path(basePath, MARKER);
// peer path has the same initial name to make sure there
// is no confusion there.
markerPeer = new Path(basePath, MARKER_PEER);
markerPeerKey = fs.pathToKey(markerPeer);
markerKey = fs.pathToKey(markerDir);
markerKeySlash = markerKey + "/";
fileKeyUnderMarker = markerKeySlash + FILENAME;
filePathUnderMarker = new Path(markerDir, FILENAME);
// put the empty dir
fs.mkdirs(markerDir);
touch(fs, markerPeer);
put(fileKeyUnderMarker, HELLO);
}
/*
=================================================================
Basic probes
=================================================================
*/
@Test
public void testMarkerExists() throws Throwable {
describe("Verify the marker exists");
head(markerKeySlash);
assertIsDirectory(markerDir);
}
@Test
public void testObjectUnderMarker() throws Throwable {
describe("verify the file under the marker dir exists");
assertIsFile(filePathUnderMarker);
head(fileKeyUnderMarker);
}
/*
=================================================================
The listing operations
=================================================================
*/
@Test
public void testListStatusMarkerDir() throws Throwable {
describe("list the marker directory and expect to see the file");
assertContainsFileUnderMarkerOnly(
toList(getFileSystem().listStatus(markerDir)));
}
@Test
public void testListFilesMarkerDirFlat() throws Throwable {
assertContainsFileUnderMarkerOnly(toList(
getFileSystem().listFiles(markerDir, false)));
}
@Test
public void testListFilesMarkerDirRecursive() throws Throwable {
List<FileStatus> statuses = toList(
getFileSystem().listFiles(markerDir, true));
assertContainsFileUnderMarkerOnly(statuses);
}
/**
* Path listing above the base dir MUST only find the file
* and not the marker.
*/
@Test
public void testListStatusBaseDirRecursive() throws Throwable {
List<FileStatus> statuses = toList(
getFileSystem().listFiles(basePath, true));
assertContainsExactlyStatusOfPaths(statuses, filePathUnderMarker,
markerPeer);
}
@Test
public void testGlobStatusBaseDirRecursive() throws Throwable {
Path escapedPath = new Path(escape(basePath.toUri().getPath()));
List<FileStatus> statuses =
exec("glob", () ->
toList(getFileSystem().globStatus(new Path(escapedPath, "*"))));
assertContainsExactlyStatusOfPaths(statuses, markerDir, markerPeer);
assertIsFileAtPath(markerPeer, statuses.get(1));
}
@Test
public void testGlobStatusMarkerDir() throws Throwable {
Path escapedPath = new Path(escape(markerDir.toUri().getPath()));
List<FileStatus> statuses =
exec("glob", () ->
toList(getFileSystem().globStatus(new Path(escapedPath, "*"))));
assertContainsFileUnderMarkerOnly(statuses);
}
/**
* Call {@code listLocatedStatus(basePath)}
* <p></p>
* The list here returns the marker peer before the
* dir. Reason: the listing iterators return
* the objects before the common prefixes, and the
* marker dir is coming back as a prefix.
*/
@Test
public void testListLocatedStatusBaseDir() throws Throwable {
List<FileStatus> statuses =
exec("listLocatedStatus", () ->
toList(getFileSystem().listLocatedStatus(basePath)));
assertContainsExactlyStatusOfPaths(statuses, markerPeer, markerDir);
}
/**
* Call {@code listLocatedStatus(markerDir)}; expect
* the file entry only.
*/
@Test
public void testListLocatedStatusMarkerDir() throws Throwable {
List<FileStatus> statuses =
exec("listLocatedStatus", () ->
toList(getFileSystem().listLocatedStatus(markerDir)));
assertContainsFileUnderMarkerOnly(statuses);
}
/*
=================================================================
Creation Rejection
=================================================================
*/
@Test
public void testCreateNoOverwriteMarkerDir() throws Throwable {
describe("create no-overwrite over the marker dir fails");
head(markerKeySlash);
intercept(FileAlreadyExistsException.class, () ->
exec("create", () ->
getFileSystem().create(markerDir, false)));
// dir is still there.
head(markerKeySlash);
}
@Test
public void testCreateNoOverwriteFile() throws Throwable {
describe("create-no-overwrite on the file fails");
head(fileKeyUnderMarker);
intercept(FileAlreadyExistsException.class, () ->
exec("create", () ->
getFileSystem().create(filePathUnderMarker, false)));
assertTestObjectsExist();
}
@Test
public void testCreateFileNoOverwrite() throws Throwable {
describe("verify the createFile() API also fails");
head(fileKeyUnderMarker);
intercept(FileAlreadyExistsException.class, () ->
exec("create", () ->
getFileSystem().createFile(filePathUnderMarker)
.overwrite(false)
.build()));
assertTestObjectsExist();
}
/*
=================================================================
Delete.
=================================================================
*/
@Test
public void testDelete() throws Throwable {
S3AFileSystem fs = getFileSystem();
// a non recursive delete MUST fail because
// it is not empty
intercept(PathIsNotEmptyDirectoryException.class, () ->
fs.delete(markerDir, false));
// file is still there
head(fileKeyUnderMarker);
// recursive delete MUST succeed
fs.delete(markerDir, true);
// and the markers are gone
head404(fileKeyUnderMarker);
head404(markerKeySlash);
// just for completeness
fs.delete(basePath, true);
}
/*
=================================================================
Rename.
=================================================================
*/
/**
* Rename the base directory, expect the source files to move.
* <p></p>
* Whether or not the marker itself is copied depends on whether
* the release's rename operation explicitly skips
* markers on renames.
*/
@Test
public void testRenameBase() throws Throwable {
describe("rename base directory");
Path src = basePath;
Path dest = new Path(methodPath(), "dest");
assertRenamed(src, dest);
assertPathDoesNotExist("source", src);
assertPathDoesNotExist("source", filePathUnderMarker);
assertPathExists("dest not found", dest);
// all the paths dest relative
Path destMarkerDir = new Path(dest, MARKER);
// peer path has the same initial name to make sure there
// is no confusion there.
Path destMarkerPeer = new Path(dest, MARKER_PEER);
String destMarkerKey = toKey(destMarkerDir);
String destMarkerKeySlash = destMarkerKey + "/";
String destFileKeyUnderMarker = destMarkerKeySlash + FILENAME;
Path destFilePathUnderMarker = new Path(destMarkerDir, FILENAME);
assertIsFile(destFilePathUnderMarker);
assertIsFile(destMarkerPeer);
head(destFileKeyUnderMarker);
// probe for the marker based on expected rename
// behavior
if (RENAME_COPIES_MARKERS) {
head(destMarkerKeySlash);
} else {
head404(destMarkerKeySlash);
}
}
/**
* Rename a file under a marker by passing in the marker
* directory as the destination; the final path is derived
* from the original filename.
* <p></p>
* After the rename:
* <ol>
* <li>The data must be at the derived destination path.</li>
* <li>The source file must not exist.</li>
* <li>The parent dir of the source file must exist.</li>
* <li>The marker above the destination file must not exist.</li>
* </ol>
*/
@Test
public void testRenameUnderMarkerDir() throws Throwable {
describe("directory rename under an existing marker");
String file = "sourceFile";
Path srcDir = new Path(basePath, "srcdir");
mkdirs(srcDir);
Path src = new Path(srcDir, file);
String srcKey = toKey(src);
put(srcKey, file);
head(srcKey);
// set the destination to be the marker directory.
Path dest = markerDir;
// rename the source file under the dest dir.
assertRenamed(src, dest);
assertIsFile(new Path(dest, file));
assertIsDirectory(srcDir);
if (isDeletingMarkers) {
head404(markerKeySlash);
} else {
head(markerKeySlash);
}
}
/**
* Rename file under a marker, giving the full path to the destination
* file.
* <p></p>
* After the rename:
* <ol>
* <li>The data must be at the explicit destination path.</li>
* <li>The source file must not exist.</li>
* <li>The parent dir of the source file must exist.</li>
* <li>The marker above the destination file must not exist.</li>
* </ol>
*/
@Test
public void testRenameUnderMarkerWithPath() throws Throwable {
describe("directory rename under an existing marker");
S3AFileSystem fs = getFileSystem();
String file = "sourceFile";
Path srcDir = new Path(basePath, "srcdir");
mkdirs(srcDir);
Path src = new Path(srcDir, file);
String srcKey = toKey(src);
put(srcKey, file);
head(srcKey);
// set the destination to be the final file
Path dest = new Path(markerDir, "destFile");
// rename the source file to the destination file
assertRenamed(src, dest);
assertIsFile(dest);
assertIsDirectory(srcDir);
if (isDeletingMarkers) {
head404(markerKeySlash);
} else {
head(markerKeySlash);
}
}
/**
* This test creates an empty dir and renames it over the directory marker.
* If the dest was considered to be empty, the rename would fail.
*/
@Test
public void testRenameEmptyDirOverMarker() throws Throwable {
describe("rename an empty directory over the marker");
S3AFileSystem fs = getFileSystem();
String dir = "sourceDir";
Path src = new Path(basePath, dir);
fs.mkdirs(src);
assertIsDirectory(src);
String srcKey = toKey(src) + "/";
head(srcKey);
Path dest = markerDir;
// renamed into the dest dir
assertFalse("rename(" + src + ", " + dest + ") should have failed",
getFileSystem().rename(src, dest));
// source is still there
assertIsDirectory(src);
head(srcKey);
// and a non-recursive delete lets us verify it is considered
// an empty dir
assertDeleted(src, false);
assertTestObjectsExist();
}
/*
=================================================================
Utility methods and assertions.
=================================================================
*/
/**
* Assert the test objects exist.
*/
private void assertTestObjectsExist() throws Exception {
head(fileKeyUnderMarker);
head(markerKeySlash);
}
/**
* Put a string to a path.
* @param key key
* @param content string
*/
private void put(final String key, final String content) throws Exception {
exec("PUT " + key, () ->
s3client.putObject(bucket, key, content));
}
/**
* Delete an object.
* @param key key
* @param content string
*/
private void deleteObject(final String key) throws Exception {
exec("DELETE " + key, () -> {
s3client.deleteObject(bucket, key);
return "deleted " + key;
});
}
/**
* Issue a HEAD request.
* @param key
* @return a description of the object.
*/
private String head(final String key) throws Exception {
ObjectMetadata md = exec("HEAD " + key, () ->
s3client.getObjectMetadata(bucket, key));
return String.format("Object %s of length %d",
key, md.getInstanceLength());
}
/**
* Issue a HEAD request and expect a 404 back.
* @param key
* @return the metadata
*/
private void head404(final String key) throws Exception {
intercept(FileNotFoundException.class, "",
"Expected 404 of " + key, () ->
head(key));
}
/**
* Execute an operation; transate AWS exceptions.
* @param op operation
* @param call call to make
* @param <T> returned type
* @return result of the call.
* @throws Exception failure
*/
private <T> T exec(String op, Callable<T> call) throws Exception {
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
try {
return call.call();
} catch (AmazonClientException ex) {
throw S3AUtils.translateException(op, "", ex);
} finally {
timer.end(op);
}
}
/**
* Assert that the listing contains only the status
* of the file under the marker.
* @param statuses status objects
*/
private void assertContainsFileUnderMarkerOnly(
final List<FileStatus> statuses) {
assertContainsExactlyStatusOfPaths(statuses, filePathUnderMarker);
assertIsFileUnderMarker(statuses.get(0));
}
/**
* Expect the list of status objects to match that of the paths.
* @param statuses status object list
* @param paths ordered varargs list of paths
* @param <T> type of status objects
*/
private <T extends FileStatus> void assertContainsExactlyStatusOfPaths(
List<T> statuses, Path... paths) {
String actual = statuses.stream()
.map(Object::toString)
.collect(Collectors.joining(";"));
String expected = Arrays.stream(paths)
.map(Object::toString)
.collect(Collectors.joining(";"));
String summary = "expected [" + expected + "]"
+ " actual = [" + actual + "]";
assertEquals("mismatch in size of listing " + summary,
paths.length, statuses.size());
for (int i = 0; i < statuses.size(); i++) {
assertEquals("Path mismatch at element " + i + " in " + summary,
paths[i], statuses.get(i).getPath());
}
}
/**
* Assert the status object refers to the file created
* under the marker.
* @param stat status object
*/
private void assertIsFileUnderMarker(final FileStatus stat) {
assertIsFileAtPath(filePathUnderMarker, stat);
}
/**
* Assert the status object refers to a path at the given name.
* @param path path
* @param stat status object
*/
private void assertIsFileAtPath(final Path path, final FileStatus stat) {
assertTrue("Is not file " + stat, stat.isFile());
assertPathEquals(path, stat);
}
/**
* Assert a status object's path matches expected.
* @param path path to expect
* @param stat status object
*/
private void assertPathEquals(final Path path, final FileStatus stat) {
assertEquals("filename is not the expected path :" + stat,
path, stat.getPath());
}
/**
* Given a remote iterator of status objects,
* build a list of the values.
* @param status status list
* @param <T> actual type.
* @return source.
* @throws IOException
*/
private <T extends FileStatus> List<FileStatus> toList(
RemoteIterator<T> status) throws IOException {
List<FileStatus> l = new ArrayList<>();
while (status.hasNext()) {
l.add(status.next());
}
return dump(l);
}
/**
* Given an array of status objects,
* build a list of the values.
* @param status status list
* @param <T> actual type.
* @return source.
* @throws IOException
*/
private <T extends FileStatus> List<FileStatus> toList(
T[] status) throws IOException {
return dump(Arrays.asList(status));
}
/**
* Dump the string values of a list to the log; return
* the list.
* @param l source.
* @param <T> source type
* @return the list
*/
private <T> List<T> dump(List<T> l) {
int c = 1;
for (T t : l) {
LOG.info("{}\t{}", c++, t);
}
return l;
}
/**
* Rename: assert the outcome is true.
* @param src source path
* @param dest dest path
*/
private void assertRenamed(final Path src, final Path dest)
throws IOException {
assertTrue("rename(" + src + ", " + dest + ") failed",
getFileSystem().rename(src, dest));
}
/**
* Convert a path to a key; does not add any trailing / .
* @param path path in
* @return key out
*/
private String toKey(final Path path) {
return getFileSystem().pathToKey(path);
}
/**
* Escape paths before handing to globStatus; this is needed as
* parameterized runs produce paths with [] in them.
* @param pathstr source path string
* @return an escaped path string
*/
private String escape(String pathstr) {
StringBuilder r = new StringBuilder();
for (char c : pathstr.toCharArray()) {
String ch = Character.toString(c);
if ("?*[{".contains(ch)) {
r.append("\\");
}
r.append(ch);
}
return r.toString();
}
}

View File

@ -0,0 +1,218 @@
/*
* 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.performance;
import java.util.Arrays;
import java.util.Collection;
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.fs.Path;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.Tristate;
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
import static org.apache.hadoop.fs.s3a.Statistic.*;
import static org.apache.hadoop.fs.s3a.performance.OperationCost.*;
import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.probe;
/**
* Use metrics to assert about the cost of file API calls.
* <p></p>
* Parameterized on guarded vs raw. and directory marker keep vs delete.
*/
@RunWith(Parameterized.class)
public class ITestS3ADeleteCost extends AbstractS3ACostTest {
private static final Logger LOG =
LoggerFactory.getLogger(ITestS3ADeleteCost.class);
/**
* Parameterization.
*/
@Parameterized.Parameters(name = "{0}")
public static Collection<Object[]> params() {
return Arrays.asList(new Object[][]{
{"raw-keep-markers", false, true, false},
{"raw-delete-markers", false, false, false},
{"nonauth-keep-markers", true, true, false},
{"auth-delete-markers", true, false, true}
});
}
public ITestS3ADeleteCost(final String name,
final boolean s3guard,
final boolean keepMarkers,
final boolean authoritative) {
super(s3guard, keepMarkers, authoritative);
}
@Override
public void teardown() throws Exception {
if (isKeepingMarkers()) {
// do this ourselves to avoid audits teardown failing
// when surplus markers are found
deleteTestDirInTeardown();
}
super.teardown();
}
/**
* This creates a directory with a child and then deletes it.
* The parent dir must be found and declared as empty.
* <p>When deleting markers, that forces the recreation of a new marker.</p>
*/
@Test
public void testDeleteSingleFileInDir() throws Throwable {
describe("delete a file");
S3AFileSystem fs = getFileSystem();
// creates the marker
Path dir = dir(methodPath());
// file creation may have deleted that marker, but it may
// still be there
Path simpleFile = file(new Path(dir, "simple.txt"));
boolean rawAndKeeping = isRaw() && isDeleting();
boolean rawAndDeleting = isRaw() && isDeleting();
verifyMetrics(() -> {
fs.delete(simpleFile, false);
return "after fs.delete(simpleFile) " + getMetricSummary();
},
probe(rawAndKeeping, OBJECT_METADATA_REQUESTS,
FILESTATUS_FILE_PROBE_H),
// if deleting markers, look for the parent too
probe(rawAndDeleting, OBJECT_METADATA_REQUESTS,
FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H),
withWhenRaw(OBJECT_LIST_REQUESTS,
FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L),
with(DIRECTORIES_DELETED, 0),
with(FILES_DELETED, 1),
// keeping: create no parent dirs or delete parents
withWhenKeeping(DIRECTORIES_CREATED, 0),
withWhenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST),
// deleting: create a parent and delete any of its parents
withWhenDeleting(DIRECTORIES_CREATED, 1),
withWhenDeleting(OBJECT_DELETE_REQUESTS,
DELETE_OBJECT_REQUEST
+ DELETE_MARKER_REQUEST)
);
// there is an empty dir for a parent
S3AFileStatus status = verifyRawInnerGetFileStatus(dir, true,
StatusProbeEnum.ALL, GET_FILE_STATUS_ON_DIR);
assertEmptyDirStatus(status, Tristate.TRUE);
}
/**
* This creates a directory with a two files and then deletes one of the
* files.
*/
@Test
public void testDeleteFileInDir() throws Throwable {
describe("delete a file in a directory with multiple files");
S3AFileSystem fs = getFileSystem();
// creates the marker
Path dir = dir(methodPath());
// file creation may have deleted that marker, but it may
// still be there
Path file1 = file(new Path(dir, "file1.txt"));
Path file2 = file(new Path(dir, "file2.txt"));
boolean rawAndKeeping = isRaw() && isDeleting();
boolean rawAndDeleting = isRaw() && isDeleting();
verifyMetrics(() -> {
fs.delete(file1, false);
return "after fs.delete(file1simpleFile) " + getMetricSummary();
},
// delete file. For keeping: that's it
probe(rawAndKeeping, OBJECT_METADATA_REQUESTS,
FILESTATUS_FILE_PROBE_H),
// if deleting markers, look for the parent too
probe(rawAndDeleting, OBJECT_METADATA_REQUESTS,
FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H),
withWhenRaw(OBJECT_LIST_REQUESTS,
FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L),
with(DIRECTORIES_DELETED, 0),
with(FILES_DELETED, 1),
// no need to create a parent
with(DIRECTORIES_CREATED, 0),
// keeping: create no parent dirs or delete parents
withWhenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST),
// deleting: create a parent and delete any of its parents
withWhenDeleting(OBJECT_DELETE_REQUESTS,
DELETE_OBJECT_REQUEST));
}
@Test
public void testDirMarkersSubdir() throws Throwable {
describe("verify cost of deep subdir creation");
Path subDir = new Path(methodPath(), "1/2/3/4/5/6");
// one dir created, possibly a parent removed
verifyMetrics(() -> {
mkdirs(subDir);
return "after mkdir(subDir) " + getMetricSummary();
},
with(DIRECTORIES_CREATED, 1),
with(DIRECTORIES_DELETED, 0),
withWhenKeeping(OBJECT_DELETE_REQUESTS, 0),
withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0),
withWhenDeleting(OBJECT_DELETE_REQUESTS, DELETE_MARKER_REQUEST),
// delete all possible fake dirs above the subdirectory
withWhenDeleting(FAKE_DIRECTORIES_DELETED,
directoriesInPath(subDir) - 1));
}
@Test
public void testDirMarkersFileCreation() throws Throwable {
describe("verify cost of file creation");
Path srcBaseDir = dir(methodPath());
Path srcDir = dir(new Path(srcBaseDir, "1/2/3/4/5/6"));
// creating a file should trigger demise of the src dir marker
// unless markers are being kept
verifyMetrics(() -> {
file(new Path(srcDir, "source.txt"));
return "after touch(fs, srcFilePath) " + getMetricSummary();
},
with(DIRECTORIES_CREATED, 0),
with(DIRECTORIES_DELETED, 0),
// keeping: no delete operations.
withWhenKeeping(OBJECT_DELETE_REQUESTS, 0),
withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0),
// delete all possible fake dirs above the file
withWhenDeleting(OBJECT_DELETE_REQUESTS, 1),
withWhenDeleting(FAKE_DIRECTORIES_DELETED,
directoriesInPath(srcDir)));
}
}

View File

@ -0,0 +1,207 @@
/*
* 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.performance;
import java.util.Arrays;
import java.util.Collection;
import java.util.UUID;
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.fs.Path;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import static org.apache.hadoop.fs.s3a.Statistic.*;
import static org.apache.hadoop.fs.s3a.performance.OperationCost.*;
/**
* Use metrics to assert about the cost of file API calls.
* <p></p>
* Parameterized on guarded vs raw. and directory marker keep vs delete
*/
@RunWith(Parameterized.class)
public class ITestS3ARenameCost extends AbstractS3ACostTest {
private static final Logger LOG =
LoggerFactory.getLogger(ITestS3ARenameCost.class);
/**
* Parameterization.
*/
@Parameterized.Parameters(name = "{0}")
public static Collection<Object[]> params() {
return Arrays.asList(new Object[][]{
{"raw-keep-markers", false, true, false},
{"raw-delete-markers", false, false, false},
{"nonauth-keep-markers", true, true, false},
{"auth-delete-markers", true, false, true}
});
}
public ITestS3ARenameCost(final String name,
final boolean s3guard,
final boolean keepMarkers,
final boolean authoritative) {
super(s3guard, keepMarkers, authoritative);
}
@Test
public void testRenameFileToDifferentDirectory() throws Throwable {
describe("rename a file to a different directory, "
+ "keeping the source dir present");
Path baseDir = dir(methodPath());
Path srcDir = new Path(baseDir, "1/2/3/4/5/6");
final Path srcFilePath = file(new Path(srcDir, "source.txt"));
// create a new source file.
// Explicitly use a new path object to guarantee that the parent paths
// are different object instances and so equals() rather than ==
// is
Path parent2 = srcFilePath.getParent();
Path srcFile2 = file(new Path(parent2, "source2.txt"));
Assertions.assertThat(srcDir)
.isNotSameAs(parent2);
Assertions.assertThat(srcFilePath.getParent())
.isEqualTo(srcFile2.getParent());
// create a directory tree, expect the dir to be created and
// possibly a request to delete all parent directories made.
Path destBaseDir = new Path(baseDir, "dest");
Path destDir = dir(new Path(destBaseDir, "a/b/c/d"));
Path destFilePath = new Path(destDir, "dest.txt");
// rename the source file to the destination file.
// this tests file rename, not dir rename
// as srcFile2 exists, the parent dir of srcFilePath must not be created.
verifyMetrics(() ->
execRename(srcFilePath, destFilePath),
whenRaw(RENAME_SINGLE_FILE_DIFFERENT_DIR),
with(DIRECTORIES_CREATED, 0),
with(DIRECTORIES_DELETED, 0),
// keeping: only the core delete operation is issued.
withWhenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST),
withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0),
// deleting: delete any fake marker above the destination.
withWhenDeleting(OBJECT_DELETE_REQUESTS,
DELETE_OBJECT_REQUEST + DELETE_MARKER_REQUEST),
withWhenDeleting(FAKE_DIRECTORIES_DELETED,
directoriesInPath(destDir)));
assertIsFile(destFilePath);
assertIsDirectory(srcDir);
assertPathDoesNotExist("should have gone in the rename", srcFilePath);
}
/**
* Same directory rename is lower cost as there's no need to
* look for the parent dir of the dest path or worry about
* deleting markers.
*/
@Test
public void testRenameSameDirectory() throws Throwable {
describe("rename a file to the same directory");
Path baseDir = dir(methodPath());
final Path sourceFile = file(new Path(baseDir, "source.txt"));
// create a new source file.
// Explicitly use a new path object to guarantee that the parent paths
// are different object instances and so equals() rather than ==
// is
Path parent2 = sourceFile.getParent();
Path destFile = new Path(parent2, "dest");
verifyMetrics(() ->
execRename(sourceFile, destFile),
whenRaw(RENAME_SINGLE_FILE_SAME_DIR),
with(OBJECT_COPY_REQUESTS, 1),
with(DIRECTORIES_CREATED, 0),
with(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST),
with(FAKE_DIRECTORIES_DELETED, 0));
}
@Test
public void testCostOfRootFileRename() throws Throwable {
describe("assert that a root file rename doesn't"
+ " do much in terms of parent dir operations");
S3AFileSystem fs = getFileSystem();
// unique name, so that even when run in parallel tests, there's no conflict
String uuid = UUID.randomUUID().toString();
Path src = file(new Path("/src-" + uuid));
Path dest = new Path("/dest-" + uuid);
try {
verifyMetrics(() -> {
fs.rename(src, dest);
return "after fs.rename(/src,/dest) " + getMetricSummary();
},
whenRaw(FILE_STATUS_FILE_PROBE
.plus(GET_FILE_STATUS_FNFE)
.plus(COPY_OP)),
// here we expect there to be no fake directories
with(DIRECTORIES_CREATED, 0),
// one for the renamed file only
with(OBJECT_DELETE_REQUESTS,
DELETE_OBJECT_REQUEST),
// no directories are deleted: This is root
with(DIRECTORIES_DELETED, 0),
// no fake directories are deleted: This is root
with(FAKE_DIRECTORIES_DELETED, 0),
with(FILES_DELETED, 1));
} finally {
fs.delete(src, false);
fs.delete(dest, false);
}
}
@Test
public void testCostOfRootFileDelete() throws Throwable {
describe("assert that a root file delete doesn't"
+ " do much in terms of parent dir operations");
S3AFileSystem fs = getFileSystem();
// unique name, so that even when run in parallel tests, there's no conflict
String uuid = UUID.randomUUID().toString();
Path src = file(new Path("/src-" + uuid));
try {
// delete that destination file, assert only the file delete was issued
verifyMetrics(() -> {
fs.delete(src, false);
return "after fs.delete(/dest) " + getMetricSummary();
},
with(DIRECTORIES_CREATED, 0),
with(DIRECTORIES_DELETED, 0),
with(FAKE_DIRECTORIES_DELETED, 0),
with(FILES_DELETED, 1),
with(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST),
whenRaw(FILE_STATUS_FILE_PROBE)); /* no need to look at parent. */
} finally {
fs.delete(src, false);
}
}
}

View File

@ -0,0 +1,201 @@
/*
* 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.performance;
/**
* Declaration of the costs of head and list calls for various FS IO
* operations.
* <p></p>
* An instance declares the number of head and list calls expected for
* various operations -with a {@link #plus(OperationCost)}
* method to add operation costs together to produce an
* aggregate cost. These can then be validated in tests
* via {@link OperationCostValidator}.
*
*/
public final class OperationCost {
/** Head costs for getFileStatus() directory probe: {@value}. */
public static final int FILESTATUS_DIR_PROBE_H = 0;
/** List costs for getFileStatus() directory probe: {@value}. */
public static final int FILESTATUS_DIR_PROBE_L = 1;
/** Head cost getFileStatus() file probe only. */
public static final int FILESTATUS_FILE_PROBE_H = 1;
/** Liast cost getFileStatus() file probe only. */
public static final int FILESTATUS_FILE_PROBE_L = 0;
/**
* Delete cost when deleting an object.
*/
public static final int DELETE_OBJECT_REQUEST = 1;
/**
* Delete cost when deleting a marker.
*/
public static final int DELETE_MARKER_REQUEST = DELETE_OBJECT_REQUEST;
/**
* No IO takes place.
*/
public static final OperationCost NO_IO =
new OperationCost(0, 0);
/** A HEAD operation. */
public static final OperationCost HEAD_OPERATION = new OperationCost(1, 0);
/** A LIST operation. */
public static final OperationCost LIST_OPERATION = new OperationCost(0, 1);
/**
* Cost of {@link org.apache.hadoop.fs.s3a.impl.StatusProbeEnum#DIRECTORIES}.
*/
public static final OperationCost FILE_STATUS_DIR_PROBE = LIST_OPERATION;
/**
* Cost of {@link org.apache.hadoop.fs.s3a.impl.StatusProbeEnum#FILE}.
*/
public static final OperationCost FILE_STATUS_FILE_PROBE = HEAD_OPERATION;
/**
* Cost of {@link org.apache.hadoop.fs.s3a.impl.StatusProbeEnum#ALL}.
*/
public static final OperationCost FILE_STATUS_ALL_PROBES =
FILE_STATUS_FILE_PROBE.plus(FILE_STATUS_DIR_PROBE);
/** getFileStatus() on a file which exists. */
public static final OperationCost GET_FILE_STATUS_ON_FILE =
FILE_STATUS_FILE_PROBE;
/** List costs for getFileStatus() on a non-empty directory: {@value}. */
public static final OperationCost GET_FILE_STATUS_ON_DIR =
FILE_STATUS_FILE_PROBE.plus(FILE_STATUS_DIR_PROBE);
/** Costs for getFileStatus() on an empty directory: {@value}. */
public static final OperationCost GET_FILE_STATUS_ON_EMPTY_DIR =
GET_FILE_STATUS_ON_DIR;
/** getFileStatus() directory marker which exists. */
public static final OperationCost GET_FILE_STATUS_ON_DIR_MARKER =
GET_FILE_STATUS_ON_EMPTY_DIR;
/** getFileStatus() call which fails to find any entry. */
public static final OperationCost GET_FILE_STATUS_FNFE =
FILE_STATUS_ALL_PROBES;
/** listLocatedStatus always does a LIST. */
public static final OperationCost LIST_LOCATED_STATUS_LIST_OP =
new OperationCost(0, 1);
/** listFiles always does a LIST. */
public static final OperationCost LIST_FILES_LIST_OP =
new OperationCost(0, 1);
/**
* Metadata cost of a copy operation, as used during rename.
* This happens even if the store is guarded.
*/
public static final OperationCost COPY_OP =
new OperationCost(1, 0);
/**
* Cost of renaming a file to a different directory.
* <p></p>
* LIST on dest not found, look for dest dir, and then, at
* end of rename, whether a parent dir needs to be created.
*/
public static final OperationCost RENAME_SINGLE_FILE_DIFFERENT_DIR =
FILE_STATUS_FILE_PROBE // source file probe
.plus(GET_FILE_STATUS_FNFE) // dest does not exist
.plus(FILE_STATUS_DIR_PROBE) // parent dir of dest
.plus(FILE_STATUS_DIR_PROBE) // recreate source parent dir?
.plus(COPY_OP); // metadata read on copy
/**
* Cost of renaming a file to the same directory
* <p></p>
* No need to look for parent directories, so only file
* existence checks and the copy.
*/
public static final OperationCost RENAME_SINGLE_FILE_SAME_DIR =
FILE_STATUS_FILE_PROBE // source file probe
.plus(GET_FILE_STATUS_FNFE) // dest must not exist
.plus(COPY_OP); // metadata read on copy
/**
* create(overwrite = true) does not look for the file existing.
*/
public static final OperationCost CREATE_FILE_OVERWRITE =
FILE_STATUS_DIR_PROBE;
/**
* create(overwrite = false) runs all the checks.
*/
public static final OperationCost CREATE_FILE_NO_OVERWRITE =
FILE_STATUS_ALL_PROBES;
/** Expected HEAD count. */
private final int head;
/** Expected LIST count. */
private final int list;
/**
* Constructor.
* @param head head requests.
* @param list list requests.
*/
public OperationCost(final int head,
final int list) {
this.head = head;
this.list = list;
}
/** Expected HEAD count. */
int head() {
return head;
}
/** Expected LIST count. */
int list() {
return list;
}
/**
* Add to create a new cost.
* @param that the other entry
* @return cost of the combined operation.
*/
public OperationCost plus(OperationCost that) {
return new OperationCost(
head + that.head,
list + that.list);
}
@Override
public String toString() {
return "OperationCost{" +
"head=" + head +
", list=" + list +
'}';
}
}

View File

@ -0,0 +1,483 @@
/*
* 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.performance;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.Callable;
import java.util.stream.Collectors;
import org.assertj.core.api.Assumptions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.Statistic;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_LIST_REQUESTS;
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_METADATA_REQUESTS;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
* Support for declarative assertions about operation cost.
* <p></p>
* Usage: A builder is used to declare the set of statistics
* to be monitored in the filesystem.
* <p></p>
* A call to {@link #exec(Callable, ExpectedProbe...)}
* executes the callable if 1+ probe is enabled; after
* invocation the probes are validated.
* The result of the callable is returned.
* <p></p>
* A call of {@link #intercepting(Class, String, Callable, ExpectedProbe...)}
* Invokes the callable if 1+ probe is enabled, expects an exception
* to be raised and then verifies metrics declared in the probes.
* <p></p>
* Probes are built up from the static method to create probes
* for metrics:
* <ul>
* <li>{@link #probe(boolean, Statistic, int)} </li>
* <li>{@link #probe(Statistic, int)} </li>
* <li>{@link #probes(boolean, ExpectedProbe...)} (Statistic, int)} </li>
* <li>{@link #always()}</li>
* </ul>
* If any probe evaluates to false, an assertion is raised.
* <p></p>
* When this happens: look in the logs!
* The logs will contain the whole set of metrics, the probe details
* and the result of the call.
*/
public final class OperationCostValidator {
private static final Logger LOG =
LoggerFactory.getLogger(OperationCostValidator.class);
/**
* The empty probe: declared as disabled.
*/
private static final ExpectedProbe EMPTY_PROBE =
new EmptyProbe("empty", false);
/**
* A probe which is always enabled.
*/
private static final ExpectedProbe ALWAYS_PROBE =
new EmptyProbe("always", true);
/**
* The map of metric diffs to track.
*/
private final Map<String, S3ATestUtils.MetricDiff> metricDiffs
= new TreeMap<>();
/**
* Build the instance.
* @param builder builder containing all options.
*/
private OperationCostValidator(Builder builder) {
builder.metrics.forEach(stat ->
metricDiffs.put(stat.getSymbol(),
new S3ATestUtils.MetricDiff(builder.filesystem, stat)));
builder.metrics.clear();
}
/**
* Reset all the metrics being tracked.
*/
public void resetMetricDiffs() {
metricDiffs.values().forEach(S3ATestUtils.MetricDiff::reset);
}
/**
* Get the diff of a statistic.
* @param stat statistic to look up
* @return the value
* @throws NullPointerException if there is no match
*/
public S3ATestUtils.MetricDiff get(Statistic stat) {
S3ATestUtils.MetricDiff diff =
requireNonNull(metricDiffs.get(stat.getSymbol()),
() -> "No metric tracking for " + stat);
return diff;
}
/**
* Execute a closure and verify the metrics.
* <p></p>
* If no probes are active, the operation will
* raise an Assumption exception for the test to be skipped.
* @param eval closure to evaluate
* @param expected varargs list of expected diffs
* @param <T> return type.
* @return the result of the evaluation
*/
public <T> T exec(
Callable<T> eval,
ExpectedProbe... expectedA) throws Exception {
List<ExpectedProbe> expected = Arrays.asList(expectedA);
resetMetricDiffs();
// verify that 1+ probe is enabled
assumeProbesEnabled(expected);
// if we get here, then yes.
// evaluate it
T r = eval.call();
// build the text for errors
String text =
"operation returning "
+ (r != null ? r.toString() : "null");
LOG.info("{}", text);
LOG.info("state {}", this);
LOG.info("probes {}", expected);
for (ExpectedProbe ed : expected) {
ed.verify(this, text);
}
return r;
}
/**
* Scan all probes for being enabled.
* <p></p>
* If none of them are enabled, the evaluation will be skipped.
* @param expected list of expected probes
*/
private void assumeProbesEnabled(List<ExpectedProbe> expected) {
boolean enabled = false;
for (ExpectedProbe ed : expected) {
enabled |= ed.isEnabled();
}
String pstr = expected.stream()
.map(Object::toString)
.collect(Collectors.joining(", "));
Assumptions.assumeThat(enabled)
.describedAs("metrics to probe for are not enabled in %s", pstr)
.isTrue();
}
/**
* Execute a closure, expecting an exception.
* Verify the metrics after the exception has been caught and
* validated.
* @param clazz type of exception
* @param text text to look for in exception (optional)
* @param eval closure to evaluate
* @param expected varargs list of expected diffs
* @param <T> return type of closure
* @param <E> exception type
* @return the exception caught.
* @throws Exception any other exception
*/
public <T, E extends Throwable> E intercepting(
Class<E> clazz,
String text,
Callable<T> eval,
ExpectedProbe... expected) throws Exception {
return exec(() ->
intercept(clazz, text, eval),
expected);
}
@Override
public String toString() {
return metricDiffs.values().stream()
.map(S3ATestUtils.MetricDiff::toString)
.collect(Collectors.joining(", "));
}
/**
* Create a builder for the cost checker.
*
* @param fs filesystem.
* @return builder.
*/
public static Builder builder(S3AFileSystem fs) {
return new Builder(fs);
}
/**
* builder.
*/
public static final class Builder {
/**
* Filesystem.
*/
private final S3AFileSystem filesystem;
/**
* Metrics to create.
*/
private final List<Statistic> metrics = new ArrayList<>();
/**
* Create with a required filesystem.
* @param filesystem monitored filesystem
*/
public Builder(final S3AFileSystem filesystem) {
this.filesystem = requireNonNull(filesystem);
}
/**
* Add a single metric.
* @param statistic statistic to monitor.
* @return this
*/
public Builder withMetric(Statistic statistic) {
return withMetric(statistic);
}
/**
* Add a varargs list of metrics.
* @param stat statistics to monitor.
* @return this.
*/
public Builder withMetrics(Statistic...stats) {
metrics.addAll(Arrays.asList(stats));
return this;
}
/**
* Instantiate.
* @return the validator.
*/
public OperationCostValidator build() {
return new OperationCostValidator(this);
}
}
/**
* Get the "always" probe.
* @return a probe which always triggers execution.
*/
public static ExpectedProbe always() {
return ALWAYS_PROBE;
}
/**
* Create a probe of a statistic which is enabled whenever the expected
* value is greater than zero.
* @param statistic statistic to check.
* @param expected expected value.
* @return a probe.
*/
public static ExpectedProbe probe(
final Statistic statistic,
final int expected) {
return probe(expected >= 0, statistic, expected);
}
/**
* Create a probe of a statistic which is conditionally enabled.
* @param enabled is the probe enabled?
* @param statistic statistic to check.
* @param expected expected value.
* @return a probe.
*/
public static ExpectedProbe probe(
final boolean enabled,
final Statistic statistic,
final int expected) {
return enabled
? new ExpectSingleStatistic(statistic, expected)
: EMPTY_PROBE;
}
/**
* Create an aggregate probe from a vararges list of probes.
* @param enabled should the probes be enabled?
* @param plist probe list
* @return a probe
*/
public static ExpectedProbe probes(
final boolean enabled,
final ExpectedProbe...plist) {
return enabled
? new ProbeList(Arrays.asList(plist))
: EMPTY_PROBE;
}
/**
* Expect the exact head and list requests of the operation
* cost supplied.
* @param enabled is the probe enabled?
* @param cost expected cost.
* @return a probe.
*/
public static ExpectedProbe expect(
boolean enabled, OperationCost cost) {
return probes(enabled,
probe(OBJECT_METADATA_REQUESTS, cost.head()),
probe(OBJECT_LIST_REQUESTS, cost.list()));
}
/**
* An expected probe to verify given criteria to trigger an eval.
* <p></p>
* Probes can be conditional, in which case they are only evaluated
* when true.
*/
public interface ExpectedProbe {
/**
* Verify a diff if the FS instance is compatible.
* @param message message to print; metric name is appended
*/
void verify(OperationCostValidator diffs, String message);
boolean isEnabled();
}
/**
* Simple probe is a single statistic.
*/
public static final class ExpectSingleStatistic implements ExpectedProbe {
private final Statistic statistic;
private final int expected;
/**
* Create.
* @param statistic statistic
* @param expected expected value.
*/
private ExpectSingleStatistic(final Statistic statistic,
final int expected) {
this.statistic = statistic;
this.expected = expected;
}
/**
* Verify a diff if the FS instance is compatible.
* @param message message to print; metric name is appended
*/
@Override
public void verify(OperationCostValidator diffs, String message) {
diffs.get(statistic).assertDiffEquals(message, expected);
}
public Statistic getStatistic() {
return statistic;
}
public int getExpected() {
return expected;
}
@Override
public boolean isEnabled() {
return true;
}
@Override
public String toString() {
String sb = "ExpectSingleStatistic{"
+ statistic
+ ", expected=" + expected
+ ", enabled=" + isEnabled()
+ '}';
return sb;
}
}
/**
* A list of probes; the verify operation
* verifies them all.
*/
public static class ProbeList implements ExpectedProbe {
/**
* Probe list.
*/
private final List<ExpectedProbe> probes;
/**
* Constructor.
* @param probes probe list.
*/
public ProbeList(final List<ExpectedProbe> probes) {
this.probes = probes;
}
@Override
public void verify(final OperationCostValidator diffs,
final String message) {
probes.forEach(p -> p.verify(diffs, message));
}
/**
* Enabled if 1+ probe is enabled.
* @return true if enabled.
*/
@Override
public boolean isEnabled() {
boolean enabled = false;
for (ExpectedProbe probe : probes) {
enabled |= probe.isEnabled();
}
return enabled;
}
@Override
public String toString() {
String pstr = probes.stream()
.map(Object::toString)
.collect(Collectors.joining(", "));
return "ProbeList{" + pstr + '}';
}
}
/**
* The empty probe always runs; it can be used to force
* a verification to execute.
*/
private static final class EmptyProbe implements ExpectedProbe {
private final String name;
private final boolean enabled;
private EmptyProbe(final String name, boolean enabled) {
this.name = name;
this.enabled = enabled;
}
@Override
public void verify(final OperationCostValidator diffs,
final String message) {
}
@Override
public boolean isEnabled() {
return enabled;
}
@Override
public String toString() {
return name;
}
}
}

View File

@ -60,10 +60,14 @@ import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_NAME_KEY;
import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_METASTORE_NULL;
import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
import static org.apache.hadoop.fs.s3a.S3AUtils.clearBucketOption;
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.BucketInfo.IS_MARKER_AWARE;
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.E_BAD_STATE;
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.INVALID_ARGUMENT;
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS;
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.exec;
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.runS3GuardCommand;
import static org.apache.hadoop.fs.s3a.tools.MarkerTool.MARKERS;
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
@ -124,7 +128,7 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
public static String expectSuccess(
String message,
S3GuardTool tool,
String... args) throws Exception {
Object... args) throws Exception {
ByteArrayOutputStream buf = new ByteArrayOutputStream();
exec(SUCCESS, message, tool, buf, args);
return buf.toString();
@ -137,9 +141,9 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
* @return the return code
* @throws Exception any exception
*/
protected int run(Configuration conf, String... args)
protected int run(Configuration conf, Object... args)
throws Exception {
return S3GuardTool.run(conf, args);
return runS3GuardCommand(conf, args);
}
/**
@ -149,8 +153,8 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
* @return the return code
* @throws Exception any exception
*/
protected int run(String... args) throws Exception {
return S3GuardTool.run(getConfiguration(), args);
protected int run(Object... args) throws Exception {
return runS3GuardCommand(getConfiguration(), args);
}
/**
@ -160,11 +164,12 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
* @param args argument list
* @throws Exception any exception
*/
protected void runToFailure(int status, String... args)
protected void runToFailure(int status, Object... args)
throws Exception {
final Configuration conf = getConfiguration();
ExitUtil.ExitException ex =
intercept(ExitUtil.ExitException.class,
() -> run(args));
intercept(ExitUtil.ExitException.class, () ->
runS3GuardCommand(conf, args));
if (ex.status != status) {
throw ex;
}
@ -445,6 +450,44 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
info.contains("S3A Client"));
}
/**
* Verify that the {@code -markers aware} option works.
* This test case is in this class for ease of backporting.
*/
@Test
public void testBucketInfoMarkerAware() throws Throwable {
final Configuration conf = getConfiguration();
URI fsUri = getFileSystem().getUri();
// run a bucket info command and look for
// confirmation that it got the output from DDB diags
S3GuardTool.BucketInfo infocmd = toClose(new S3GuardTool.BucketInfo(conf));
String info = exec(infocmd, S3GuardTool.BucketInfo.NAME,
"-" + MARKERS, S3GuardTool.BucketInfo.MARKERS_AWARE,
fsUri.toString());
assertTrue("Output should contain information about S3A client " + info,
info.contains(IS_MARKER_AWARE));
}
/**
* Verify that the {@code -markers} option fails on unknown options.
* This test case is in this class for ease of backporting.
*/
@Test
public void testBucketInfoMarkerPolicyUnknown() throws Throwable {
final Configuration conf = getConfiguration();
URI fsUri = getFileSystem().getUri();
// run a bucket info command and look for
// confirmation that it got the output from DDB diags
S3GuardTool.BucketInfo infocmd = toClose(new S3GuardTool.BucketInfo(conf));
intercept(ExitUtil.ExitException.class, ""+ EXIT_NOT_ACCEPTABLE, () ->
exec(infocmd, S3GuardTool.BucketInfo.NAME,
"-" + MARKERS, "unknown",
fsUri.toString()));
}
@Test
public void testSetCapacityFailFastIfNotGuarded() throws Exception{
Configuration conf = getConfiguration();
@ -654,4 +697,5 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
assertEquals("Mismatched s3 outputs: " + actualOut, filesOnS3, actualOnS3);
assertFalse("Diff contained duplicates", duplicates);
}
}

View File

@ -38,9 +38,12 @@ import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE;
import static org.apache.hadoop.fs.s3a.Constants.ENABLE_MULTI_DELETE;
import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides;
import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles;
@ -52,6 +55,8 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles;
* integration tests.
* <p>
* The tests only run if DynamoDB is the metastore.
* <p></p>
* The marker policy is fixed to "delete"
*/
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
public class ITestS3GuardDDBRootOperations extends AbstractS3ATestBase {
@ -82,9 +87,15 @@ public class ITestS3GuardDDBRootOperations extends AbstractS3ATestBase {
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
String bucketName = getTestBucketName(conf);
disableFilesystemCaching(conf);
removeBucketOverrides(bucketName, conf,
S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY,
ENABLE_MULTI_DELETE,
DIRECTORY_MARKER_POLICY);
conf.set(DIRECTORY_MARKER_POLICY,
DIRECTORY_MARKER_POLICY_DELETE);
// set a sleep time of 0 on pruning, for speedier test runs.
removeBucketOverrides(bucketName, conf, ENABLE_MULTI_DELETE);
conf.setTimeDuration(
S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY,
0,

View File

@ -20,12 +20,16 @@ package org.apache.hadoop.fs.s3a.s3guard;
import java.io.ByteArrayOutputStream;
import java.io.PrintStream;
import java.util.Arrays;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.ExitCodeProvider;
import org.apache.hadoop.util.ExitUtil;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
/**
@ -48,7 +52,7 @@ public final class S3GuardToolTestHelper {
* @param args argument list
* @throws Exception on any failure
*/
public static String exec(S3GuardTool cmd, String... args) throws Exception {
public static String exec(S3GuardTool cmd, Object... args) throws Exception {
return expectExecResult(0, cmd, args);
}
@ -64,7 +68,7 @@ public final class S3GuardToolTestHelper {
public static String expectExecResult(
final int expectedResult,
final S3GuardTool cmd,
final String... args) throws Exception {
final Object... args) throws Exception {
ByteArrayOutputStream buf = new ByteArrayOutputStream();
try {
exec(expectedResult, "", cmd, buf, args);
@ -77,6 +81,17 @@ public final class S3GuardToolTestHelper {
}
}
/**
* Given an array of objects, conver to an array of strings.
* @param oargs object args
* @return string equivalent
*/
public static String[] varargsToString(final Object[] oargs) {
return Arrays.stream(oargs)
.map(Object::toString)
.toArray(String[]::new);
}
/**
* Execute a command, saving the output into the buffer.
* @param expectedResult expected result of the command.
@ -91,8 +106,9 @@ public final class S3GuardToolTestHelper {
final String errorText,
final S3GuardTool cmd,
final ByteArrayOutputStream buf,
final String... args)
final Object... oargs)
throws Exception {
final String[] args = varargsToString(oargs);
LOG.info("exec {}", (Object) args);
int r;
try (PrintStream out = new PrintStream(buf)) {
@ -116,4 +132,43 @@ public final class S3GuardToolTestHelper {
}
}
/**
* Run a S3GuardTool command from a varags list.
* <p></p>
* Warning: if the filesystem is retrieved from the cache,
* it will be closed afterwards.
* @param conf configuration
* @param args argument list
* @return the return code
* @throws Exception any exception
*/
public static int runS3GuardCommand(Configuration conf, Object... args)
throws Exception {
return S3GuardTool.run(conf, varargsToString(args));
}
/**
* Run a S3GuardTool command from a varags list, catch any raised
* ExitException and verify the status code matches that expected.
* @param conf configuration
* @param status expected status code of the exception
* @param args argument list
* @throws Exception any exception
*/
public static void runS3GuardCommandToFailure(Configuration conf,
int status,
Object... args) throws Exception {
ExitUtil.ExitException ex =
intercept(ExitUtil.ExitException.class,
() -> {
int ec = runS3GuardCommand(conf, args);
if (ec != 0) {
throw new ExitUtil.ExitException(ec, "exit code " + ec);
}
});
if (ex.status != status) {
throw ex;
}
}
}

View File

@ -316,18 +316,23 @@ public class TestDirListingMetadata {
List<PathMetadata> listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3);
DirListingMetadata meta = new DirListingMetadata(path, listing, false);
meta.removeExpiredEntriesFromListing(ttl, now);
List<PathMetadata> expired = meta.removeExpiredEntriesFromListing(ttl,
now);
Assertions.assertThat(meta.getListing())
.describedAs("Metadata listing for %s", path)
.doesNotContain(pathMeta1)
.contains(pathMeta2)
.contains(pathMeta3);
Assertions.assertThat(expired)
.describedAs("Expire entries underr %s", path)
.doesNotContain(pathMeta2)
.contains(pathMeta1);
}
/*
/**
* Create DirListingMetadata with two dirs and one file living in directory
* 'parent'
* 'parent'.
*/
private static DirListingMetadata makeTwoDirsOneFile(Path parent) {
PathMetadata pathMeta1 = new PathMetadata(

View File

@ -0,0 +1,334 @@
/*
* 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.tools;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.net.URI;
import java.util.List;
import org.assertj.core.api.Assertions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.StringUtils;
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides;
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.VERBOSE;
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.runS3GuardCommand;
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.runS3GuardCommandToFailure;
import static org.apache.hadoop.fs.s3a.tools.MarkerTool.UNLIMITED_LISTING;
/**
* Class for marker tool tests -sets up keeping/deleting filesystems,
* has methods to invoke.
*/
public class AbstractMarkerToolTest extends AbstractS3ATestBase {
private static final Logger LOG =
LoggerFactory.getLogger(AbstractMarkerToolTest.class);
/** the -verbose option. */
protected static final String V = AbstractMarkerToolTest.m(VERBOSE);
/** FS which keeps markers. */
private S3AFileSystem keepingFS;
/** FS which deletes markers. */
private S3AFileSystem deletingFS;
/** FS which mixes markers; only created in some tests. */
private S3AFileSystem mixedFS;
@Override
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
String bucketName = getTestBucketName(conf);
removeBaseAndBucketOverrides(bucketName, conf,
S3A_BUCKET_PROBE,
DIRECTORY_MARKER_POLICY,
S3_METADATA_STORE_IMPL,
METADATASTORE_AUTHORITATIVE,
AUTHORITATIVE_PATH);
// base FS is legacy
conf.set(DIRECTORY_MARKER_POLICY, DIRECTORY_MARKER_POLICY_DELETE);
conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL);
// turn off bucket probes for a bit of speedup in the connectors we create.
conf.setInt(S3A_BUCKET_PROBE, 0);
return conf;
}
@Override
public void setup() throws Exception {
super.setup();
setKeepingFS(createFS(DIRECTORY_MARKER_POLICY_KEEP, null));
setDeletingFS(createFS(DIRECTORY_MARKER_POLICY_DELETE, null));
}
@Override
public void teardown() throws Exception {
// do this ourselves to avoid audits teardown failing
// when surplus markers are found
deleteTestDirInTeardown();
super.teardown();
IOUtils.cleanupWithLogger(LOG, getKeepingFS(),
getMixedFS(), getDeletingFS());
}
/**
* FS which deletes markers.
*/
public S3AFileSystem getDeletingFS() {
return deletingFS;
}
public void setDeletingFS(final S3AFileSystem deletingFS) {
this.deletingFS = deletingFS;
}
/**
* FS which keeps markers.
*/
protected S3AFileSystem getKeepingFS() {
return keepingFS;
}
private void setKeepingFS(S3AFileSystem keepingFS) {
this.keepingFS = keepingFS;
}
/** only created on demand. */
private S3AFileSystem getMixedFS() {
return mixedFS;
}
protected void setMixedFS(S3AFileSystem mixedFS) {
this.mixedFS = mixedFS;
}
/**
* Get a filename for a temp file.
* The generated file is deleted.
*
* @return a file path for a output file
*/
protected File tempAuditFile() throws IOException {
final File audit = File.createTempFile("audit", ".txt");
audit.delete();
return audit;
}
/**
* Read the audit output and verify it has the expected number of lines.
* @param auditFile audit file to read
* @param expected expected line count
*/
protected void expectMarkersInOutput(final File auditFile,
final int expected)
throws IOException {
final List<String> lines = readOutput(auditFile);
Assertions.assertThat(lines)
.describedAs("Content of %s", auditFile)
.hasSize(expected);
}
/**
* Read the output file in. Logs the contents at info.
* @param outputFile audit output file.
* @return the lines
*/
protected List<String> readOutput(final File outputFile)
throws IOException {
try (FileReader reader = new FileReader(outputFile)) {
final List<String> lines =
org.apache.commons.io.IOUtils.readLines(reader);
LOG.info("contents of output file {}\n{}", outputFile,
StringUtils.join("\n", lines));
return lines;
}
}
/**
* Create a new FS with given marker policy and path.
* This filesystem MUST be closed in test teardown.
* @param markerPolicy markers
* @param authPath authoritative path. If null: no path.
* @return a new FS.
*/
protected S3AFileSystem createFS(String markerPolicy,
String authPath) throws Exception {
S3AFileSystem testFS = getFileSystem();
Configuration conf = new Configuration(testFS.getConf());
URI testFSUri = testFS.getUri();
String bucketName = getTestBucketName(conf);
removeBucketOverrides(bucketName, conf,
DIRECTORY_MARKER_POLICY,
S3_METADATA_STORE_IMPL,
BULK_DELETE_PAGE_SIZE,
AUTHORITATIVE_PATH);
if (authPath != null) {
conf.set(AUTHORITATIVE_PATH, authPath);
}
// Use a very small page size to force the paging
// code to be tested.
conf.setInt(BULK_DELETE_PAGE_SIZE, 2);
conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL);
conf.set(DIRECTORY_MARKER_POLICY, markerPolicy);
S3AFileSystem fs2 = new S3AFileSystem();
fs2.initialize(testFSUri, conf);
LOG.info("created new filesystem with policy {} and auth path {}",
markerPolicy,
(authPath == null ? "(null)": authPath));
return fs2;
}
/**
* Execute the marker tool, expecting the execution to succeed.
* @param sourceFS filesystem to use
* @param path path to scan
* @param doPurge should markers be purged
* @param expectedMarkerCount number of markers expected
* @return the result
*/
protected MarkerTool.ScanResult markerTool(
final FileSystem sourceFS,
final Path path,
final boolean doPurge,
final int expectedMarkerCount)
throws IOException {
return markerTool(0, sourceFS, path, doPurge,
expectedMarkerCount,
UNLIMITED_LISTING, false);
}
/**
* Run a S3GuardTool command from a varags list and the
* configuration returned by {@code getConfiguration()}.
* @param args argument list
* @return the return code
* @throws Exception any exception
*/
protected int run(Object... args) throws Exception {
return runS3GuardCommand(uncachedFSConfig(getConfiguration()), args);
}
/**
* Take a configuration, copy it and disable FS Caching on
* the new one.
* @param conf source config
* @return a new, patched, config
*/
protected Configuration uncachedFSConfig(final Configuration conf) {
Configuration c = new Configuration(conf);
disableFilesystemCaching(c);
return c;
}
/**
* given an FS instance, create a matching configuration where caching
* is disabled.
* @param fs source
* @return new config.
*/
protected Configuration uncachedFSConfig(final FileSystem fs) {
return uncachedFSConfig(fs.getConf());
}
/**
* Run a S3GuardTool command from a varags list, catch any raised
* ExitException and verify the status code matches that expected.
* @param status expected status code of the exception
* @param args argument list
* @throws Exception any exception
*/
protected void runToFailure(int status, Object... args)
throws Exception {
Configuration conf = uncachedFSConfig(getConfiguration());
runS3GuardCommandToFailure(conf, status, args);
}
/**
* Given a base and a filename, create a new path.
* @param base base path
* @param name name: may be empty, in which case the base path is returned
* @return a path
*/
protected static Path toPath(final Path base, final String name) {
return name.isEmpty() ? base : new Path(base, name);
}
/**
* Execute the marker tool, expecting the execution to
* return a specific exit code.
*
* @param sourceFS filesystem to use
* @param exitCode exit code to expect.
* @param path path to scan
* @param doPurge should markers be purged
* @param expectedMarkers number of markers expected
* @param limit limit of files to scan; -1 for 'unlimited'
* @param nonAuth only use nonauth path count for failure rules
* @return the result
*/
public static MarkerTool.ScanResult markerTool(
final int exitCode,
final FileSystem sourceFS,
final Path path,
final boolean doPurge,
final int expectedMarkers,
final int limit,
final boolean nonAuth) throws IOException {
MarkerTool.ScanResult result = MarkerTool.execMarkerTool(
sourceFS,
path,
doPurge,
expectedMarkers,
limit, nonAuth);
Assertions.assertThat(result.getExitCode())
.describedAs("Exit code of marker(%s, %s, %d) -> %s",
path, doPurge, expectedMarkers, result)
.isEqualTo(exitCode);
return result;
}
/**
* Add a "-" prefix to a string.
* @param s string to prefix
* @return a string for passing into the CLI
*/
protected static String m(String s) {
return "-" + s;
}
}

View File

@ -0,0 +1,533 @@
/*
* 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.tools;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.assertj.core.api.Assertions;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
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.S3AFileSystem;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_AUTHORITATIVE;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP;
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.BucketInfo.BUCKET_INFO;
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.runS3GuardCommand;
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.runS3GuardCommandToFailure;
import static org.apache.hadoop.fs.s3a.tools.MarkerTool.*;
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_INTERRUPTED;
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE;
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_FOUND;
import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE;
/**
* Test the marker tool and use it to compare the behavior
* of keeping vs legacy S3A FS instances.
*/
public class ITestMarkerTool extends AbstractMarkerToolTest {
protected static final Logger LOG =
LoggerFactory.getLogger(ITestMarkerTool.class);
/**
* How many files to expect.
*/
private int expectedFileCount;
/**
* How many markers to expect under dir1.
*/
private int expectedMarkersUnderDir1;
/**
* How many markers to expect under dir2.
*/
private int expectedMarkersUnderDir2;
/**
* How many markers to expect across both dirs?
*/
private int expectedMarkers;
/**
* How many markers to expect including the base directory?
*/
private int expectedMarkersWithBaseDir;
@Test
public void testCleanMarkersLegacyDir() throws Throwable {
describe("Clean markers under a deleting FS -expect none");
CreatedPaths createdPaths = createPaths(getDeletingFS(), methodPath());
markerTool(getDeletingFS(), createdPaths.base, false, 0);
markerTool(getDeletingFS(), createdPaths.base, true, 0);
}
@Test
public void testCleanMarkersFileLimit() throws Throwable {
describe("Clean markers under a keeping FS -with file limit");
CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath());
// audit will be interrupted
markerTool(EXIT_INTERRUPTED, getDeletingFS(),
createdPaths.base, false, 0, 1, false);
}
@Test
public void testCleanMarkersKeepingDir() throws Throwable {
describe("Audit then clean markers under a deleting FS "
+ "-expect markers to be found and then cleaned up");
CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath());
// audit will find the expected entries
int expectedMarkerCount = createdPaths.dirs.size();
S3AFileSystem fs = getDeletingFS();
LOG.info("Auditing a directory with retained markers -expect failure");
markerTool(EXIT_NOT_ACCEPTABLE, fs,
createdPaths.base, false, 0, UNLIMITED_LISTING, false);
LOG.info("Auditing a directory expecting retained markers");
markerTool(fs, createdPaths.base, false,
expectedMarkerCount);
// we require that a purge didn't take place, so run the
// audit again.
LOG.info("Auditing a directory expecting retained markers");
markerTool(fs, createdPaths.base, false,
expectedMarkerCount);
LOG.info("Purging a directory of retained markers");
// purge cleans up
assertMarkersDeleted(expectedMarkerCount,
markerTool(fs, createdPaths.base, true, expectedMarkerCount));
// and a rerun doesn't find markers
LOG.info("Auditing a directory with retained markers -expect success");
assertMarkersDeleted(0,
markerTool(fs, createdPaths.base, true, 0));
}
@Test
public void testRenameKeepingFS() throws Throwable {
describe("Rename with the keeping FS -verify that no markers"
+ " exist at far end");
Path base = methodPath();
Path source = new Path(base, "source");
Path dest = new Path(base, "dest");
S3AFileSystem fs = getKeepingFS();
CreatedPaths createdPaths = createPaths(fs, source);
// audit will find three entries
int expectedMarkerCount = createdPaths.dirs.size();
markerTool(fs, source, false, expectedMarkerCount);
fs.rename(source, dest);
assertIsDirectory(dest);
// there are no markers
markerTool(fs, dest, false, 0);
LOG.info("Auditing destination paths");
verifyRenamed(dest, createdPaths);
}
/**
* Create a FS where only dir2 in the source tree keeps markers;
* verify all is good.
*/
@Test
public void testAuthPathIsMixed() throws Throwable {
describe("Create a source tree with mixed semantics");
Path base = methodPath();
Path source = new Path(base, "source");
Path dest = new Path(base, "dest");
Path dir2 = new Path(source, "dir2");
S3AFileSystem mixedFSDir2 = createFS(DIRECTORY_MARKER_POLICY_AUTHORITATIVE,
dir2.toUri().toString());
// line up for close in teardown
setMixedFS(mixedFSDir2);
// some of these paths will retain markers, some will not
CreatedPaths createdPaths = createPaths(mixedFSDir2, source);
// markers are only under dir2
markerTool(mixedFSDir2, toPath(source, "dir1"), false, 0);
markerTool(mixedFSDir2, source, false, expectedMarkersUnderDir2);
// full scan of source will fail
markerTool(EXIT_NOT_ACCEPTABLE,
mixedFSDir2, source, false, 0, 0, false);
// but add the -nonauth option and the markers under dir2 are skipped
markerTool(0, mixedFSDir2, source, false, 0, 0, true);
// if we now rename, all will be good
LOG.info("Executing rename");
mixedFSDir2.rename(source, dest);
assertIsDirectory(dest);
// there are no markers
MarkerTool.ScanResult scanResult = markerTool(mixedFSDir2, dest, false, 0);
// there are exactly the files we want
Assertions.assertThat(scanResult)
.describedAs("Scan result %s", scanResult)
.extracting(s -> s.getTracker().getFilesFound())
.isEqualTo(expectedFileCount);
verifyRenamed(dest, createdPaths);
}
/**
* Assert that an expected number of markers were deleted.
* @param expected expected count.
* @param result scan result
*/
private static void assertMarkersDeleted(int expected,
MarkerTool.ScanResult result) {
Assertions.assertThat(result.getPurgeSummary())
.describedAs("Purge result of scan %s", result)
.isNotNull()
.extracting(f -> f.getMarkersDeleted())
.isEqualTo(expected);
}
/**
* Marker tool with no args.
*/
@Test
public void testRunNoArgs() throws Throwable {
runToFailure(EXIT_USAGE, MARKERS);
}
@Test
public void testRunWrongBucket() throws Throwable {
runToFailure(EXIT_NOT_FOUND, MARKERS,
AUDIT,
"s3a://this-bucket-does-not-exist-hopefully");
}
/**
* Run with a path that doesn't exist.
*/
@Test
public void testRunUnknownPath() throws Throwable {
runToFailure(EXIT_NOT_FOUND, MARKERS,
AUDIT,
methodPath());
}
/**
* Having both -audit and -clean on the command line is an error.
*/
@Test
public void testRunTooManyActions() throws Throwable {
runToFailure(EXIT_USAGE, MARKERS,
AUDIT, CLEAN,
methodPath());
}
@Test
public void testRunAuditWithExpectedMarkers() throws Throwable {
describe("Run a verbose audit expecting some markers");
// a run under the keeping FS will create paths
CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath());
final File audit = tempAuditFile();
run(MARKERS, V,
AUDIT,
m(OPT_LIMIT), 0,
m(OPT_OUT), audit,
m(OPT_EXPECTED), expectedMarkersWithBaseDir,
createdPaths.base);
expectMarkersInOutput(audit, expectedMarkersWithBaseDir);
}
@Test
public void testRunAuditWithExcessMarkers() throws Throwable {
describe("Run a verbose audit failing as surplus markers were found");
// a run under the keeping FS will create paths
CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath());
final File audit = tempAuditFile();
runToFailure(EXIT_NOT_ACCEPTABLE, MARKERS, V,
AUDIT,
m(OPT_OUT), audit,
createdPaths.base);
expectMarkersInOutput(audit, expectedMarkersWithBaseDir);
}
@Test
public void testRunLimitedAudit() throws Throwable {
describe("Audit with a limited number of files (2)");
CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath());
runToFailure(EXIT_INTERRUPTED,
MARKERS, V,
m(OPT_LIMIT), 2,
CLEAN,
createdPaths.base);
run(MARKERS, V,
AUDIT,
createdPaths.base);
}
/**
* Run an audit against the landsat bucket.
* <p></p>
* This tests paging/scale against a larger bucket without
* worrying about setup costs.
*/
@Test
public void testRunLimitedLandsatAudit() throws Throwable {
describe("Audit a few thousand landsat objects");
final File audit = tempAuditFile();
run(MARKERS,
AUDIT,
m(OPT_LIMIT), 3000,
m(OPT_OUT), audit,
LANDSAT_BUCKET);
readOutput(audit);
}
@Test
public void testBucketInfoKeepingOnDeleting() throws Throwable {
describe("Run bucket info with the keeping config on the deleting fs");
runS3GuardCommandToFailure(uncachedFSConfig(getDeletingFS()),
EXIT_NOT_ACCEPTABLE,
BUCKET_INFO,
m(MARKERS), DIRECTORY_MARKER_POLICY_KEEP,
methodPath());
}
@Test
public void testBucketInfoKeepingOnKeeping() throws Throwable {
describe("Run bucket info with the keeping config on the keeping fs");
runS3GuardCommand(uncachedFSConfig(getKeepingFS()),
BUCKET_INFO,
m(MARKERS), DIRECTORY_MARKER_POLICY_KEEP,
methodPath());
}
@Test
public void testBucketInfoDeletingOnDeleting() throws Throwable {
describe("Run bucket info with the deleting config on the deleting fs");
runS3GuardCommand(uncachedFSConfig(getDeletingFS()),
BUCKET_INFO,
m(MARKERS), DIRECTORY_MARKER_POLICY_DELETE,
methodPath());
}
@Test
public void testBucketInfoAuthOnAuth() throws Throwable {
describe("Run bucket info with the auth FS");
Path base = methodPath();
S3AFileSystem authFS = createFS(DIRECTORY_MARKER_POLICY_AUTHORITATIVE,
base.toUri().toString());
// line up for close in teardown
setMixedFS(authFS);
runS3GuardCommand(uncachedFSConfig(authFS),
BUCKET_INFO,
m(MARKERS), DIRECTORY_MARKER_POLICY_AUTHORITATIVE,
methodPath());
}
/**
* Tracker of created paths.
*/
private static final class CreatedPaths {
private final FileSystem fs;
private final Path base;
private List<Path> files = new ArrayList<>();
private List<Path> dirs = new ArrayList<>();
private List<Path> emptyDirs = new ArrayList<>();
private List<String> filesUnderBase = new ArrayList<>();
private List<String> dirsUnderBase = new ArrayList<>();
private List<String> emptyDirsUnderBase = new ArrayList<>();
/**
* Constructor.
* @param fs filesystem.
* @param base base directory for all creation operations.
*/
private CreatedPaths(final FileSystem fs,
final Path base) {
this.fs = fs;
this.base = base;
}
/**
* Make a set of directories.
* @param names varargs list of paths under the base.
* @return number of entries created.
* @throws IOException failure
*/
private int dirs(String... names) throws IOException {
for (String name : names) {
mkdir(name);
}
return names.length;
}
/**
* Create a single directory under the base.
* @param name name/relative names of the directory
* @return the path of the new entry.
*/
private Path mkdir(String name) throws IOException {
Path dir = toPath(base, name);
fs.mkdirs(dir);
dirs.add(dir);
dirsUnderBase.add(name);
return dir;
}
/**
* Make a set of empty directories.
* @param names varargs list of paths under the base.
* @return number of entries created.
* @throws IOException failure
*/
private int emptydirs(String... names) throws IOException {
for (String name : names) {
emptydir(name);
}
return names.length;
}
/**
* Create an empty directory.
* @param name name under the base dir
* @return the path
* @throws IOException failure
*/
private Path emptydir(String name) throws IOException {
Path dir = toPath(base, name);
fs.mkdirs(dir);
emptyDirs.add(dir);
emptyDirsUnderBase.add(name);
return dir;
}
/**
* Make a set of files.
* @param names varargs list of paths under the base.
* @return number of entries created.
* @throws IOException failure
*/
private int files(String... names) throws IOException {
for (String name : names) {
mkfile(name);
}
return names.length;
}
/**
* Create a 0-byte file.
* @param name name under the base dir
* @return the path
* @throws IOException failure
*/
private Path mkfile(String name)
throws IOException {
Path file = toPath(base, name);
ContractTestUtils.touch(fs, file);
files.add(file);
filesUnderBase.add(name);
return file;
}
}
/**
* Create the "standard" test paths.
* @param fs filesystem
* @param base base dir
* @return the details on what was created.
*/
private CreatedPaths createPaths(FileSystem fs, Path base)
throws IOException {
CreatedPaths r = new CreatedPaths(fs, base);
// the directories under which we will create files,
// so expect to have markers
r.mkdir("");
// create the empty dirs
r.emptydir("empty");
// dir 1 has a file underneath
r.mkdir("dir1");
expectedFileCount = r.files("dir1/file1");
expectedMarkersUnderDir1 = 1;
// dir2 has a subdir
r.dirs("dir2", "dir2/dir3");
// an empty subdir
r.emptydir("dir2/empty2");
// and a file under itself and dir3
expectedFileCount += r.files(
"dir2/file2",
"dir2/dir3/file3");
// wrap up the expectations.
expectedMarkersUnderDir2 = 2;
expectedMarkers = expectedMarkersUnderDir1 + expectedMarkersUnderDir2;
expectedMarkersWithBaseDir = expectedMarkers + 1;
return r;
}
/**
* Verify that all the paths renamed from the source exist
* under the destination, including all empty directories.
* @param dest destination to look under.
* @param createdPaths list of created paths.
*/
void verifyRenamed(final Path dest,
final CreatedPaths createdPaths) throws IOException {
// all leaf directories exist
for (String p : createdPaths.emptyDirsUnderBase) {
assertIsDirectory(toPath(dest, p));
}
// non-empty dirs
for (String p : createdPaths.dirsUnderBase) {
assertIsDirectory(toPath(dest, p));
}
// all files exist
for (String p : createdPaths.filesUnderBase) {
assertIsFile(toPath(dest, p));
}
}
}

View File

@ -0,0 +1,70 @@
/*
* 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.tools;
import java.io.File;
import org.junit.FixMethodOrder;
import org.junit.Test;
import org.junit.runners.MethodSorters;
import org.apache.hadoop.fs.Path;
import static org.apache.hadoop.fs.s3a.tools.MarkerTool.AUDIT;
import static org.apache.hadoop.fs.s3a.tools.MarkerTool.CLEAN;
import static org.apache.hadoop.fs.s3a.tools.MarkerTool.MARKERS;
import static org.apache.hadoop.fs.s3a.tools.MarkerTool.OPT_OUT;
/**
* Marker tool tests against the root FS; run in the sequential phase.
*/
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
public class ITestMarkerToolRootOperations extends AbstractMarkerToolTest {
private Path rootPath;
@Override
public void setup() throws Exception {
super.setup();
rootPath = getFileSystem().makeQualified(new Path("/"));
}
@Test
public void test_100_audit_root_noauth() throws Throwable {
describe("Run a verbose audit");
final File audit = tempAuditFile();
run(MARKERS, V,
AUDIT,
m(OPT_OUT), audit,
rootPath);
readOutput(audit);
}
@Test
public void test_200_clean_root() throws Throwable {
describe("Clean the root path");
final File audit = tempAuditFile();
run(MARKERS, V,
CLEAN,
m(OPT_OUT), audit,
rootPath);
readOutput(audit);
}
}