From ad9e1524c7ec337a21b47145701187e0e279a3ec Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Mon, 25 Feb 2019 16:41:54 -0600 Subject: [PATCH 01/28] HADOOP-16085-003.patch Rebase of previous work after merge of HADOOP-15625. --- .../org/apache/hadoop/fs/s3a/Listing.java | 75 ++--- .../apache/hadoop/fs/s3a/S3AFileStatus.java | 93 +++++- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 174 ++++++++--- .../apache/hadoop/fs/s3a/S3AInputStream.java | 8 +- .../org/apache/hadoop/fs/s3a/S3AUtils.java | 21 +- .../hadoop/fs/s3a/S3LocatedFileStatus.java | 42 +++ .../hadoop/fs/s3a/S3ObjectAttributes.java | 16 +- .../hadoop/fs/s3a/WriteOperationHelper.java | 6 +- .../fs/s3a/impl/ChangeDetectionPolicy.java | 83 +++++- .../hadoop/fs/s3a/impl/ChangeTracker.java | 74 ++++- .../fs/s3a/s3guard/DDBPathMetadata.java | 8 +- .../fs/s3a/s3guard/DescendantsIterator.java | 6 +- .../fs/s3a/s3guard/DirListingMetadata.java | 5 +- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 58 ++-- .../fs/s3a/s3guard/LocalMetadataStore.java | 3 +- .../MetadataStoreListFilesIterator.java | 12 +- .../hadoop/fs/s3a/s3guard/PathMetadata.java | 22 +- .../PathMetadataDynamoDBTranslation.java | 20 +- .../apache/hadoop/fs/s3a/s3guard/S3Guard.java | 33 ++- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 14 +- .../site/markdown/tools/hadoop-aws/s3guard.md | 15 + .../hadoop/fs/s3a/AbstractS3AMockTest.java | 12 +- .../fs/s3a/ITestS3ARemoteFileChanged.java | 90 +++++- .../fs/s3a/ITestS3GuardListConsistency.java | 2 +- .../hadoop/fs/s3a/MockS3AFileSystem.java | 2 +- .../org/apache/hadoop/fs/s3a/TestListing.java | 32 +- .../fs/s3a/TestStreamChangeTracker.java | 51 +++- .../s3guard/AbstractS3GuardToolTestBase.java | 2 +- .../s3guard/ITestDynamoDBMetadataStore.java | 5 +- .../ITestDynamoDBMetadataStoreScale.java | 3 +- .../fs/s3a/s3guard/MetadataStoreTestBase.java | 41 ++- .../s3a/s3guard/TestDirListingMetadata.java | 14 +- .../s3a/s3guard/TestLocalMetadataStore.java | 6 +- .../hadoop/fs/s3a/s3guard/TestObjectETag.java | 276 ++++++++++++++++++ .../TestPathMetadataDynamoDBTranslation.java | 14 +- .../hadoop/fs/s3a/s3guard/TestS3Guard.java | 15 +- .../AbstractITestS3AMetadataStoreScale.java | 6 +- 37 files changed, 1119 insertions(+), 240 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java 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 b016eadcfbb46..fa3d6f7bb81e7 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 @@ -68,7 +68,7 @@ public Listing(S3AFileSystem owner) { * @return the file status iterator */ ProvidedFileStatusIterator createProvidedFileStatusIterator( - FileStatus[] fileStatuses, + S3AFileStatus[] fileStatuses, PathFilter filter, FileStatusAcceptor acceptor) { return new ProvidedFileStatusIterator(fileStatuses, filter, acceptor); @@ -114,7 +114,7 @@ FileStatusListingIterator createFileStatusListingIterator( S3ListRequest request, PathFilter filter, Listing.FileStatusAcceptor acceptor, - RemoteIterator providedStatus) throws IOException { + RemoteIterator providedStatus) throws IOException { return new FileStatusListingIterator( new ObjectListingIterator(listPath, request), filter, @@ -129,7 +129,7 @@ FileStatusListingIterator createFileStatusListingIterator( */ @VisibleForTesting LocatedFileStatusIterator createLocatedFileStatusIterator( - RemoteIterator statusIterator) { + RemoteIterator statusIterator) { return new LocatedFileStatusIterator(statusIterator); } @@ -143,7 +143,7 @@ LocatedFileStatusIterator createLocatedFileStatusIterator( */ @VisibleForTesting TombstoneReconcilingIterator createTombstoneReconcilingIterator( - RemoteIterator iterator, Set tombstones) { + RemoteIterator iterator, Set tombstones) { return new TombstoneReconcilingIterator(iterator, tombstones); } @@ -189,19 +189,19 @@ interface FileStatusAcceptor { * iterator returned. */ static final class SingleStatusRemoteIterator - implements RemoteIterator { + implements RemoteIterator { /** * The status to return; set to null after the first iteration. */ - private LocatedFileStatus status; + private S3LocatedFileStatus status; /** * Constructor. * @param status status value: may be null, in which case * the iterator is empty. */ - public SingleStatusRemoteIterator(LocatedFileStatus status) { + public SingleStatusRemoteIterator(S3LocatedFileStatus status) { this.status = status; } @@ -226,9 +226,9 @@ public boolean hasNext() throws IOException { * to the constructor. */ @Override - public LocatedFileStatus next() throws IOException { + public S3LocatedFileStatus next() throws IOException { if (hasNext()) { - LocatedFileStatus s = this.status; + S3LocatedFileStatus s = this.status; status = null; return s; } else { @@ -247,16 +247,16 @@ public LocatedFileStatus next() throws IOException { * There is no remote data to fetch. */ static class ProvidedFileStatusIterator - implements RemoteIterator { - private final ArrayList filteredStatusList; + implements RemoteIterator { + private final ArrayList filteredStatusList; private int index = 0; - ProvidedFileStatusIterator(FileStatus[] fileStatuses, PathFilter filter, + ProvidedFileStatusIterator(S3AFileStatus[] fileStatuses, PathFilter filter, FileStatusAcceptor acceptor) { Preconditions.checkArgument(fileStatuses != null, "Null status list!"); filteredStatusList = new ArrayList<>(fileStatuses.length); - for (FileStatus status : fileStatuses) { + for (S3AFileStatus status : fileStatuses) { if (filter.accept(status.getPath()) && acceptor.accept(status)) { filteredStatusList.add(status); } @@ -270,7 +270,7 @@ public boolean hasNext() throws IOException { } @Override - public FileStatus next() throws IOException { + public S3AFileStatus next() throws IOException { if (!hasNext()) { throw new NoSuchElementException(); } @@ -305,7 +305,7 @@ public FileStatus next() throws IOException { * Thread safety: None. */ class FileStatusListingIterator - implements RemoteIterator { + implements RemoteIterator { /** Source of objects. */ private final ObjectListingIterator source; @@ -316,10 +316,10 @@ class FileStatusListingIterator /** request batch size. */ private int batchSize; /** Iterator over the current set of results. */ - private ListIterator statusBatchIterator; + private ListIterator statusBatchIterator; - private final Set providedStatus; - private Iterator providedStatusIterator; + private final Set providedStatus; + private Iterator providedStatusIterator; /** * Create an iterator over file status entries. @@ -335,13 +335,13 @@ class FileStatusListingIterator FileStatusListingIterator(ObjectListingIterator source, PathFilter filter, FileStatusAcceptor acceptor, - RemoteIterator providedStatus) throws IOException { + RemoteIterator providedStatus) throws IOException { this.source = source; this.filter = filter; this.acceptor = acceptor; this.providedStatus = new HashSet<>(); for (; providedStatus != null && providedStatus.hasNext();) { - final FileStatus status = providedStatus.next(); + final S3AFileStatus status = providedStatus.next(); if (filter.accept(status.getPath()) && acceptor.accept(status)) { this.providedStatus.add(status); } @@ -384,8 +384,8 @@ private boolean sourceHasNext() throws IOException { @Override @Retries.RetryTranslated - public FileStatus next() throws IOException { - final FileStatus status; + public S3AFileStatus next() throws IOException { + final S3AFileStatus status; if (sourceHasNext()) { status = statusBatchIterator.next(); // We remove from provided list the file status listed by S3 so that @@ -441,7 +441,7 @@ private boolean buildNextStatusBatch(S3ListResult objects) { // counters for debug logs int added = 0, ignored = 0; // list to fill in with results. Initial size will be list maximum. - List stats = new ArrayList<>( + List stats = new ArrayList<>( objects.getObjectSummaries().size() + objects.getCommonPrefixes().size()); // objects @@ -453,8 +453,9 @@ private boolean buildNextStatusBatch(S3ListResult objects) { } // Skip over keys that are ourselves and old S3N _$folder$ files if (acceptor.accept(keyPath, summary) && filter.accept(keyPath)) { - FileStatus status = createFileStatus(keyPath, summary, - owner.getDefaultBlockSize(keyPath), owner.getUsername()); + S3AFileStatus status = createFileStatus(keyPath, summary, + owner.getDefaultBlockSize(keyPath), owner.getUsername(), + null, null); LOG.debug("Adding: {}", status); stats.add(status); added++; @@ -468,7 +469,7 @@ private boolean buildNextStatusBatch(S3ListResult objects) { for (String prefix : objects.getCommonPrefixes()) { Path keyPath = owner.keyToQualifiedPath(prefix); if (acceptor.accept(keyPath, prefix) && filter.accept(keyPath)) { - FileStatus status = new S3AFileStatus(Tristate.FALSE, keyPath, + S3AFileStatus status = new S3AFileStatus(Tristate.FALSE, keyPath, owner.getUsername()); LOG.debug("Adding directory: {}", status); added++; @@ -679,14 +680,14 @@ public boolean accept(FileStatus status) { * return a remote iterator of {@link LocatedFileStatus} instances. */ class LocatedFileStatusIterator - implements RemoteIterator { - private final RemoteIterator statusIterator; + implements RemoteIterator { + private final RemoteIterator statusIterator; /** * Constructor. * @param statusIterator an iterator over the remote status entries */ - LocatedFileStatusIterator(RemoteIterator statusIterator) { + LocatedFileStatusIterator(RemoteIterator statusIterator) { this.statusIterator = statusIterator; } @@ -696,7 +697,7 @@ public boolean hasNext() throws IOException { } @Override - public LocatedFileStatus next() throws IOException { + public S3LocatedFileStatus next() throws IOException { return owner.toLocatedFileStatus(statusIterator.next()); } } @@ -708,16 +709,16 @@ public LocatedFileStatus next() throws IOException { * remain in the source iterator. */ static class TombstoneReconcilingIterator implements - RemoteIterator { - private LocatedFileStatus next = null; - private final RemoteIterator iterator; + RemoteIterator { + private S3LocatedFileStatus next = null; + private final RemoteIterator iterator; private final Set tombstones; /** * @param iterator Source iterator to filter * @param tombstones set of tombstone markers to filter out of results */ - TombstoneReconcilingIterator(RemoteIterator + TombstoneReconcilingIterator(RemoteIterator iterator, Set tombstones) { this.iterator = iterator; if (tombstones != null) { @@ -729,7 +730,7 @@ static class TombstoneReconcilingIterator implements private boolean fetch() throws IOException { while (next == null && iterator.hasNext()) { - LocatedFileStatus candidate = iterator.next(); + S3LocatedFileStatus candidate = iterator.next(); if (!tombstones.contains(candidate.getPath())) { next = candidate; return true; @@ -745,9 +746,9 @@ public boolean hasNext() throws IOException { return fetch(); } - public LocatedFileStatus next() throws IOException { + public S3LocatedFileStatus next() throws IOException { if (hasNext()) { - LocatedFileStatus result = next; + S3LocatedFileStatus result = next; next = null; fetch(); return result; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java index be08afe4b1044..469ea1a1c9fe8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java @@ -21,6 +21,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; /** * File status for an S3A "file". @@ -32,6 +33,8 @@ @InterfaceStability.Evolving public class S3AFileStatus extends FileStatus { private Tristate isEmptyDirectory; + private String eTag; + private String versionId; /** * Create a directory status. @@ -42,7 +45,7 @@ public class S3AFileStatus extends FileStatus { public S3AFileStatus(boolean isemptydir, Path path, String owner) { - this(Tristate.fromBool(isemptydir), path, owner); + this(Tristate.fromBool(isemptydir), path, owner, owner, 0, 0, null); } /** @@ -54,10 +57,29 @@ public S3AFileStatus(boolean isemptydir, public S3AFileStatus(Tristate isemptydir, Path path, String owner) { - super(0, true, 1, 0, 0, path); + this(isemptydir, path, owner, owner, 0, 0, null); + } + + /** + * Create a directory status. + * @param isemptydir is this an empty directory? + * @param path the path + * @param owner the owner + * @param group the group + * @param modification_time the modification time + */ + public S3AFileStatus(Tristate isemptydir, + Path path, + String owner, + String group, + long modification_time, + long access_time, + FsPermission permission) { + super(0, true, 1, 0, modification_time, + access_time, permission, owner, group, path); isEmptyDirectory = isemptydir; setOwner(owner); - setGroup(owner); + setGroup(group); } /** @@ -67,31 +89,64 @@ public S3AFileStatus(Tristate isemptydir, * @param path path * @param blockSize block size * @param owner owner + * @param eTag eTag of the S3 object if available, else null + * @param versionId versionId of the S3 object if available, else null */ public S3AFileStatus(long length, long modification_time, Path path, - long blockSize, String owner) { - super(length, false, 1, blockSize, modification_time, path); + long blockSize, String owner, String eTag, String versionId) { + super(length, false, 1, blockSize, modification_time, + path); isEmptyDirectory = Tristate.FALSE; + this.eTag = eTag; + this.versionId = versionId; setOwner(owner); setGroup(owner); } + /** + * A simple file. + * @param length file length + * @param modification_time mod time + * @param access_time access time + * @param path path + * @param blockSize block size + * @param owner owner + * @param group group + * @param permission persmission + * @param eTag eTag of the S3 object if available, else null + * @param versionId versionId of the S3 object if available, else null + */ + public S3AFileStatus(long length, long modification_time, long access_time, + Path path, long blockSize, String owner, String group, + FsPermission permission, String eTag, String versionId) { + super(length, false, 1, blockSize, modification_time, + access_time, permission, owner, group, path); + isEmptyDirectory = Tristate.FALSE; + this.eTag = eTag; + this.versionId = versionId; + } + /** * Convenience constructor for creating from a vanilla FileStatus plus * an isEmptyDirectory flag. * @param source FileStatus to convert to S3AFileStatus * @param isEmptyDirectory TRUE/FALSE if known to be / not be an empty * directory, UNKNOWN if that information was not computed. + * @param eTag eTag of the S3 object if available, else null + * @param versionId versionId of the S3 object if available, else null * @return a new S3AFileStatus */ public static S3AFileStatus fromFileStatus(FileStatus source, - Tristate isEmptyDirectory) { + Tristate isEmptyDirectory, String eTag, String versionId) { if (source.isDirectory()) { return new S3AFileStatus(isEmptyDirectory, source.getPath(), - source.getOwner()); + source.getOwner(), source.getGroup(), source.getModificationTime(), + source.getAccessTime(), source.getPermission()); } else { return new S3AFileStatus(source.getLen(), source.getModificationTime(), - source.getPath(), source.getBlockSize(), source.getOwner()); + source.getAccessTime(), source.getPath(), source.getBlockSize(), + source.getOwner(), source.getGroup(), source.getPermission(), + eTag, versionId); } } @@ -105,6 +160,20 @@ public Tristate isEmptyDirectory() { return isEmptyDirectory; } + /** + * @return the S3 object eTag when available, else null. + */ + public String getETag() { + return eTag; + } + + /** + * @return the S3 object versionId when available, else null. + */ + public String getVersionId() { + return versionId; + } + /** Compare if this object is equal to another object. * @param o the object to be compared. * @return true if two file status has the same path name; false if not. @@ -137,7 +206,7 @@ public int hashCode() { */ @Override public long getModificationTime(){ - if(isDirectory()){ + if(isDirectory() && super.getModificationTime() == 0){ return System.currentTimeMillis(); } else { return super.getModificationTime(); @@ -146,8 +215,10 @@ public long getModificationTime(){ @Override public String toString() { - return super.toString() + - String.format(" isEmptyDirectory=%s", isEmptyDirectory().name()); + return super.toString() + + String.format(" isEmptyDirectory=%s", isEmptyDirectory().name() + + String.format(" eTag=%s", eTag) + + String.format(" versionId=%s", versionId)); } } 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 1f560d064a9bf..3ee76877d5867 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 @@ -74,6 +74,7 @@ import com.amazonaws.services.s3.transfer.TransferManager; import com.amazonaws.services.s3.transfer.TransferManagerConfiguration; import com.amazonaws.services.s3.transfer.Upload; +import com.amazonaws.services.s3.transfer.model.CopyResult; import com.amazonaws.services.s3.transfer.model.UploadResult; import com.amazonaws.event.ProgressListener; import com.google.common.annotations.VisibleForTesting; @@ -115,6 +116,7 @@ import org.apache.hadoop.fs.s3a.commit.CommitConstants; 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.select.SelectBinding; import org.apache.hadoop.fs.s3a.select.SelectConstants; import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata; @@ -873,7 +875,7 @@ private FSDataInputStream open( throws IOException { entryPoint(INVOCATION_OPEN); - final FileStatus fileStatus = getFileStatus(path); + final S3AFileStatus fileStatus = (S3AFileStatus) getFileStatus(path); if (fileStatus.isDirectory()) { throw new FileNotFoundException("Can't open " + path + " because it is a directory"); @@ -906,7 +908,10 @@ private FSDataInputStream open( return new FSDataInputStream( new S3AInputStream( readContext, - createObjectAttributes(path), + createObjectAttributes( + path, + fileStatus.getETag(), + fileStatus.getVersionId()), fileStatus.getLen(), s3)); } @@ -939,13 +944,20 @@ private S3AReadOpContext createReadContext( /** * Create the attributes of an object for a get/select request. * @param f path path of the request. + * @param eTag the eTag of the S3 object + * @param versionId S3 object version ID * @return attributes to use when building the query. */ - private S3ObjectAttributes createObjectAttributes(final Path f) { + private S3ObjectAttributes createObjectAttributes( + final Path f, + final String eTag, + final String versionId) { return new S3ObjectAttributes(bucket, pathToKey(f), getServerSideEncryptionAlgorithm(), - encryptionSecrets.getEncryptionKey()); + encryptionSecrets.getEncryptionKey(), + eTag, + versionId); } /** @@ -1209,19 +1221,27 @@ private boolean innerRename(Path source, Path dest) if (srcStatus.isFile()) { LOG.debug("rename: renaming file {} to {}", src, dst); long length = srcStatus.getLen(); + S3ObjectAttributes objectAttributes = + createObjectAttributes(srcStatus.getPath(), + srcStatus.getETag(), srcStatus.getVersionId()); + S3AReadOpContext readContext = createReadContext(srcStatus, inputPolicy, + changeDetectionPolicy, readAhead); if (dstStatus != null && dstStatus.isDirectory()) { String newDstKey = maybeAddTrailingSlash(dstKey); String filename = srcKey.substring(pathToKey(src.getParent()).length()+1); newDstKey = newDstKey + filename; - copyFile(srcKey, newDstKey, length); + CopyResult copyResult = copyFile(srcKey, newDstKey, length, + objectAttributes, readContext); S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src, keyToQualifiedPath(newDstKey), length, getDefaultBlockSize(dst), - username); + username, copyResult.getETag(), copyResult.getVersionId()); } else { - copyFile(srcKey, dstKey, srcStatus.getLen()); + CopyResult copyResult = copyFile(srcKey, dstKey, srcStatus.getLen(), + objectAttributes, readContext); S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src, dst, - length, getDefaultBlockSize(dst), username); + length, getDefaultBlockSize(dst), username, + copyResult.getETag(), copyResult.getVersionId()); } innerDelete(srcStatus, false); } else { @@ -1244,10 +1264,10 @@ private boolean innerRename(Path source, Path dest) } Path parentPath = keyToQualifiedPath(srcKey); - RemoteIterator iterator = listFilesAndEmptyDirectories( - parentPath, true); + RemoteIterator iterator = + listFilesAndEmptyDirectories(parentPath, true); while (iterator.hasNext()) { - LocatedFileStatus status = iterator.next(); + S3LocatedFileStatus status = iterator.next(); long length = status.getLen(); String key = pathToKey(status.getPath()); if (status.isDirectory() && !key.endsWith("/")) { @@ -1257,7 +1277,13 @@ private boolean innerRename(Path source, Path dest) .add(new DeleteObjectsRequest.KeyVersion(key)); String newDstKey = dstKey + key.substring(srcKey.length()); - copyFile(key, newDstKey, length); + S3ObjectAttributes objectAttributes = + createObjectAttributes(status.getPath(), + status.getETag(), status.getVersionId()); + S3AReadOpContext readContext = createReadContext(status, inputPolicy, + changeDetectionPolicy, readAhead); + CopyResult copyResult = copyFile(key, newDstKey, length, + objectAttributes, readContext); if (hasMetadataStore()) { // with a metadata store, the object entries need to be updated, @@ -1269,7 +1295,8 @@ private boolean innerRename(Path source, Path dest) childDst, username); } else { S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, childSrc, - childDst, length, getDefaultBlockSize(childDst), username); + childDst, length, getDefaultBlockSize(childDst), username, + copyResult.getETag(), copyResult.getVersionId()); } // Ancestor directories may not be listed, so we explicitly add them S3Guard.addMoveAncestors(metadataStore, srcPaths, dstMetas, @@ -1785,7 +1812,8 @@ PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest) PutObjectResult result = s3.putObject(putObjectRequest); incrementPutCompletedStatistics(true, len); // update metadata - finishedWrite(putObjectRequest.getKey(), len); + finishedWrite(putObjectRequest.getKey(), len, + result.getETag(), result.getVersionId()); return result; } catch (AmazonClientException e) { incrementPutCompletedStatistics(false, len); @@ -2143,7 +2171,7 @@ public FileStatus[] innerListStatus(Path f) throws FileNotFoundException, LOG.debug("List status for path: {}", path); entryPoint(INVOCATION_LIST_STATUS); - List result; + List result; final FileStatus fileStatus = getFileStatus(path); if (fileStatus.isDirectory()) { @@ -2382,11 +2410,11 @@ S3AFileStatus innerGetFileStatus(final Path f, "deleted by S3Guard"); } - FileStatus msStatus = pm.getFileStatus(); + S3AFileStatus msStatus = pm.getFileStatus(); if (needEmptyDirectoryFlag && msStatus.isDirectory()) { if (pm.isEmptyDirectory() != Tristate.UNKNOWN) { // We have a definitive true / false from MetadataStore, we are done. - return S3AFileStatus.fromFileStatus(msStatus, pm.isEmptyDirectory()); + return msStatus; } else { DirListingMetadata children = S3Guard.listChildrenWithTtl(metadataStore, path, ttlTimeProvider); @@ -2397,7 +2425,7 @@ S3AFileStatus innerGetFileStatus(final Path f, } } else { // Either this is not a directory, or we don't care if it is empty - return S3AFileStatus.fromFileStatus(msStatus, pm.isEmptyDirectory()); + return msStatus; } // If the metadata store has no children for it and it's not listed in @@ -2406,7 +2434,8 @@ S3AFileStatus innerGetFileStatus(final Path f, try { s3FileStatus = s3GetFileStatus(path, key, tombstones); } catch (FileNotFoundException e) { - return S3AFileStatus.fromFileStatus(msStatus, Tristate.TRUE); + return S3AFileStatus.fromFileStatus(msStatus, Tristate.TRUE, + null, null); } // entry was found, save in S3Guard return S3Guard.putAndReturn(metadataStore, s3FileStatus, instrumentation); @@ -2445,7 +2474,9 @@ private S3AFileStatus s3GetFileStatus(final Path path, String key, dateToLong(meta.getLastModified()), path, getDefaultBlockSize(path), - username); + username, + meta.getETag(), + meta.getVersionId()); } } catch (AmazonServiceException e) { if (e.getStatusCode() != 404) { @@ -2472,7 +2503,9 @@ private S3AFileStatus s3GetFileStatus(final Path path, String key, dateToLong(meta.getLastModified()), path, getDefaultBlockSize(path), - username); + username, + meta.getETag(), + meta.getVersionId()); } } catch (AmazonServiceException e) { if (e.getStatusCode() != 404) { @@ -2694,7 +2727,8 @@ UploadResult executePut(PutObjectRequest putObjectRequest, UploadResult result = waitForUploadCompletion(key, info); listener.uploadCompleted(); // post-write actions - finishedWrite(key, info.getLength()); + finishedWrite(key, info.getLength(), + result.getETag(), result.getVersionId()); return result; } @@ -2859,12 +2893,15 @@ public List listAWSPolicyRules( * @param srcKey source object path * @param dstKey destination object path * @param size object size + * @param srcAttributes S3 attributes of the source object + * @param readContext the read context * @throws AmazonClientException on failures inside the AWS SDK * @throws InterruptedIOException the operation was interrupted * @throws IOException Other IO problems */ @Retries.RetryMixed - private void copyFile(String srcKey, String dstKey, long size) + private CopyResult copyFile(String srcKey, String dstKey, long size, + S3ObjectAttributes srcAttributes, S3AReadOpContext readContext) throws IOException, InterruptedIOException { LOG.debug("copyFile {} -> {} ", srcKey, dstKey); @@ -2878,22 +2915,33 @@ private void copyFile(String srcKey, String dstKey, long size) } }; - once("copyFile(" + srcKey + ", " + dstKey + ")", srcKey, + return once("copyFile(" + srcKey + ", " + dstKey + ")", srcKey, () -> { ObjectMetadata srcom = getObjectMetadata(srcKey); ObjectMetadata dstom = cloneObjectMetadata(srcom); setOptionalObjectMetadata(dstom); CopyObjectRequest copyObjectRequest = new CopyObjectRequest(bucket, srcKey, bucket, dstKey); + + ChangeTracker changeTracker = new ChangeTracker( + keyToQualifiedPath(srcKey).toString(), + changeDetectionPolicy, + readContext.instrumentation.newInputStreamStatistics() + .getVersionMismatchCounter(), + srcAttributes); + changeTracker.maybeApplyConstraint(copyObjectRequest); + setOptionalCopyObjectRequestParameters(copyObjectRequest); copyObjectRequest.setCannedAccessControlList(cannedACL); copyObjectRequest.setNewObjectMetadata(dstom); Copy copy = transfers.copy(copyObjectRequest); copy.addProgressListener(progressListener); try { - copy.waitForCopyResult(); + CopyResult result = copy.waitForCopyResult(); + changeTracker.processResponse(result); incrementWriteOperations(); instrumentation.filesCopied(1, size); + return result; } catch (InterruptedException e) { throw new InterruptedIOException("Interrupted copying " + srcKey + " to " + dstKey + ", cancelling"); @@ -3004,10 +3052,12 @@ private Optional generateSSECustomerKey() { * * @param key key written to * @param length total length of file written + * @param eTag eTag of the written object + * @param versionId S3 object versionId of the written object */ @InterfaceAudience.Private @Retries.RetryExceptionsSwallowed - void finishedWrite(String key, long length) { + void finishedWrite(String key, long length, String eTag, String versionId) { LOG.debug("Finished write to {}, len {}", key, length); Path p = keyToQualifiedPath(key); Preconditions.checkArgument(length >= 0, "content length is negative"); @@ -3019,7 +3069,7 @@ void finishedWrite(String key, long length) { S3Guard.addAncestors(metadataStore, p, username); S3AFileStatus status = createUploadFileStatus(p, S3AUtils.objectRepresentsDirectory(key, length), length, - getDefaultBlockSize(p), username); + getDefaultBlockSize(p), username, eTag, versionId); S3Guard.putAndReturn(metadataStore, status, instrumentation); } } catch (IOException e) { @@ -3387,26 +3437,41 @@ public EtagChecksum getFileChecksum(Path f, final long length) @Retries.OnceTranslated public RemoteIterator listFiles(Path f, boolean recursive) throws FileNotFoundException, IOException { - return innerListFiles(f, recursive, - new Listing.AcceptFilesOnly(qualify(f))); + return toLocatedFileStatusIterator(innerListFiles(f, recursive, + new Listing.AcceptFilesOnly(qualify(f)))); + } + + private static RemoteIterator toLocatedFileStatusIterator( + RemoteIterator iterator) { + return new RemoteIterator() { + @Override + public boolean hasNext() throws IOException { + return iterator.hasNext(); + } + + @Override + public LocatedFileStatus next() throws IOException { + return iterator.next(); + } + }; } @Retries.OnceTranslated - public RemoteIterator listFilesAndEmptyDirectories(Path f, - boolean recursive) throws IOException { + public RemoteIterator listFilesAndEmptyDirectories( + Path f, boolean recursive) throws IOException { return innerListFiles(f, recursive, new Listing.AcceptAllButS3nDirs()); } @Retries.OnceTranslated - private RemoteIterator innerListFiles(Path f, boolean + private RemoteIterator innerListFiles(Path f, boolean recursive, Listing.FileStatusAcceptor acceptor) throws IOException { entryPoint(INVOCATION_LIST_FILES); Path path = qualify(f); LOG.debug("listFiles({}, {})", path, recursive); try { // lookup dir triggers existence check - final FileStatus fileStatus = getFileStatus(path); + final S3AFileStatus fileStatus = (S3AFileStatus) getFileStatus(path); if (fileStatus.isFile()) { // simple case: File LOG.debug("Path is a file"); @@ -3418,7 +3483,7 @@ private RemoteIterator innerListFiles(Path f, boolean String delimiter = recursive ? null : "/"; LOG.debug("Requesting all entries under {} with delimiter '{}'", key, delimiter); - final RemoteIterator cachedFilesIterator; + final RemoteIterator cachedFilesIterator; final Set tombstones; if (recursive) { final PathMetadata pm = metadataStore.get(path, true); @@ -3490,10 +3555,11 @@ public RemoteIterator listLocatedStatus(final Path f, entryPoint(INVOCATION_LIST_LOCATED_STATUS); Path path = qualify(f); LOG.debug("listLocatedStatus({}, {}", path, filter); - return once("listLocatedStatus", path.toString(), + RemoteIterator iterator = + once("listLocatedStatus", path.toString(), () -> { // lookup dir triggers existence check - final FileStatus fileStatus = getFileStatus(path); + final S3AFileStatus fileStatus = (S3AFileStatus) getFileStatus(path); if (fileStatus.isFile()) { // simple case: File LOG.debug("Path is a file"); @@ -3507,7 +3573,7 @@ public RemoteIterator listLocatedStatus(final Path f, DirListingMetadata meta = S3Guard.listChildrenWithTtl(metadataStore, path, ttlTimeProvider); - final RemoteIterator cachedFileStatusIterator = + final RemoteIterator cachedFileStatusIterator = listing.createProvidedFileStatusIterator( S3Guard.dirMetaToStatuses(meta), filter, acceptor); return (allowAuthoritative && meta != null @@ -3522,20 +3588,21 @@ public RemoteIterator listLocatedStatus(final Path f, cachedFileStatusIterator)); } }); + return toLocatedFileStatusIterator(iterator); } /** - * Build a {@link LocatedFileStatus} from a {@link FileStatus} instance. + * Build a {@link S3LocatedFileStatus} from a {@link FileStatus} instance. * @param status file status * @return a located status with block locations set up from this FS. * @throws IOException IO Problems. */ - LocatedFileStatus toLocatedFileStatus(FileStatus status) + S3LocatedFileStatus toLocatedFileStatus(S3AFileStatus status) throws IOException { - return new LocatedFileStatus(status, + return new S3LocatedFileStatus(status, status.isFile() ? getFileBlockLocations(status, 0, status.getLen()) - : null); + : null, status.getETag(), status.getVersionId()); } /** @@ -3694,17 +3761,36 @@ private FSDataInputStream select(final Path source, // so the operation will fail if it is not there or S3Guard believes it has // been deleted. // validation of the file status are delegated to the binding. - final FileStatus fileStatus = getFileStatus(path); + final S3AFileStatus fileStatus = (S3AFileStatus) getFileStatus(path); // readahead range can be dynamically set long ra = options.getLong(READAHEAD_RANGE, readAhead); + S3ObjectAttributes objectAttributes = createObjectAttributes( + path, fileStatus.getETag(), fileStatus.getVersionId()); + S3AReadOpContext readContext = createReadContext(fileStatus, inputPolicy, + changeDetectionPolicy, ra); + + if (!fileStatus.isDirectory()) { + // check that the object metadata lines up with what is expected + // based on the object attributes (which may contain an eTag or versionId) + // from S3Guard + ChangeTracker changeTracker = + new ChangeTracker(uri.toString(), + changeDetectionPolicy, + readContext.instrumentation.newInputStreamStatistics() + .getVersionMismatchCounter(), + objectAttributes); + ObjectMetadata objectMetadata = getObjectMetadata(path); + changeTracker.processMetadata(objectMetadata, "select", 0); + } + // build and execute the request return selectBinding.select( - createReadContext(fileStatus, inputPolicy, changeDetectionPolicy, ra), + readContext, expression, options, generateSSECustomerKey(), - createObjectAttributes(path)); + objectAttributes); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java index d0966013074d9..a68f85ed8160c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java @@ -33,6 +33,7 @@ import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ChangeTracker; import org.slf4j.Logger; @@ -146,9 +147,12 @@ public S3AInputStream(S3AReadOpContext ctx, this.serverSideEncryptionAlgorithm = s3Attributes.getServerSideEncryptionAlgorithm(); this.serverSideEncryptionKey = s3Attributes.getServerSideEncryptionKey(); + ChangeDetectionPolicy changeDetectionPolicy = + ctx.getChangeDetectionPolicy(); this.changeTracker = new ChangeTracker(uri, - ctx.getChangeDetectionPolicy(), - streamStatistics.getVersionMismatchCounter()); + changeDetectionPolicy, + streamStatistics.getVersionMismatchCounter(), + s3Attributes); setInputPolicy(ctx.getInputPolicy()); setReadahead(ctx.getReadahead()); } 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 f3235545c49c5..a5216b0e80a5c 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 @@ -531,16 +531,20 @@ public static String stringify(AmazonS3Exception e) { * @param summary summary from AWS * @param blockSize block size to declare. * @param owner owner of the file + * @param eTag S3 object eTag or null if unavailable + * @param versionId S3 object versionId or null if unavailable * @return a status entry */ public static S3AFileStatus createFileStatus(Path keyPath, S3ObjectSummary summary, long blockSize, - String owner) { + String owner, + String eTag, + String versionId) { long size = summary.getSize(); return createFileStatus(keyPath, objectRepresentsDirectory(summary.getKey(), size), - size, summary.getLastModified(), blockSize, owner); + size, summary.getLastModified(), blockSize, owner, eTag, versionId); } /** @@ -553,22 +557,27 @@ public static S3AFileStatus createFileStatus(Path keyPath, * @param size file length * @param blockSize block size for file status * @param owner Hadoop username + * @param eTag S3 object eTag or null if unavailable + * @param versionId S3 object versionId or null if unavailable * @return a status entry */ public static S3AFileStatus createUploadFileStatus(Path keyPath, - boolean isDir, long size, long blockSize, String owner) { + boolean isDir, long size, long blockSize, String owner, + String eTag, String versionId) { Date date = isDir ? null : new Date(); - return createFileStatus(keyPath, isDir, size, date, blockSize, owner); + return createFileStatus(keyPath, isDir, size, date, blockSize, owner, + eTag, versionId); } /* Date 'modified' is ignored when isDir is true. */ private static S3AFileStatus createFileStatus(Path keyPath, boolean isDir, - long size, Date modified, long blockSize, String owner) { + long size, Date modified, long blockSize, String owner, + String eTag, String versionId) { if (isDir) { return new S3AFileStatus(Tristate.UNKNOWN, keyPath, owner); } else { return new S3AFileStatus(size, dateToLong(modified), keyPath, blockSize, - owner); + owner, eTag, versionId); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java new file mode 100644 index 0000000000000..56da324cee9ca --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java @@ -0,0 +1,42 @@ +/** + * 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; + +import org.apache.hadoop.fs.BlockLocation; +import org.apache.hadoop.fs.LocatedFileStatus; + +public class S3LocatedFileStatus extends LocatedFileStatus { + private final String eTag; + private final String versionId; + + public S3LocatedFileStatus(S3AFileStatus status, BlockLocation[] locations, + String eTag, String versionId) { + super(status, locations); + this.eTag = eTag; + this.versionId = versionId; + } + + public String getETag() { + return eTag; + } + + public String getVersionId() { + return versionId; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java index d67e3e1e8cbc6..2e62ff6728206 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java @@ -34,16 +34,22 @@ public class S3ObjectAttributes { private final String key; private final S3AEncryptionMethods serverSideEncryptionAlgorithm; private final String serverSideEncryptionKey; + private final String eTag; + private final String versionId; public S3ObjectAttributes( String bucket, String key, S3AEncryptionMethods serverSideEncryptionAlgorithm, - String serverSideEncryptionKey) { + String serverSideEncryptionKey, + String eTag, + String versionId) { this.bucket = bucket; this.key = key; this.serverSideEncryptionAlgorithm = serverSideEncryptionAlgorithm; this.serverSideEncryptionKey = serverSideEncryptionKey; + this.eTag = eTag; + this.versionId = versionId; } public String getBucket() { @@ -61,4 +67,12 @@ public S3AEncryptionMethods getServerSideEncryptionAlgorithm() { public String getServerSideEncryptionKey() { return serverSideEncryptionKey; } + + public String getETag() { + return eTag; + } + + public String getVersionId() { + return versionId; + } } 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 73303f4d92f79..4ea916f8eed2a 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 @@ -226,7 +226,8 @@ public String initiateMultiPartUpload(String destKey) throws IOException { /** * Finalize a multipart PUT operation. * This completes the upload, and, if that works, calls - * {@link S3AFileSystem#finishedWrite(String, long)} to update the filesystem. + * {@link S3AFileSystem#finishedWrite(String, long, String, String)} + * to update the filesystem. * Retry policy: retrying, translated. * @param destKey destination of the commit * @param uploadId multipart operation Id @@ -259,7 +260,8 @@ private CompleteMultipartUploadResult finalizeMultipartUpload( destKey, uploadId, new ArrayList<>(partETags))); - owner.finishedWrite(destKey, length); + owner.finishedWrite(destKey, length, result.getETag(), + result.getVersionId()); return result; } ); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java index f3d8bc20c824b..bf596cefcce70 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java @@ -20,9 +20,12 @@ import java.util.Locale; +import com.amazonaws.services.s3.model.CopyObjectRequest; import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.transfer.model.CopyResult; import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.fs.s3a.S3ObjectAttributes; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,6 +54,10 @@ public abstract class ChangeDetectionPolicy { private final Mode mode; private final boolean requireVersion; + + public abstract String getRevisionId(S3ObjectAttributes s3Attributes); + + public abstract String getRevisionId(CopyResult copyResult); /** * Version support is only warned about once per S3A instance. @@ -210,6 +217,16 @@ public abstract String getRevisionId(ObjectMetadata objectMetadata, public abstract void applyRevisionConstraint(GetObjectRequest request, String revisionId); + /** + * Applies the given {@link #getRevisionId(ObjectMetadata, String) revisionId} + * as a server-side qualification on the {@code CopyObjectRequest}. + * + * @param request the request + * @param revisionId the revision id + */ + public abstract void applyRevisionConstraint(CopyObjectRequest request, + String revisionId); + /** * Takes appropriate action based on {@link #getMode() mode} when a change has * been detected. @@ -277,11 +294,32 @@ public String getRevisionId(ObjectMetadata objectMetadata, String uri) { return objectMetadata.getETag(); } + @Override + public String getRevisionId(S3ObjectAttributes s3Attributes) { + return s3Attributes.getETag(); + } + + @Override + public String getRevisionId(CopyResult copyResult) { + return copyResult.getETag(); + } + @Override public void applyRevisionConstraint(GetObjectRequest request, String revisionId) { - LOG.debug("Restricting request to etag {}", revisionId); - request.withMatchingETagConstraint(revisionId); + if (revisionId != null) { + LOG.debug("Restricting request to etag {}", revisionId); + request.withMatchingETagConstraint(revisionId); + } + } + + @Override + public void applyRevisionConstraint(CopyObjectRequest request, + String revisionId) { + if (revisionId != null) { + LOG.debug("Restricting request to etag {}", revisionId); + request.withMatchingETagConstraint(revisionId); + } } @Override @@ -322,12 +360,33 @@ public String getRevisionId(ObjectMetadata objectMetadata, String uri) { } return versionId; } + + @Override + public String getRevisionId(S3ObjectAttributes s3Attributes) { + return s3Attributes.getVersionId(); + } + + @Override + public String getRevisionId(CopyResult copyResult) { + return copyResult.getVersionId(); + } @Override public void applyRevisionConstraint(GetObjectRequest request, String revisionId) { - LOG.debug("Restricting request to version {}", revisionId); - request.withVersionId(revisionId); + if (revisionId != null) { + LOG.debug("Restricting request to version {}", revisionId); + request.withVersionId(revisionId); + } + } + + @Override + public void applyRevisionConstraint(CopyObjectRequest request, + String revisionId) { + if (revisionId != null) { + LOG.debug("Restricting request to version {}", revisionId); + request.withSourceVersionId(revisionId); + } } @Override @@ -361,12 +420,28 @@ public String getRevisionId(final ObjectMetadata objectMetadata, return null; } + @Override + public String getRevisionId(final S3ObjectAttributes s3ObjectAttributes) { + return null; + } + + @Override + public String getRevisionId(CopyResult copyResult) { + return null; + } + @Override public void applyRevisionConstraint(final GetObjectRequest request, final String revisionId) { } + @Override + public void applyRevisionConstraint(CopyObjectRequest request, + String revisionId) { + + } + @Override public String toString() { return "NoChangeDetection"; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java index f76602b953259..d3598d81daf68 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java @@ -20,11 +20,14 @@ import java.util.concurrent.atomic.AtomicLong; +import com.amazonaws.services.s3.model.CopyObjectRequest; import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.S3Object; +import com.amazonaws.services.s3.transfer.model.CopyResult; import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.fs.s3a.NoVersionAttributeException; +import org.apache.hadoop.fs.s3a.S3ObjectAttributes; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,9 +40,10 @@ import static com.google.common.base.Preconditions.checkNotNull; /** - * Change tracking for input streams: the revision ID/etag - * the previous request is recorded and when the next request comes in, - * it is compared. + * Change tracking for input streams: the version ID or etag of the object is + * tracked and compared on open/re-open. An initial version ID or etag may or + * may not be available, depending on usage (e.g. if S3Guard is utilized). + * * Self-contained for testing and use in different streams. */ @InterfaceAudience.Private @@ -76,13 +80,20 @@ public class ChangeTracker { * @param uri URI of object being tracked * @param policy policy to track. * @param versionMismatches reference to the version mismatch counter + * @param s3ObjectAttributes attributes of the object, potentially including + * an eTag or versionId to match depending on {@code policy} */ public ChangeTracker(final String uri, final ChangeDetectionPolicy policy, - final AtomicLong versionMismatches) { + final AtomicLong versionMismatches, + final S3ObjectAttributes s3ObjectAttributes) { this.policy = checkNotNull(policy); this.uri = uri; this.versionMismatches = versionMismatches; + this.revisionId = policy.getRevisionId(s3ObjectAttributes); + if (revisionId != null) { + LOG.debug("Revision ID for object at {}: {}", uri, revisionId); + } } public String getRevisionId() { @@ -115,6 +126,23 @@ public boolean maybeApplyConstraint( return false; } + /** + * Apply any revision control set by the policy if it is to be + * enforced on the server. + * @param request request to modify + * @return true iff a constraint was added. + */ + public boolean maybeApplyConstraint( + final CopyObjectRequest request) { + + if (policy.getMode() == ChangeDetectionPolicy.Mode.Server + && revisionId != null) { + policy.applyRevisionConstraint(request, revisionId); + return true; + } + return false; + } + /** * Process the response from the server for validation against the @@ -148,16 +176,48 @@ public void processResponse(final S3Object object, } } - final ObjectMetadata metadata = object.getObjectMetadata(); + processMetadata(object.getObjectMetadata(), operation, pos); + } + + /** + * Process the response from the server for validation against the + * change policy. + * @param copyResult result of a copy operation + * @throws PathIOException raised on failure + * @throws RemoteFileChangedException if the remote file has changed. + */ + public void processResponse(final CopyResult copyResult) + throws PathIOException { + final String newRevisionId = policy.getRevisionId(copyResult); + processNewRevision(newRevisionId, "copy", 0); + } + + /** + * Process metadata response from server for validation against the change + * policy. + * @param metadata metadata returned from server + * @param operation operation in progress + * @param pos offset of read + * @throws PathIOException raised on failure + * @throws RemoteFileChangedException if the remote file has changed. + */ + public void processMetadata(final ObjectMetadata metadata, + final String operation, + final long pos) throws PathIOException { final String newRevisionId = policy.getRevisionId(metadata, uri); + processNewRevision(newRevisionId, operation, pos); + } + + private void processNewRevision(final String newRevisionId, + final String operation, final long pos) throws PathIOException { if (newRevisionId == null && policy.isRequireVersion()) { throw new NoVersionAttributeException(uri, String.format( "Change detection policy requires %s", policy.getSource())); } if (revisionId == null) { - // revisionId is null on first (re)open. Pin it so change can be detected - // if object has been updated + // revisionId may be null on first (re)open. Pin it so change can be + // detected if object has been updated LOG.debug("Setting revision ID for object at {}: {}", uri, newRevisionId); revisionId = newRevisionId; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DDBPathMetadata.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DDBPathMetadata.java index 78568dc4bbbf9..e3a529ac14f2d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DDBPathMetadata.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DDBPathMetadata.java @@ -18,7 +18,7 @@ package org.apache.hadoop.fs.s3a.s3guard; -import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.Tristate; /** @@ -36,18 +36,18 @@ public DDBPathMetadata(PathMetadata pmd) { this.setLastUpdated(pmd.getLastUpdated()); } - public DDBPathMetadata(FileStatus fileStatus) { + public DDBPathMetadata(S3AFileStatus fileStatus) { super(fileStatus); this.isAuthoritativeDir = false; } - public DDBPathMetadata(FileStatus fileStatus, Tristate isEmptyDir, + public DDBPathMetadata(S3AFileStatus fileStatus, Tristate isEmptyDir, boolean isDeleted) { super(fileStatus, isEmptyDir, isDeleted); this.isAuthoritativeDir = false; } - public DDBPathMetadata(FileStatus fileStatus, Tristate isEmptyDir, + public DDBPathMetadata(S3AFileStatus fileStatus, Tristate isEmptyDir, boolean isDeleted, boolean isAuthoritativeDir, long lastUpdated) { super(fileStatus, isEmptyDir, isDeleted); this.isAuthoritativeDir = isAuthoritativeDir; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DescendantsIterator.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DescendantsIterator.java index dcee35824ed0f..88a46745b11bf 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DescendantsIterator.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DescendantsIterator.java @@ -28,9 +28,9 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.s3a.S3AFileStatus; /** * {@code DescendantsIterator} is a {@link RemoteIterator} that implements @@ -83,7 +83,7 @@ */ @InterfaceAudience.Private @InterfaceStability.Evolving -public class DescendantsIterator implements RemoteIterator { +public class DescendantsIterator implements RemoteIterator { private final MetadataStore metadataStore; private final Queue queue = new LinkedList<>(); @@ -121,7 +121,7 @@ public boolean hasNext() throws IOException { } @Override - public FileStatus next() throws IOException { + public S3AFileStatus next() throws IOException { if (!hasNext()) { throw new NoSuchElementException("No more descendants."); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java index 88f24aa9841e7..1059dd148623b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java @@ -34,6 +34,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.Tristate; /** @@ -61,7 +62,7 @@ public class DirListingMetadata extends ExpirableMetadata { * Create a directory listing metadata container. * * @param path Path of the directory. If this path has a host component, then - * all paths added later via {@link #put(FileStatus)} must also have + * all paths added later via {@link #put(S3AFileStatus)} must also have * the same host. * @param listing Entries in the directory. * @param isAuthoritative true iff listing is the full contents of the @@ -225,7 +226,7 @@ public void remove(Path childPath) { * @return true if the status was added or replaced with a new value. False * if the same FileStatus value was already present. */ - public boolean put(FileStatus childFileStatus) { + public boolean put(S3AFileStatus childFileStatus) { Preconditions.checkNotNull(childFileStatus, "childFileStatus must be non-null"); Path childPath = childStatusToPathKey(childFileStatus); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java index d2676f70e7a62..db91ca8e04c89 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java @@ -88,6 +88,7 @@ import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3AUtils; @@ -129,6 +130,14 @@ * This attribute is meaningful only to file items. *

  • optional long attribute revealing block size of the file. * This attribute is meaningful only to file items.
  • + *
  • optional string attribute tracking the s3 eTag of the file. + * May be absent if the metadata was entered with a version of S3Guard + * before this was tracked. + * This attribute is meaningful only to file items.
  • + *
  • optional string attribute tracking the s3 versionId of the file. + * May be absent if the metadata was entered with a version of S3Guard + * before this was tracked. + * This attribute is meaningful only to file items.
  • * * * The DynamoDB partition key is the parent, and the range key is the child. @@ -155,20 +164,20 @@ * This is persisted to a single DynamoDB table as: * *
    - * =========================================================================
    - * | parent                 | child | is_dir | mod_time | len |     ...    |
    - * =========================================================================
    - * | /bucket                | dir1  | true   |          |     |            |
    - * | /bucket/dir1           | dir2  | true   |          |     |            |
    - * | /bucket/dir1           | dir3  | true   |          |     |            |
    - * | /bucket/dir1/dir2      | file1 |        |   100    | 111 |            |
    - * | /bucket/dir1/dir2      | file2 |        |   200    | 222 |            |
    - * | /bucket/dir1/dir3      | dir4  | true   |          |     |            |
    - * | /bucket/dir1/dir3      | dir5  | true   |          |     |            |
    - * | /bucket/dir1/dir3/dir4 | file3 |        |   300    | 333 |            |
    - * | /bucket/dir1/dir3/dir5 | file4 |        |   400    | 444 |            |
    - * | /bucket/dir1/dir3      | dir6  | true   |          |     |            |
    - * =========================================================================
    + * ====================================================================================
    + * | parent                 | child | is_dir | mod_time | len | etag | ver_id |  ...  |
    + * ====================================================================================
    + * | /bucket                | dir1  | true   |          |     |      |        |       |
    + * | /bucket/dir1           | dir2  | true   |          |     |      |        |       |
    + * | /bucket/dir1           | dir3  | true   |          |     |      |        |       |
    + * | /bucket/dir1/dir2      | file1 |        |   100    | 111 | abc  |  mno   |       |
    + * | /bucket/dir1/dir2      | file2 |        |   200    | 222 | def  |  pqr   |       |
    + * | /bucket/dir1/dir3      | dir4  | true   |          |     |      |        |       |
    + * | /bucket/dir1/dir3      | dir5  | true   |          |     |      |        |       |
    + * | /bucket/dir1/dir3/dir4 | file3 |        |   300    | 333 | ghi  |  stu   |       |
    + * | /bucket/dir1/dir3/dir5 | file4 |        |   400    | 444 | jkl  |  vwx   |       |
    + * | /bucket/dir1/dir3      | dir6  | true   |          |     |      |        |       |
    + * ====================================================================================
      * 
    * * This choice of schema is efficient for read access patterns. @@ -610,9 +619,12 @@ private DDBPathMetadata innerGet(Path path, boolean wantEmptyDirectoryFlag) * @param path path to dir * @return new FileStatus */ - private FileStatus makeDirStatus(String owner, Path path) { - return new FileStatus(0, true, 1, 0, 0, 0, null, - owner, null, path); + private S3AFileStatus makeDirStatus(String owner, Path path) { + FileStatus fileStatus = new FileStatus(0, true, 1, 0, 0, 0, null, + owner, null, path); + + return S3AFileStatus.fromFileStatus(fileStatus, Tristate.UNKNOWN, + null, null); } @Override @@ -675,7 +687,7 @@ Collection completeAncestry( while (!parent.isRoot() && !ancestry.containsKey(parent)) { LOG.debug("auto-create ancestor path {} for child path {}", parent, path); - final FileStatus status = makeDirStatus(parent, username); + final S3AFileStatus status = makeDirStatus(parent, username); ancestry.put(parent, new DDBPathMetadata(status, Tristate.FALSE, false)); parent = parent.getParent(); @@ -880,7 +892,7 @@ Collection fullPathsToPut(DDBPathMetadata meta) while (path != null && !path.isRoot()) { final Item item = getConsistentItem(path); if (!itemExists(item)) { - final FileStatus status = makeDirStatus(path, username); + final S3AFileStatus status = makeDirStatus(path, username); metasToPut.add(new DDBPathMetadata(status, Tristate.FALSE, false, meta.isAuthoritativeDir(), meta.getLastUpdated())); path = path.getParent(); @@ -903,9 +915,13 @@ private boolean itemExists(Item item) { } /** Create a directory FileStatus using current system time as mod time. */ - static FileStatus makeDirStatus(Path f, String owner) { - return new FileStatus(0, true, 1, 0, System.currentTimeMillis(), 0, + static S3AFileStatus makeDirStatus(Path f, String owner) { + FileStatus fileStatus = + new FileStatus(0, true, 1, 0, System.currentTimeMillis(), 0, null, owner, owner, f); + S3AFileStatus s3aStatus = S3AFileStatus.fromFileStatus( + fileStatus, Tristate.UNKNOWN, null, null); + return s3aStatus; } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java index b8f9635dcd283..9276388679866 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.Tristate; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -231,7 +232,7 @@ public void move(Collection pathsToDelete, public void put(PathMetadata meta) throws IOException { Preconditions.checkNotNull(meta); - FileStatus status = meta.getFileStatus(); + S3AFileStatus status = meta.getFileStatus(); Path path = standardize(status.getPath()); synchronized (this) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreListFilesIterator.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreListFilesIterator.java index 378d10980c835..e4e76c50d6ce5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreListFilesIterator.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreListFilesIterator.java @@ -33,9 +33,9 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.s3a.S3AFileStatus; /** * {@code MetadataStoreListFilesIterator} is a {@link RemoteIterator} that @@ -85,14 +85,14 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class MetadataStoreListFilesIterator implements - RemoteIterator { + RemoteIterator { public static final Logger LOG = LoggerFactory.getLogger( MetadataStoreListFilesIterator.class); private final boolean allowAuthoritative; private final MetadataStore metadataStore; private final Set tombstones = new HashSet<>(); - private Iterator leafNodesIterator = null; + private Iterator leafNodesIterator = null; public MetadataStoreListFilesIterator(MetadataStore ms, PathMetadata meta, boolean allowAuthoritative) throws IOException { @@ -104,7 +104,7 @@ public MetadataStoreListFilesIterator(MetadataStore ms, PathMetadata meta, private void prefetch(PathMetadata meta) throws IOException { final Queue queue = new LinkedList<>(); - final Collection leafNodes = new ArrayList<>(); + final Collection leafNodes = new ArrayList<>(); if (meta != null) { final Path path = meta.getFileStatus().getPath(); @@ -121,7 +121,7 @@ private void prefetch(PathMetadata meta) throws IOException { while(!queue.isEmpty()) { PathMetadata nextMetadata = queue.poll(); - FileStatus nextStatus = nextMetadata.getFileStatus(); + S3AFileStatus nextStatus = nextMetadata.getFileStatus(); if (nextStatus.isFile()) { // All files are leaf nodes by definition leafNodes.add(nextStatus); @@ -159,7 +159,7 @@ public boolean hasNext() { } @Override - public FileStatus next() { + public S3AFileStatus next() { return leafNodesIterator.next(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java index 56645fead73c4..24d47861ff2e7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java @@ -23,6 +23,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.Tristate; /** @@ -33,7 +34,7 @@ @InterfaceStability.Evolving public class PathMetadata extends ExpirableMetadata { - private final FileStatus fileStatus; + private final S3AFileStatus fileStatus; private Tristate isEmptyDirectory; private boolean isDeleted; @@ -45,22 +46,24 @@ public class PathMetadata extends ExpirableMetadata { public static PathMetadata tombstone(Path path) { long now = System.currentTimeMillis(); FileStatus status = new FileStatus(0, false, 0, 0, now, path); - return new PathMetadata(status, Tristate.UNKNOWN, true); + S3AFileStatus s3aStatus = S3AFileStatus.fromFileStatus( + status, Tristate.UNKNOWN, null, null); + return new PathMetadata(s3aStatus, Tristate.UNKNOWN, true); } /** * Creates a new {@code PathMetadata} containing given {@code FileStatus}. * @param fileStatus file status containing an absolute path. */ - public PathMetadata(FileStatus fileStatus) { - this(fileStatus, Tristate.UNKNOWN); + public PathMetadata(S3AFileStatus fileStatus) { + this(fileStatus, Tristate.UNKNOWN, false); } - public PathMetadata(FileStatus fileStatus, Tristate isEmptyDir) { + public PathMetadata(S3AFileStatus fileStatus, Tristate isEmptyDir) { this(fileStatus, isEmptyDir, false); } - public PathMetadata(FileStatus fileStatus, Tristate isEmptyDir, boolean + public PathMetadata(S3AFileStatus fileStatus, Tristate isEmptyDir, boolean isDeleted) { Preconditions.checkNotNull(fileStatus, "fileStatus must be non-null"); Preconditions.checkNotNull(fileStatus.getPath(), "fileStatus path must be" + @@ -75,7 +78,7 @@ public PathMetadata(FileStatus fileStatus, Tristate isEmptyDir, boolean /** * @return {@code FileStatus} contained in this {@code PathMetadata}. */ - public final FileStatus getFileStatus() { + public final S3AFileStatus getFileStatus() { return fileStatus; } @@ -128,10 +131,11 @@ public String toString() { * @param sb target StringBuilder */ public void prettyPrint(StringBuilder sb) { - sb.append(String.format("%-5s %-20s %-7d %-8s %-6s", + sb.append(String.format("%-5s %-20s %-7d %-8s %-6s %-20s %-20s", fileStatus.isDirectory() ? "dir" : "file", fileStatus.getPath().toString(), fileStatus.getLen(), - isEmptyDirectory.name(), isDeleted)); + isEmptyDirectory.name(), isDeleted, + fileStatus.getETag(), fileStatus.getVersionId())); sb.append(fileStatus); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java index c6f70bf277f44..1fbe314748139 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java @@ -43,6 +43,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Constants; +import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.Tristate; /** @@ -70,6 +71,8 @@ final class PathMetadataDynamoDBTranslation { static final String IS_DELETED = "is_deleted"; static final String IS_AUTHORITATIVE = "is_authoritative"; static final String LAST_UPDATED = "last_updated"; + static final String ETAG = "etag"; + static final String VERSION_ID = "version_id"; /** Used while testing backward compatibility. */ @VisibleForTesting @@ -135,7 +138,7 @@ static DDBPathMetadata itemToPathMetadata(Item item, String username) { boolean isDir = item.hasAttribute(IS_DIR) && item.getBoolean(IS_DIR); boolean isAuthoritativeDir = false; - final FileStatus fileStatus; + final S3AFileStatus fileStatus; long lastUpdated = 0; if (isDir) { isAuthoritativeDir = !IGNORED_FIELDS.contains(IS_AUTHORITATIVE) @@ -146,8 +149,11 @@ static DDBPathMetadata itemToPathMetadata(Item item, String username) { long len = item.hasAttribute(FILE_LENGTH) ? item.getLong(FILE_LENGTH) : 0; long modTime = item.hasAttribute(MOD_TIME) ? item.getLong(MOD_TIME) : 0; long block = item.hasAttribute(BLOCK_SIZE) ? item.getLong(BLOCK_SIZE) : 0; - fileStatus = new FileStatus(len, false, 1, block, modTime, 0, null, - username, username, path); + String eTag = item.getString(ETAG); + String versionId = item.getString(VERSION_ID); + fileStatus = S3AFileStatus.fromFileStatus( + new FileStatus(len, false, 1, block, modTime, 0, null, + username, username, path), Tristate.UNKNOWN, eTag, versionId); } lastUpdated = !IGNORED_FIELDS.contains(LAST_UPDATED) @@ -172,7 +178,7 @@ static DDBPathMetadata itemToPathMetadata(Item item, String username) { */ static Item pathMetadataToItem(DDBPathMetadata meta) { Preconditions.checkNotNull(meta); - final FileStatus status = meta.getFileStatus(); + final S3AFileStatus status = meta.getFileStatus(); final Item item = new Item().withPrimaryKey(pathToKey(status.getPath())); if (status.isDirectory()) { item.withBoolean(IS_DIR, true); @@ -183,6 +189,12 @@ static Item pathMetadataToItem(DDBPathMetadata meta) { item.withLong(FILE_LENGTH, status.getLen()) .withLong(MOD_TIME, status.getModificationTime()) .withLong(BLOCK_SIZE, status.getBlockSize()); + if (status.getETag() != null) { + item.withString(ETAG, status.getETag()); + } + if (status.getVersionId() != null) { + item.withString(VERSION_ID, status.getVersionId()); + } } item.withBoolean(IS_DELETED, meta.isDeleted()); 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 b462489f68ecf..3ece30527211a 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 @@ -65,7 +65,7 @@ public final class S3Guard { static final Class S3GUARD_DDB_CLIENT_FACTORY_IMPL_DEFAULT = DynamoDBClientFactory.DefaultDynamoDBClientFactory.class; - private static final FileStatus[] EMPTY_LISTING = new FileStatus[0]; + private static final S3AFileStatus[] EMPTY_LISTING = new S3AFileStatus[0]; // Utility class. All static functions. private S3Guard() { } @@ -160,7 +160,7 @@ public static S3AFileStatus putAndReturn(MetadataStore ms, * @param dirMeta directory listing -may be null * @return a possibly-empty array of file status entries */ - public static FileStatus[] dirMetaToStatuses(DirListingMetadata dirMeta) { + public static S3AFileStatus[] dirMetaToStatuses(DirListingMetadata dirMeta) { if (dirMeta == null) { return EMPTY_LISTING; } @@ -174,7 +174,7 @@ public static FileStatus[] dirMetaToStatuses(DirListingMetadata dirMeta) { } } - return statuses.toArray(new FileStatus[0]); + return statuses.toArray(new S3AFileStatus[0]); } /** @@ -194,7 +194,7 @@ public static FileStatus[] dirMetaToStatuses(DirListingMetadata dirMeta) { * @throws IOException if metadata store update failed */ public static FileStatus[] dirListingUnion(MetadataStore ms, Path path, - List backingStatuses, DirListingMetadata dirMeta, + List backingStatuses, DirListingMetadata dirMeta, boolean isAuthoritative, ITtlTimeProvider timeProvider) throws IOException { @@ -221,7 +221,7 @@ public static FileStatus[] dirListingUnion(MetadataStore ms, Path path, // DirListingMetadata, and add any that only exist in the backingStatuses. boolean changed = false; - for (FileStatus s : backingStatuses) { + for (S3AFileStatus s : backingStatuses) { if (deleted.contains(s.getPath())) { continue; } @@ -303,7 +303,7 @@ public static void makeDirsOrdered(MetadataStore ms, List dirs, * [/a/b/file0, /a/b/file1, /a/b/file2, /a/b/file3], isAuthoritative = * true */ - FileStatus prevStatus = null; + S3AFileStatus prevStatus = null; // Use new batched put to reduce round trips. List pathMetas = new ArrayList<>(dirs.size()); @@ -314,8 +314,8 @@ public static void makeDirsOrdered(MetadataStore ms, List dirs, boolean isLeaf = (prevStatus == null); Path f = dirs.get(i); assertQualified(f); - FileStatus status = - createUploadFileStatus(f, true, 0, 0, owner); + S3AFileStatus status = + createUploadFileStatus(f, true, 0, 0, owner, null, null); // We only need to put a DirListingMetadata if we are setting // authoritative bit @@ -363,7 +363,8 @@ public static void addMoveDir(MetadataStore ms, Collection srcPaths, } assertQualified(srcPath, dstPath); - FileStatus dstStatus = createUploadFileStatus(dstPath, true, 0, 0, owner); + S3AFileStatus dstStatus = createUploadFileStatus(dstPath, true, 0, + 0, owner, null, null); addMoveStatus(srcPaths, dstMetas, srcPath, dstStatus); } @@ -379,16 +380,18 @@ public static void addMoveDir(MetadataStore ms, Collection srcPaths, * @param size length of file moved * @param blockSize blocksize to associate with destination file * @param owner file owner to use in created records + * @param eTag the s3 object eTag of file moved + * @param versionId the s3 object versionId of file moved */ public static void addMoveFile(MetadataStore ms, Collection srcPaths, Collection dstMetas, Path srcPath, Path dstPath, - long size, long blockSize, String owner) { + long size, long blockSize, String owner, String eTag, String versionId) { if (isNullMetadataStore(ms)) { return; } assertQualified(srcPath, dstPath); - FileStatus dstStatus = createUploadFileStatus(dstPath, false, - size, blockSize, owner); + S3AFileStatus dstStatus = createUploadFileStatus(dstPath, false, + size, blockSize, owner, eTag, versionId); addMoveStatus(srcPaths, dstMetas, srcPath, dstStatus); } @@ -447,7 +450,9 @@ public static void addAncestors(MetadataStore metadataStore, if (directory == null || directory.isDeleted()) { FileStatus status = new FileStatus(0, true, 1, 0, 0, 0, null, username, null, parent); - PathMetadata meta = new PathMetadata(status, Tristate.FALSE, false); + S3AFileStatus s3aStatus = S3AFileStatus.fromFileStatus( + status, Tristate.FALSE, null, null); + PathMetadata meta = new PathMetadata(s3aStatus, Tristate.FALSE, false); newDirs.add(meta); } else { break; @@ -460,7 +465,7 @@ public static void addAncestors(MetadataStore metadataStore, private static void addMoveStatus(Collection srcPaths, Collection dstMetas, Path srcPath, - FileStatus dstStatus) { + S3AFileStatus dstStatus) { srcPaths.add(srcPath); dstMetas.add(new PathMetadata(dstStatus)); } 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 1ac167f5a6dc6..f1f65cc582d58 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 @@ -51,6 +51,7 @@ import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AUtils; +import org.apache.hadoop.fs.s3a.S3LocatedFileStatus; import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens; import org.apache.hadoop.fs.s3a.commit.CommitConstants; import org.apache.hadoop.fs.s3a.select.SelectTool; @@ -703,7 +704,7 @@ private void putParentsIfNotPresent(FileStatus f) throws IOException { if (dirCache.contains(parent)) { return; } - FileStatus dir = DynamoDBMetadataStore.makeDirStatus(parent, + S3AFileStatus dir = DynamoDBMetadataStore.makeDirStatus(parent, f.getOwner()); getStore().put(new PathMetadata(dir)); dirCache.add(parent); @@ -718,13 +719,13 @@ private void putParentsIfNotPresent(FileStatus f) throws IOException { */ private long importDir(FileStatus status) throws IOException { Preconditions.checkArgument(status.isDirectory()); - RemoteIterator it = getFilesystem() + RemoteIterator it = getFilesystem() .listFilesAndEmptyDirectories(status.getPath(), true); long items = 0; while (it.hasNext()) { LocatedFileStatus located = it.next(); - FileStatus child; + S3AFileStatus child; if (located.isDirectory()) { child = DynamoDBMetadataStore.makeDirStatus(located.getPath(), located.getOwner()); @@ -734,7 +735,9 @@ private long importDir(FileStatus status) throws IOException { located.getModificationTime(), located.getPath(), located.getBlockSize(), - located.getOwner()); + located.getOwner(), + null, + null); } putParentsIfNotPresent(child); getStore().put(new PathMetadata(child)); @@ -761,7 +764,8 @@ public int run(String[] args, PrintStream out) throws Exception { filePath = "/"; } Path path = new Path(filePath); - FileStatus status = getFilesystem().getFileStatus(path); + S3AFileStatus status = (S3AFileStatus) getFilesystem() + .getFileStatus(path); try { initMetadataStore(false); 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 67f050cd36802..bb9483a92c303 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 @@ -923,6 +923,21 @@ from previous days, and and choosing a combination of retry counts and an interval which allow for the clients to cope with some throttling, but not to time out other applications. +## Read-After-Overwrite Consistency + +S3Guard provides read-after-overwrite consistency through ETags (default) or +object versioning. This works such that a reader reading a file after an +overwrite either sees the new version of the file or an error. Without S3Guard, +new readers may see the original version. Once S3 reaches eventual consistency, +new readers will see the new version. + +Readers using S3Guard will usually see the new file version, but may +in rare cases see `RemoteFileChangedException` instead. This would occur if +an S3 object read cannot provide the version tracked in S3Guard metadata. + +The configuration controlling this behavior is the same as for handling +[read-during-overwrite](./index.html#Handling_Read-During-Overwrite). + ## Troubleshooting ### Error: `S3Guard table lacks version marker.` diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java index 03c91e62cedce..6b7ebdbc515d4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java @@ -27,8 +27,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.s3a.commit.CommitConstants; +import org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; -import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore; import org.junit.After; import org.junit.Before; @@ -36,7 +36,7 @@ import org.junit.rules.ExpectedException; /** - * Abstract base class for S3A unit tests using a mock S3 client and a null + * Abstract base class for S3A unit tests using a mock S3 client and a local * metadata store. */ public abstract class AbstractS3AMockTest { @@ -59,12 +59,14 @@ public void setup() throws Exception { Configuration conf = new Configuration(); conf.setClass(S3_CLIENT_FACTORY_IMPL, MockS3ClientFactory.class, S3ClientFactory.class); - // We explicitly disable MetadataStore even if it's configured. For unit - // test we don't issue request to AWS DynamoDB service. - conf.setClass(S3_METADATA_STORE_IMPL, NullMetadataStore.class, + // We explicitly use local MetadataStore even if something else is + // configured. For unit test we don't issue request to AWS DynamoDB service + conf.setClass(S3_METADATA_STORE_IMPL, LocalMetadataStore.class, MetadataStore.class); // FS is always magic conf.setBoolean(CommitConstants.MAGIC_COMMITTER_ENABLED, true); + // use minimum multipart size for faster triggering + conf.setLong(Constants.MULTIPART_SIZE, MULTIPART_MIN_SIZE); fs = new S3AFileSystem(); URI uri = URI.create(FS_S3A + "://" + BUCKET); fs.initialize(uri, conf); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java index 98dd2026f5f0d..0976bef3849e4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a; import java.io.FileNotFoundException; +import java.io.IOException; import java.util.Arrays; import java.util.Collection; @@ -33,14 +34,20 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source; +import org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore; +import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; +import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore; +import org.apache.hadoop.fs.s3a.s3guard.PathMetadata; 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.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; +import static org.apache.hadoop.fs.s3a.select.SelectConstants.SELECT_SQL; import static org.apache.hadoop.test.LambdaTestUtils.eventually; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture; /** * Test S3A remote file change detection. @@ -101,12 +108,21 @@ protected Configuration createConfiguration() { CHANGE_DETECT_MODE); conf.set(CHANGE_DETECT_SOURCE, changeDetectionSource); conf.set(CHANGE_DETECT_MODE, changeDetectionMode); + if (conf.getClass(S3_METADATA_STORE_IMPL, MetadataStore.class) == + NullMetadataStore.class) { + // favor LocalMetadataStore over NullMetadataStore + conf.setClass(S3_METADATA_STORE_IMPL, + LocalMetadataStore.class, MetadataStore.class); + } S3ATestUtils.disableFilesystemCaching(conf); return conf; } + /** + * Tests reading a file that is changed while the reader's InputStream is open + */ @Test - public void testReadFileChanged() throws Throwable { + public void testReadFileChangedStreamOpen() throws Throwable { final int originalLength = 8192; final byte[] originalDataset = dataset(originalLength, 'a', 32); final int newLength = originalLength + 1; @@ -194,4 +210,76 @@ public void testReadFileChanged() throws Throwable { } } } + + /** + * Tests reading a file where the version visible in S3 does not match the + * version tracked in the metadata store. This simulates the condition + * of an eventually consistent read-after-overwrite. The metadata store + * will track the new version but reads from S3 may not yet return that + * version. + */ + @Test + public void testReadFileChangedNotVisibleInS3() throws Throwable { + // This test is invalid for server-side versionId. Passing a bogus + // versionId as a server-side qualification results in a 400 (Bad Request) + // saying the versionId is invalid. Maybe that's what happens in a + // read-after-overwrite inconsistency scenario? Or maybe if you read + // with versionId server-side qualified then read-after-overwrite + // inconsistency is impossible. + Assume.assumeFalse( + changeDetectionMode.equals(CHANGE_DETECT_MODE_SERVER) && + changeDetectionSource.equals(CHANGE_DETECT_SOURCE_VERSION_ID)); + final S3AFileSystem fs = getFileSystem(); + final Path testpath = writeOutOfSyncFileVersion(fs); + final FSDataInputStream instream = fs.open(testpath); + if (expectChangeException) { + intercept(RemoteFileChangedException.class, "", "read()", + () -> { + instream.read(); + }); + } else { + instream.read(); + } + } + + /** + * Tests using S3 Select on a file where the version visible in S3 does not + * match the version tracked in the metadata store. + */ + @Test + public void testSelectChangedFile() throws Throwable { + final S3AFileSystem fs = getFileSystem(); + final Path testpath = writeOutOfSyncFileVersion(fs); + if (expectChangeException) { + interceptFuture(RemoteFileChangedException.class, "select", + fs.openFile(testpath) + .must(SELECT_SQL, "SELECT * FROM S3OBJECT").build()); + } + else { + fs.openFile(testpath) + .must(SELECT_SQL, "SELECT * FROM S3OBJECT").build(); + } + } + + /** + * Writes a file with bogus ETag and versionId in the metadata store such + * that the metadata is out of sync with S3. Attempts to read such a file + * should always result in {@link RemoteFileChangedException}. + */ + private Path writeOutOfSyncFileVersion(S3AFileSystem fs) throws IOException { + final Path testpath = path("outOfSync" + + System.currentTimeMillis() + ".txt"); + final byte[] dataset = dataset(8192, 'a', 32); + writeDataset(fs, testpath, dataset, dataset.length, + 1024, false); + S3AFileStatus fileStatus = (S3AFileStatus) fs.getFileStatus(testpath); + S3AFileStatus newStatus = new S3AFileStatus(fileStatus.getLen(), fileStatus.getModificationTime(), + fileStatus.getAccessTime(), + fileStatus.getPath(), fileStatus.getBlockSize(), fileStatus.getOwner(), fileStatus.getGroup(), + fileStatus.getPermission(), "bogusETag", "bogusVersionId"); + + fs.getMetadataStore().put( + new PathMetadata(newStatus, Tristate.FALSE, false)); + return testpath; + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java index d3b3c21c926df..e71bf87494230 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java @@ -253,7 +253,7 @@ public void testConsistentRenameAfterDelete() throws Exception { DEFAULT_DELAY_KEY_SUBSTRING))); try { - RemoteIterator old = fs.listFilesAndEmptyDirectories( + RemoteIterator old = fs.listFilesAndEmptyDirectories( path("a"), true); fail("Recently renamed dir should not be visible"); } catch(FileNotFoundException e) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java index 51ff299e7be08..0e091a9e9cf0a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java @@ -177,7 +177,7 @@ public boolean exists(Path f) throws IOException { } @Override - void finishedWrite(String key, long length) { + void finishedWrite(String key, long length, String eTag, String versionId) { } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java index 39a5e3bd87d25..fb8c202e1c8c0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java @@ -40,11 +40,11 @@ */ public class TestListing extends AbstractS3AMockTest { - private static class MockRemoteIterator implements - RemoteIterator { - private Iterator iterator; + private static class MockRemoteIterator implements + RemoteIterator { + private Iterator iterator; - MockRemoteIterator(Collection source) { + MockRemoteIterator(Collection source) { iterator = source.iterator(); } @@ -52,13 +52,16 @@ public boolean hasNext() { return iterator.hasNext(); } - public FileStatus next() { + public S3AFileStatus next() { return iterator.next(); } } - private FileStatus blankFileStatus(Path path) { - return new FileStatus(0, true, 0, 0, 0, path); + private S3AFileStatus blankFileStatus(Path path) { + FileStatus fileStatus = new FileStatus(0, true, 0, 0, 0, path); + S3AFileStatus s3aStatus = S3AFileStatus.fromFileStatus( + fileStatus, Tristate.UNKNOWN, null, null); + return s3aStatus; } @Test @@ -78,11 +81,11 @@ public void testTombstoneReconcilingIterator() throws Exception { Set tombstones = new HashSet<>(); tombstones.add(deletedChild); - RemoteIterator sourceIterator = new MockRemoteIterator( + RemoteIterator sourceIterator = new MockRemoteIterator( statuses); - RemoteIterator locatedIterator = + RemoteIterator locatedIterator = listing.createLocatedFileStatusIterator(sourceIterator); - RemoteIterator reconcilingIterator = + RemoteIterator reconcilingIterator = listing.createTombstoneReconcilingIterator(locatedIterator, tombstones); Set expectedPaths = new HashSet<>(); @@ -98,8 +101,13 @@ public void testTombstoneReconcilingIterator() throws Exception { @Test public void testProvidedFileStatusIteratorEnd() throws Exception { - FileStatus[] statuses = { - new FileStatus(100, false, 1, 8192, 0, new Path("s3a://blah/blah")) + FileStatus status = new FileStatus(100, false, 1, 8192, 0, + new Path("s3a://blah/blah")); + S3AFileStatus s3aStatus = S3AFileStatus.fromFileStatus(status, + Tristate.FALSE, null, null); + + S3AFileStatus[] statuses = { + s3aStatus }; ProvidedFileStatusIterator it = new ProvidedFileStatusIterator(statuses, ACCEPT_ALL, new Listing.AcceptAllButS3nDirs()); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java index f073c4c486d2f..bbd1e401b2c33 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java @@ -161,6 +161,28 @@ public void testVersionCheckingOnServer() throws Throwable { CHANGE_DETECTED); } + @Test + public void testVersionCheckingUpfrontETag() throws Throwable { + ChangeTracker tracker = newTracker( + ChangeDetectionPolicy.Mode.Server, + ChangeDetectionPolicy.Source.ETag, + false, + objectAttributes("etag1", "versionid1")); + + assertEquals("etag1", tracker.getRevisionId()); + } + + @Test + public void testVersionCheckingUpfrontVersionId() throws Throwable { + ChangeTracker tracker = newTracker( + ChangeDetectionPolicy.Mode.Server, + ChangeDetectionPolicy.Source.VersionId, + false, + objectAttributes("etag1", "versionid1")); + + assertEquals("versionid1", tracker.getRevisionId()); + } + protected void assertConstraintApplied(final ChangeTracker tracker, final GetObjectRequest request) { assertTrue("Tracker should have applied contraints " + tracker, @@ -218,14 +240,29 @@ protected void assertTrackerMismatchCount( */ protected ChangeTracker newTracker(final ChangeDetectionPolicy.Mode mode, final ChangeDetectionPolicy.Source source, boolean requireVersion) { + return newTracker(mode, source, requireVersion, + objectAttributes(null, null)); + } + + /** + * Create tracker. + * Contains standard assertions(s). + * @return the tracker. + */ + protected ChangeTracker newTracker(final ChangeDetectionPolicy.Mode mode, + final ChangeDetectionPolicy.Source source, boolean requireVersion, + S3ObjectAttributes objectAttributes) { ChangeDetectionPolicy policy = createPolicy( mode, source, requireVersion); ChangeTracker tracker = new ChangeTracker(URI, policy, - new AtomicLong(0)); + new AtomicLong(0), objectAttributes); + if (objectAttributes.getVersionId() == null + && objectAttributes.getETag() == null) { assertFalse("Tracker should not have applied constraints " + tracker, - tracker.maybeApplyConstraint(newGetObjectRequest())); + tracker.maybeApplyConstraint(newGetObjectRequest())); + } return tracker; } @@ -252,4 +289,14 @@ private S3Object emptyResponse() { response.setKey(OBJECT); return response; } + + private S3ObjectAttributes objectAttributes( + String etag, String versionId) { + return new S3ObjectAttributes(BUCKET, + OBJECT, + null, + null, + etag, + versionId); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java index d4e4122eaee5b..f208dc11e6568 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java @@ -220,7 +220,7 @@ protected void createFile(Path path, boolean onS3, boolean onMetadataStore) ContractTestUtils.touch(fs, path); } else if (onMetadataStore) { S3AFileStatus status = new S3AFileStatus(100L, System.currentTimeMillis(), - fs.qualify(path), 512L, "hdfs"); + fs.qualify(path), 512L, "hdfs", null, null); putFile(ms, status); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java index 2f8146dfd314a..6495ed39edbf3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java @@ -246,12 +246,13 @@ public DynamoDBMSContract createContract(Configuration conf) { } @Override - FileStatus basicFileStatus(Path path, int size, boolean isDir) + S3AFileStatus basicFileStatus(Path path, int size, boolean isDir) throws IOException { String owner = UserGroupInformation.getCurrentUser().getShortUserName(); return isDir ? new S3AFileStatus(true, path, owner) - : new S3AFileStatus(size, getModTime(), path, BLOCK_SIZE, owner); + : new S3AFileStatus(size, getModTime(), path, BLOCK_SIZE, owner, + null, null); } private DynamoDBMetadataStore getDynamoMetadataStore() throws IOException { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java index 48dbce98a77ee..24f93d724c536 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java @@ -41,7 +41,6 @@ import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.StorageStatistics; import org.apache.hadoop.fs.contract.ContractTestUtils; @@ -227,7 +226,7 @@ public void test_030_BatchedWrite() throws Exception { long pruneItems = 0; for (long i = 0; i < iterations; i++) { Path longPath = pathOfDepth(BATCH_SIZE, String.valueOf(i)); - FileStatus status = basicFileStatus(longPath, 0, false, 12345, + S3AFileStatus status = basicFileStatus(longPath, 0, false, 12345, 12345); PathMetadata pm = new PathMetadata(status); synchronized (toCleanup) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java index 799c5a046bc22..150b0acf3d8f6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java @@ -38,6 +38,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Tristate; import org.apache.hadoop.io.IOUtils; @@ -148,14 +149,14 @@ private void doTestDescendantsIterator( String[] checkNodes) throws Exception { // we set up the example file system tree in metadata store for (String pathStr : createNodes) { - final FileStatus status = pathStr.contains("file") + final S3AFileStatus status = pathStr.contains("file") ? basicFileStatus(strToPath(pathStr), 100, false) : basicFileStatus(strToPath(pathStr), 0, true); ms.put(new PathMetadata(status)); } final PathMetadata rootMeta = new PathMetadata(makeDirStatus("/")); - RemoteIterator iterator; + RemoteIterator iterator; if (implementation == DescendantsIterator.class) { iterator = new DescendantsIterator(ms, rootMeta); } else if (implementation == MetadataStoreListFilesIterator.class) { @@ -715,10 +716,10 @@ public void testPruneUnsetsAuthoritative() throws Exception { createNewDirs(rootDir, grandparentDir, parentDir); long time = System.currentTimeMillis(); ms.put(new PathMetadata( - new FileStatus(0, false, 0, 0, time - 1, strToPath(staleFile)), + basicFileStatus(0, false, 0, 0, time - 1, strToPath(staleFile)), Tristate.FALSE, false)); ms.put(new PathMetadata( - new FileStatus(0, false, 0, 0, time + 1, strToPath(freshFile)), + basicFileStatus(0, false, 0, 0, time + 1, strToPath(freshFile)), Tristate.FALSE, false)); // set parent dir as authoritative @@ -752,10 +753,10 @@ public void testPrunePreservesAuthoritative() throws Exception { createNewDirs(rootDir, grandparentDir, parentDir); long time = System.currentTimeMillis(); ms.put(new PathMetadata( - new FileStatus(0, false, 0, 0, time + 1, strToPath(staleFile)), + basicFileStatus(0, false, 0, 0, time + 1, strToPath(staleFile)), Tristate.FALSE, false)); ms.put(new PathMetadata( - new FileStatus(0, false, 0, 0, time + 1, strToPath(freshFile)), + basicFileStatus(0, false, 0, 0, time + 1, strToPath(freshFile)), Tristate.FALSE, false)); if (!allowMissing()) { @@ -811,7 +812,7 @@ public void testPutDirListingMetadataPutsFileMetadata() @Test public void testPutRetainsIsDeletedInParentListing() throws Exception { final Path path = strToPath("/a/b"); - final FileStatus fileStatus = basicFileStatus(path, 0, false); + final S3AFileStatus fileStatus = basicFileStatus(path, 0, false); PathMetadata pm = new PathMetadata(fileStatus); pm.setIsDeleted(true); ms.put(pm); @@ -948,23 +949,33 @@ private void assertEmptyDirs(String ...dirs) throws IOException { } } - FileStatus basicFileStatus(Path path, int size, boolean isDir) throws + S3AFileStatus basicFileStatus(Path path, int size, boolean isDir) throws IOException { return basicFileStatus(path, size, isDir, modTime, accessTime); } - public static FileStatus basicFileStatus(Path path, int size, boolean isDir, - long newModTime, long newAccessTime) throws IOException { - return new FileStatus(size, isDir, REPLICATION, BLOCK_SIZE, newModTime, - newAccessTime, PERMISSION, OWNER, GROUP, path); + S3AFileStatus basicFileStatus(int size, boolean isDir, int blockReplication, + long blockSize, long modificationTime, Path path) { + FileStatus fileStatus = new FileStatus(size, isDir, blockReplication, + blockSize, modificationTime, path); + return S3AFileStatus.fromFileStatus(fileStatus, Tristate.UNKNOWN, + null, null); } - private FileStatus makeFileStatus(String pathStr, int size) throws + public static S3AFileStatus basicFileStatus(Path path, int size, + boolean isDir, long newModTime, long newAccessTime) throws IOException { + FileStatus fileStatus = new FileStatus(size, isDir, REPLICATION, BLOCK_SIZE, + newModTime, newAccessTime, PERMISSION, OWNER, GROUP, path); + return S3AFileStatus.fromFileStatus(fileStatus, Tristate.UNKNOWN, + null, null); + } + + private S3AFileStatus makeFileStatus(String pathStr, int size) throws IOException { return makeFileStatus(pathStr, size, modTime, accessTime); } - private FileStatus makeFileStatus(String pathStr, int size, long newModTime, + private S3AFileStatus makeFileStatus(String pathStr, int size, long newModTime, long newAccessTime) throws IOException { return basicFileStatus(strToPath(pathStr), size, false, newModTime, newAccessTime); @@ -974,7 +985,7 @@ void verifyFileStatus(FileStatus status, long size) { S3ATestUtils.verifyFileStatus(status, size, BLOCK_SIZE, modTime); } - private FileStatus makeDirStatus(String pathStr) throws IOException { + private S3AFileStatus makeDirStatus(String pathStr) throws IOException { return basicFileStatus(strToPath(pathStr), 0, true, modTime, accessTime); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java index 8458252af763c..a0404d7e63750 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java @@ -79,7 +79,8 @@ public void testListing() { PathMetadata pathMeta2 = new PathMetadata( new S3AFileStatus(true, new Path(path, "dir2"), TEST_OWNER)); PathMetadata pathMeta3 = new PathMetadata( - new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER)); + new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER, + "abc", "def")); List listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3); DirListingMetadata meta = new DirListingMetadata(path, listing, false); assertEquals(path, meta.getPath()); @@ -130,7 +131,8 @@ public void testGet() { PathMetadata pathMeta2 = new PathMetadata( new S3AFileStatus(true, new Path(path, "dir2"), TEST_OWNER)); PathMetadata pathMeta3 = new PathMetadata( - new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER)); + new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER, + "abc", "def")); List listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3); DirListingMetadata meta = new DirListingMetadata(path, listing, false); assertEquals(path, meta.getPath()); @@ -181,7 +183,8 @@ public void testPut() { PathMetadata pathMeta2 = new PathMetadata( new S3AFileStatus(true, new Path(path, "dir2"), TEST_OWNER)); PathMetadata pathMeta3 = new PathMetadata( - new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER)); + new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER, + "abc", "def")); List listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3); DirListingMetadata meta = new DirListingMetadata(path, listing, false); assertEquals(path, meta.getPath()); @@ -243,7 +246,8 @@ public void testRemove() { PathMetadata pathMeta2 = new PathMetadata( new S3AFileStatus(true, new Path(path, "dir2"), TEST_OWNER)); PathMetadata pathMeta3 = new PathMetadata( - new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER)); + new S3AFileStatus(123, 456, new Path(path, "file1"), 8192, TEST_OWNER, + "abc", "def")); List listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3); DirListingMetadata meta = new DirListingMetadata(path, listing, false); assertEquals(path, meta.getPath()); @@ -296,7 +300,7 @@ private static DirListingMetadata makeTwoDirsOneFile(Path parent) { new S3AFileStatus(true, new Path(parent, "dir2"), TEST_OWNER)); PathMetadata pathMeta3 = new PathMetadata( new S3AFileStatus(123, 456, new Path(parent, "file1"), 8192, - TEST_OWNER)); + TEST_OWNER, "abc", "def")); List listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3); return new DirListingMetadata(parent, listing, false); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestLocalMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestLocalMetadataStore.java index 2ea20b26b023d..a9372600f34ca 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestLocalMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestLocalMetadataStore.java @@ -30,7 +30,9 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.apache.hadoop.fs.s3a.Tristate; /** * MetadataStore unit test for {@link LocalMetadataStore}. @@ -169,7 +171,9 @@ private static void populateMap(Cache cache, private static void populateEntry(Cache cache, Path path) { FileStatus fileStatus = new FileStatus(0, true, 0, 0, 0, path); - cache.put(path, new LocalMetadataEntry(new PathMetadata(fileStatus))); + S3AFileStatus s3aStatus = S3AFileStatus.fromFileStatus( + fileStatus, Tristate.UNKNOWN, null, null); + cache.put(path, new LocalMetadataEntry(new PathMetadata(s3aStatus))); } private static long sizeOfMap(Cache cache) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java new file mode 100644 index 0000000000000..826474059ec97 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java @@ -0,0 +1,276 @@ +package org.apache.hadoop.fs.s3a.s3guard; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.when; +import static org.mockito.hamcrest.MockitoHamcrest.argThat; + +import com.amazonaws.services.s3.Headers; +import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; +import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; +import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import com.amazonaws.services.s3.model.GetObjectRequest; +import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; +import com.amazonaws.services.s3.model.InitiateMultipartUploadResult; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.ListObjectsV2Result; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.PutObjectResult; +import com.amazonaws.services.s3.model.S3Object; +import com.amazonaws.services.s3.model.UploadPartRequest; +import com.amazonaws.services.s3.model.UploadPartResult; +import java.io.ByteArrayInputStream; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.AbstractS3AMockTest; +import org.apache.hadoop.fs.s3a.Constants; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.junit.Test; + +/** + * Tests to ensure eTag is captured on S3 PUT and used on GET. + */ +public class TestObjectETag extends AbstractS3AMockTest { + + /** + * Tests a file uploaded with a single PUT to ensure eTag is captured and used + * on file read. + */ + @Test + public void testCreateAndReadFileSinglePart() throws Exception { + Path path = new Path("s3a://mock-bucket/file"); + String content = "content"; + + PutObjectResult putObjectResult = new PutObjectResult(); + ObjectMetadata objectMetadata = new ObjectMetadata(); + objectMetadata.setContentLength(content.length()); + putObjectResult.setMetadata(objectMetadata); + String eTag = "abc"; + putObjectResult.setETag(eTag); + + when(s3.getObjectMetadata(any(GetObjectMetadataRequest.class))) + .thenThrow(NOT_FOUND); + when(s3.putObject(argThat(correctPutObjectRequest("file")))) + .thenReturn(putObjectResult); + ListObjectsV2Result emptyListing = new ListObjectsV2Result(); + when(s3.listObjectsV2(argThat(correctListObjectsRequest("file/")))) + .thenReturn(emptyListing); + + FSDataOutputStream outputStream = fs.create(path); + outputStream.writeChars(content); + outputStream.close(); + + // make sure the eTag was put into the metadataStore + MetadataStore metadataStore = fs.getMetadataStore(); + PathMetadata pathMetadata = metadataStore.get(path); + assertNotNull(pathMetadata); + String storedETag = pathMetadata.getFileStatus().getETag(); + assertEquals(eTag, storedETag); + + // Ensure underlying S3 getObject call uses the stored eTag when reading + // data back. If it doesn't, the read won't work and the assert will + // fail. + S3Object s3Object = new S3Object(); + s3Object.setObjectContent(new ByteArrayInputStream(content.getBytes())); + ObjectMetadata metadata = new ObjectMetadata(); + metadata.setHeader(Headers.ETAG, eTag); + s3Object.setObjectMetadata(metadata); + when(s3.getObject(argThat(correctGetObjectRequest("file", eTag)))) + .thenReturn(s3Object); + FSDataInputStream inputStream = fs.open(path); + String readContent = IOUtils.toString(inputStream); + assertEquals(content, readContent); + } + + /** + * Tests a file uploaded with multi-part upload to ensure eTag is captured + * and used on file read. + */ + @Test + public void testCreateAndReadFileMultiPart() throws Exception { + Path path = new Path("s3a://mock-bucket/file"); + byte[] content = new byte[Constants.MULTIPART_MIN_SIZE + 1]; + + CompleteMultipartUploadResult uploadResult = + new CompleteMultipartUploadResult(); + String eTag = "abc"; + uploadResult.setETag(eTag); + + when(s3.getObjectMetadata(any(GetObjectMetadataRequest.class))) + .thenThrow(NOT_FOUND); + + InitiateMultipartUploadResult initiateMultipartUploadResult = + new InitiateMultipartUploadResult(); + initiateMultipartUploadResult.setUploadId("uploadId"); + when(s3.initiateMultipartUpload( + argThat(correctInitiateMultipartUploadRequest("file")))) + .thenReturn(initiateMultipartUploadResult); + + UploadPartResult uploadPartResult = new UploadPartResult(); + uploadPartResult.setETag("partETag"); + when(s3.uploadPart(argThat(correctUploadPartRequest("file")))) + .thenReturn(uploadPartResult); + + CompleteMultipartUploadResult multipartUploadResult = + new CompleteMultipartUploadResult(); + multipartUploadResult.setETag(eTag); + when(s3.completeMultipartUpload( + argThat(correctMultipartUploadRequest("file")))) + .thenReturn(multipartUploadResult); + + ListObjectsV2Result emptyListing = new ListObjectsV2Result(); + when(s3.listObjectsV2(argThat(correctListObjectsRequest("file/")))) + .thenReturn(emptyListing); + + FSDataOutputStream outputStream = fs.create(path); + outputStream.write(content); + outputStream.close(); + + // make sure the eTag was put into the metadataStore + MetadataStore metadataStore = fs.getMetadataStore(); + PathMetadata pathMetadata = metadataStore.get(path); + assertNotNull(pathMetadata); + String storedETag = pathMetadata.getFileStatus().getETag(); + assertEquals(eTag, storedETag); + + // Ensure underlying S3 getObject call uses the stored eTag when reading + // data back. If it doesn't, the read won't work and the assert will + // fail. + S3Object s3Object = new S3Object(); + ObjectMetadata metadata = new ObjectMetadata(); + metadata.setHeader(Headers.ETAG, eTag); + s3Object.setObjectMetadata(metadata); + s3Object.setObjectContent(new ByteArrayInputStream(content)); + when(s3.getObject(argThat(correctGetObjectRequest("file", eTag)))) + .thenReturn(s3Object); + FSDataInputStream inputStream = fs.open(path); + byte[] readContent = IOUtils.toByteArray(inputStream); + assertArrayEquals(content, readContent); + } + + private Matcher correctUploadPartRequest( + final String key) { + return new BaseMatcher() { + @Override + public boolean matches(Object item) { + if (item instanceof UploadPartRequest) { + UploadPartRequest request = (UploadPartRequest) item; + return request.getKey().equals(key); + } + return false; + } + + @Override + public void describeTo(Description description) { + description.appendText("key matches"); + } + }; + } + + private Matcher + correctInitiateMultipartUploadRequest(final String key) { + return new BaseMatcher() { + @Override + public void describeTo(Description description) { + description.appendText("key matches"); + } + + @Override + public boolean matches(Object item) { + if (item instanceof InitiateMultipartUploadRequest) { + InitiateMultipartUploadRequest request = + (InitiateMultipartUploadRequest) item; + return request.getKey().equals(key); + } + return false; + } + }; + } + + private Matcher + correctMultipartUploadRequest(final String key) { + return new BaseMatcher() { + @Override + public boolean matches(Object item) { + if (item instanceof CompleteMultipartUploadRequest) { + CompleteMultipartUploadRequest request = + (CompleteMultipartUploadRequest) item; + return request.getKey().equals(key); + } + return false; + } + + @Override + public void describeTo(Description description) { + description.appendText("key matches"); + } + }; + } + + private Matcher correctListObjectsRequest( + final String key) { + return new BaseMatcher() { + @Override + public boolean matches(Object item) { + if (item instanceof ListObjectsV2Request) { + ListObjectsV2Request listObjectsRequest = + (ListObjectsV2Request) item; + return listObjectsRequest.getPrefix().equals(key); + } + return false; + } + + @Override + public void describeTo(Description description) { + description.appendText("key matches"); + } + }; + } + + private Matcher correctPutObjectRequest( + final String key) { + return new BaseMatcher() { + @Override + public boolean matches(Object item) { + if (item instanceof PutObjectRequest) { + PutObjectRequest putObjectRequest = (PutObjectRequest) item; + return putObjectRequest.getKey().equals(key); + } + return false; + } + + @Override + public void describeTo(Description description) { + description.appendText("key matches"); + } + }; + } + + private Matcher correctGetObjectRequest(final String key, + final String eTag) { + return new BaseMatcher() { + @Override + public boolean matches(Object item) { + if (item instanceof GetObjectRequest) { + GetObjectRequest getObjectRequest = (GetObjectRequest) item; + return getObjectRequest.getKey().equals(key) + && getObjectRequest.getMatchingETagConstraints() + .contains(eTag); + } + return false; + } + + @Override + public void describeTo(Description description) { + description.appendText("key and eTag matches"); + } + }; + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java index 704f51e3c083b..59e85f87381d7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java @@ -67,6 +67,8 @@ public class TestPathMetadataDynamoDBTranslation extends Assert { private static final long TEST_FILE_LENGTH = 100; private static final long TEST_MOD_TIME = 9999; private static final long TEST_BLOCK_SIZE = 128; + private static final String TEST_ETAG = "abc"; + private static final String TEST_VERSION_ID = "def"; private static final Path TEST_FILE_PATH = new Path(TEST_DIR_PATH, "myFile"); private static final Item TEST_FILE_ITEM = new Item(); private static DDBPathMetadata testFilePathMetadata; @@ -84,7 +86,7 @@ public static void setUpBeforeClass() throws IOException { testFilePathMetadata = new DDBPathMetadata( new S3AFileStatus(TEST_FILE_LENGTH, TEST_MOD_TIME, TEST_FILE_PATH, - TEST_BLOCK_SIZE, username)); + TEST_BLOCK_SIZE, username, TEST_ETAG, TEST_VERSION_ID)); TEST_FILE_ITEM .withPrimaryKey(PARENT, pathToParentKey(TEST_FILE_PATH.getParent()), @@ -92,7 +94,9 @@ public static void setUpBeforeClass() throws IOException { .withBoolean(IS_DIR, false) .withLong(FILE_LENGTH, TEST_FILE_LENGTH) .withLong(MOD_TIME, TEST_MOD_TIME) - .withLong(BLOCK_SIZE, TEST_BLOCK_SIZE); + .withLong(BLOCK_SIZE, TEST_BLOCK_SIZE) + .withString(ETAG, TEST_ETAG) + .withString(VERSION_ID, TEST_VERSION_ID); } /** @@ -147,7 +151,7 @@ public void testItemToPathMetadata() throws IOException { */ private static void verify(Item item, PathMetadata meta) { assertNotNull(meta); - final FileStatus status = meta.getFileStatus(); + final S3AFileStatus status = meta.getFileStatus(); final Path path = status.getPath(); assertEquals(item.get(PARENT), pathToParentKey(path.getParent())); assertEquals(item.get(CHILD), path.getName()); @@ -157,6 +161,10 @@ private static void verify(Item item, PathMetadata meta) { assertEquals(len, status.getLen()); long bSize = item.hasAttribute(BLOCK_SIZE) ? item.getLong(BLOCK_SIZE) : 0; assertEquals(bSize, status.getBlockSize()); + String eTag = item.getString(ETAG); + assertEquals(eTag, status.getETag()); + String versionId = item.getString(VERSION_ID); + assertEquals(versionId, status.getVersionId()); /* * S3AFileStatue#getModificationTime() reports the current time, so the diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java index 1ddfed414d825..77489833a15fb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java @@ -26,6 +26,8 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.Tristate; import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_METADATASTORE_AUTHORITATIVE_DIR_TTL; @@ -51,7 +53,7 @@ public void testDirListingUnion() throws Exception { Arrays.asList(m1, m2), false); // Two other files in s3 - List s3Listing = Arrays.asList( + List s3Listing = Arrays.asList( makeFileStatus("s3a://bucket/dir/s3-file3", false), makeFileStatus("s3a://bucket/dir/s3-file4", false) ); @@ -86,12 +88,17 @@ private PathMetadata makePathMeta(String pathStr, boolean isDir) { return new PathMetadata(makeFileStatus(pathStr, isDir)); } - private FileStatus makeFileStatus(String pathStr, boolean isDir) { + private S3AFileStatus makeFileStatus(String pathStr, boolean isDir) { Path p = new Path(pathStr); + FileStatus fileStatus; if (isDir) { - return new FileStatus(0, true, 1, 1, System.currentTimeMillis(), p); + fileStatus = new FileStatus(0, true, 1, 1, + System.currentTimeMillis(), p); } else { - return new FileStatus(100, false, 1, 1, System.currentTimeMillis(), p); + fileStatus = new FileStatus(100, false, 1, 1, + System.currentTimeMillis(), p); } + return S3AFileStatus.fromFileStatus(fileStatus, Tristate.UNKNOWN, + null , null); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java index 0e6a1d8d09245..b843392ebfa07 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java @@ -170,7 +170,8 @@ protected S3AFileStatus copyStatus(S3AFileStatus status) { status.getOwner()); } else { return new S3AFileStatus(status.getLen(), status.getModificationTime(), - status.getPath(), status.getBlockSize(), status.getOwner()); + status.getPath(), status.getBlockSize(), status.getOwner(), + status.getETag(), status.getVersionId()); } } @@ -207,7 +208,8 @@ private static void printTiming(Logger log, String op, NanoTimer timer, } protected static S3AFileStatus makeFileStatus(Path path) throws IOException { - return new S3AFileStatus(SIZE, ACCESS_TIME, path, BLOCK_SIZE, OWNER); + return new S3AFileStatus(SIZE, ACCESS_TIME, path, BLOCK_SIZE, OWNER, + null, null); } protected static S3AFileStatus makeDirStatus(Path p) throws IOException { From 0eca6f38be44879011df7099a89ec338851f9c36 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Mon, 25 Mar 2019 12:43:51 -0500 Subject: [PATCH 02/28] Don't process response from copy --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 9 +++-- .../hadoop/fs/s3a/impl/ChangeTracker.java | 6 ++-- .../fs/s3a/TestStreamChangeTracker.java | 34 +++++++++++++++++++ .../hadoop/fs/s3a/s3guard/TestObjectETag.java | 8 +++++ .../fs/s3a/select/ITestS3SelectCLI.java | 8 +++++ .../fs/s3a/select/ITestS3SelectLandsat.java | 8 +++++ 6 files changed, 68 insertions(+), 5 deletions(-) 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 3ee76877d5867..81bb8e208ce56 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 @@ -698,7 +698,7 @@ public S3AInputPolicy getInputPolicy() { * @return the change detection policy */ @VisibleForTesting - ChangeDetectionPolicy getChangeDetectionPolicy() { + public ChangeDetectionPolicy getChangeDetectionPolicy() { return changeDetectionPolicy; } @@ -2920,8 +2920,6 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, ObjectMetadata srcom = getObjectMetadata(srcKey); ObjectMetadata dstom = cloneObjectMetadata(srcom); setOptionalObjectMetadata(dstom); - CopyObjectRequest copyObjectRequest = - new CopyObjectRequest(bucket, srcKey, bucket, dstKey); ChangeTracker changeTracker = new ChangeTracker( keyToQualifiedPath(srcKey).toString(), @@ -2929,6 +2927,11 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, readContext.instrumentation.newInputStreamStatistics() .getVersionMismatchCounter(), srcAttributes); + + changeTracker.processMetadata(srcom, "copy", 0); + + CopyObjectRequest copyObjectRequest = + new CopyObjectRequest(bucket, srcKey, bucket, dstKey); changeTracker.maybeApplyConstraint(copyObjectRequest); setOptionalCopyObjectRequestParameters(copyObjectRequest); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java index d3598d81daf68..f81eb95ddadf2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java @@ -188,8 +188,10 @@ public void processResponse(final S3Object object, */ public void processResponse(final CopyResult copyResult) throws PathIOException { - final String newRevisionId = policy.getRevisionId(copyResult); - processNewRevision(newRevisionId, "copy", 0); + // ETag (sometimes, depending on encryption and/or multipart) is not the + // same on the copied object as the original. Version Id seems to never + // be the same on the copy. As such, there isn't really anything that + // can be verified on the response. } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java index bbd1e401b2c33..1a16a9a8fc26d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java @@ -21,6 +21,7 @@ import java.util.concurrent.atomic.AtomicLong; import com.amazonaws.services.s3.Headers; +import com.amazonaws.services.s3.model.CopyObjectRequest; import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.S3Object; @@ -50,6 +51,8 @@ public class TestStreamChangeTracker extends HadoopTestBase { public static final String OBJECT = "object"; + public static final String DEST_OBJECT = "new_object"; + public static final String URI = "s3a://" + BUCKET + "/" + OBJECT; @Test @@ -183,12 +186,39 @@ public void testVersionCheckingUpfrontVersionId() throws Throwable { assertEquals("versionid1", tracker.getRevisionId()); } + @Test + public void testVersionCheckingETagCopyServer() throws Throwable { + ChangeTracker tracker = newTracker( + ChangeDetectionPolicy.Mode.Server, + ChangeDetectionPolicy.Source.VersionId, + false, + objectAttributes("etag1", "versionid1")); + assertConstraintApplied(tracker, newCopyObjectRequest()); + } + + @Test + public void testVersionCheckingETagCopyClient() throws Throwable { + ChangeTracker tracker = newTracker( + ChangeDetectionPolicy.Mode.Client, + ChangeDetectionPolicy.Source.VersionId, + false, + objectAttributes("etag1", "versionid1")); + assertFalse("Tracker should not have applied contraints " + tracker, + tracker.maybeApplyConstraint(newCopyObjectRequest())); + } + protected void assertConstraintApplied(final ChangeTracker tracker, final GetObjectRequest request) { assertTrue("Tracker should have applied contraints " + tracker, tracker.maybeApplyConstraint(request)); } + protected void assertConstraintApplied(final ChangeTracker tracker, + final CopyObjectRequest request) throws PathIOException { + assertTrue("Tracker should have applied contraints " + tracker, + tracker.maybeApplyConstraint(request)); + } + protected RemoteFileChangedException expectChangeException( final ChangeTracker tracker, final S3Object response, @@ -270,6 +300,10 @@ private GetObjectRequest newGetObjectRequest() { return new GetObjectRequest(BUCKET, OBJECT); } + private CopyObjectRequest newCopyObjectRequest() { + return new CopyObjectRequest(BUCKET, OBJECT, BUCKET, DEST_OBJECT); + } + private S3Object newResponse(String etag, String versionId) { ObjectMetadata md = new ObjectMetadata(); if (etag != null) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java index 826474059ec97..569bc963e2642 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java @@ -29,15 +29,23 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.AbstractS3AMockTest; import org.apache.hadoop.fs.s3a.Constants; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.hamcrest.Matcher; +import org.junit.Assume; +import org.junit.Before; import org.junit.Test; /** * Tests to ensure eTag is captured on S3 PUT and used on GET. */ public class TestObjectETag extends AbstractS3AMockTest { + @Before + public void before() { + Assume.assumeTrue("change detection source should be etag", + fs.getChangeDetectionPolicy().getSource() == Source.ETag); + } /** * Tests a file uploaded with a single PUT to ensure eTag is captured and used diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectCLI.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectCLI.java index fccf708fef4e8..e31b48e5b5862 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectCLI.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectCLI.java @@ -25,6 +25,7 @@ import java.nio.charset.Charset; import java.util.List; +import org.junit.Assume; import org.junit.Test; import org.apache.commons.io.IOUtils; @@ -34,6 +35,8 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source; import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.OperationDuration; @@ -80,6 +83,11 @@ public void setup() throws Exception { selectConf = new Configuration(getConfiguration()); localFile = getTempFilename(); landsatSrc = getLandsatGZ().toString(); + ChangeDetectionPolicy changeDetectionPolicy = + getLandsatFS().getChangeDetectionPolicy(); + Assume.assumeFalse("the standard landsat bucket doesn't have versioning", + changeDetectionPolicy.getSource() == Source.VersionId + && changeDetectionPolicy.isRequireVersion()); } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java index 78f3a6d1fe558..1d45852c98946 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java @@ -21,6 +21,9 @@ import java.io.IOException; import java.util.List; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source; +import org.junit.Assume; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -190,6 +193,11 @@ public void setup() throws Exception { // disable the gzip codec, so that the record readers do not // get confused enablePassthroughCodec(selectConf, ".gz"); + ChangeDetectionPolicy changeDetectionPolicy = + getLandsatFS().getChangeDetectionPolicy(); + Assume.assumeFalse("the standard landsat bucket doesn't have versioning", + changeDetectionPolicy.getSource() == Source.VersionId + && changeDetectionPolicy.isRequireVersion()); } protected int getMaxLines() { From 068a55d62201284393f15448acbb8b4b439eefd5 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Mon, 25 Mar 2019 15:43:45 -0500 Subject: [PATCH 03/28] Add copy exception handling --- .../fs/s3a/RemoteFileChangedException.java | 20 ++++++++ .../apache/hadoop/fs/s3a/S3AFileSystem.java | 51 ++++++++++++++++++- .../hadoop/fs/s3a/impl/ChangeTracker.java | 28 ++++++++++ 3 files changed, 98 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/RemoteFileChangedException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/RemoteFileChangedException.java index cfa5935bbf3e3..4a91fb7780afb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/RemoteFileChangedException.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/RemoteFileChangedException.java @@ -32,6 +32,9 @@ @InterfaceStability.Unstable public class RemoteFileChangedException extends PathIOException { + public static final String PRECONDITIONS_NOT_MET = + "Constraints of request were unsatisfiable"; + /** * Constructs a RemoteFileChangedException. * @@ -46,4 +49,21 @@ public RemoteFileChangedException(String path, super(path, message); setOperation(operation); } + + /** + * Constructs a RemoteFileChangedException. + * + * @param path the path accessed when the change was detected + * @param operation the operation (e.g. open, re-open) performed when the + * change was detected + * @param message a message providing more details about the condition + * @param cause inner cause. + */ + public RemoteFileChangedException(String path, + String operation, + String message, + Throwable cause) { + super(path, message, cause); + setOperation(operation); + } } 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 81bb8e208ce56..291700f93ed64 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 @@ -2940,7 +2940,19 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, Copy copy = transfers.copy(copyObjectRequest); copy.addProgressListener(progressListener); try { - CopyResult result = copy.waitForCopyResult(); + CopyOutcome copyOutcome = copyOutcome(copy); + InterruptedException interruptedException = + copyOutcome.getInterruptedException(); + if (interruptedException != null) { + throw interruptedException; + } + RuntimeException runtimeException = + copyOutcome.getRuntimeException(); + if (runtimeException != null) { + changeTracker.processException(runtimeException, "copy"); + throw runtimeException; + } + CopyResult result = copyOutcome.getCopyResult(); changeTracker.processResponse(result); incrementWriteOperations(); instrumentation.filesCopied(1, size); @@ -2952,6 +2964,43 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, }); } + private static CopyOutcome copyOutcome(Copy copy) { + try { + CopyResult result = copy.waitForCopyResult(); + return new CopyOutcome(result, null, null); + } catch (RuntimeException e) { + return new CopyOutcome(null, null, e); + } catch (InterruptedException e) { + return new CopyOutcome(null, e, null); + } + } + + private static class CopyOutcome { + private final CopyResult copyResult; + private final InterruptedException interruptedException; + private final RuntimeException runtimeException; + + private CopyOutcome(CopyResult copyResult, + InterruptedException interruptedException, + RuntimeException runtimeException) { + this.copyResult = copyResult; + this.interruptedException = interruptedException; + this.runtimeException = runtimeException; + } + + public CopyResult getCopyResult() { + return copyResult; + } + + public InterruptedException getInterruptedException() { + return interruptedException; + } + + public RuntimeException getRuntimeException() { + return runtimeException; + } + } + /** * Set the optional parameters when initiating the request (encryption, * headers, storage, etc). diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java index f81eb95ddadf2..b8f217ac7a94a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java @@ -20,6 +20,7 @@ import java.util.concurrent.atomic.AtomicLong; +import com.amazonaws.AmazonServiceException; import com.amazonaws.services.s3.model.CopyObjectRequest; import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.ObjectMetadata; @@ -194,6 +195,33 @@ public void processResponse(final CopyResult copyResult) // can be verified on the response. } + /** + * Process an exception generated against the change policy. + * If the exception indicates the file has changed, this method throws + * {@code RemoteFileChangedException} with the original exception as the + * cause. + * @param e the exception + * @param operation the operation performed when the exception was + * generated. + * @throws RemoteFileChangedException if the remote file has changed. + */ + public void processException(Exception e, String operation) throws + RemoteFileChangedException { + if (e instanceof AmazonServiceException) { + AmazonServiceException serviceException = (AmazonServiceException) e; + if (serviceException.getStatusCode() == 412) { + versionMismatches.incrementAndGet(); + throw new RemoteFileChangedException(uri, operation, String.format( + RemoteFileChangedException.PRECONDITIONS_NOT_MET + + " on %s." + + " Version %s was unavailable", + getSource(), + getRevisionId()), + serviceException); + } + } + } + /** * Process metadata response from server for validation against the change * policy. From 2ae7d1621ee893b9e1282d8741509dadf31d9225 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Mon, 25 Mar 2019 16:54:49 -0500 Subject: [PATCH 04/28] Add constants in TestDirListingMetadata --- .../apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java index a0404d7e63750..891edb27e28e9 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java @@ -38,6 +38,8 @@ public class TestDirListingMetadata { private static final String TEST_OWNER = "hadoop"; + public static final String TEST_ETAG = "abc"; + public static final String TEST_VERSION_ID = "def"; @Rule public ExpectedException exception = ExpectedException.none(); @@ -300,7 +302,7 @@ private static DirListingMetadata makeTwoDirsOneFile(Path parent) { new S3AFileStatus(true, new Path(parent, "dir2"), TEST_OWNER)); PathMetadata pathMeta3 = new PathMetadata( new S3AFileStatus(123, 456, new Path(parent, "file1"), 8192, - TEST_OWNER, "abc", "def")); + TEST_OWNER, TEST_ETAG, TEST_VERSION_ID)); List listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3); return new DirListingMetadata(parent, listing, false); } From bbf8365815c8e60da04158a805d49f02bcf48e62 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Mon, 25 Mar 2019 16:55:24 -0500 Subject: [PATCH 05/28] TestPathMetadataDynamoDBTranslation tests null etag, versonId --- .../TestPathMetadataDynamoDBTranslation.java | 87 +++++++++++++------ 1 file changed, 59 insertions(+), 28 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java index 59e85f87381d7..70bf901514bab 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.net.URI; +import java.util.Arrays; import java.util.Collection; import java.util.concurrent.Callable; @@ -31,16 +32,16 @@ import com.google.common.base.Preconditions; import org.junit.After; import org.junit.Assert; -import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.rules.Timeout; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.test.LambdaTestUtils; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.mockito.Mockito; import static com.amazonaws.services.dynamodbv2.model.KeyType.HASH; @@ -58,45 +59,75 @@ * Test the PathMetadataDynamoDBTranslation is able to translate between domain * model objects and DynamoDB items. */ +@RunWith(Parameterized.class) public class TestPathMetadataDynamoDBTranslation extends Assert { private static final Path TEST_DIR_PATH = new Path("s3a://test-bucket/myDir"); - private static final Item TEST_DIR_ITEM = new Item(); - private static DDBPathMetadata testDirPathMetadata; - private static final long TEST_FILE_LENGTH = 100; private static final long TEST_MOD_TIME = 9999; private static final long TEST_BLOCK_SIZE = 128; private static final String TEST_ETAG = "abc"; private static final String TEST_VERSION_ID = "def"; private static final Path TEST_FILE_PATH = new Path(TEST_DIR_PATH, "myFile"); - private static final Item TEST_FILE_ITEM = new Item(); - private static DDBPathMetadata testFilePathMetadata; - @BeforeClass - public static void setUpBeforeClass() throws IOException { + private final Item testFileItem; + private final DDBPathMetadata testFilePathMetadata; + private final Item testDirItem; + private final DDBPathMetadata testDirPathMetadata; + + @Parameterized.Parameters + public static Collection params() throws IOException { + String username = UserGroupInformation.getCurrentUser().getShortUserName(); + + return Arrays.asList(new Object[][]{ + // with etag and versionId + { + new Item() + .withPrimaryKey(PARENT, + pathToParentKey(TEST_FILE_PATH.getParent()), + CHILD, TEST_FILE_PATH.getName()) + .withBoolean(IS_DIR, false) + .withLong(FILE_LENGTH, TEST_FILE_LENGTH) + .withLong(MOD_TIME, TEST_MOD_TIME) + .withLong(BLOCK_SIZE, TEST_BLOCK_SIZE) + .withString(ETAG, TEST_ETAG) + .withString(VERSION_ID, TEST_VERSION_ID), + new DDBPathMetadata( + new S3AFileStatus(TEST_FILE_LENGTH, TEST_MOD_TIME, + TEST_FILE_PATH, TEST_BLOCK_SIZE, username, TEST_ETAG, + TEST_VERSION_ID)) + }, + // without etag or versionId + { + new Item() + .withPrimaryKey(PARENT, + pathToParentKey(TEST_FILE_PATH.getParent()), + CHILD, TEST_FILE_PATH.getName()) + .withBoolean(IS_DIR, false) + .withLong(FILE_LENGTH, TEST_FILE_LENGTH) + .withLong(MOD_TIME, TEST_MOD_TIME) + .withLong(BLOCK_SIZE, TEST_BLOCK_SIZE), + new DDBPathMetadata( + new S3AFileStatus(TEST_FILE_LENGTH, TEST_MOD_TIME, + TEST_FILE_PATH, TEST_BLOCK_SIZE, username, null, null)) + } + }); + } + + public TestPathMetadataDynamoDBTranslation(Item item, + DDBPathMetadata metadata) throws IOException { + testFileItem = item; + testFilePathMetadata = metadata; + String username = UserGroupInformation.getCurrentUser().getShortUserName(); testDirPathMetadata = new DDBPathMetadata(new S3AFileStatus(false, TEST_DIR_PATH, username)); - TEST_DIR_ITEM + testDirItem = new Item(); + testDirItem .withPrimaryKey(PARENT, "/test-bucket", CHILD, TEST_DIR_PATH.getName()) .withBoolean(IS_DIR, true); - - testFilePathMetadata = new DDBPathMetadata( - new S3AFileStatus(TEST_FILE_LENGTH, TEST_MOD_TIME, TEST_FILE_PATH, - TEST_BLOCK_SIZE, username, TEST_ETAG, TEST_VERSION_ID)); - - TEST_FILE_ITEM - .withPrimaryKey(PARENT, pathToParentKey(TEST_FILE_PATH.getParent()), - CHILD, TEST_FILE_PATH.getName()) - .withBoolean(IS_DIR, false) - .withLong(FILE_LENGTH, TEST_FILE_LENGTH) - .withLong(MOD_TIME, TEST_MOD_TIME) - .withLong(BLOCK_SIZE, TEST_BLOCK_SIZE) - .withString(ETAG, TEST_ETAG) - .withString(VERSION_ID, TEST_VERSION_ID); } /** @@ -142,8 +173,8 @@ public void testItemToPathMetadata() throws IOException { UserGroupInformation.getCurrentUser().getShortUserName(); assertNull(itemToPathMetadata(null, user)); - verify(TEST_DIR_ITEM, itemToPathMetadata(TEST_DIR_ITEM, user)); - verify(TEST_FILE_ITEM, itemToPathMetadata(TEST_FILE_ITEM, user)); + verify(testDirItem, itemToPathMetadata(testDirItem, user)); + verify(testFileItem, itemToPathMetadata(testFileItem, user)); } /** @@ -260,7 +291,7 @@ public void testVersionMarkerNotStatusIllegalPath() throws Throwable { @Test public void testIsAuthoritativeCompatibilityItemToPathMetadata() throws Exception { - Item item = Mockito.spy(TEST_DIR_ITEM); + Item item = Mockito.spy(testDirItem); item.withBoolean(IS_AUTHORITATIVE, true); PathMetadataDynamoDBTranslation.IGNORED_FIELDS.add(IS_AUTHORITATIVE); @@ -296,7 +327,7 @@ public void testIsAuthoritativeCompatibilityPathMetadataToItem() { @Test public void testIsLastUpdatedCompatibilityItemToPathMetadata() throws Exception { - Item item = Mockito.spy(TEST_DIR_ITEM); + Item item = Mockito.spy(testDirItem); item.withLong(LAST_UPDATED, 100); PathMetadataDynamoDBTranslation.IGNORED_FIELDS.add(LAST_UPDATED); From 34b0c8028ee8a711b1bf15ce127bef58ae231ff1 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Tue, 26 Mar 2019 11:35:34 -0500 Subject: [PATCH 06/28] Fix isEmptyDir inconsistency --- .../java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java index 24d47861ff2e7..cbfafee8321bd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadata.java @@ -34,7 +34,7 @@ @InterfaceStability.Evolving public class PathMetadata extends ExpirableMetadata { - private final S3AFileStatus fileStatus; + private S3AFileStatus fileStatus; private Tristate isEmptyDirectory; private boolean isDeleted; @@ -94,6 +94,10 @@ public Tristate isEmptyDirectory() { void setIsEmptyDirectory(Tristate isEmptyDirectory) { this.isEmptyDirectory = isEmptyDirectory; + if (fileStatus.isDirectory()) { + fileStatus = new S3AFileStatus( + isEmptyDirectory, fileStatus.getPath(), fileStatus.getOwner()); + } } public boolean isDeleted() { From 1e4fa85cc153575d60c25189f7ddeb15e3a44533 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Tue, 26 Mar 2019 11:37:48 -0500 Subject: [PATCH 07/28] Increase local metastore cache timeout --- .../src/main/java/org/apache/hadoop/fs/s3a/Constants.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 e0b1629b40ae8..8e68a3ca41a39 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 @@ -545,7 +545,7 @@ private Constants() { public static final String S3GUARD_METASTORE_LOCAL_ENTRY_TTL = "fs.s3a.s3guard.local.ttl"; public static final int DEFAULT_S3GUARD_METASTORE_LOCAL_ENTRY_TTL - = 10 * 1000; + = 120 * 1000; /** * Use DynamoDB for the metadata: {@value}. From ca2f0e9352da8d11997b9eb56fdebfe52972d4d2 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Tue, 26 Mar 2019 14:29:03 -0500 Subject: [PATCH 08/28] Fix ITestS3ARemoteFileChanged --- .../fs/s3a/ITestS3ARemoteFileChanged.java | 164 ++++++++++++------ 1 file changed, 107 insertions(+), 57 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java index 0976bef3849e4..d6a062244f13d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java @@ -18,12 +18,15 @@ package org.apache.hadoop.fs.s3a; +import java.io.ByteArrayOutputStream; import java.io.FileNotFoundException; import java.io.IOException; +import java.io.PrintWriter; import java.util.Arrays; import java.util.Collection; import org.junit.Assume; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -57,46 +60,84 @@ public class ITestS3ARemoteFileChanged extends AbstractS3ATestBase { private static final Logger LOG = LoggerFactory.getLogger(ITestS3ARemoteFileChanged.class); + private enum InteractionType { + READ, READ_AFTER_DELETE, COPY, SELECT + } + private final String changeDetectionSource; private final String changeDetectionMode; - private final boolean expectChangeException; - private final boolean expectFileNotFoundException; + private final Collection expectedExceptionInteractions; + private S3AFileSystem fs; @Parameterized.Parameters public static Collection params() { return Arrays.asList(new Object[][]{ // make sure it works with invalid config - {"bogus", "bogus", true, true}, + {"bogus", "bogus", + Arrays.asList( + InteractionType.READ, + InteractionType.READ_AFTER_DELETE, + InteractionType.COPY, + InteractionType.SELECT)}, // test with etag - {CHANGE_DETECT_SOURCE_ETAG, CHANGE_DETECT_MODE_SERVER, true, true}, - {CHANGE_DETECT_SOURCE_ETAG, CHANGE_DETECT_MODE_CLIENT, true, true}, - {CHANGE_DETECT_SOURCE_ETAG, CHANGE_DETECT_MODE_WARN, false, true}, - {CHANGE_DETECT_SOURCE_ETAG, CHANGE_DETECT_MODE_NONE, false, true}, + {CHANGE_DETECT_SOURCE_ETAG, CHANGE_DETECT_MODE_SERVER, + Arrays.asList( + InteractionType.READ, + InteractionType.READ_AFTER_DELETE, + InteractionType.COPY, + InteractionType.SELECT)}, + {CHANGE_DETECT_SOURCE_ETAG, CHANGE_DETECT_MODE_CLIENT, + Arrays.asList( + InteractionType.READ, + InteractionType.READ_AFTER_DELETE, + InteractionType.COPY, + InteractionType.SELECT)}, + {CHANGE_DETECT_SOURCE_ETAG, CHANGE_DETECT_MODE_WARN, + Arrays.asList( + InteractionType.READ_AFTER_DELETE)}, + {CHANGE_DETECT_SOURCE_ETAG, CHANGE_DETECT_MODE_NONE, + Arrays.asList( + InteractionType.READ_AFTER_DELETE)}, // test with versionId - // when using server-side versionId, the exceptions shouldn't happen - // since the previous version will still be available - {CHANGE_DETECT_SOURCE_VERSION_ID, CHANGE_DETECT_MODE_SERVER, false, - false}, + // when using server-side versionId, the read exceptions shouldn't + // happen since the previous version will still be available, but + // they will still happen on rename and select since we always do a + // client-side check against the current version + {CHANGE_DETECT_SOURCE_VERSION_ID, CHANGE_DETECT_MODE_SERVER, + Arrays.asList( + InteractionType.COPY, + InteractionType.SELECT)}, // with client-side versionId it will behave similar to client-side eTag - {CHANGE_DETECT_SOURCE_VERSION_ID, CHANGE_DETECT_MODE_CLIENT, true, - true}, - - {CHANGE_DETECT_SOURCE_VERSION_ID, CHANGE_DETECT_MODE_WARN, false, true}, - {CHANGE_DETECT_SOURCE_VERSION_ID, CHANGE_DETECT_MODE_NONE, false, true} + {CHANGE_DETECT_SOURCE_VERSION_ID, CHANGE_DETECT_MODE_CLIENT, + Arrays.asList( + InteractionType.READ, + InteractionType.READ_AFTER_DELETE, + InteractionType.COPY, + InteractionType.SELECT)}, + + {CHANGE_DETECT_SOURCE_VERSION_ID, CHANGE_DETECT_MODE_WARN, + Arrays.asList( + InteractionType.READ_AFTER_DELETE)}, + {CHANGE_DETECT_SOURCE_VERSION_ID, CHANGE_DETECT_MODE_NONE, + Arrays.asList( + InteractionType.READ_AFTER_DELETE)} }); } public ITestS3ARemoteFileChanged(String changeDetectionSource, String changeDetectionMode, - boolean expectException, - boolean expectFileNotFoundException) { + Collection expectedExceptionInteractions) { this.changeDetectionSource = changeDetectionSource; this.changeDetectionMode = changeDetectionMode; - this.expectChangeException = expectException; - this.expectFileNotFoundException = expectFileNotFoundException; + this.expectedExceptionInteractions = expectedExceptionInteractions; + } + + @Before + public void setUp() { + fs = getFileSystem(); } @Override @@ -127,7 +168,6 @@ public void testReadFileChangedStreamOpen() throws Throwable { final byte[] originalDataset = dataset(originalLength, 'a', 32); final int newLength = originalLength + 1; final byte[] newDataset = dataset(newLength, 'A', 32); - final S3AFileSystem fs = getFileSystem(); final Path testpath = path("readFileToChange.txt"); // initial write writeDataset(fs, testpath, originalDataset, originalDataset.length, @@ -168,7 +208,7 @@ public void testReadFileChangedStreamOpen() throws Throwable { // now check seek backward instream.seek(instream.getPos() - 100); - if (expectChangeException) { + if (expectedExceptionInteractions.contains(InteractionType.READ)) { intercept(RemoteFileChangedException.class, "", "read", () -> instream.read()); } else { @@ -180,7 +220,7 @@ public void testReadFileChangedStreamOpen() throws Throwable { // seek backward instream.seek(0); - if (expectChangeException) { + if (expectedExceptionInteractions.contains(InteractionType.READ)) { intercept(RemoteFileChangedException.class, "", "read", () -> instream.read(buf)); intercept(RemoteFileChangedException.class, "", "read", @@ -199,7 +239,8 @@ public void testReadFileChangedStreamOpen() throws Throwable { // seek backward instream.seek(0); - if (expectFileNotFoundException) { + if (expectedExceptionInteractions.contains( + InteractionType.READ_AFTER_DELETE)) { intercept(FileNotFoundException.class, "", "read()", () -> instream.read()); intercept(FileNotFoundException.class, "", "readfully", @@ -213,26 +254,13 @@ public void testReadFileChangedStreamOpen() throws Throwable { /** * Tests reading a file where the version visible in S3 does not match the - * version tracked in the metadata store. This simulates the condition - * of an eventually consistent read-after-overwrite. The metadata store - * will track the new version but reads from S3 may not yet return that - * version. + * version tracked in the metadata store. */ @Test - public void testReadFileChangedNotVisibleInS3() throws Throwable { - // This test is invalid for server-side versionId. Passing a bogus - // versionId as a server-side qualification results in a 400 (Bad Request) - // saying the versionId is invalid. Maybe that's what happens in a - // read-after-overwrite inconsistency scenario? Or maybe if you read - // with versionId server-side qualified then read-after-overwrite - // inconsistency is impossible. - Assume.assumeFalse( - changeDetectionMode.equals(CHANGE_DETECT_MODE_SERVER) && - changeDetectionSource.equals(CHANGE_DETECT_SOURCE_VERSION_ID)); - final S3AFileSystem fs = getFileSystem(); - final Path testpath = writeOutOfSyncFileVersion(fs); + public void testReadFileChangedOutOfSyncMetadata() throws Throwable { + final Path testpath = writeOutOfSyncFileVersion("read.dat"); final FSDataInputStream instream = fs.open(testpath); - if (expectChangeException) { + if (expectedExceptionInteractions.contains(InteractionType.READ)) { intercept(RemoteFileChangedException.class, "", "read()", () -> { instream.read(); @@ -248,38 +276,60 @@ public void testReadFileChangedNotVisibleInS3() throws Throwable { */ @Test public void testSelectChangedFile() throws Throwable { - final S3AFileSystem fs = getFileSystem(); - final Path testpath = writeOutOfSyncFileVersion(fs); - if (expectChangeException) { + final Path testpath = writeOutOfSyncFileVersion("select.dat"); + if (expectedExceptionInteractions.contains(InteractionType.SELECT)) { interceptFuture(RemoteFileChangedException.class, "select", fs.openFile(testpath) .must(SELECT_SQL, "SELECT * FROM S3OBJECT").build()); } else { fs.openFile(testpath) - .must(SELECT_SQL, "SELECT * FROM S3OBJECT").build(); + .must(SELECT_SQL, "SELECT * FROM S3OBJECT").build().get(); + } + } + + /** + * Tests doing a rename() on a file where the version visible in S3 does not + * match the version tracked in the metadata store. + * @throws Throwable + */ + @Test + public void testRenameChangedFile() throws Throwable { + final Path testpath = writeOutOfSyncFileVersion("rename.dat"); + final Path dest = path("dest.dat"); + if (expectedExceptionInteractions.contains(InteractionType.COPY)) { + intercept(RemoteFileChangedException.class, "", "copy()", + () -> { + fs.rename(testpath, dest); + }); + } else { + fs.rename(testpath, dest); } } /** - * Writes a file with bogus ETag and versionId in the metadata store such + * Writes a file with old ETag and versionId in the metadata store such * that the metadata is out of sync with S3. Attempts to read such a file - * should always result in {@link RemoteFileChangedException}. + * should result in {@link RemoteFileChangedException}. */ - private Path writeOutOfSyncFileVersion(S3AFileSystem fs) throws IOException { - final Path testpath = path("outOfSync" + - System.currentTimeMillis() + ".txt"); - final byte[] dataset = dataset(8192, 'a', 32); + private Path writeOutOfSyncFileVersion(String filename) throws IOException { + final Path testpath = path(filename); + ByteArrayOutputStream out = new ByteArrayOutputStream(); + PrintWriter printWriter = new PrintWriter(out); + printWriter.println("Some test data"); + printWriter.close(); + final byte[] dataset = out.toByteArray(); writeDataset(fs, testpath, dataset, dataset.length, 1024, false); - S3AFileStatus fileStatus = (S3AFileStatus) fs.getFileStatus(testpath); - S3AFileStatus newStatus = new S3AFileStatus(fileStatus.getLen(), fileStatus.getModificationTime(), - fileStatus.getAccessTime(), - fileStatus.getPath(), fileStatus.getBlockSize(), fileStatus.getOwner(), fileStatus.getGroup(), - fileStatus.getPermission(), "bogusETag", "bogusVersionId"); + S3AFileStatus originalStatus = (S3AFileStatus) fs.getFileStatus(testpath); + + // overwrite with half the content + writeDataset(fs, testpath, dataset, dataset.length / 2, + 1024, true); + // put back the original metadata (etag, versionId) fs.getMetadataStore().put( - new PathMetadata(newStatus, Tristate.FALSE, false)); + new PathMetadata(originalStatus, Tristate.FALSE, false)); return testpath; } } From d82069bb2b21c2a5df5f04bd8ab7004b239e56da Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Tue, 26 Mar 2019 16:05:01 -0500 Subject: [PATCH 09/28] Improve S3Guard doc --- .../site/markdown/tools/hadoop-aws/s3guard.md | 120 +++++++++++++++++- 1 file changed, 118 insertions(+), 2 deletions(-) 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 bb9483a92c303..9f948ed134ee5 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 @@ -935,8 +935,89 @@ Readers using S3Guard will usually see the new file version, but may in rare cases see `RemoteFileChangedException` instead. This would occur if an S3 object read cannot provide the version tracked in S3Guard metadata. -The configuration controlling this behavior is the same as for handling -[read-during-overwrite](./index.html#Handling_Read-During-Overwrite). +S3Guard achieves this behavior by storing ETags and object version IDs in the +S3Guard metadata store (e.g. DynamoDB). On opening a file, S3AFileSystem +will look in S3 for the version of the file indicated by the ETag or object +version ID stored in the metadata store. If that version is unavailable, +`RemoteFileChangedException` is thrown. Whether ETag or version ID is used is +determed by the +[fs.s3a.change.detection configuration options](./index.html#Handling_Read-During-Overwrite). + +### No Versioning Metadata Available + +When the first S3AFileSystem clients are upgraded to the version of +S3AFileSystem that contains these change tracking features, any existing +S3Guard metadata will not contain ETags or object version IDs. Reads of files +tracked in such S3Guard metadata will access whatever version of the file is +available in S3 at the time of read. Only if the file is subsequently updated +will S3Guard start tracking ETag and object version ID and as such generating +`RemoteFileChangedException` if an inconsistency is detected. + +Similarly, when S3Guard metadata is pruned, S3Guard will no longer be able to +detect an inconsistent read. S3Guard metadata should be retained for at least +as long as the perceived read-after-overwrite eventual consistency window. +That window is expected to be short, but there are no guarantees so it is at the +administrator's discretion to weigh the risk. + +### Known Limitations + +#### S3 Select + +S3 Select does not provide a capability for server-side ETag or object +version ID qualification. Whether fs.s3a.change.detection.mode is client or +server, S3Guard will cause a client-side check of the file version before +opening the file with S3 Select. If the current version does not match the +version tracked in S3Guard, `RemoteFileChangedException` is thrown. + +It is still possible that the S3 Select read will access a different version of +the file, if the visible file version changes between the version check and +the opening of the file. This can happen due to eventual consistency or +an overwrite of the file between the version check and the open of the file. + +#### Rename + +Rename is implemented via copy in S3. With fs.s3a.change.detection.mode=client, +a fully reliable mechansim for ensuring the copied content is the expected +content is not possible. This is the case since there isn't necessarily a way +to know the expected ETag or version ID to appear on the object resulting from +the copy. + +Furthermore, if fs.s3a.change.detection.mode=server and a third-party S3 +implemntation is used that doesn't honor the provided ETag or version ID, +S3AFileSystem and S3Guard cannot detect it. + +In either fs.s3.change.detection.mode=server or client, a client-side check +will be performed before the copy to ensure the current version of the file +matches S3Guard metadata. If not, `RemoteFileChangedException` is thrown. +Similar to as discussed with regard to S3 Select, this is not sufficient to +guarantee that same version is the version copied. + +When fs.s3.change.detection.mode=server, the expected version is also specified +in the underlying S3 CopyObjectRequest. As long as the server honors it, the +copied object will be correct. + +All this said, with the defaults of fs.s3.change.detection.mode=server and +fs.s3.change.detection.source=etag against Amazon's S3, copy should in fact +either copy the expected file version or, in the case of an eventual consistency +anamoly, generate `RemoteFileChangedException`. The same should be true with +fs.s3.change.detection.source=versionid. + +#### Out of Sync Metadata + +The S3Guard version tracking metadata (ETag or object version ID) could become +out of sync with the true current object metadata in S3. For example, S3Guard +is still tracking v1 of some file after v2 has been written. This could occur +for reasons such as a writer writing without utilizing S3Guard and/or +S3AFileSystem or simply due to a write with S3AFileSystem and S3Guard that wrote +successfully to S3, but failed in communication with S3Guard's metadata store +(e.g. DynamoDB). + +If this happens, reads of the affected file(s) will result in +`RemoteFileChangedException` until one of: + +* the S3Guard metadata is corrected out-of-band +* the file is overwritten (causing an S3Guard update) +* the S3Guard metadata is pruned ## Troubleshooting @@ -1036,6 +1117,41 @@ java.io.IOException: Invalid region specified "iceland-2": The region specified in `fs.s3a.s3guard.ddb.region` is invalid. +### Error `RemoteFileChangedException` + +An exception like the following could occur for a couple of reasons: + +* the S3Guard metadata is out of sync with the true S3 metadata. For +example, the S3Guard DynamoDB table is tracking a different ETag than the ETag +shown in the exception. This may suggest the object was updated in S3 without +involvement from S3Guard or there was a transient failure when S3Guard tried to +write to S3. + +* S3 is exhibiting read-after-overwrite eventual consistency. The S3Guard +metadata was updated with a new ETag during a recent write, but the current read +is not seeing that ETag due to S3 eventual consistency. This exception prevents +the reader from an inconsistent read where the reader sees an older version of +the file. + +``` +org.apache.hadoop.fs.s3a.RemoteFileChangedException: open 's3a://my-bucket/test/file.txt': + ETag change reported by S3 while reading at position 0. + Version 4e886e26c072fef250cfaf8037675405 was unavailable + at org.apache.hadoop.fs.s3a.impl.ChangeTracker.processResponse(ChangeTracker.java:167) + at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:207) + at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$lazySeek$1(S3AInputStream.java:355) + at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$2(Invoker.java:195) + at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:109) + at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:265) + at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:322) + at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:261) + at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:193) + at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:215) + at org.apache.hadoop.fs.s3a.S3AInputStream.lazySeek(S3AInputStream.java:348) + at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:381) + at java.io.FilterInputStream.read(FilterInputStream.java:83) +``` + ## Other Topics For details on how to test S3Guard, see [Testing S3Guard](./testing.html#s3guard) From 6110a11d22d9eb07ed7ca065461a5b7e12e0db37 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Wed, 27 Mar 2019 08:28:37 -0500 Subject: [PATCH 10/28] Remove trailing whitespace --- .../org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java index bf596cefcce70..5668636545a7c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java @@ -54,7 +54,7 @@ public abstract class ChangeDetectionPolicy { private final Mode mode; private final boolean requireVersion; - + public abstract String getRevisionId(S3ObjectAttributes s3Attributes); public abstract String getRevisionId(CopyResult copyResult); @@ -360,7 +360,7 @@ public String getRevisionId(ObjectMetadata objectMetadata, String uri) { } return versionId; } - + @Override public String getRevisionId(S3ObjectAttributes s3Attributes) { return s3Attributes.getVersionId(); From 5b7fadbe3f3d10219c0068aff1b44a5e3d088913 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Wed, 27 Mar 2019 09:00:39 -0500 Subject: [PATCH 11/28] Fix checkstyle issues --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 65 ++++++++++--------- .../hadoop/fs/s3a/S3LocatedFileStatus.java | 3 + .../fs/s3a/impl/ChangeDetectionPolicy.java | 3 +- .../fs/s3a/ITestS3ARemoteFileChanged.java | 6 +- .../org/apache/hadoop/fs/s3a/TestListing.java | 1 - .../fs/s3a/TestStreamChangeTracker.java | 2 +- .../hadoop/fs/s3a/s3guard/TestS3Guard.java | 2 +- 7 files changed, 43 insertions(+), 39 deletions(-) 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 291700f93ed64..9bc2215a17136 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 @@ -2975,7 +2975,7 @@ private static CopyOutcome copyOutcome(Copy copy) { } } - private static class CopyOutcome { + private static final class CopyOutcome { private final CopyResult copyResult; private final InterruptedException interruptedException; private final RuntimeException runtimeException; @@ -3609,37 +3609,38 @@ public RemoteIterator listLocatedStatus(final Path f, LOG.debug("listLocatedStatus({}, {}", path, filter); RemoteIterator iterator = once("listLocatedStatus", path.toString(), - () -> { - // lookup dir triggers existence check - final S3AFileStatus fileStatus = (S3AFileStatus) getFileStatus(path); - if (fileStatus.isFile()) { - // simple case: File - LOG.debug("Path is a file"); - return new Listing.SingleStatusRemoteIterator( - filter.accept(path) ? toLocatedFileStatus(fileStatus) : null); - } else { - // directory: trigger a lookup - final String key = maybeAddTrailingSlash(pathToKey(path)); - final Listing.FileStatusAcceptor acceptor = - new Listing.AcceptAllButSelfAndS3nDirs(path); - DirListingMetadata meta = - S3Guard.listChildrenWithTtl(metadataStore, path, - ttlTimeProvider); - final RemoteIterator cachedFileStatusIterator = - listing.createProvidedFileStatusIterator( - S3Guard.dirMetaToStatuses(meta), filter, acceptor); - return (allowAuthoritative && meta != null - && meta.isAuthoritative()) - ? listing.createLocatedFileStatusIterator( - cachedFileStatusIterator) - : listing.createLocatedFileStatusIterator( - listing.createFileStatusListingIterator(path, - createListObjectsRequest(key, "/"), - filter, - acceptor, - cachedFileStatusIterator)); - } - }); + () -> { + // lookup dir triggers existence check + final S3AFileStatus fileStatus = + (S3AFileStatus) getFileStatus(path); + if (fileStatus.isFile()) { + // simple case: File + LOG.debug("Path is a file"); + return new Listing.SingleStatusRemoteIterator( + filter.accept(path) ? toLocatedFileStatus(fileStatus) : null); + } else { + // directory: trigger a lookup + final String key = maybeAddTrailingSlash(pathToKey(path)); + final Listing.FileStatusAcceptor acceptor = + new Listing.AcceptAllButSelfAndS3nDirs(path); + DirListingMetadata meta = + S3Guard.listChildrenWithTtl(metadataStore, path, + ttlTimeProvider); + final RemoteIterator cachedFileStatusIterator = + listing.createProvidedFileStatusIterator( + S3Guard.dirMetaToStatuses(meta), filter, acceptor); + return (allowAuthoritative && meta != null + && meta.isAuthoritative()) + ? listing.createLocatedFileStatusIterator( + cachedFileStatusIterator) + : listing.createLocatedFileStatusIterator( + listing.createFileStatusListingIterator(path, + createListObjectsRequest(key, "/"), + filter, + acceptor, + cachedFileStatusIterator)); + } + }); return toLocatedFileStatusIterator(iterator); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java index 56da324cee9ca..d89fb7f0ac869 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java @@ -21,6 +21,9 @@ import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.LocatedFileStatus; +/** + * {@link LocatedFileStatus} extended to also carry ETag and object version ID. + */ public class S3LocatedFileStatus extends LocatedFileStatus { private final String eTag; private final String versionId; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java index 5668636545a7c..2a78dc6696cad 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java @@ -259,7 +259,8 @@ public ImmutablePair onChangeDetected( if (timesAlreadyDetected == 0) { // only warn on the first detection to avoid a noisy log LOG.warn( - String.format("%s change detected on %s %s at %d. Expected %s got %s", + String.format( + "%s change detected on %s %s at %d. Expected %s got %s", getSource(), operation, uri, position, revisionId, newRevisionId)); return new ImmutablePair<>(true, null); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java index d6a062244f13d..ddb76c1115f45 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java @@ -160,7 +160,8 @@ protected Configuration createConfiguration() { } /** - * Tests reading a file that is changed while the reader's InputStream is open + * Tests reading a file that is changed while the reader's InputStream is + * open. */ @Test public void testReadFileChangedStreamOpen() throws Throwable { @@ -281,8 +282,7 @@ public void testSelectChangedFile() throws Throwable { interceptFuture(RemoteFileChangedException.class, "select", fs.openFile(testpath) .must(SELECT_SQL, "SELECT * FROM S3OBJECT").build()); - } - else { + } else { fs.openFile(testpath) .must(SELECT_SQL, "SELECT * FROM S3OBJECT").build().get(); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java index fb8c202e1c8c0..78e5617b05c92 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestListing.java @@ -19,7 +19,6 @@ package org.apache.hadoop.fs.s3a; import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.junit.Assert; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java index 1a16a9a8fc26d..d5aedc0067088 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java @@ -290,7 +290,7 @@ protected ChangeTracker newTracker(final ChangeDetectionPolicy.Mode mode, new AtomicLong(0), objectAttributes); if (objectAttributes.getVersionId() == null && objectAttributes.getETag() == null) { - assertFalse("Tracker should not have applied constraints " + tracker, + assertFalse("Tracker should not have applied constraints " + tracker, tracker.maybeApplyConstraint(newGetObjectRequest())); } return tracker; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java index 77489833a15fb..0d7c29900b255 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java @@ -99,6 +99,6 @@ private S3AFileStatus makeFileStatus(String pathStr, boolean isDir) { System.currentTimeMillis(), p); } return S3AFileStatus.fromFileStatus(fileStatus, Tristate.UNKNOWN, - null , null); + null, null); } } From 33bb5f9c2b4167db1149057834af2d4dbf8e418a Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Wed, 27 Mar 2019 09:19:32 -0500 Subject: [PATCH 12/28] Fix findbugs issue --- .../org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java index d89fb7f0ac869..322fd25ffee31 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java @@ -42,4 +42,14 @@ public String getETag() { public String getVersionId() { return versionId; } + + @Override + public boolean equals(Object o) { + return super.equals(o); + } + + @Override + public int hashCode() { + return super.hashCode(); + } } From b8e1569b1c30b591459ade7c22d01a5758e7f22d Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Wed, 27 Mar 2019 09:19:46 -0500 Subject: [PATCH 13/28] Fix license issue --- .../hadoop/fs/s3a/s3guard/TestObjectETag.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java index 569bc963e2642..0950d0880301d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.fs.s3a.s3guard; import static org.junit.Assert.assertArrayEquals; From 21d37ddec4554e24640f4aad0f640568bb62ade6 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Wed, 27 Mar 2019 09:25:46 -0500 Subject: [PATCH 14/28] Fix compiler deprecation warning --- .../org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java index 0950d0880301d..a521ee03b80ea 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java @@ -41,6 +41,7 @@ import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; import java.io.ByteArrayInputStream; +import java.nio.charset.Charset; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -90,7 +91,7 @@ public void testCreateAndReadFileSinglePart() throws Exception { .thenReturn(emptyListing); FSDataOutputStream outputStream = fs.create(path); - outputStream.writeChars(content); + outputStream.writeUTF(content); outputStream.close(); // make sure the eTag was put into the metadataStore @@ -111,7 +112,8 @@ public void testCreateAndReadFileSinglePart() throws Exception { when(s3.getObject(argThat(correctGetObjectRequest("file", eTag)))) .thenReturn(s3Object); FSDataInputStream inputStream = fs.open(path); - String readContent = IOUtils.toString(inputStream); + String readContent = IOUtils.toString(inputStream, + Charset.forName("UTF-8")); assertEquals(content, readContent); } From ddbf68bb1ad67c72f87eed29149929af26694377 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Wed, 27 Mar 2019 11:56:38 -0500 Subject: [PATCH 15/28] Add tests of case where no version metadata is present --- .../fs/s3a/ITestS3ARemoteFileChanged.java | 75 ++++++++++++++++++- 1 file changed, 74 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java index ddb76c1115f45..005903390ee53 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java @@ -22,9 +22,11 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.PrintWriter; +import java.nio.charset.Charset; import java.util.Arrays; import java.util.Collection; +import org.apache.commons.io.IOUtils; import org.junit.Assume; import org.junit.Before; import org.junit.Test; @@ -57,9 +59,14 @@ */ @RunWith(Parameterized.class) public class ITestS3ARemoteFileChanged extends AbstractS3ATestBase { + private static final Logger LOG = LoggerFactory.getLogger(ITestS3ARemoteFileChanged.class); + private static final String TEST_DATA = "Some test data"; + private static final String QUOTED_TEST_DATA = + "\"" + TEST_DATA + "\""; + private enum InteractionType { READ, READ_AFTER_DELETE, COPY, SELECT } @@ -271,6 +278,18 @@ public void testReadFileChangedOutOfSyncMetadata() throws Throwable { } } + /** + * Ensures a file can be read when there is no version metadata + * (ETag, versionId). + */ + @Test + public void testReadWithNoVersionMetadata() throws Throwable { + final Path testpath = writeFileWithNoVersionMetadata("readnoversion.dat"); + final FSDataInputStream instream = fs.open(testpath); + assertEquals(TEST_DATA, + IOUtils.toString(instream, Charset.forName("UTF-8")).trim()); + } + /** * Tests using S3 Select on a file where the version visible in S3 does not * match the version tracked in the metadata store. @@ -288,6 +307,20 @@ public void testSelectChangedFile() throws Throwable { } } + /** + * Ensures a file can be read via S3 Select when there is no version metadata + * (ETag, versionId). + */ + @Test + public void testSelectWithNoVersionMetadata() throws Throwable { + final Path testpath = + writeFileWithNoVersionMetadata("selectnoversion.dat"); + FSDataInputStream instream = fs.openFile(testpath) + .must(SELECT_SQL, "SELECT * FROM S3OBJECT").build().get(); + assertEquals(QUOTED_TEST_DATA, + IOUtils.toString(instream, Charset.forName("UTF-8")).trim()); + } + /** * Tests doing a rename() on a file where the version visible in S3 does not * match the version tracked in the metadata store. @@ -307,6 +340,21 @@ public void testRenameChangedFile() throws Throwable { } } + /** + * Ensures a file can be renamed when there is no version metadata + * (ETag, versionId). + */ + @Test + public void testRenameWithNoVersionMetadata() throws Throwable { + final Path testpath = + writeFileWithNoVersionMetadata("renamenoversion.dat"); + final Path dest = path("noversiondest.dat"); + fs.rename(testpath, dest); + FSDataInputStream inputStream = fs.open(dest); + assertEquals(TEST_DATA, + IOUtils.toString(inputStream, Charset.forName("UTF-8")).trim()); + } + /** * Writes a file with old ETag and versionId in the metadata store such * that the metadata is out of sync with S3. Attempts to read such a file @@ -316,7 +364,7 @@ private Path writeOutOfSyncFileVersion(String filename) throws IOException { final Path testpath = path(filename); ByteArrayOutputStream out = new ByteArrayOutputStream(); PrintWriter printWriter = new PrintWriter(out); - printWriter.println("Some test data"); + printWriter.println(TEST_DATA); printWriter.close(); final byte[] dataset = out.toByteArray(); writeDataset(fs, testpath, dataset, dataset.length, @@ -330,6 +378,31 @@ private Path writeOutOfSyncFileVersion(String filename) throws IOException { // put back the original metadata (etag, versionId) fs.getMetadataStore().put( new PathMetadata(originalStatus, Tristate.FALSE, false)); + + return testpath; + } + + /** + * Writes a file with null ETag and versionId in the metadata store. + */ + private Path writeFileWithNoVersionMetadata(String filename) + throws IOException { + final Path testpath = path(filename); + ByteArrayOutputStream out = new ByteArrayOutputStream(); + PrintWriter printWriter = new PrintWriter(out); + printWriter.println(TEST_DATA); + printWriter.close(); + final byte[] dataset = out.toByteArray(); + writeDataset(fs, testpath, dataset, dataset.length, + 1024, false); + S3AFileStatus originalStatus = (S3AFileStatus) fs.getFileStatus(testpath); + + // remove ETag and versionId + S3AFileStatus newStatus = S3AFileStatus.fromFileStatus(originalStatus, + Tristate.FALSE, null, null); + fs.getMetadataStore().put(new PathMetadata(newStatus, Tristate.FALSE, + false)); + return testpath; } } From 3e9ea192305603d5a4bc096d04aa9b7fad3f9421 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Wed, 27 Mar 2019 13:15:58 -0500 Subject: [PATCH 16/28] Skip tests that aren't applicable with change.detection.source=versionId --- .../org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java | 9 +++++++++ .../apache/hadoop/fs/s3a/ITestS3AInconsistency.java | 9 +++++++++ .../hadoop/fs/s3a/select/ITestS3SelectMRJob.java | 10 ++++++++++ 3 files changed, 28 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java index 7abd47497646e..94c5233f19a15 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java @@ -22,7 +22,10 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source; import org.apache.hadoop.test.LambdaTestUtils; +import org.junit.Assume; import org.junit.Test; import java.io.FileNotFoundException; @@ -43,6 +46,12 @@ public class ITestS3ADelayedFNF extends AbstractS3ATestBase { @Test public void testNotFoundFirstRead() throws Exception { FileSystem fs = getFileSystem(); + ChangeDetectionPolicy changeDetectionPolicy = + ((S3AFileSystem) fs).getChangeDetectionPolicy(); + Assume.assumeFalse("FNF not expected when using a bucket with" + + " object versioning", + changeDetectionPolicy.getSource() == Source.VersionId); + Path p = path("some-file"); ContractTestUtils.createFile(fs, p, false, new byte[] {20, 21, 22}); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInconsistency.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInconsistency.java index 6ac803e3085eb..c82a8b9789486 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInconsistency.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInconsistency.java @@ -24,9 +24,12 @@ import org.apache.hadoop.fs.contract.AbstractFSContract; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.contract.s3a.S3AContract; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore; import org.apache.hadoop.test.LambdaTestUtils; +import org.junit.Assume; import org.junit.Test; import java.io.FileNotFoundException; @@ -106,6 +109,12 @@ public void testGetFileStatus() throws Exception { @Test public void testOpenDeleteRead() throws Exception { S3AFileSystem fs = getFileSystem(); + ChangeDetectionPolicy changeDetectionPolicy = + ((S3AFileSystem) fs).getChangeDetectionPolicy(); + Assume.assumeFalse("FNF not expected when using a bucket with" + + " object versioning", + changeDetectionPolicy.getSource() == Source.VersionId); + Path p = path("testOpenDeleteRead.txt"); writeTextFile(fs, p, "1337c0d3z", true); try (InputStream s = fs.open(p)) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java index ee7de8c7ac2f2..181d797767397 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java @@ -21,6 +21,9 @@ import java.io.IOException; import java.util.concurrent.atomic.AtomicLong; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source; +import org.junit.Assume; import org.junit.Test; import org.apache.hadoop.conf.Configuration; @@ -90,6 +93,13 @@ public class ITestS3SelectMRJob extends AbstractS3SelectTest { public void setup() throws Exception { super.setup(); fs = S3ATestUtils.createTestFileSystem(conf); + + ChangeDetectionPolicy changeDetectionPolicy = + getLandsatFS().getChangeDetectionPolicy(); + Assume.assumeFalse("the standard landsat bucket doesn't have versioning", + changeDetectionPolicy.getSource() == Source.VersionId + && changeDetectionPolicy.isRequireVersion()); + rootPath = path("ITestS3SelectMRJob"); Path workingDir = path("working"); fs.setWorkingDirectory(workingDir); From e296275971f3b7f1f74fd646b10f50a20a8673fd Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Wed, 27 Mar 2019 16:04:50 -0500 Subject: [PATCH 17/28] Minor javadoc improvements from PR review --- .../apache/hadoop/fs/s3a/S3AFileStatus.java | 4 ++- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 4 ++- .../hadoop/fs/s3a/S3LocatedFileStatus.java | 3 ++ .../fs/s3a/impl/ChangeDetectionPolicy.java | 34 ++++++++++++++----- .../hadoop/fs/s3a/impl/ChangeTracker.java | 2 +- 5 files changed, 36 insertions(+), 11 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java index 469ea1a1c9fe8..41df62d7aaae9 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java @@ -67,6 +67,8 @@ public S3AFileStatus(Tristate isemptydir, * @param owner the owner * @param group the group * @param modification_time the modification time + * @param access_time the access time + * @param permission the permission */ public S3AFileStatus(Tristate isemptydir, Path path, @@ -112,7 +114,7 @@ public S3AFileStatus(long length, long modification_time, Path path, * @param blockSize block size * @param owner owner * @param group group - * @param permission persmission + * @param permission permission * @param eTag eTag of the S3 object if available, else null * @param versionId versionId of the S3 object if available, else null */ 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 9bc2215a17136..8269e067c2fcd 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 @@ -694,7 +694,8 @@ public S3AInputPolicy getInputPolicy() { } /** - * Get the change detection policy for this FS instance. + * Get the change detection policy for this FS instance. Only public to allow + * access in tests in other packages. * @return the change detection policy */ @VisibleForTesting @@ -2895,6 +2896,7 @@ public List listAWSPolicyRules( * @param size object size * @param srcAttributes S3 attributes of the source object * @param readContext the read context + * @return the result of the copy * @throws AmazonClientException on failures inside the AWS SDK * @throws InterruptedIOException the operation was interrupted * @throws IOException Other IO problems diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java index 322fd25ffee31..4cd0cfcf9bebd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3LocatedFileStatus.java @@ -43,6 +43,9 @@ public String getVersionId() { return versionId; } + // equals() and hashCode() overridden to avoid FindBugs warning. + // Base implementation is equality on Path only, which is still appropriate. + @Override public boolean equals(Object o) { return super.equals(o); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java index 2a78dc6696cad..7972183796aa3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java @@ -55,10 +55,6 @@ public abstract class ChangeDetectionPolicy { private final Mode mode; private final boolean requireVersion; - public abstract String getRevisionId(S3ObjectAttributes s3Attributes); - - public abstract String getRevisionId(CopyResult copyResult); - /** * Version support is only warned about once per S3A instance. * This still means that on a long-lived application which destroys @@ -207,6 +203,28 @@ public static ChangeDetectionPolicy createPolicy(final Mode mode, public abstract String getRevisionId(ObjectMetadata objectMetadata, String uri); + /** + * Like {{@link #getRevisionId(ObjectMetadata, String)}}, but retrieves the + * revision identifier from {@link S3ObjectAttributes}. + * + * @param s3Attributes the object attributes + * @return the revisionId string as interpreted by this policy, or potentially + * null if the attribute is unavailable (such as when the policy says to use + * versionId but object versioning is not enabled for the bucket). + */ + public abstract String getRevisionId(S3ObjectAttributes s3Attributes); + + /** + * Like {{@link #getRevisionId(ObjectMetadata, String)}}, but retrieves the + * revision identifier from {@link CopyResult}. + * + * @param copyResult the copy result + * @return the revisionId string as interpreted by this policy, or potentially + * null if the attribute is unavailable (such as when the policy says to use + * versionId but object versioning is not enabled for the bucket). + */ + public abstract String getRevisionId(CopyResult copyResult); + /** * Applies the given {@link #getRevisionId(ObjectMetadata, String) revisionId} * as a server-side qualification on the {@code GetObjectRequest}. @@ -309,7 +327,7 @@ public String getRevisionId(CopyResult copyResult) { public void applyRevisionConstraint(GetObjectRequest request, String revisionId) { if (revisionId != null) { - LOG.debug("Restricting request to etag {}", revisionId); + LOG.debug("Restricting get request to etag {}", revisionId); request.withMatchingETagConstraint(revisionId); } } @@ -318,7 +336,7 @@ public void applyRevisionConstraint(GetObjectRequest request, public void applyRevisionConstraint(CopyObjectRequest request, String revisionId) { if (revisionId != null) { - LOG.debug("Restricting request to etag {}", revisionId); + LOG.debug("Restricting copy request to etag {}", revisionId); request.withMatchingETagConstraint(revisionId); } } @@ -376,7 +394,7 @@ public String getRevisionId(CopyResult copyResult) { public void applyRevisionConstraint(GetObjectRequest request, String revisionId) { if (revisionId != null) { - LOG.debug("Restricting request to version {}", revisionId); + LOG.debug("Restricting get request to version {}", revisionId); request.withVersionId(revisionId); } } @@ -385,7 +403,7 @@ public void applyRevisionConstraint(GetObjectRequest request, public void applyRevisionConstraint(CopyObjectRequest request, String revisionId) { if (revisionId != null) { - LOG.debug("Restricting request to version {}", revisionId); + LOG.debug("Restricting copy request to version {}", revisionId); request.withSourceVersionId(revisionId); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java index b8f217ac7a94a..2d45ac394eee6 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java @@ -202,7 +202,7 @@ public void processResponse(final CopyResult copyResult) * cause. * @param e the exception * @param operation the operation performed when the exception was - * generated. + * generated (e.g. "copy", "read", "select"). * @throws RemoteFileChangedException if the remote file has changed. */ public void processException(Exception e, String operation) throws From 1ff8bef49f86c863300166d26a331ab8817f90ed Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Wed, 27 Mar 2019 16:05:59 -0500 Subject: [PATCH 18/28] check version.required on CopyResult --- .../hadoop/fs/s3a/impl/ChangeTracker.java | 10 +++- .../fs/s3a/TestStreamChangeTracker.java | 60 +++++++++++++++++++ 2 files changed, 69 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java index 2d45ac394eee6..13a9d18b2e8f9 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java @@ -192,7 +192,15 @@ public void processResponse(final CopyResult copyResult) // ETag (sometimes, depending on encryption and/or multipart) is not the // same on the copied object as the original. Version Id seems to never // be the same on the copy. As such, there isn't really anything that - // can be verified on the response. + // can be verified on the response, except that a revision ID is present + // if required. + String newRevisionId = policy.getRevisionId(copyResult); + LOG.debug("Copy result {}: {}", policy.getSource(), newRevisionId); + if (newRevisionId == null && policy.isRequireVersion()) { + throw new NoVersionAttributeException(uri, String.format( + "Change detection policy requires %s", + policy.getSource())); + } } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java index d5aedc0067088..e5090653495f2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java @@ -25,6 +25,7 @@ import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.S3Object; +import com.amazonaws.services.s3.transfer.model.CopyResult; import org.apache.hadoop.fs.PathIOException; import org.junit.Test; import org.slf4j.Logger; @@ -207,6 +208,32 @@ public void testVersionCheckingETagCopyClient() throws Throwable { tracker.maybeApplyConstraint(newCopyObjectRequest())); } + @Test + public void testCopyVersionIdRequired() throws Throwable { + ChangeTracker tracker = newTracker( + ChangeDetectionPolicy.Mode.Client, + ChangeDetectionPolicy.Source.VersionId, + true, + objectAttributes("etag1", "versionId")); + + expectNoVersionAttributeException(tracker, newCopyResult("etag1", + null), + "policy requires VersionId"); + } + + @Test + public void testCopyETagRequired() throws Throwable { + ChangeTracker tracker = newTracker( + ChangeDetectionPolicy.Mode.Client, + ChangeDetectionPolicy.Source.ETag, + true, + objectAttributes("etag1", "versionId")); + + expectNoVersionAttributeException(tracker, newCopyResult(null, + "versionId"), + "policy requires ETag"); + } + protected void assertConstraintApplied(final ChangeTracker tracker, final GetObjectRequest request) { assertTrue("Tracker should have applied contraints " + tracker, @@ -235,6 +262,14 @@ protected PathIOException expectNoVersionAttributeException( NoVersionAttributeException.class); } + protected PathIOException expectNoVersionAttributeException( + final ChangeTracker tracker, + final CopyResult response, + final String message) throws Exception { + return expectException(tracker, response, message, + NoVersionAttributeException.class); + } + protected T expectException( final ChangeTracker tracker, final S3Object response, @@ -249,6 +284,20 @@ protected T expectException( }); } + protected T expectException( + final ChangeTracker tracker, + final CopyResult response, + final String message, + final Class clazz) throws Exception { + return intercept( + clazz, + message, + () -> { + tracker.processResponse(response); + return tracker; + }); + } + protected void assertRevisionId(final ChangeTracker tracker, final String revId) { assertEquals("Wrong revision ID in " + tracker, @@ -304,6 +353,17 @@ private CopyObjectRequest newCopyObjectRequest() { return new CopyObjectRequest(BUCKET, OBJECT, BUCKET, DEST_OBJECT); } + private CopyResult newCopyResult(String eTag, String versionId) { + CopyResult copyResult = new CopyResult(); + copyResult.setSourceBucketName(BUCKET); + copyResult.setSourceKey(OBJECT); + copyResult.setDestinationBucketName(BUCKET); + copyResult.setDestinationKey(DEST_OBJECT); + copyResult.setETag(eTag); + copyResult.setVersionId(versionId); + return copyResult; + } + private S3Object newResponse(String etag, String versionId) { ObjectMetadata md = new ObjectMetadata(); if (etag != null) { From 6e62a3aed4ef316d7015757ab2cc73070e1b470f Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Wed, 27 Mar 2019 21:17:48 -0500 Subject: [PATCH 19/28] Documentation updates per PR feedback --- .../site/markdown/tools/hadoop-aws/s3guard.md | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) 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 9f948ed134ee5..e1e504daa7ab6 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 @@ -926,10 +926,11 @@ some throttling, but not to time out other applications. ## Read-After-Overwrite Consistency S3Guard provides read-after-overwrite consistency through ETags (default) or -object versioning. This works such that a reader reading a file after an -overwrite either sees the new version of the file or an error. Without S3Guard, -new readers may see the original version. Once S3 reaches eventual consistency, -new readers will see the new version. +object versioning checked either on the server (default) or client. This works +such that a reader reading a file after an overwrite either sees the new version +of the file or an error. Without S3Guard, new readers may see the original +version. Once S3 reaches eventual consistency, new readers will see the new +version. Readers using S3Guard will usually see the new file version, but may in rare cases see `RemoteFileChangedException` instead. This would occur if @@ -939,8 +940,8 @@ S3Guard achieves this behavior by storing ETags and object version IDs in the S3Guard metadata store (e.g. DynamoDB). On opening a file, S3AFileSystem will look in S3 for the version of the file indicated by the ETag or object version ID stored in the metadata store. If that version is unavailable, -`RemoteFileChangedException` is thrown. Whether ETag or version ID is used is -determed by the +`RemoteFileChangedException` is thrown. Whether ETag or version ID and +server or client mode is used is determed by the [fs.s3a.change.detection configuration options](./index.html#Handling_Read-During-Overwrite). ### No Versioning Metadata Available @@ -955,9 +956,9 @@ will S3Guard start tracking ETag and object version ID and as such generating Similarly, when S3Guard metadata is pruned, S3Guard will no longer be able to detect an inconsistent read. S3Guard metadata should be retained for at least -as long as the perceived read-after-overwrite eventual consistency window. -That window is expected to be short, but there are no guarantees so it is at the -administrator's discretion to weigh the risk. +as long as the perceived possible read-after-overwrite temporary inconsistency +window. That window is expected to be short, but there are no guarantees so it +is at the administrator's discretion to weigh the risk. ### Known Limitations @@ -999,7 +1000,7 @@ copied object will be correct. All this said, with the defaults of fs.s3.change.detection.mode=server and fs.s3.change.detection.source=etag against Amazon's S3, copy should in fact either copy the expected file version or, in the case of an eventual consistency -anamoly, generate `RemoteFileChangedException`. The same should be true with +anomaly, generate `RemoteFileChangedException`. The same should be true with fs.s3.change.detection.source=versionid. #### Out of Sync Metadata @@ -1016,7 +1017,7 @@ If this happens, reads of the affected file(s) will result in `RemoteFileChangedException` until one of: * the S3Guard metadata is corrected out-of-band -* the file is overwritten (causing an S3Guard update) +* the file is overwritten (causing an S3Guard metadata update) * the S3Guard metadata is pruned ## Troubleshooting @@ -1125,9 +1126,9 @@ An exception like the following could occur for a couple of reasons: example, the S3Guard DynamoDB table is tracking a different ETag than the ETag shown in the exception. This may suggest the object was updated in S3 without involvement from S3Guard or there was a transient failure when S3Guard tried to -write to S3. +write to DynamoDB. -* S3 is exhibiting read-after-overwrite eventual consistency. The S3Guard +* S3 is exhibiting read-after-overwrite temporary inconsistency. The S3Guard metadata was updated with a new ETag during a recent write, but the current read is not seeing that ETag due to S3 eventual consistency. This exception prevents the reader from an inconsistent read where the reader sees an older version of From 2a2bba772c830411de554ccd1f39a4779273142d Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Wed, 27 Mar 2019 21:27:27 -0500 Subject: [PATCH 20/28] Clarify log message --- .../org/apache/hadoop/fs/s3a/impl/ChangeTracker.java | 9 ++++----- .../src/site/markdown/tools/hadoop-aws/s3guard.md | 4 ++-- .../markdown/tools/hadoop-aws/troubleshooting_s3a.md | 4 ++-- 3 files changed, 8 insertions(+), 9 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java index 13a9d18b2e8f9..7521fb55571ee 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java @@ -54,7 +54,7 @@ public class ChangeTracker { private static final Logger LOG = LoggerFactory.getLogger(ChangeTracker.class); - public static final String CHANGE_REPORTED_BY_S3 = "reported by S3"; + public static final String CHANGE_REPORTED_BY_S3 = "Change reported by S3"; /** Policy to use. */ private final ChangeDetectionPolicy policy; @@ -164,13 +164,12 @@ public void processResponse(final S3Object object, // object was not returned. versionMismatches.incrementAndGet(); throw new RemoteFileChangedException(uri, operation, - String.format("%s change " - + CHANGE_REPORTED_BY_S3 + String.format(CHANGE_REPORTED_BY_S3 + " while reading" + " at position %s." - + " Version %s was unavailable", - getSource(), + + " %s %s was unavailable", pos, + getSource(), getRevisionId())); } else { throw new PathIOException(uri, "No data returned from GET request"); 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 e1e504daa7ab6..a9dbc1a074d9f 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 @@ -1136,8 +1136,8 @@ the file. ``` org.apache.hadoop.fs.s3a.RemoteFileChangedException: open 's3a://my-bucket/test/file.txt': - ETag change reported by S3 while reading at position 0. - Version 4e886e26c072fef250cfaf8037675405 was unavailable + Change reported by S3 while reading at position 0. + ETag 4e886e26c072fef250cfaf8037675405 was unavailable at org.apache.hadoop.fs.s3a.impl.ChangeTracker.processResponse(ChangeTracker.java:167) at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:207) at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$lazySeek$1(S3AInputStream.java:355) diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md index 3123221bd8293..8cdac9e35263f 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md @@ -970,8 +970,8 @@ and the like. The standard strategy here is to save to HDFS and then copy to S3. ``` org.apache.hadoop.fs.s3a.RemoteFileChangedException: re-open `s3a://my-bucket/test/file.txt': - ETag change reported by S3 while reading at position 1949. - Version f9c186d787d4de9657e99f280ba26555 was unavailable + Change reported by S3 while reading at position 1949. + ETag f9c186d787d4de9657e99f280ba26555 was unavailable at org.apache.hadoop.fs.s3a.impl.ChangeTracker.processResponse(ChangeTracker.java:137) at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:200) at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$lazySeek$1(S3AInputStream.java:346) From 324be6d2b09521292df74a581bf38d27b58d9ace Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Wed, 27 Mar 2019 22:00:57 -0500 Subject: [PATCH 21/28] S3GuardTool updates to correct ETag or versionId metadata --- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 6 +-- .../site/markdown/tools/hadoop-aws/s3guard.md | 8 ++++ .../fs/s3a/s3guard/ITestS3GuardToolLocal.java | 37 +++++++++++++++++++ 3 files changed, 48 insertions(+), 3 deletions(-) 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 f1f65cc582d58..9e4871659b023 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 @@ -724,7 +724,7 @@ private long importDir(FileStatus status) throws IOException { long items = 0; while (it.hasNext()) { - LocatedFileStatus located = it.next(); + S3LocatedFileStatus located = it.next(); S3AFileStatus child; if (located.isDirectory()) { child = DynamoDBMetadataStore.makeDirStatus(located.getPath(), @@ -736,8 +736,8 @@ private long importDir(FileStatus status) throws IOException { located.getPath(), located.getBlockSize(), located.getOwner(), - null, - null); + located.getETag(), + located.getVersionId()); } putParentsIfNotPresent(child); getStore().put(new PathMetadata(child)); 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 a9dbc1a074d9f..ec96876e1489d 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 @@ -1020,6 +1020,14 @@ If this happens, reads of the affected file(s) will result in * the file is overwritten (causing an S3Guard metadata update) * the S3Guard metadata is pruned +The S3Guard metadata for a file can be corrected with the `s3guard import` +command as discussed above. The command can take a file URI instead of a +bucket URI to correct the metdata for a single file. For example: + +```bash +hadoop s3guard import [-meta URI] s3a://my-bucket/file-with-bad-metadata +``` + ## Troubleshooting ### Error: `S3Guard table lacks version marker.` diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java index 6a4d45e9ea170..f81f0e2bc13b7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java @@ -37,7 +37,9 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.Tristate; import static org.apache.hadoop.fs.s3a.MultipartTestUtils.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getLandsatCSVFile; @@ -95,6 +97,41 @@ public void testImportCommand() throws Exception { // assertTrue(children.isAuthoritative()); } + @Test + public void testImportCommandRepairsETagAndVersionId() throws Exception { + S3AFileSystem fs = getFileSystem(); + MetadataStore ms = getMetadataStore(); + Path path = path("test-version-metadata"); + try (FSDataOutputStream out = fs.create(path)) { + out.write(1); + } + S3AFileStatus originalStatus = (S3AFileStatus) fs.getFileStatus(path); + + // put in bogus ETag and versionId + S3AFileStatus bogusStatus = S3AFileStatus.fromFileStatus(originalStatus, + Tristate.FALSE, "bogusETag", "bogusVersionId"); + ms.put(new PathMetadata(bogusStatus)); + + // sanity check that bogus status is actually persisted + S3AFileStatus retrievedBogusStatus = (S3AFileStatus) fs.getFileStatus(path); + assertEquals("bogus ETag was not persisted", + "bogusETag", retrievedBogusStatus.getETag()); + assertEquals("bogus versionId was not persisted", + "bogusVersionId", retrievedBogusStatus.getVersionId()); + + // execute the import + S3GuardTool.Import cmd = new S3GuardTool.Import(fs.getConf()); + cmd.setStore(ms); + exec(cmd, "import", path.toString()); + + // make sure ETag and versionId were corrected + S3AFileStatus updatedStatus = (S3AFileStatus) fs.getFileStatus(path); + assertEquals("ETag was not corrected", + originalStatus.getETag(), updatedStatus.getETag()); + assertEquals("VersionId was not corrected", + originalStatus.getVersionId(), updatedStatus.getVersionId()); + } + @Test public void testDestroyBucketExistsButNoTable() throws Throwable { run(Destroy.NAME, From 0d71f327f17ec74c5d0f1d867948f4a6bad35743 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Thu, 28 Mar 2019 08:45:42 -0500 Subject: [PATCH 22/28] Fix trailing whitespace --- .../hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 ec96876e1489d..31ea3e3ddb476 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 @@ -940,7 +940,7 @@ S3Guard achieves this behavior by storing ETags and object version IDs in the S3Guard metadata store (e.g. DynamoDB). On opening a file, S3AFileSystem will look in S3 for the version of the file indicated by the ETag or object version ID stored in the metadata store. If that version is unavailable, -`RemoteFileChangedException` is thrown. Whether ETag or version ID and +`RemoteFileChangedException` is thrown. Whether ETag or version ID and server or client mode is used is determed by the [fs.s3a.change.detection configuration options](./index.html#Handling_Read-During-Overwrite). From dc83cef506975f541ecfa1e9ddc0837dc8ed3165 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Thu, 28 Mar 2019 10:28:09 -0500 Subject: [PATCH 23/28] Generalize TestObjectETag to cover versionId and test overwrite --- ... TestObjectChangeDetectionAttributes.java} | 217 +++++++++++------- 1 file changed, 132 insertions(+), 85 deletions(-) rename hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/{TestObjectETag.java => TestObjectChangeDetectionAttributes.java} (66%) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectChangeDetectionAttributes.java similarity index 66% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectChangeDetectionAttributes.java index a521ee03b80ea..615e90d3ce0e6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectETag.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestObjectChangeDetectionAttributes.java @@ -41,29 +41,33 @@ import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; import java.io.ByteArrayInputStream; -import java.nio.charset.Charset; +import java.io.IOException; +import java.util.Arrays; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.AbstractS3AMockTest; import org.apache.hadoop.fs.s3a.Constants; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.hamcrest.Matcher; -import org.junit.Assume; import org.junit.Before; import org.junit.Test; /** - * Tests to ensure eTag is captured on S3 PUT and used on GET. + * Tests to ensure object eTag and versionId are captured on S3 PUT and used on + * GET. */ -public class TestObjectETag extends AbstractS3AMockTest { +public class TestObjectChangeDetectionAttributes extends AbstractS3AMockTest { + private ChangeDetectionPolicy changeDetectionPolicy; + @Before public void before() { - Assume.assumeTrue("change detection source should be etag", - fs.getChangeDetectionPolicy().getSource() == Source.ETag); + changeDetectionPolicy = fs.getChangeDetectionPolicy(); } /** @@ -72,64 +76,94 @@ public void before() { */ @Test public void testCreateAndReadFileSinglePart() throws Exception { - Path path = new Path("s3a://mock-bucket/file"); - String content = "content"; + String bucket = "s3a://mock-bucket/"; + String file = "single-part-file"; + Path path = new Path(bucket, file); + byte[] content = "content".getBytes(); + String eTag = "abc"; + String versionId = "def"; + + putObject(file, path, content, eTag, versionId); + + // make sure the eTag and versionId were put into the metadataStore + assertVersionAttributes(path, eTag, versionId); + + // Ensure underlying S3 getObject call uses the stored eTag or versionId + // when reading data back. If it doesn't, the read won't work and the + // assert will fail. + assertContent(file, path, content, eTag, versionId); + + // test overwrite + byte[] newConent = "newcontent".getBytes(); + String newETag = "newETag"; + String newVersionId = "newVersionId"; + + putObject(file, path, newConent, newETag, newVersionId); + assertVersionAttributes(path, newETag, newVersionId); + assertContent(file, path, newConent, newETag, newVersionId); + } + + /** + * Tests a file uploaded with multi-part upload to ensure eTag is captured + * and used on file read. + */ + @Test + public void testCreateAndReadFileMultiPart() throws Exception { + String bucket = "s3a://mock-bucket/"; + String file = "multi-part-file"; + Path path = new Path(bucket, file); + byte[] content = new byte[Constants.MULTIPART_MIN_SIZE + 1]; + String eTag = "abc"; + String versionId = "def"; + multipartUpload(file, path, content, eTag, versionId); + + // make sure the eTag and versionId were put into the metadataStore + assertVersionAttributes(path, eTag, versionId); + + // Ensure underlying S3 getObject call uses the stored eTag or versionId + // when reading data back. If it doesn't, the read won't work and the + // assert will fail. + assertContent(file, path, content, eTag, versionId); + + // test overwrite + byte[] newConent = new byte[Constants.MULTIPART_MIN_SIZE + 1]; + Arrays.fill(newConent, (byte) 1); + String newETag = "newETag"; + String newVersionId = "newVersionId"; + + multipartUpload(file, path, newConent, newETag, newVersionId); + assertVersionAttributes(path, newETag, newVersionId); + assertContent(file, path, newConent, newETag, newVersionId); + } + + private void putObject(String file, Path path, byte[] content, + String eTag, String versionId) throws IOException { PutObjectResult putObjectResult = new PutObjectResult(); ObjectMetadata objectMetadata = new ObjectMetadata(); - objectMetadata.setContentLength(content.length()); + objectMetadata.setContentLength(content.length); putObjectResult.setMetadata(objectMetadata); - String eTag = "abc"; putObjectResult.setETag(eTag); + putObjectResult.setVersionId(versionId); when(s3.getObjectMetadata(any(GetObjectMetadataRequest.class))) .thenThrow(NOT_FOUND); - when(s3.putObject(argThat(correctPutObjectRequest("file")))) + when(s3.putObject(argThat(correctPutObjectRequest(file)))) .thenReturn(putObjectResult); ListObjectsV2Result emptyListing = new ListObjectsV2Result(); - when(s3.listObjectsV2(argThat(correctListObjectsRequest("file/")))) + when(s3.listObjectsV2(argThat(correctListObjectsRequest(file + "/")))) .thenReturn(emptyListing); FSDataOutputStream outputStream = fs.create(path); - outputStream.writeUTF(content); + outputStream.write(content); outputStream.close(); - - // make sure the eTag was put into the metadataStore - MetadataStore metadataStore = fs.getMetadataStore(); - PathMetadata pathMetadata = metadataStore.get(path); - assertNotNull(pathMetadata); - String storedETag = pathMetadata.getFileStatus().getETag(); - assertEquals(eTag, storedETag); - - // Ensure underlying S3 getObject call uses the stored eTag when reading - // data back. If it doesn't, the read won't work and the assert will - // fail. - S3Object s3Object = new S3Object(); - s3Object.setObjectContent(new ByteArrayInputStream(content.getBytes())); - ObjectMetadata metadata = new ObjectMetadata(); - metadata.setHeader(Headers.ETAG, eTag); - s3Object.setObjectMetadata(metadata); - when(s3.getObject(argThat(correctGetObjectRequest("file", eTag)))) - .thenReturn(s3Object); - FSDataInputStream inputStream = fs.open(path); - String readContent = IOUtils.toString(inputStream, - Charset.forName("UTF-8")); - assertEquals(content, readContent); } - /** - * Tests a file uploaded with multi-part upload to ensure eTag is captured - * and used on file read. - */ - @Test - public void testCreateAndReadFileMultiPart() throws Exception { - Path path = new Path("s3a://mock-bucket/file"); - byte[] content = new byte[Constants.MULTIPART_MIN_SIZE + 1]; - + private void multipartUpload(String file, Path path, byte[] content, + String eTag, String versionId) throws IOException { CompleteMultipartUploadResult uploadResult = new CompleteMultipartUploadResult(); - String eTag = "abc"; - uploadResult.setETag(eTag); + uploadResult.setVersionId(versionId); when(s3.getObjectMetadata(any(GetObjectMetadataRequest.class))) .thenThrow(NOT_FOUND); @@ -138,51 +172,85 @@ public void testCreateAndReadFileMultiPart() throws Exception { new InitiateMultipartUploadResult(); initiateMultipartUploadResult.setUploadId("uploadId"); when(s3.initiateMultipartUpload( - argThat(correctInitiateMultipartUploadRequest("file")))) + argThat(correctInitiateMultipartUploadRequest(file)))) .thenReturn(initiateMultipartUploadResult); UploadPartResult uploadPartResult = new UploadPartResult(); uploadPartResult.setETag("partETag"); - when(s3.uploadPart(argThat(correctUploadPartRequest("file")))) + when(s3.uploadPart(argThat(correctUploadPartRequest(file)))) .thenReturn(uploadPartResult); CompleteMultipartUploadResult multipartUploadResult = new CompleteMultipartUploadResult(); multipartUploadResult.setETag(eTag); + multipartUploadResult.setVersionId(versionId); when(s3.completeMultipartUpload( - argThat(correctMultipartUploadRequest("file")))) + argThat(correctMultipartUploadRequest(file)))) .thenReturn(multipartUploadResult); ListObjectsV2Result emptyListing = new ListObjectsV2Result(); - when(s3.listObjectsV2(argThat(correctListObjectsRequest("file/")))) + when(s3.listObjectsV2(argThat(correctListObjectsRequest(file + "/")))) .thenReturn(emptyListing); FSDataOutputStream outputStream = fs.create(path); outputStream.write(content); outputStream.close(); + } - // make sure the eTag was put into the metadataStore - MetadataStore metadataStore = fs.getMetadataStore(); - PathMetadata pathMetadata = metadataStore.get(path); - assertNotNull(pathMetadata); - String storedETag = pathMetadata.getFileStatus().getETag(); - assertEquals(eTag, storedETag); - - // Ensure underlying S3 getObject call uses the stored eTag when reading - // data back. If it doesn't, the read won't work and the assert will - // fail. + private void assertContent(String file, Path path, byte[] content, + String eTag, String versionId) throws IOException { S3Object s3Object = new S3Object(); ObjectMetadata metadata = new ObjectMetadata(); - metadata.setHeader(Headers.ETAG, eTag); + metadata.setHeader(Headers.S3_VERSION_ID, versionId); s3Object.setObjectMetadata(metadata); s3Object.setObjectContent(new ByteArrayInputStream(content)); - when(s3.getObject(argThat(correctGetObjectRequest("file", eTag)))) + when(s3.getObject(argThat(correctGetObjectRequest(file, eTag, versionId)))) .thenReturn(s3Object); FSDataInputStream inputStream = fs.open(path); byte[] readContent = IOUtils.toByteArray(inputStream); assertArrayEquals(content, readContent); } + private void assertVersionAttributes(Path path, String eTag, String versionId) + throws IOException { + MetadataStore metadataStore = fs.getMetadataStore(); + PathMetadata pathMetadata = metadataStore.get(path); + assertNotNull(pathMetadata); + S3AFileStatus fileStatus = pathMetadata.getFileStatus(); + assertEquals(eTag, fileStatus.getETag()); + assertEquals(versionId, fileStatus.getVersionId()); + } + + private Matcher correctGetObjectRequest(final String key, + final String eTag, final String versionId) { + return new BaseMatcher() { + @Override + public boolean matches(Object item) { + if (item instanceof GetObjectRequest) { + GetObjectRequest getObjectRequest = (GetObjectRequest) item; + if (getObjectRequest.getKey().equals(key)) { + if (changeDetectionPolicy.getSource().equals( + Source.ETag)) { + return getObjectRequest.getMatchingETagConstraints() + .contains(eTag); + } else if (changeDetectionPolicy.getSource().equals( + Source.VersionId)) { + return getObjectRequest.getVersionId().equals(versionId); + } + } + } + return false; + } + + @Override + public void describeTo(Description description) { + description.appendText("key and " + + changeDetectionPolicy.getSource() + + " matches"); + } + }; + } + private Matcher correctUploadPartRequest( final String key) { return new BaseMatcher() { @@ -203,7 +271,7 @@ public void describeTo(Description description) { } private Matcher - correctInitiateMultipartUploadRequest(final String key) { + correctInitiateMultipartUploadRequest(final String key) { return new BaseMatcher() { @Override public void describeTo(Description description) { @@ -223,7 +291,7 @@ public boolean matches(Object item) { } private Matcher - correctMultipartUploadRequest(final String key) { + correctMultipartUploadRequest(final String key) { return new BaseMatcher() { @Override public boolean matches(Object item) { @@ -280,25 +348,4 @@ public void describeTo(Description description) { } }; } - - private Matcher correctGetObjectRequest(final String key, - final String eTag) { - return new BaseMatcher() { - @Override - public boolean matches(Object item) { - if (item instanceof GetObjectRequest) { - GetObjectRequest getObjectRequest = (GetObjectRequest) item; - return getObjectRequest.getKey().equals(key) - && getObjectRequest.getMatchingETagConstraints() - .contains(eTag); - } - return false; - } - - @Override - public void describeTo(Description description) { - description.appendText("key and eTag matches"); - } - }; - } } From 2d235f89d125e47e0122cf9cde92f2903bdf7350 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Thu, 28 Mar 2019 11:51:26 -0500 Subject: [PATCH 24/28] Fix typo --- .../hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 31ea3e3ddb476..2a1fd9a5afff5 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 @@ -1022,7 +1022,7 @@ If this happens, reads of the affected file(s) will result in The S3Guard metadata for a file can be corrected with the `s3guard import` command as discussed above. The command can take a file URI instead of a -bucket URI to correct the metdata for a single file. For example: +bucket URI to correct the metadata for a single file. For example: ```bash hadoop s3guard import [-meta URI] s3a://my-bucket/file-with-bad-metadata From 1b6be405b72fa00465829298b6ccb33615befdcb Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Thu, 28 Mar 2019 14:23:53 -0500 Subject: [PATCH 25/28] Skip invalid test when object versioning enabled --- .../hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java index adaf538c6c8c4..6dbe6f91d48e3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java @@ -26,6 +26,7 @@ import java.util.UUID; import java.util.stream.Collectors; +import org.junit.Assume; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -37,6 +38,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source; import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; @@ -243,6 +246,12 @@ public void testLongerLengthOverwrite() throws Exception { @Test public void testOutOfBandDeletes() throws Exception { + ChangeDetectionPolicy changeDetectionPolicy = + ((S3AFileSystem) getFileSystem()).getChangeDetectionPolicy(); + Assume.assumeFalse("FNF not expected when using a bucket with" + + " object versioning", + changeDetectionPolicy.getSource() == Source.VersionId); + Path testFileName = path("OutOfBandDelete-" + UUID.randomUUID()); outOfBandDeletes(testFileName, authoritative); } From 5e1f3e343168913dff7c3d169c994e78f001e4dd Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Thu, 28 Mar 2019 15:49:26 -0500 Subject: [PATCH 26/28] Speed up ITestS3ARemoteFileChanged --- .../org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java index 005903390ee53..ff31c1e6a5ae9 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java @@ -156,6 +156,11 @@ protected Configuration createConfiguration() { CHANGE_DETECT_MODE); conf.set(CHANGE_DETECT_SOURCE, changeDetectionSource); conf.set(CHANGE_DETECT_MODE, changeDetectionMode); + + // reduce retry limit so FileNotFoundException cases timeout faster, + // speeding up the tests + conf.set(RETRY_LIMIT, "5"); + if (conf.getClass(S3_METADATA_STORE_IMPL, MetadataStore.class) == NullMetadataStore.class) { // favor LocalMetadataStore over NullMetadataStore From dc0a3fb1930d831434b576cfe236cd3196fd2024 Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Thu, 28 Mar 2019 16:53:46 -0500 Subject: [PATCH 27/28] Update tests that started failing due to HADOOP-15999 --- .../apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java index ff31c1e6a5ae9..a6f36d832db5a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java @@ -380,9 +380,14 @@ private Path writeOutOfSyncFileVersion(String filename) throws IOException { writeDataset(fs, testpath, dataset, dataset.length / 2, 1024, true); - // put back the original metadata (etag, versionId) + S3AFileStatus newStatus = (S3AFileStatus) fs.getFileStatus(testpath); + + // put back the original etag, versionId + S3AFileStatus forgedStatus = + S3AFileStatus.fromFileStatus(newStatus, Tristate.FALSE, + originalStatus.getETag(), originalStatus.getVersionId()); fs.getMetadataStore().put( - new PathMetadata(originalStatus, Tristate.FALSE, false)); + new PathMetadata(forgedStatus, Tristate.FALSE, false)); return testpath; } From 9f4ad8845d8ee040ffd009d61ebec2d302feb27f Mon Sep 17 00:00:00 2001 From: Ben Roling Date: Mon, 1 Apr 2019 09:29:35 -0500 Subject: [PATCH 28/28] Add test for 412 response --- .../fs/s3a/TestStreamChangeTracker.java | 44 +++++++++++++++++++ 1 file changed, 44 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java index e5090653495f2..2b44620685b40 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a; +import com.amazonaws.AmazonServiceException; import java.util.concurrent.atomic.AtomicLong; import com.amazonaws.services.s3.Headers; @@ -234,6 +235,25 @@ public void testCopyETagRequired() throws Throwable { "policy requires ETag"); } + @Test + public void testCopyVersionMismatch() throws Throwable { + ChangeTracker tracker = newTracker( + ChangeDetectionPolicy.Mode.Server, + ChangeDetectionPolicy.Source.ETag, + true, + objectAttributes("etag", "versionId")); + + // 412 is translated to RemoteFileChangedException + AmazonServiceException awsException = + new AmazonServiceException("aws exception"); + awsException.setStatusCode(412); + expectChangeException(tracker, awsException, "copy", + RemoteFileChangedException.PRECONDITIONS_NOT_MET); + + // processing another type of exception does nothing + tracker.processException(new RuntimeException(), "copy"); + } + protected void assertConstraintApplied(final ChangeTracker tracker, final GetObjectRequest request) { assertTrue("Tracker should have applied contraints " + tracker, @@ -254,6 +274,15 @@ protected RemoteFileChangedException expectChangeException( RemoteFileChangedException.class); } + protected RemoteFileChangedException expectChangeException( + final ChangeTracker tracker, + final Exception exception, + final String operation, + final String message) throws Exception { + return expectException(tracker, exception, operation, message, + RemoteFileChangedException.class); + } + protected PathIOException expectNoVersionAttributeException( final ChangeTracker tracker, final S3Object response, @@ -298,6 +327,21 @@ protected T expectException( }); } + protected T expectException( + final ChangeTracker tracker, + final Exception exception, + final String operation, + final String message, + final Class clazz) throws Exception { + return intercept( + clazz, + message, + () -> { + tracker.processException(exception, operation); + return tracker; + }); + } + protected void assertRevisionId(final ChangeTracker tracker, final String revId) { assertEquals("Wrong revision ID in " + tracker,