diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index d28704b7c334e..1d04107ff5b48 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -48,10 +48,6 @@
200000
-
- false
-
- unset
@@ -126,9 +122,6 @@
${fs.s3a.scale.test.huge.filesize}${fs.s3a.scale.test.huge.partitionsize}${fs.s3a.scale.test.timeout}
-
- ${fs.s3a.directory.marker.retention}
- ${fs.s3a.directory.marker.audit}${fs.s3a.prefetch.enabled}
@@ -167,8 +160,6 @@
${fs.s3a.scale.test.huge.filesize}${fs.s3a.scale.test.huge.partitionsize}${fs.s3a.scale.test.timeout}
- ${fs.s3a.directory.marker.retention}
-
${test.integration.timeout}${fs.s3a.prefetch.enabled}
@@ -221,9 +212,6 @@
${fs.s3a.scale.test.huge.filesize}${fs.s3a.scale.test.huge.partitionsize}${fs.s3a.scale.test.timeout}
-
- ${fs.s3a.directory.marker.retention}
- ${fs.s3a.directory.marker.audit}${fs.s3a.prefetch.enabled}
@@ -285,9 +273,6 @@
${fs.s3a.scale.test.enabled}${fs.s3a.scale.test.huge.filesize}${fs.s3a.scale.test.timeout}
-
- ${fs.s3a.directory.marker.retention}
- ${fs.s3a.directory.marker.audit}${fs.s3a.prefetch.enabled}job-${job.id}
@@ -314,44 +299,6 @@
-
-
- keep-markers
-
-
- markers
- keep
-
-
-
- keep
-
-
-
- delete-markers
-
-
- markers
- delete
-
-
-
- delete
-
-
-
- auth-markers
-
-
- markers
- authoritative
-
-
-
- authoritative
-
-
-
prefetch
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
index b03c41c7bb1d2..0a6944c6d293f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
@@ -848,6 +848,7 @@ private Constants() {
"fs.s3a." + Constants.AWS_SERVICE_IDENTIFIER_STS.toLowerCase()
+ ".signing-algorithm";
+ @Deprecated
public static final String S3N_FOLDER_SUFFIX = "_$folder$";
public static final String FS_S3A_BLOCK_SIZE = "fs.s3a.block.size";
public static final String FS_S3A = "s3a";
@@ -868,10 +869,13 @@ private Constants() {
/**
* Paths considered "authoritative".
* When S3guard was supported, this skipped checks to s3 on directory listings.
- * It is also use to optionally disable marker retentation purely on these
- * paths -a feature which is still retained/available.
+ * It was also possilbe to use to optionally disable marker retentation purely on these
+ * paths -a feature which is no longer available.
+ * As no feature uses this any more, it is declared as deprecated.
* */
+ @Deprecated
public static final String AUTHORITATIVE_PATH = "fs.s3a.authoritative.path";
+ @Deprecated
public static final String[] DEFAULT_AUTHORITATIVE_PATH = {};
/**
@@ -1339,37 +1343,44 @@ private Constants() {
/**
* 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.
+ * No longer supported as "keep" is the sole policy.
*/
+ @Deprecated
public static final String DIRECTORY_MARKER_POLICY =
"fs.s3a.directory.marker.retention";
/**
- * Delete directory markers. This is the backwards compatible option.
+ * Delete directory markers.
+ * No longer supported as "keep" is the sole policy.
* Value: {@value}.
*/
+ @Deprecated
public static final String DIRECTORY_MARKER_POLICY_DELETE =
"delete";
/**
* Retain directory markers.
+ * No longer needed, so marked as deprecated to flag usages.
* Value: {@value}.
*/
+ @Deprecated
public static final String DIRECTORY_MARKER_POLICY_KEEP =
"keep";
/**
* Retain directory markers in authoritative directory trees only.
+ * No longer required as "keep" is the sole policy.
* Value: {@value}.
*/
+ @Deprecated
public static final String DIRECTORY_MARKER_POLICY_AUTHORITATIVE =
"authoritative";
/**
* Default retention policy: {@value}.
+ * No longer required as "keep" is the sole policy.
*/
+ @Deprecated
public static final String DEFAULT_DIRECTORY_MARKER_POLICY =
DIRECTORY_MARKER_POLICY_KEEP;
@@ -1377,7 +1388,7 @@ private Constants() {
/**
* {@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.
+ * directory markers have not been deleted.
* Value: {@value}.
*/
public static final String STORE_CAPABILITY_DIRECTORY_MARKER_AWARE
@@ -1394,16 +1405,20 @@ private Constants() {
/**
* {@code PathCapabilities} probe to indicate that the filesystem
* deletes directory markers.
+ * Always false.
* Value: {@value}.
*/
+ @Deprecated
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.
+ * This probe always returns false.
* Value: {@value}.
*/
+ @Deprecated
public static final String
STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_AUTHORITATIVE =
"fs.s3a.capability.directory.marker.policy.authoritative";
@@ -1411,6 +1426,7 @@ private Constants() {
/**
* {@code PathCapabilities} probe to indicate that a path
* keeps directory markers.
+ * This probe always returns true.
* Value: {@value}.
*/
public static final String STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP
@@ -1419,6 +1435,7 @@ private Constants() {
/**
* {@code PathCapabilities} probe to indicate that a path
* deletes directory markers.
+ * This probe always returns false.
* Value: {@value}.
*/
public static final String STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
index d8f68b4840339..a9d531b5b4621 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
@@ -50,9 +50,7 @@
import java.util.concurrent.CompletableFuture;
import java.util.StringJoiner;
-import static org.apache.hadoop.fs.s3a.Constants.S3N_FOLDER_SUFFIX;
import static org.apache.hadoop.fs.s3a.Invoker.onceInTheFuture;
-import static org.apache.hadoop.fs.s3a.S3AUtils.ACCEPT_ALL;
import static org.apache.hadoop.fs.s3a.S3AUtils.createFileStatus;
import static org.apache.hadoop.fs.s3a.S3AUtils.maybeAddTrailingSlash;
import static org.apache.hadoop.fs.s3a.S3AUtils.objectRepresentsDirectory;
@@ -76,8 +74,8 @@ public class Listing extends AbstractStoreOperation {
private static final Logger LOG = S3AFileSystem.LOG;
- static final FileStatusAcceptor ACCEPT_ALL_BUT_S3N =
- new AcceptAllButS3nDirs();
+ static final FileStatusAcceptor ACCEPT_ALL_OBJECTS =
+ new AcceptAllObjects();
private final ListingOperationCallbacks listingOperationCallbacks;
@@ -116,7 +114,7 @@ public static RemoteIterator toProvidedFileStatusIterator(
S3AFileStatus[] fileStatuses) {
return filteringRemoteIterator(
remoteIteratorFromArray(fileStatuses),
- Listing.ACCEPT_ALL_BUT_S3N::accept);
+ Listing.ACCEPT_ALL_OBJECTS::accept);
}
/**
@@ -132,7 +130,7 @@ public static RemoteIterator toProvidedFileStatusIterator(
* @throws IOException IO Problems
*/
@Retries.RetryRaw
- public FileStatusListingIterator createFileStatusListingIterator(
+ public RemoteIterator createFileStatusListingIterator(
Path listPath,
S3ListRequest request,
PathFilter filter,
@@ -212,7 +210,7 @@ public RemoteIterator getListFilesAssumingDir(
.createListObjectsRequest(key,
delimiter,
span),
- ACCEPT_ALL,
+ S3AUtils.ACCEPT_ALL,
acceptor,
span));
}
@@ -235,7 +233,7 @@ public RemoteIterator getLocatedFileStatusIteratorForDir(
listingOperationCallbacks
.createListObjectsRequest(key, "/", span),
filter,
- new AcceptAllButSelfAndS3nDirs(dir),
+ new AcceptAllButSelf(dir),
span));
}
@@ -263,8 +261,8 @@ public RemoteIterator getLocatedFileStatusIteratorForDir(
return createFileStatusListingIterator(
path,
request,
- ACCEPT_ALL,
- new AcceptAllButSelfAndS3nDirs(path),
+ S3AUtils.ACCEPT_ALL,
+ new AcceptAllButSelf(path),
span);
}
@@ -462,7 +460,7 @@ private boolean buildNextStatusBatch(S3ListResult objects) throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("{}: {}", keyPath, stringify(s3Object));
}
- // Skip over keys that are ourselves and old S3N _$folder$ files
+ // Skip over keys that are ourselves
if (acceptor.accept(keyPath, s3Object) && filter.accept(keyPath)) {
S3AFileStatus status = createFileStatus(keyPath, s3Object,
blockSize, userName, s3Object.eTag(),
@@ -722,13 +720,12 @@ public void close() {
}
/**
- * Accept all entries except the base path and those which map to S3N
- * pseudo directory markers.
+ * Accept all entries except the base path.
*/
static class AcceptFilesOnly implements FileStatusAcceptor {
private final Path qualifiedPath;
- public AcceptFilesOnly(Path qualifiedPath) {
+ AcceptFilesOnly(Path qualifiedPath) {
this.qualifiedPath = qualifiedPath;
}
@@ -743,7 +740,6 @@ public AcceptFilesOnly(Path qualifiedPath) {
@Override
public boolean accept(Path keyPath, S3Object s3Object) {
return !keyPath.equals(qualifiedPath)
- && !s3Object.key().endsWith(S3N_FOLDER_SUFFIX)
&& !objectRepresentsDirectory(s3Object.key());
}
@@ -765,29 +761,28 @@ public boolean accept(FileStatus status) {
}
/**
- * Accept all entries except those which map to S3N pseudo directory markers.
+ * Accept all entries.
*/
- static class AcceptAllButS3nDirs implements FileStatusAcceptor {
+ static class AcceptAllObjects implements FileStatusAcceptor {
public boolean accept(Path keyPath, S3Object s3Object) {
- return !s3Object.key().endsWith(S3N_FOLDER_SUFFIX);
+ return true;
}
public boolean accept(Path keyPath, String prefix) {
- return !keyPath.toString().endsWith(S3N_FOLDER_SUFFIX);
+ return true;
}
public boolean accept(FileStatus status) {
- return !status.getPath().toString().endsWith(S3N_FOLDER_SUFFIX);
+ return true;
}
}
/**
- * Accept all entries except the base path and those which map to S3N
- * pseudo directory markers.
+ * Accept all entries except the base path.
*/
- public static class AcceptAllButSelfAndS3nDirs implements FileStatusAcceptor {
+ public static class AcceptAllButSelf implements FileStatusAcceptor {
/** Base path. */
private final Path qualifiedPath;
@@ -796,13 +791,12 @@ public static class AcceptAllButSelfAndS3nDirs implements FileStatusAcceptor {
* Constructor.
* @param qualifiedPath an already-qualified path.
*/
- public AcceptAllButSelfAndS3nDirs(Path qualifiedPath) {
+ public AcceptAllButSelf(Path qualifiedPath) {
this.qualifiedPath = qualifiedPath;
}
/**
- * Reject a s3Object entry if the key path is the qualified Path, or
- * it ends with {@code "_$folder$"}.
+ * Reject a s3Object entry if the key path is the qualified Path.
* @param keyPath key path of the entry
* @param s3Object s3Object entry
* @return true if the entry is accepted (i.e. that a status entry
@@ -810,8 +804,7 @@ public AcceptAllButSelfAndS3nDirs(Path qualifiedPath) {
*/
@Override
public boolean accept(Path keyPath, S3Object s3Object) {
- return !keyPath.equals(qualifiedPath) &&
- !s3Object.key().endsWith(S3N_FOLDER_SUFFIX);
+ return !keyPath.equals(qualifiedPath);
}
/**
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index c0e530cb5ce40..14031ed007ef8 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -30,7 +30,6 @@
import java.time.Duration;
import java.time.Instant;
import java.util.ArrayList;
-import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.EnumSet;
@@ -130,8 +129,6 @@
import org.apache.hadoop.fs.s3a.impl.CSEV1CompatibleS3AFileSystemOperations;
import org.apache.hadoop.fs.s3a.impl.CSEMaterials;
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.GetContentSummaryOperation;
import org.apache.hadoop.fs.s3a.impl.HeaderProcessing;
import org.apache.hadoop.fs.s3a.impl.InternalConstants;
@@ -198,7 +195,6 @@
import org.apache.hadoop.fs.s3a.commit.PutTracker;
import org.apache.hadoop.fs.s3a.commit.MagicCommitIntegration;
import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
-import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
@@ -395,7 +391,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
private ChangeDetectionPolicy changeDetectionPolicy;
private final AtomicBoolean closed = new AtomicBoolean(false);
private volatile boolean isClosed = false;
- private Collection allowAuthoritativePaths;
/** Delegation token integration; non-empty when DT support is enabled. */
private Optional delegationTokens = Optional.empty();
@@ -431,11 +426,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
*/
private OpenFileSupport openFileHelper;
- /**
- * Directory policy.
- */
- private DirectoryPolicy directoryPolicy;
-
/**
* Context accessors for re-use.
*/
@@ -771,12 +761,7 @@ public void initialize(URI name, Configuration originalConf)
performanceFlags.makeImmutable();
LOG.debug("{} = {}", FS_S3A_CREATE_PERFORMANCE, performanceCreation);
- allowAuthoritativePaths = S3Guard.getAuthoritativePaths(this);
- // directory policy, which may look at authoritative paths
- directoryPolicy = DirectoryPolicyImpl.getDirectoryPolicy(conf,
- this::allowAuthoritative);
- LOG.debug("Directory marker retention policy is {}", directoryPolicy);
pageSize = intOption(getConf(), BULK_DELETE_PAGE_SIZE,
BULK_DELETE_PAGE_SIZE_DEFAULT, 0);
checkArgument(pageSize <= InternalConstants.MAX_ENTRIES_TO_DELETE,
@@ -2010,33 +1995,6 @@ public UploadPartResponse uploadPart(
return store.uploadPart(request, body, durationTrackerFactory);
}
- /**
- * Perform post-write actions.
- *
- * This operation MUST be called after any PUT/multipart PUT completes
- * successfully.
- *
- * The actions include calling
- * {@link #deleteUnnecessaryFakeDirectories(Path)}
- * if directory markers are not being retained.
- * @param eTag eTag of the written object
- * @param versionId S3 object versionId of the written object
- * @param key key written to
- * @param length total length of file written
- * @param putOptions put object options
- */
- @Override
- @Retries.RetryExceptionsSwallowed
- public void finishedWrite(
- String key,
- long length,
- PutObjectOptions putOptions) {
- S3AFileSystem.this.finishedWrite(
- key,
- length,
- putOptions);
-
- }
}
/**
@@ -2153,8 +2111,7 @@ public FSDataOutputStream create(Path f, FsPermission permission,
* Retry policy: retrying, translated on the getFileStatus() probe.
* No data is uploaded to S3 in this call, so no retry issues related to that.
* The "performance" flag disables safety checks for the path being a file,
- * parent directory existing, and doesn't attempt to delete
- * dir markers, irrespective of FS settings.
+ * or parent directory existing.
* If true, this method call does no IO at all.
* @param path the file name to open
* @param progress the progress reporter.
@@ -2213,11 +2170,9 @@ private FSDataOutputStream innerCreateFile(
committerIntegration.createTracker(path, key, outputStreamStatistics);
String destKey = putTracker.getDestKey();
- // put options are derived from the path and the
- // option builder.
- boolean keep = options.isPerformance() || keepDirectoryMarkers(path);
+ // put options are derived from the option builder.
final PutObjectOptions putOptions =
- new PutObjectOptions(keep, null, options.getHeaders());
+ new PutObjectOptions(null, options.getHeaders());
validateOutputStreamConfiguration(path, getConf());
@@ -2650,8 +2605,8 @@ public RemoteIterator listFilesAndDirectoryMarkers(
path,
true,
includeSelf
- ? Listing.ACCEPT_ALL_BUT_S3N
- : new Listing.AcceptAllButSelfAndS3nDirs(path),
+ ? Listing.ACCEPT_ALL_OBJECTS
+ : new Listing.AcceptAllButSelf(path),
status
);
}
@@ -2682,9 +2637,6 @@ public void finishRename(final Path sourceRenamed, final Path destCreated)
Path destParent = destCreated.getParent();
if (!sourceRenamed.getParent().equals(destParent)) {
LOG.debug("source & dest parents are different; fix up dir markers");
- if (!keepDirectoryMarkers(destParent)) {
- deleteUnnecessaryFakeDirectories(destParent);
- }
maybeCreateFakeParentDirectory(sourceRenamed);
}
}
@@ -2699,7 +2651,7 @@ public RemoteIterator listObjects(
listing.createFileStatusListingIterator(path,
createListObjectsRequest(key, null),
ACCEPT_ALL,
- Listing.ACCEPT_ALL_BUT_S3N,
+ Listing.ACCEPT_ALL_OBJECTS,
auditSpan));
}
@@ -3359,8 +3311,6 @@ PutObjectResponse putObjectDirect(PutObjectRequest putObjectRequest,
provider.getSize(),
CONTENT_TYPE_OCTET_STREAM)));
incrementPutCompletedStatistics(true, len);
- // apply any post-write actions.
- finishedWrite(putObjectRequest.key(), len, putOptions);
return response;
} catch (SdkException e) {
incrementPutCompletedStatistics(false, len);
@@ -3637,7 +3587,7 @@ private void createFakeDirectoryIfNecessary(Path f)
// is mostly harmless to create a new one.
if (!key.isEmpty() && !s3Exists(f, StatusProbeEnum.DIRECTORIES)) {
LOG.debug("Creating new fake directory at {}", f);
- createFakeDirectory(key, putOptionsForPath(f));
+ createFakeDirectory(key, PutObjectOptions.defaultOptions());
}
}
@@ -3729,25 +3679,13 @@ private RemoteIterator innerListStatus(Path f)
return listing.createProvidedFileStatusIterator(
stats,
ACCEPT_ALL,
- Listing.ACCEPT_ALL_BUT_S3N);
+ Listing.ACCEPT_ALL_OBJECTS);
}
}
// Here we have a directory which may or may not be empty.
return statusIt;
}
- /**
- * Is a path to be considered as authoritative?
- * is a store with the supplied path under
- * one of the paths declared as authoritative.
- * @param path path
- * @return true if the path is auth
- */
- public boolean allowAuthoritative(final Path path) {
- return S3Guard.allowAuthoritative(path, this,
- allowAuthoritativePaths);
- }
-
/**
* Create a {@code ListObjectsRequest} request against this bucket,
* with the maximum keys returned in a query set by {@link #maxKeys}.
@@ -3873,13 +3811,11 @@ public S3AFileStatus probePathStatus(final Path path,
}
@Override
- public void createFakeDirectory(final Path dir, final boolean keepMarkers)
+ public void createFakeDirectory(final Path dir)
throws IOException {
S3AFileSystem.this.createFakeDirectory(
pathToKey(dir),
- keepMarkers
- ? PutObjectOptions.keepingDirs()
- : putOptionsForPath(dir));
+ PutObjectOptions.defaultOptions());
}
}
@@ -3927,7 +3863,7 @@ public S3AFileStatus probePathStatus(final Path path,
@Override
public RemoteIterator listFilesIterator(final Path path,
final boolean recursive) throws IOException {
- return S3AFileSystem.this.innerListFiles(path, recursive, Listing.ACCEPT_ALL_BUT_S3N, null);
+ return S3AFileSystem.this.innerListFiles(path, recursive, Listing.ACCEPT_ALL_OBJECTS, null);
}
}
@@ -4276,7 +4212,8 @@ public void copyLocalFileFromTo(File file, Path from, Path to) throws IOExceptio
newPutObjectRequestBuilder(key, file.length(), false);
final String dest = to.toString();
S3AFileSystem.this.invoker.retry("putObject(" + dest + ")", dest, true, () ->
- executePut(putObjectRequestBuilder.build(), null, putOptionsForPath(to), file));
+ executePut(putObjectRequestBuilder.build(), null,
+ PutObjectOptions.defaultOptions(), file));
return null;
});
}
@@ -4319,15 +4256,11 @@ PutObjectResponse executePut(
final File file)
throws IOException {
String key = putObjectRequest.key();
- long len = getPutRequestLength(putObjectRequest);
ProgressableProgressListener listener =
- new ProgressableProgressListener(store, putObjectRequest.key(), progress);
+ new ProgressableProgressListener(store, key, progress);
UploadInfo info = putObject(putObjectRequest, file, listener);
PutObjectResponse result = store.waitForUploadCompletion(key, info).response();
listener.uploadCompleted(info.getFileUpload());
-
- // post-write actions
- finishedWrite(key, len, putOptions);
return result;
}
@@ -4579,7 +4512,7 @@ public List listAWSPolicyRules(
/**
* Copy a single object in the bucket via a COPY operation.
- * There's no update of metadata, directory markers, etc.
+ * There's no update of metadata, etc.
* Callers must implement.
* @param srcKey source object path
* @param dstKey destination object path
@@ -4711,10 +4644,6 @@ CreateMultipartUploadResponse initiateMultipartUpload(
*
* This operation MUST be called after any PUT/multipart PUT completes
* successfully.
- *
- * The actions include calling
- * {@link #deleteUnnecessaryFakeDirectories(Path)}
- * if directory markers are not being retained.
* @param key key written to
* @param length total length of file written
* @param putOptions put object options
@@ -4728,70 +4657,6 @@ void finishedWrite(
LOG.debug("Finished write to {}, len {}.",
key, length);
Preconditions.checkArgument(length >= 0, "content length is negative");
- if (!putOptions.isKeepMarkers()) {
- Path p = keyToQualifiedPath(key);
- deleteUnnecessaryFakeDirectories(p.getParent());
- }
- }
-
- /**
- * Should we keep directory markers under the path being created
- * by mkdir/file creation/rename?
- * This is done if marker retention is enabled for the path,
- * or it is under a magic path where we are saving IOPs
- * knowing that all committers are on the same code version and
- * therefore marker aware.
- * @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)
- || isUnderMagicCommitPath(path);
- }
-
- /**
- * Should we keep directory markers under the path being created
- * by mkdir/file creation/rename?
- * See {@link #keepDirectoryMarkers(Path)} for the policy.
- *
- * @param path path to probe
- * @return the options to use with the put request
- */
- private PutObjectOptions putOptionsForPath(Path path) {
- return keepDirectoryMarkers(path)
- ? PutObjectOptions.keepingDirs()
- : PutObjectOptions.deletingDirs();
- }
-
- /**
- * Delete mock parent directories which are no longer needed.
- * Retry policy: retrying; exceptions swallowed.
- * @param path path
- *
- */
- @Retries.RetryExceptionsSwallowed
- private void deleteUnnecessaryFakeDirectories(Path path) {
- List keysToRemove = new ArrayList<>();
- while (!path.isRoot()) {
- String key = pathToKey(path);
- key = (key.endsWith("/")) ? key : (key + "/");
- LOG.trace("To delete unnecessary fake directory {} for {}", key, path);
- keysToRemove.add(ObjectIdentifier.builder().key(key).build());
- path = path.getParent();
- }
- try {
- removeKeys(keysToRemove, true);
- } catch (AwsServiceException | IOException e) {
- instrumentation.errorIgnored();
- if (LOG.isDebugEnabled()) {
- StringBuilder sb = new StringBuilder();
- for (ObjectIdentifier objectIdentifier : keysToRemove) {
- sb.append(objectIdentifier.key()).append(",");
- }
- LOG.debug("While deleting keys {} ", sb.toString(), e);
- }
- }
}
/**
@@ -4810,8 +4675,6 @@ private void createFakeDirectory(final String objectName,
/**
* Used to create an empty file that represents an empty directory.
- * The policy for deleting parent dirs depends on the path, dir
- * status and the putOptions value.
* Retry policy: retrying; translated.
* @param objectName object to create
* @param putOptions put object options
@@ -4842,14 +4705,6 @@ public long getDefaultBlockSize() {
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(
@@ -4879,7 +4734,6 @@ public String toString() {
sb.append(", blockFactory=").append(blockFactory);
}
sb.append(", auditManager=").append(auditManager);
- sb.append(", authoritativePath=").append(allowAuthoritativePaths);
sb.append(", useListV1=").append(useListV1);
if (committerIntegration != null) {
sb.append(", magicCommitter=").append(isMagicCommitEnabled());
@@ -4889,7 +4743,6 @@ public String toString() {
sb.append(", credentials=").append(credentials);
sb.append(", delegation tokens=")
.append(delegationTokens.map(Objects::toString).orElse("disabled"));
- sb.append(", ").append(directoryPolicy);
// if logging at debug, toString returns the entire IOStatistics set.
if (getInstrumentation() != null) {
sb.append(", instrumentation {")
@@ -5237,7 +5090,7 @@ public RemoteIterator listFilesAndEmptyDirectories(
final Path path = qualify(f);
return trackDurationAndSpan(INVOCATION_LIST_FILES, path, () ->
innerListFiles(path, recursive,
- Listing.ACCEPT_ALL_BUT_S3N,
+ Listing.ACCEPT_ALL_OBJECTS,
null));
}
@@ -5549,20 +5402,13 @@ public boolean hasPathCapability(final Path path, final String capability)
case CommonPathCapabilities.VIRTUAL_BLOCK_LOCATIONS:
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:
- return getDirectoryMarkerPolicy().hasPathCapability(path, cap);
-
- // keep for a magic path or if the policy retains it
case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP:
- return keepDirectoryMarkers(path);
- // delete is the opposite of keep
+ return true;
+ // never true
+ case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_AUTHORITATIVE:
case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE:
- return !keepDirectoryMarkers(path);
+ return false;
case STORE_CAPABILITY_DIRECTORY_MARKER_MULTIPART_UPLOAD_ENABLED:
return isMultipartUploadEnabled();
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index a1da63329a9ea..63ad42dab7adb 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -724,7 +724,6 @@ public static boolean setIfDefined(Configuration config, String key,
*/
public static S3xLoginHelper.Login getAWSAccessKeys(URI name,
Configuration conf) throws IOException {
- S3xLoginHelper.rejectSecretsInURIs(name);
Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
conf, S3AFileSystem.class);
String bucket = name != null ? name.getHost() : "";
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
index a7a87bdfcfb29..969c1023d7347 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
@@ -288,9 +288,7 @@ public String initiateMultiPartUpload(
/**
* Finalize a multipart PUT operation.
- * This completes the upload, and, if that works, calls
- * {@link WriteOperationHelperCallbacks#finishedWrite(String, long, PutObjectOptions)}
- * to update the filesystem.
+ * This completes the upload.
* Retry policy: retrying, translated.
* @param destKey destination of the commit
* @param uploadId multipart operation Id
@@ -324,8 +322,6 @@ private CompleteMultipartUploadResponse finalizeMultipartUpload(
destKey, uploadId, partETags);
return writeOperationHelperCallbacks.completeMultipartUpload(requestBuilder.build());
});
- writeOperationHelperCallbacks.finishedWrite(destKey, length,
- putOptions);
return uploadResult;
}
}
@@ -547,8 +543,6 @@ public void revertCommit(String destKey) throws IOException {
/**
* This completes a multipart upload to the destination key via
* {@code finalizeMultipartUpload()}.
- * Markers are never deleted on commit; this avoids having to
- * issue many duplicate deletions.
* Retry policy: retrying, translated.
* Retries increment the {@code errorCount} counter.
* @param destKey destination
@@ -574,7 +568,7 @@ public CompleteMultipartUploadResponse commitUpload(
uploadId,
partETags,
length,
- PutObjectOptions.keepingDirs(),
+ PutObjectOptions.defaultOptions(),
Invoker.NO_OP);
}
@@ -670,21 +664,6 @@ UploadPartResponse uploadPart(
RequestBody body,
DurationTrackerFactory durationTrackerFactory)
throws AwsServiceException, UncheckedIOException;
-
- /**
- * Perform post-write actions.
- *
- * This operation MUST be called after any PUT/multipart PUT completes
- * successfully.
- * @param key key written to
- * @param length total length of file written
- * @param putOptions put object options
- */
- @Retries.RetryExceptionsSwallowed
- void finishedWrite(
- String key,
- long length,
- PutObjectOptions putOptions);
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/CredentialProviderListFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/CredentialProviderListFactory.java
index 941ce741151d5..c0daa6a3aecf9 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/CredentialProviderListFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/CredentialProviderListFactory.java
@@ -47,7 +47,6 @@
import org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider;
import org.apache.hadoop.fs.s3a.adapter.AwsV1BindingSupport;
import org.apache.hadoop.fs.s3a.impl.InstantiationIOException;
-import org.apache.hadoop.fs.s3native.S3xLoginHelper;
import org.apache.hadoop.fs.store.LogExactlyOnce;
import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
@@ -139,15 +138,12 @@ private CredentialProviderListFactory() {
public static AWSCredentialProviderList createAWSCredentialProviderList(
@Nullable URI binding,
Configuration conf) throws IOException {
- // this will reject any user:secret entries in the URI
- S3xLoginHelper.rejectSecretsInURIs(binding);
AWSCredentialProviderList credentials =
buildAWSProviderList(binding,
conf,
AWS_CREDENTIALS_PROVIDER,
STANDARD_AWS_PROVIDERS,
new HashSet<>());
- // make sure the logging message strips out any auth details
LOG.debug("For URI {}, using credentials {}",
binding, credentials);
return credentials;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java
index 14bd4cc2f7da1..b9fcc1f06f2d8 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java
@@ -547,7 +547,7 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile,
statistics.commitCreated();
uploadId = writeOperations.initiateMultiPartUpload(destKey,
- PutObjectOptions.keepingDirs());
+ PutObjectOptions.defaultOptions());
long length = localFile.length();
SinglePendingCommit commitData = new SinglePendingCommit();
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/S3MagicCommitTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/S3MagicCommitTracker.java
index 1f6c9123bae62..ecc3496ce8f3a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/S3MagicCommitTracker.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/S3MagicCommitTracker.java
@@ -79,7 +79,7 @@ public boolean aboutToComplete(String uploadId,
PutObjectRequest originalDestPut = getWriter().createPutObjectRequest(
getOriginalDestKey(),
0,
- new PutObjectOptions(true, null, headers));
+ new PutObjectOptions(null, headers));
upload(originalDestPut, EMPTY);
// build the commit summary
@@ -117,7 +117,7 @@ public boolean aboutToComplete(String uploadId,
@Retries.RetryTranslated
private void upload(PutObjectRequest request, byte[] bytes) throws IOException {
trackDurationOfInvocation(getTrackerStatistics(), COMMITTER_MAGIC_MARKER_PUT.getSymbol(),
- () -> getWriter().putObject(request, PutObjectOptions.keepingDirs(),
+ () -> getWriter().putObject(request, PutObjectOptions.defaultOptions(),
new S3ADataBlocks.BlockUploadData(bytes, null), null));
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java
index 9390c699335b7..d4e0b5db38850 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java
@@ -19,7 +19,6 @@
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;
@@ -267,31 +266,6 @@ public String toString() {
'}';
}
- /**
- * 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 removeAllowedMarkers(DirectoryPolicy policy) {
- List removed = new ArrayList<>();
- Iterator> entries =
- surplusMarkers.entrySet().iterator();
- while (entries.hasNext()) {
- Map.Entry 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.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java
deleted file mode 100644
index 6ba74c7e971d2..0000000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * 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.
- *
- * This is the classic S3A policy,
- */
- Delete(DIRECTORY_MARKER_POLICY_DELETE),
-
- /**
- * Keep markers.
- *
- * This is Not backwards compatible.
- */
- Keep(DIRECTORY_MARKER_POLICY_KEEP),
-
- /**
- * Keep markers in authoritative paths only.
- *
- * This is Not backwards compatible 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;
- }
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java
deleted file mode 100644
index 7d501944308bd..0000000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * 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 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 authoritativeness;
-
- /**
- * Constructor.
- * @param markerPolicy marker policy
- * @param authoritativeness function for authoritativeness
- */
- public DirectoryPolicyImpl(final MarkerPolicy markerPolicy,
- final Predicate 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 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.debug("Directory markers will be kept");
- policy = KEEP;
- break;
- case DIRECTORY_MARKER_POLICY_AUTHORITATIVE:
- LOG.debug("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 availablePolicies() {
- return AVAILABLE_POLICIES;
- }
-
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java
index a027cabffd46d..a2d79271a8a67 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java
@@ -145,8 +145,7 @@ public Boolean execute() throws IOException {
// If so, we declare success without looking any further
if (isMagicPath) {
// Create the marker file immediately,
- // and don't delete markers
- callbacks.createFakeDirectory(dir, true);
+ callbacks.createFakeDirectory(dir);
return true;
}
@@ -159,9 +158,8 @@ public Boolean execute() throws IOException {
// if we get here there is no directory at the destination.
// so create one.
- // Create the marker file, delete the parent entries
- // if the filesystem isn't configured to retain them
- callbacks.createFakeDirectory(dir, false);
+ // Create the directory marker file
+ callbacks.createFakeDirectory(dir);
return true;
}
@@ -264,15 +262,10 @@ S3AFileStatus probePathStatus(Path path,
/**
* Create a fake directory, always ending in "/".
* Retry policy: retrying; translated.
- * the keepMarkers flag controls whether or not markers
- * are automatically kept (this is set when creating
- * directories under a magic path, always)
* @param dir dir to create
- * @param keepMarkers always keep markers
- *
* @throws IOException IO failure
*/
@Retries.RetryTranslated
- void createFakeDirectory(Path dir, boolean keepMarkers) throws IOException;
+ void createFakeDirectory(Path dir) throws IOException;
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PutObjectOptions.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PutObjectOptions.java
index e14285a1ca8b1..1ca502c44cde6 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PutObjectOptions.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PutObjectOptions.java
@@ -26,11 +26,6 @@
*/
public final class PutObjectOptions {
- /**
- * Can the PUT operation skip marker deletion?
- */
- private final boolean keepMarkers;
-
/**
* Storage class, if not null.
*/
@@ -43,27 +38,16 @@ public final class PutObjectOptions {
/**
* Constructor.
- * @param keepMarkers Can the PUT operation skip marker deletion?
* @param storageClass Storage class, if not null.
* @param headers Headers; may be null.
*/
public PutObjectOptions(
- final boolean keepMarkers,
@Nullable final String storageClass,
@Nullable final Map headers) {
- this.keepMarkers = keepMarkers;
this.storageClass = storageClass;
this.headers = headers;
}
- /**
- * Get the marker retention flag.
- * @return true if markers are to be retained.
- */
- public boolean isKeepMarkers() {
- return keepMarkers;
- }
-
/**
* Headers for the put/post request.
* @return headers or null.
@@ -75,30 +59,22 @@ public Map getHeaders() {
@Override
public String toString() {
return "PutObjectOptions{" +
- "keepMarkers=" + keepMarkers +
", storageClass='" + storageClass + '\'' +
'}';
}
- private static final PutObjectOptions KEEP_DIRS = new PutObjectOptions(true,
- null, null);
- private static final PutObjectOptions DELETE_DIRS = new PutObjectOptions(false,
- null, null);
-
/**
- * Get the options to keep directories.
- * @return an instance which keeps dirs
+ * Empty options.
*/
- public static PutObjectOptions keepingDirs() {
- return KEEP_DIRS;
- }
+ private static final PutObjectOptions EMPTY_OPTIONS = new PutObjectOptions(
+ null, null);
/**
- * Get the options to delete directory markers.
- * @return an instance which deletes dirs
+ * Get the default options.
+ * @return an instance with no storage class or headers.
*/
- public static PutObjectOptions deletingDirs() {
- return DELETE_DIRS;
+ public static PutObjectOptions defaultOptions() {
+ return EMPTY_OPTIONS;
}
}
\ No newline at end of file
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java
index e00319c3dc581..32d7faad856e5 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java
@@ -128,7 +128,7 @@ public CompletableFuture startUpload(
return context.submit(new CompletableFuture<>(),
trackDurationOfCallable(statistics, OBJECT_MULTIPART_UPLOAD_INITIATED.getSymbol(), () -> {
String uploadId = writeOperations.initiateMultiPartUpload(key,
- PutObjectOptions.keepingDirs());
+ PutObjectOptions.defaultOptions());
statistics.uploadStarted();
return BBUploadHandle.from(ByteBuffer.wrap(
uploadId.getBytes(StandardCharsets.UTF_8)));
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
index 5e3e5dcab77d4..83d6d92c95ea3 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
@@ -121,12 +121,6 @@ public static boolean checkNoS3Guard(URI fsURI, Configuration conf) throws PathI
return true;
}
- public static Collection getAuthoritativePaths(S3AFileSystem fs) {
- return getAuthoritativePaths(
- fs.getUri(),
- fs.getConf(),
- p -> fs.maybeAddTrailingSlash(fs.qualify(p).toString()));
- }
/**
* Get the authoritative paths of a filesystem.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
index 57fd879c38cf6..4e691ff10009f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
@@ -25,15 +25,13 @@
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Arrays;
-import java.util.Collection;
import java.util.Date;
import java.util.List;
+import java.util.Locale;
import java.util.Scanner;
import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
import software.amazon.awssdk.services.s3.model.MultipartUpload;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -55,8 +53,6 @@
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.SelectConstants;
import org.apache.hadoop.fs.s3a.tools.BucketTool;
import org.apache.hadoop.fs.s3a.tools.MarkerTool;
@@ -389,8 +385,7 @@ public static class BucketInfo extends S3GuardTool {
@VisibleForTesting
public static final String IS_MARKER_AWARE =
- "\tThe S3A connector is compatible with buckets where"
- + " directory markers are not deleted";
+ "\tThe S3A connector does not delete markers";
public static final String CAPABILITY_FORMAT = "\t%s %s%n";
@@ -446,16 +441,6 @@ public int run(String[] args, PrintStream out)
fs.listXAttrs(new Path("/"));
}
- // print any auth paths for directory marker info
- final Collection authoritativePaths
- = S3Guard.getAuthoritativePaths(fs);
- if (!authoritativePaths.isEmpty()) {
- println(out, "Qualified Authoritative Paths:");
- for (String path : authoritativePaths) {
- println(out, "\t%s", path);
- }
- println(out, "");
- }
println(out, "%nS3A Client");
printOption(out, "\tSigning Algorithm", SIGNING_ALGORITHM, "(unset)");
String endpoint = conf.getTrimmed(ENDPOINT, "");
@@ -556,7 +541,7 @@ public int run(String[] args, PrintStream out)
}
// directory markers
- processMarkerOption(out, fs,
+ processMarkerOption(out,
getCommandFormat().getOptValue(MARKERS_FLAG));
// and check for capabilities
@@ -583,43 +568,29 @@ public int run(String[] args, PrintStream out)
/**
* Validate the marker options.
* @param out output stream
- * @param fs filesystem
* @param marker desired marker option -may be null.
*/
private void processMarkerOption(final PrintStream out,
- final S3AFileSystem fs,
final String marker) {
- println(out, "%nDirectory Markers");
- DirectoryPolicy markerPolicy = fs.getDirectoryMarkerPolicy();
- String desc = markerPolicy.describe();
- println(out, "\tThe directory marker policy is \"%s\"", desc);
-
- String pols = DirectoryPolicyImpl.availablePolicies()
- .stream()
- .map(DirectoryPolicy.MarkerPolicy::getOptionName)
- .collect(Collectors.joining(", "));
- println(out, "\tAvailable Policies: %s", pols);
- printOption(out, "\tAuthoritative paths",
- AUTHORITATIVE_PATH, "");
- DirectoryPolicy.MarkerPolicy mp = markerPolicy.getMarkerPolicy();
+ println(out, "%nThis version of Hadoop always retains directory markers");
+
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);
- } 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);
- }
- }
+ : marker.trim().toLowerCase(Locale.ROOT);
+ switch(desiredMarker) {
+ case "":
+ case DIRECTORY_MARKER_POLICY_KEEP:
+ break;
+
+ case MARKERS_AWARE:
+ // simple awareness test -provides a way to validate compatibility
+ // on the command line
+ println(out, IS_MARKER_AWARE);
+ break;
+
+ default:
+ throw badState("Unsupported Marker Policy \"%s\"", desiredMarker);
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java
index b2ff63b39865b..0c396f2f5c9ce 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java
@@ -53,8 +53,6 @@
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.DirectoryPolicyImpl;
import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteException;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool;
@@ -62,8 +60,6 @@
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;
@@ -175,7 +171,6 @@ public final class MarkerTool extends S3GuardTool {
+ " [-" + OPT_MAX + " ]"
+ " [-" + OPT_OUT + " ]"
+ " [-" + OPT_LIMIT + " ]"
- + " [-" + OPT_NONAUTH + "]"
+ " [-" + VERBOSE + "]"
+ " \n"
@@ -207,8 +202,7 @@ public MarkerTool(final Configuration conf) {
super(conf,
OPT_AUDIT,
OPT_CLEAN,
- VERBOSE,
- OPT_NONAUTH);
+ VERBOSE);
CommandFormat format = getCommandFormat();
format.addOptionWithValue(OPT_MIN);
format.addOptionWithValue(OPT_MAX);
@@ -275,7 +269,6 @@ public int run(final String[] args, final PrintStream stream)
path = new Path(path, "/");
}
FileSystem fs = path.getFileSystem(getConf());
- boolean nonAuth = command.getOpt(OPT_NONAUTH);
ScanResult result;
try {
result = execute(
@@ -286,7 +279,6 @@ public int run(final String[] args, final PrintStream stream)
.withMinMarkerCount(expectedMin)
.withMaxMarkerCount(expectedMax)
.withLimit(limit)
- .withNonAuth(nonAuth)
.build());
} catch (UnknownStoreException ex) {
// bucket doesn't exist.
@@ -356,21 +348,6 @@ ScanResult execute(final ScanArgs scanArgs)
// extract the callbacks needed for the rest of the work
storeContext = fs.createStoreContext();
- // 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 path = scanArgs.getPath();
Path target = path.makeQualified(fs.getUri(), new Path("/"));
@@ -389,26 +366,8 @@ ScanResult execute(final ScanArgs scanArgs)
}
// the default filter policy is that all entries should be deleted
- DirectoryPolicy filterPolicy;
- if (scanArgs.isNonAuth()) {
- filterPolicy = new DirectoryPolicyImpl(
- DirectoryPolicy.MarkerPolicy.Authoritative,
- fs::allowAuthoritative);
- } else {
- filterPolicy = null;
- }
int minMarkerCount = scanArgs.getMinMarkerCount();
int maxMarkerCount = scanArgs.getMaxMarkerCount();
- if (minMarkerCount > maxMarkerCount) {
- // swap min and max if they are wrong.
- // this is to ensure any test scripts written to work around
- // HADOOP-17332 and min/max swapping continue to work.
- println(out, "Swapping -min (%d) and -max (%d) values",
- minMarkerCount, maxMarkerCount);
- int m = minMarkerCount;
- minMarkerCount = maxMarkerCount;
- maxMarkerCount = m;
- }
// extract the callbacks needed for the rest of the work
operations = fs.createMarkerToolOperations(
target.toString());
@@ -416,8 +375,8 @@ ScanResult execute(final ScanArgs scanArgs)
scanArgs.isDoPurge(),
minMarkerCount,
maxMarkerCount,
- scanArgs.getLimit(),
- filterPolicy);
+ scanArgs.getLimit()
+ );
}
/**
@@ -520,7 +479,6 @@ public int finish() throws ExitUtil.ExitException {
* @param minMarkerCount min marker count (ignored on purge)
* @param maxMarkerCount max marker count (ignored on purge)
* @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
@@ -531,8 +489,7 @@ private ScanResult scan(
final boolean doPurge,
final int minMarkerCount,
final int maxMarkerCount,
- final int limit,
- final DirectoryPolicy filterPolicy)
+ final int limit)
throws IOException, ExitUtil.ExitException {
// safety check: min and max are correctly ordered at this point.
@@ -598,21 +555,6 @@ private ScanResult scan(
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 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
- markerCount = surplusMarkers.size();
- result.filteredMarkerCount = markerCount;
- }
if (markerCount < minMarkerCount || markerCount > maxMarkerCount) {
// failure
return failScan(result, EXIT_NOT_ACCEPTABLE,
@@ -902,9 +844,6 @@ public static final class ScanArgs {
/** Limit of files to scan; 0 for 'unlimited'. */
private final int limit;
- /** Consider only markers in nonauth paths as errors. */
- private final boolean nonAuth;
-
/**
* @param sourceFS source FS; must be or wrap an S3A FS.
* @param path path to scan.
@@ -912,22 +851,19 @@ public static final class ScanArgs {
* @param minMarkerCount min marker count (ignored on purge)
* @param maxMarkerCount max marker count (ignored on purge)
* @param limit limit of files to scan; 0 for 'unlimited'
- * @param nonAuth consider only markers in nonauth paths as errors
*/
private ScanArgs(final FileSystem sourceFS,
final Path path,
final boolean doPurge,
final int minMarkerCount,
final int maxMarkerCount,
- final int limit,
- final boolean nonAuth) {
+ final int limit) {
this.sourceFS = sourceFS;
this.path = path;
this.doPurge = doPurge;
this.minMarkerCount = minMarkerCount;
this.maxMarkerCount = maxMarkerCount;
this.limit = limit;
- this.nonAuth = nonAuth;
}
FileSystem getSourceFS() {
@@ -954,9 +890,6 @@ int getLimit() {
return limit;
}
- boolean isNonAuth() {
- return nonAuth;
- }
}
/**
@@ -982,9 +915,6 @@ public static final class ScanArgsBuilder {
/** Limit of files to scan; 0 for 'unlimited'. */
private int limit = UNLIMITED_LISTING;
- /** Consider only markers in nonauth paths as errors. */
- private boolean nonAuth = false;
-
/**
* Source FS; must be or wrap an S3A FS.
* @param source Source FileSystem
@@ -1045,16 +975,6 @@ public ScanArgsBuilder withLimit(final int l) {
return this;
}
- /**
- * Consider only markers in non-authoritative paths as errors.
- * @param b True if tool should only consider markers in non-authoritative paths
- * @return builder class for method chaining
- */
- public ScanArgsBuilder withNonAuth(final boolean b) {
- this.nonAuth = b;
- return this;
- }
-
/**
* Build the actual argument instance.
* @return the arguments to pass in
@@ -1065,8 +985,7 @@ public ScanArgs build() {
doPurge,
minMarkerCount,
maxMarkerCount,
- limit,
- nonAuth);
+ limit);
}
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
deleted file mode 100644
index 5a7129f7d5c96..0000000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * 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.s3native;
-
-import java.io.IOException;
-import java.net.URI;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.util.Progressable;
-
-/**
- * This is a stub filesystem purely present to fail meaningfully when
- * someone who explicitly declares
- * {@code fs.s3n.impl=org.apache.hadoop.fs.s3native.NativeS3FileSystem}
- * and then tries to create a filesystem off an s3n:// URL.
- *
- * The {@link #initialize(URI, Configuration)} method will throw
- * an IOException informing the user of their need to migrate.
- * @deprecated Replaced by the S3A client.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public final class NativeS3FileSystem extends FileSystem {
-
- public static final Logger LOG =
- LoggerFactory.getLogger(NativeS3FileSystem.class);
-
- /**
- * Message in thrown exceptions: {@value}.
- */
- private static final String UNSUPPORTED =
- "The s3n:// client to Amazon S3 is no longer available:"
- + " please migrate to the s3a:// client";
-
- public NativeS3FileSystem() {
- }
-
- /**
- * Return the protocol scheme for the FileSystem.
- *
- * @return s3n
- */
- @Override
- public String getScheme() {
- return "s3n";
- }
-
- /**
- * Always fail to initialize.
- * @throws IOException always.
- */
- @Override
- public void initialize(URI uri, Configuration conf) throws IOException {
- super.initialize(uri, conf);
- throw new IOException(UNSUPPORTED);
- }
-
- @Override
- public FileStatus getFileStatus(Path f) throws IOException {
- throw new UnsupportedOperationException(UNSUPPORTED);
- }
-
- @Override
- public URI getUri() {
- throw new UnsupportedOperationException(UNSUPPORTED);
- }
-
- @Override
- public FSDataInputStream open(Path f, int bufferSize) throws IOException {
- throw new UnsupportedOperationException(UNSUPPORTED);
- }
-
- @Override
- public FSDataOutputStream create(Path f,
- FsPermission permission,
- boolean overwrite,
- int bufferSize,
- short replication,
- long blockSize,
- Progressable progress) throws IOException {
- throw new UnsupportedOperationException(UNSUPPORTED);
- }
-
- @Override
- public FSDataOutputStream append(Path f,
- int bufferSize,
- Progressable progress) throws IOException {
- throw new UnsupportedOperationException(UNSUPPORTED);
- }
-
- @Override
- public boolean rename(Path src, Path dst) throws IOException {
- throw new UnsupportedOperationException(UNSUPPORTED);
- }
-
- @Override
- public boolean delete(Path f, boolean recursive) throws IOException {
- throw new UnsupportedOperationException(UNSUPPORTED);
- }
-
- @Override
- public FileStatus[] listStatus(Path f)
- throws IOException {
- throw new UnsupportedOperationException(UNSUPPORTED);
- }
-
- @Override
- public void setWorkingDirectory(Path new_dir) {
- throw new UnsupportedOperationException(UNSUPPORTED);
- }
-
- @Override
- public Path getWorkingDirectory() {
- throw new UnsupportedOperationException(UNSUPPORTED);
- }
-
- @Override
- public boolean mkdirs(Path f, FsPermission permission) throws IOException {
- throw new UnsupportedOperationException(UNSUPPORTED);
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
index b1a8c96880237..f10f28d9f7b01 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
@@ -22,11 +22,6 @@
import java.net.URISyntaxException;
import java.util.Objects;
-import org.apache.hadoop.classification.VisibleForTesting;
-import org.apache.hadoop.util.Preconditions;
-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;
@@ -41,9 +36,8 @@
* It is in S3N so that it can be used across all S3 filesystems.
*
* The core function of this class was the extraction and decoding of user:secret
- * information from filesystems URIs. As this is no longer supported,
- * its role has been reduced to checking for secrets in the URI and rejecting
- * them where found.
+ * information from filesystems URIs.
+ * All that is left now is some URI canonicalization and checking.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
@@ -52,87 +46,20 @@ public final class S3xLoginHelper {
private S3xLoginHelper() {
}
- public static final String LOGIN_WARNING =
- "The Filesystem URI contains login details."
- +" This authentication mechanism is no longer supported.";
-
/**
* Build the filesystem URI.
* @param uri filesystem uri
* @return the URI to use as the basis for FS operation and qualifying paths.
- * @throws IllegalArgumentException if the URI is in some way invalid.
+ * @throws NullPointerException if the URI has null parts.
*/
public static URI buildFSURI(URI uri) {
// look for login secrets and fail if they are present.
- rejectSecretsInURIs(uri);
Objects.requireNonNull(uri, "null uri");
Objects.requireNonNull(uri.getScheme(), "null uri.getScheme()");
- if (uri.getHost() == null && uri.getAuthority() != null) {
- Objects.requireNonNull(uri.getHost(), "null uri host.");
- }
Objects.requireNonNull(uri.getHost(), "null uri host.");
return URI.create(uri.getScheme() + "://" + uri.getHost());
}
- /**
- * Create a stripped down string value for error messages.
- * @param pathUri URI
- * @return a shortened schema://host/path value
- */
- public static String toString(URI pathUri) {
- return pathUri != null
- ? String.format("%s://%s/%s",
- pathUri.getScheme(), pathUri.getHost(), pathUri.getPath())
- : "(null URI)";
- }
-
- /**
- * Extract the login details from a URI, raising an exception if
- * the URI contains them.
- * @param name URI of the filesystem, can be null
- * @throws IllegalArgumentException if there is a secret in the URI.
- */
- public static void rejectSecretsInURIs(URI name) {
- Login login = extractLoginDetails(name);
- Preconditions.checkArgument(!login.hasLogin(), LOGIN_WARNING);
- }
-
- /**
- * Extract the login details from a URI.
- * @param name URI of the filesystem, may be null
- * @return a login tuple, possibly empty.
- */
- @VisibleForTesting
- static Login extractLoginDetails(URI name) {
- if (name == null) {
- return Login.EMPTY;
- }
-
- String authority = name.getAuthority();
- if (authority == null) {
- return Login.EMPTY;
- }
- int loginIndex = authority.indexOf('@');
- if (loginIndex < 0) {
- // no login
- return Login.EMPTY;
- }
- String login = authority.substring(0, loginIndex);
- int loginSplit = login.indexOf(':');
- if (loginSplit > 0) {
- String user = login.substring(0, loginSplit);
- String encodedPassword = login.substring(loginSplit + 1);
- return new Login(user, encodedPassword.isEmpty()? "": "password removed");
- } else if (loginSplit == 0) {
- // there is no user, just a password. In this case, there's no login
- return Login.EMPTY;
- } else {
- // loginSplit < 0: there is no ":".
- // return a login with a null password
- return new Login(login, "");
- }
- }
-
/**
* Canonicalize the given URI.
*
@@ -164,8 +91,7 @@ public static URI canonicalizeUri(URI uri, int defaultPort) {
/**
* Check the path, ignoring authentication details.
- * See {@link FileSystem#checkPath(Path)} for the operation of this.
- *
+ * See {@code FileSystem.checkPath(Path)} for the operation of this.
* Essentially
*
*
The URI is canonicalized.
@@ -221,8 +147,7 @@ public static void checkPath(Configuration conf,
}
// make sure the exception strips out any auth details
throw new IllegalArgumentException(
- "Wrong FS " + S3xLoginHelper.toString(pathUri)
- + " -expected " + fsUri);
+ "Wrong FS " + pathUri + " -expected " + fsUri);
}
/**
@@ -232,8 +157,6 @@ public static class Login {
private final String user;
private final String password;
- public static final Login EMPTY = new Login();
-
/**
* Create an instance with no login details.
* Calls to {@link #hasLogin()} return false.
diff --git a/hadoop-tools/hadoop-aws/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem b/hadoop-tools/hadoop-aws/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
deleted file mode 100644
index 1938bdefa4107..0000000000000
--- a/hadoop-tools/hadoop-aws/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
+++ /dev/null
@@ -1,16 +0,0 @@
-# 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.
-
-org.apache.hadoop.fs.s3native.NativeS3FileSystem
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md
index ba1bc4b362c47..c226000d99a3a 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md
@@ -295,13 +295,10 @@ views or operations.
Particular troublespots are "directory markers" and
failures of non-atomic operations, particularly `rename()` and `delete()`.
-A directory marker such as `/users/` will not be deleted if the user `alice`
-creates a directory `/users/alice` *and* she only has access to `/users/alice`.
-
-When a path or directory is deleted, the parent directory may not exist afterwards.
-In the example above, if `alice` deletes `/users/alice` and there are no
-other entries under `/users/alice`, then the directory marker `/users/` cannot
-be created. The directory `/users` will not exist in listings,
+If `alice` deletes `/users/alice` and there are no
+other entries under `/users/alice`, or a directory marker `/users` then that
+directory marker cannot be created.
+The directory `/users` will not exist in listings,
`getFileStatus("/users")` or similar.
Rename will fail if it cannot delete the items it has just copied, that is
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md
index 4e31edfbbb757..1a2a0aa852f1b 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md
@@ -12,112 +12,104 @@
limitations under the License. See accompanying LICENSE file.
-->
-# Controlling the S3A Directory Marker Behavior
+# S3A Directory Marker Behavior
-This document discusses a performance feature of the S3A
-connector: directory markers are not deleted unless the
-client is explicitly configured to do so.
-
-## Critical: this is not backwards compatible!
+This document discusses directory markers and a change to the S3A
+connector: surplus directory markers are no longer deleted.
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.
-The default policy in this release of hadoop is "keep",
-which _is not backwards compatible_ with hadoop versions
+This release always retains markers
+and _is potentially not backwards compatible_ with hadoop versions
released before 2021.
The compatibility table of older releases is as follows:
-| Branch | Compatible Since | Supported | Released |
-|------------|------------------|-----------|----------|
-| Hadoop 2.x | 2.10.2 | Read-only | 05/2022 |
-| Hadoop 3.0 | n/a | WONTFIX | |
-| Hadoop 3.1 | n/a | WONTFIX | |
-| Hadoop 3.2 | 3.2.2 | Read-only | 01/2022 |
-| Hadoop 3.3 | 3.3.1 | Done | 01/2021 |
+| Branch | Compatible Since | Support | Released |
+|------------|------------------|---------------------------------------|----------|
+| Hadoop 2.x | 2.10.2 | Partial | 05/2022 |
+| Hadoop 3.0 | n/a | WONTFIX | |
+| Hadoop 3.1 | n/a | WONTFIX | |
+| Hadoop 3.2 | 3.2.2 | Partial | 01/2022 |
+| Hadoop 3.3 | 3.3.1 | Full: deletion is enabled | 01/2021 |
+| Hadoop 3.4 | 3.4.0 | Full: deletion is disabled | 03/2024 |
+| Hadoop 3.5 | 3.5.0 | Full: markers will always be retained | |
+*Full*
-*WONTFIX*
+These releases are full marker-aware and will (possibly only optionally)
+not delete them on file/dir creation.
-The Hadoop 3.0 and 3.1 lines will have no further releases, so will
-not be upgraded.
-The compatibility patch "HADOOP-17199. S3A Directory Marker HADOOP-13230 backport"
-is present in both source code branches, for anyone wishing to make a private
-release.
+*Partial*
-*Read-only*
+These branches have partial compatibility, and are safe to use with hadoop versions
+that do not delete markers.
-These branches have read-only compatibility.
-
-* They may list directories with directory markers, and correctly identify when
+* They can list directories with surplus directory markers, and correctly identify when
such directories have child entries.
-* They will open files under directories with such markers.
-
-## How to re-enable backwards compatibility
+* They can open files under directories with such markers.
+* They will always delete parent directory markers when creating their own files and directories.
-The option `fs.s3a.directory.marker.retention` can be changed to "delete" to re-enable
-the original policy.
+All these branches are no longer supported by Apache for bugs and security fixes
+-users should upgrade to more recent versions for those reasons alone.
-```xml
-
- fs.s3a.directory.marker.retention
- delete
-
-```
-## Verifying read compatibility.
+*WONTFIX*
-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.
+These stores may misinterpret a surplus directory marker for an empty directory.
+This does not happen on listing, but it may be misinterpreted on rename operations.
-External Hadoop-based applications should also be assumed to be incompatible
-unless otherwise stated/known.
+The compatibility patch "HADOOP-17199. S3A Directory Marker HADOOP-13230 backport"
+is present in both the Hadoop 3.0.x and 3.1.x source branches, for anyone wishing to make a private
+release.
+However, there will be no further Apache releases of the obsolete branches.
-It is only safe change the directory marker policy if the following
- conditions are met:
+Everyone using these branches should upgrade to a supported version.
-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.
+## History
+### Hadoop 3.3.1 Directory marker retention is optional
-### Applications backing up data.
+[HADOOP-13230](https://issues.apache.org/jira/browse/HADOOP-13230)
+ _S3A to optionally retain directory markers_
-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.
+### Hadoop 3.4.0: markers are not deleted by default
-### How will incompatible applications/versions fail?
+[HADOOP-18752](https://issues.apache.org/jira/browse/HADOOP-18752)
+_Change fs.s3a.directory.marker.retention to "keep"_ changed the default
+policy.
-Applications using an incompatible version of the S3A connector will mistake
-directories containing data for empty directories. This means that:
+Marker deletion can still be enabled.
-* 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.
+Since this release there have been no reports of incompatibilities
+surfacing "in the wild". That is: out of date hadoop versions are not
+being used to work into the same parts of S3 buckets as modern releases.
-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.
+### Hadoop 3.5: markers are never deleted
-### If an application has updated a directory tree incompatibly-- what can be done?
+[HADOOP-19278](https://issues.apache.org/jira/browse/HADOOP-19278)
+_S3A: remove option to delete directory markers_
-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.
+Surplus directory markers are neither checked for nor deleted.
-Now that this is all clear, let's explain the problem.
+Removing the option to delete markers simplifies the code and significantly improves testing:
+* There is no need to parameterize many tests based on the marker policy.
+* Tests which make assertions about the number of http requests which take places no longer have to
+ contain separate assertions for the keeping/deleting options.
+Notes
+* During the directory tree copy which takes place in a rename, surplus directory
+markers are not copied. They are, after all, surplus.
+* The `hadoop s3guard markers` command (see below) can purge directory markers from a bucket or path.
## Background: Directory Markers: what and why?
-Amazon S3 is not a filesystem, it is an object store.
+Amazon S3 is not a Posix-like 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.
+data in S3, it tries to maintain the filesystem metaphor for applications
+written to expect it.
One key aspect of the metaphor of a file system is "directories"
@@ -157,7 +149,6 @@ 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.
@@ -172,10 +163,13 @@ in `_$folder$` was considered to be a sign that a directory existed. A call to
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
+In older versions of Hadoop, when a file was 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.
+This release does not delete directory markers.
+
And, historically, when a path is listed, if a marker to that path is found, *it
has been interpreted as an empty directory.*
@@ -186,11 +180,11 @@ It is that little detail which is the cause of the incompatibility issues.
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
+Whenever a file is created the S3A client had to delete any marker which could exist in the
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
+parent markers existing, the connector issued 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.
+`DELETE` request containing the keys `/a/` and `/a/b/` was issued.
If no markers exists, this is harmless.
When a file is deleted, a check for the parent directory continuing to exist
@@ -226,182 +220,21 @@ 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.
-## 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 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.
-
-## 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
-
-1. `delete`: a request is issued to delete any parental directory markers
-whenever a file or directory is created.
-2. `keep`: No delete request is issued.
-Any directory markers which exist are not deleted.
-This is *not* backwards compatible
-3. `authoritative`: directory markers are deleted _except for files created
-in "authoritative" directories_. This is backwards compatible _outside authoritative directories_.
-
-The setting, `fs.s3a.directory.marker.retention = delete` is compatible with
-every shipping Hadoop release; that of `keep` compatible with
-all releases since 2021.
-
-## Directory Markers and Authoritative paths
-
-
-The now-deleted S3Guard feature included the concept of "authoritative paths";
-paths where all clients were required to be using S3Guard and sharing the
-same metadata store.
-In such a setup, listing authoritative paths would skip all queries of the S3
-store -potentially being much faster.
-
-In production, authoritative paths were usually only ever for Hive managed
-tables, where access was strictly restricted to the Hive services.
-
-
-When the S3A client is configured to treat some directories as "Authoritative"
-then an S3A connector with a retention policy of `fs.s3a.directory.marker.retention` of
-`authoritative` will omit deleting markers in authoritative directories.
-
-```xml
-
- fs.s3a.bucket.hive.authoritative.path
- /tables
-
-```
-This an option to consider if not 100% confident that all
-applications interacting with a store are using an S3A client
-which is marker aware.
-
## Verifying marker policy with `s3guard bucket-info`
-The `bucket-info` command has been enhanced to support verification from the command
+Although it is now moot, 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 |
+| option | verifies | result |
+|--------------------------|------------------------------------------------------------------|--------|
+| `-markers aware` | The hadoop release is "aware" of directory markers. Always true | 0 |
+| `-markers keep` | Directory markers are kept. Always true | 0 |
+| `-markers delete` | Directory markers are deleted. Always false | 1 |
+| `-markers authoritative` | Directory markers are kept in authoritative paths. Always false | 1 |
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://noaa-isd-pds/
-Filesystem s3a://noaa-isd-pds
-
-...
-
-Directory Markers
- The directory marker policy is "keep"
- Available Policies: delete, keep, authoritative
- Authoritative paths: fs.s3a.authoritative.path=
- The S3A connector is compatible with buckets where directory markers are not deleted
-
-...
-```
-
-The same command will fail on older releases, because the `-markers` option
-is unknown
-
-```
-> hadoop s3guard bucket-info -markers aware s3a://noaa-isd-pds/
-Illegal option -markers
-Usage: hadoop bucket-info [OPTIONS] s3a://BUCKET
- provide/check information about a specific bucket
-
-Common options:
- -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 - specify an application configuration file
- -D - 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 keep s3a://noaa-isd-pds/
-Filesystem s3a://noaa-isd-pds
-
-...
-
-Directory Markers
- The directory marker policy is "keep"
- Available Policies: delete, keep, authoritative
- Authoritative paths: fs.s3a.authoritative.path=
-
-```
-
-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 delete s3a://noaa-isd-pds/
-Filesystem s3a://noaa-isd-pds
-
-S3A Client
- Signing Algorithm: fs.s3a.signing-algorithm=(unset)
- Endpoint: fs.s3a.endpoint=s3.amazonaws.com
- Encryption: fs.s3a.encryption.algorithm=none
- Input seek policy: fs.s3a.experimental.input.fadvise=normal
- Change Detection Source: fs.s3a.change.detection.source=etag
- Change Detection Mode: fs.s3a.change.detection.mode=server
-
-S3A Committers
- The "magic" committer is supported in the filesystem
- S3A Committer factory class: mapreduce.outputcommitter.factory.scheme.s3a=org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory
- S3A Committer name: fs.s3a.committer.name=magic
- Store magic committer integration: fs.s3a.committer.magic.enabled=true
-
-Security
- Delegation token support is disabled
-
-Directory Markers
- The directory marker policy is "delete"
- Available Policies: delete, keep, authoritative
- Authoritative paths: fs.s3a.authoritative.path=
-
-2021-11-22 16:03:59,175 [main] INFO util.ExitUtil (ExitUtil.java:terminate(210))
- -Exiting with status 46: 46: Bucket s3a://noaa-isd-pds: required marker polic is
- "keep" but actual policy is "delete"
-
-```
-
-
## The marker tool: `hadoop s3guard markers`
The marker tool aims to help migration by scanning/auditing directory trees
@@ -415,7 +248,6 @@ Syntax
> hadoop s3guard markers -verbose -nonauth
markers (-audit | -clean) [-min ] [-max ] [-out ] [-limit ] [-nonauth] [-verbose]
View and manipulate S3 directory markers
-
```
*Options*
@@ -446,7 +278,6 @@ All other non-zero status code also indicate errors of some form or other.
Audit the path and fail if any markers were found.
-
```
> hadoop s3guard markers -limit 8000 -audit s3a://noaa-isd-pds/
@@ -499,35 +330,6 @@ Found 5 empty directory 'leaf' markers under s3a://london/
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/
-
-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:(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 use.
-
### `markers clean`
@@ -535,9 +337,9 @@ 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
```
-bin/hadoop s3guard markers -clean -verbose s3a://stevel-london/
-The directory marker policy of s3a://stevel-london is "Keep"
-2023-06-06 17:15:52,110 [main] INFO tools.MarkerTool (DurationInfo.java:(77)) - Starting: marker scan s3a://stevel-london/
+bin/hadoop s3guard markers -clean -verbose s3a://london/
+The directory marker policy of s3a://london is "Keep"
+2023-06-06 17:15:52,110 [main] INFO tools.MarkerTool (DurationInfo.java:(77)) - Starting: marker scan s3a://london/
Directory Marker user/stevel/target/test/data/4so7pZebRx/
Directory Marker user/stevel/target/test/data/OKvfC3oxlD/
Directory Marker user/stevel/target/test/data/VSTQ1O4dMi/
@@ -550,38 +352,20 @@ maximums=((object_continue_list_request.failures.max=-1) (object_list_request.fa
means=((object_list_request.mean=(samples=1, sum=540, mean=540.0000)) (object_continue_list_request.failures.mean=(samples=0, sum=0, mean=0.0000)) (object_list_request.failures.mean=(samples=0, sum=0, mean=0.0000)) (object_continue_list_request.mean=(samples=0, sum=0, mean=0.0000)));
-2023-06-06 17:15:52,662 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - marker scan s3a://stevel-london/: duration 0:00.553s
-Listed 3 objects under s3a://stevel-london/
+2023-06-06 17:15:52,662 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - marker scan s3a://london/: duration 0:00.553s
+Listed 3 objects under s3a://london/
-No surplus directory markers were found under s3a://stevel-london/
-Found 3 empty directory 'leaf' markers under s3a://stevel-london/
- s3a://stevel-london/user/stevel/target/test/data/4so7pZebRx/
- s3a://stevel-london/user/stevel/target/test/data/OKvfC3oxlD/
- s3a://stevel-london/user/stevel/target/test/data/VSTQ1O4dMi/
+No surplus directory markers were found under s3a://london/
+Found 3 empty directory 'leaf' markers under s3a://london/
+ s3a://london/user/stevel/target/test/data/4so7pZebRx/
+ s3a://london/user/stevel/target/test/data/OKvfC3oxlD/
+ s3a://london/user/stevel/target/test/data/VSTQ1O4dMi/
These are required to indicate empty directories
0 markers to delete in 0 pages of 250 keys/page
2023-06-06 17:15:52,664 [main] INFO tools.MarkerTool (DurationInfo.java:(77)) - Starting: Deleting markers
2023-06-06 17:15:52,664 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - Deleting markers: duration 0:00.000s
-IO Statistics for s3a://stevel-london
-
-counters=((audit_request_execution=1)
-(audit_span_creation=3)
-(object_list_request=1)
-(op_get_file_status=1)
-(store_io_request=1));
-
-gauges=();
-
-minimums=((object_list_request.min=540)
-(op_get_file_status.min=2));
-
-maximums=((object_list_request.max=540)
-(op_get_file_status.max=2));
-
-means=((object_list_request.mean=(samples=1, sum=540, mean=540.0000))
-(op_get_file_status.mean=(samples=1, sum=2, mean=2.0000)));
```
The `markers -clean` command _does not_ delete markers above empty directories -only those which have
@@ -600,16 +384,14 @@ An instance of the filesystem can be probed for its directory marker retention a
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? |
-
+| Probe | Meaning | Current value |
+|-----------------------------------------------------------|--------------------------------------------------------------------------|---------------|
+| `fs.s3a.capability.directory.marker.aware` | Does the filesystem support surplus directory markers? | true |
+| `fs.s3a.capability.directory.marker.policy.delete` | Is the bucket policy "delete"? | false |
+| `fs.s3a.capability.directory.marker.policy.keep` | Is the bucket policy "keep"? | true |
+| `fs.s3a.capability.directory.marker.policy.authoritative` | Is the bucket policy "authoritative"? | false |
+| `fs.s3a.capability.directory.marker.action.delete` | If a file was created at this path, would directory markers be deleted? | false |
+| `fs.s3a.capability.directory.marker.action.keep` | If a file was created at this path, would directory markers be retained? | true |
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
@@ -625,68 +407,6 @@ 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
-
- fs.s3a.bucket.london.directory.marker.retention
- authoritative
-
-
- fs.s3a.bucket.london.authoritative.path
- /tables
-
-```
-
-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:
-```
-
## Glossary
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index 1b4b2e8b21b38..340fd80498e39 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -23,12 +23,12 @@
### Directory Marker Compatibility
-This release does not delete directory markers when creating
+This release never delete directory markers when creating
files or directories underneath.
This is incompatible with versions of the Hadoop S3A client released
before 2021.
-Consult [Controlling the S3A Directory Marker Behavior](directory_markers.html) for
+Consult [S3A and Directory Markers](directory_markers.html) for
full details.
## Documents
@@ -40,7 +40,7 @@ full details.
* [Working with Third-party S3 Stores](./third_party_stores.html)
* [Troubleshooting](./troubleshooting_s3a.html)
* [Prefetching](./prefetching.html)
-* [Controlling the S3A Directory Marker Behavior](directory_markers.html).
+* [S3A and Directory Markers](directory_markers.html).
* [Auditing](./auditing.html).
* [Committing work to S3 with the "S3A Committers"](./committers.html)
* [S3A Committers Architecture](./committer_architecture.html)
@@ -77,7 +77,7 @@ and compatible implementations.
* Directly reads and writes S3 objects.
* Compatible with standard S3 clients.
-* Compatible with files created by the older `s3n://` client and Amazon EMR's `s3://` client.
+* Compatible with files created by Amazon EMR's `s3://` client (EMRFS).
* Supports partitioned uploads for many-GB objects.
* Offers a high-performance random IO mode for working with columnar data such
as Apache ORC and Apache Parquet files.
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md
index b8cb3ff732b36..8fddddf79c74f 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md
@@ -110,8 +110,7 @@ it is by default, then the page size is limited to that defined in
will, if the path references an object, cause that object to be deleted.
* If the path does not reference an object: the path will not be deleted
"This is for deleting objects, not directories"
-* No probes for the existence of parent directories will take place; no
- parent directory markers will be created.
+* No probes for the existence of parent directories will take place.
"If you need parent directories, call mkdir() yourself"
* The list of failed keys listed in the `DeleteObjectsResponse` response
are converted into paths and returned along with their error messages.
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
index af4c6a76becb1..76583b760435d 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
@@ -141,7 +141,7 @@ Options
|----------------------|---------------------------------------------------------------------|
| `-fips` | Require FIPS endopint to be in use |
| `-magic` | Require the S3 filesystem to be support the "magic" committer |
-| `-markers` | Directory marker status: `aware`, `keep`, `delete`, `authoritative` |
+| `-markers` | Directory marker status: `aware`, `keep` |
| `-encryption ` | Require a specific encryption algorithm |
The server side encryption options are not directly related to S3Guard, but
@@ -172,10 +172,7 @@ S3A Committers
Security
Delegation token support is disabled
-Directory Markers
- The directory marker policy is "keep"
- Available Policies: delete, keep, authoritative
- Authoritative paths: fs.s3a.authoritative.path=
+This version of Hadoop always retains directory markers
```
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3n.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3n.md
deleted file mode 100644
index 16beed920b137..0000000000000
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3n.md
+++ /dev/null
@@ -1,52 +0,0 @@
-
-
-# The S3N Client
-
-
-
-S3N was a Hadoop filesystem client which can read or write data stored
-in Amazon S3. It uses URLs with the schema `s3n://`.
-
-- - -
-
-**Hadoop's S3N client for Amazon S3 has been superceded by
-the S3A connector**
-
-**Please upgrade to S3A for a supported, higher-performance S3 Client**
-
-- - -
-
-
-## How to migrate to the S3A client
-
-1. Keep the `hadoop-aws` JAR on your classpath.
-
-1. Add the `aws-java-sdk-bundle.jar` JAR which Hadoop ships
-with to your classpath.
-
-1. Change the authentication keys
-
- | old key | new key |
- |---------|---------|
- | `fs.s3n.awsAccessKeyId` | `fs.s3a.access.key` |
- | `fs.s3n.awsSecretAccessKey` | `fs.s3a.secret.key` |
-
- Do make sure the property names are correct. For S3A, they are
- `fs.s3a.access.key` and `fs.s3a.secret.key` —you cannot just copy the S3N
- properties and replace `s3n` with `s3a`.
-
-1. Replace URLs which began with `s3n://` with `s3a://`
-
-1. You may now remove the `jets3t` JAR, as it is no longer needed.
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
index 7222eee98baeb..48e112f4f29b4 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
@@ -337,53 +337,6 @@ then delete.
Once a bucket is converted to being versioned, it cannot be converted back
to being unversioned.
-
-## 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=keep`
-
-```
-mvn verify -Dparallel-tests -DtestsThreadCount=4 -Dmarkers=keep
-```
-
-This is the default and does not need to be explicitly set.
-
-Example: test with `markers=delete`
-
-```
-mvn verify -Dparallel-tests -DtestsThreadCount=4 -Dmarkers=delete
-```
-
-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.
-
## Enabling prefetch for all tests
The tests are run with prefetch if the `prefetch` property is set in the
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
index 93f41cfaa81bb..a8f509727e4a9 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
@@ -25,7 +25,6 @@
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.fs.statistics.IOStatisticsSnapshot;
import org.apache.hadoop.fs.statistics.IOStatisticsContext;
import org.apache.hadoop.fs.store.audit.AuditSpan;
@@ -37,15 +36,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicReference;
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.getTestPropertyBool;
-import static org.apache.hadoop.fs.s3a.S3AUtils.E_FS_CLOSED;
-import static org.apache.hadoop.fs.s3a.tools.MarkerTool.UNLIMITED_LISTING;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
@@ -119,8 +114,6 @@ public void setup() throws Exception {
public void teardown() throws Exception {
Thread.currentThread().setName("teardown");
- maybeAuditTestPath();
-
super.teardown();
if (getFileSystem() != null) {
FILESYSTEM_IOSTATS.aggregate(getFileSystem().getIOStatistics());
@@ -138,50 +131,6 @@ public static void dumpFileSystemIOStatistics() {
ioStatisticsToPrettyString(FILESYSTEM_IOSTATS));
}
- /**
- * 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(
- new MarkerTool.ScanArgsBuilder()
- .withSourceFS(fs)
- .withPath(methodPath)
- .withDoPurge(true)
- .withMinMarkerCount(0)
- .withMaxMarkerCount(0)
- .withLimit(UNLIMITED_LISTING)
- .withNonAuth(false)
- .build());
- final String resultStr = result.toString();
- assertEquals("Audit of " + methodPath + " failed: "
- + resultStr,
- 0, result.getExitCode());
- assertEquals("Marker Count under " + methodPath
- + " non-zero: " + resultStr,
- 0, result.getFilteredMarkerCount());
- }
- } 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
protected int getTestTimeoutMillis() {
return S3A_TEST_TIMEOUT;
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestDowngradeSyncable.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestDowngradeSyncable.java
index 3ad679e6728d3..86e9f15568155 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestDowngradeSyncable.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestDowngradeSyncable.java
@@ -42,11 +42,6 @@ public class ITestDowngradeSyncable extends AbstractS3ACostTest {
protected static final Logger LOG =
LoggerFactory.getLogger(ITestDowngradeSyncable.class);
-
- public ITestDowngradeSyncable() {
- super(true);
- }
-
@Override
public Configuration createConfiguration() {
final Configuration conf = super.createConfiguration();
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryption.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryption.java
index 8a955a5869b2b..508e1a38356ec 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryption.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryption.java
@@ -336,7 +336,7 @@ public void testSizeOfEncryptedObjectFromHeaderWithV1Compatibility() throws Exce
putObjectRequestBuilder.contentLength(Long.parseLong(String.valueOf(SMALL_FILE_SIZE)));
putObjectRequestBuilder.metadata(metadata);
fs.putObjectDirect(putObjectRequestBuilder.build(),
- PutObjectOptions.deletingDirs(),
+ PutObjectOptions.defaultOptions(),
new S3ADataBlocks.BlockUploadData(new byte[SMALL_FILE_SIZE], null),
null);
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
index 585317c8daf3a..3e3f8245e7c85 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
@@ -21,12 +21,11 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.api.PerformanceFlagEnum;
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
import org.assertj.core.api.Assertions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -34,8 +33,6 @@
import java.io.File;
import java.io.FileNotFoundException;
import java.net.URI;
-import java.util.Arrays;
-import java.util.Collection;
import java.util.EnumSet;
@@ -48,39 +45,17 @@
/**
* Use metrics to assert about the cost of file API calls.
- * Parameterized on directory marker keep vs delete.
- * When the FS is instantiated with creation performance, things
- * behave differently...its value is that of the marker keep flag,
- * so deletion costs are the same.
*/
-@RunWith(Parameterized.class)
public class ITestS3AFileOperationCost extends AbstractS3ACostTest {
private static final Logger LOG =
LoggerFactory.getLogger(ITestS3AFileOperationCost.class);
- /**
- * Parameterization.
- */
- @Parameterized.Parameters(name = "{0}")
- public static Collection