|
@@ -102,6 +102,7 @@ import org.apache.hadoop.fs.s3a.impl.InternalConstants;
|
|
import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport;
|
|
import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport;
|
|
import org.apache.hadoop.fs.s3a.impl.OperationCallbacks;
|
|
import org.apache.hadoop.fs.s3a.impl.OperationCallbacks;
|
|
import org.apache.hadoop.fs.s3a.impl.RenameOperation;
|
|
import org.apache.hadoop.fs.s3a.impl.RenameOperation;
|
|
|
|
+import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
|
|
import org.apache.hadoop.fs.s3a.impl.StoreContext;
|
|
import org.apache.hadoop.fs.s3a.impl.StoreContext;
|
|
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
|
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
|
|
import org.apache.hadoop.fs.s3a.select.InternalSelectConstants;
|
|
import org.apache.hadoop.fs.s3a.select.InternalSelectConstants;
|
|
@@ -160,6 +161,7 @@ import static org.apache.hadoop.fs.s3a.auth.RolePolicies.STATEMENT_ALLOW_SSE_KMS
|
|
import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3Operations;
|
|
import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3Operations;
|
|
import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.TokenIssuingPolicy.NoTokensAvailable;
|
|
import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.TokenIssuingPolicy.NoTokensAvailable;
|
|
import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.hasDelegationTokenBinding;
|
|
import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.hasDelegationTokenBinding;
|
|
|
|
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404;
|
|
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
|
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1059,8 +1061,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
String key = pathToKey(path);
|
|
String key = pathToKey(path);
|
|
FileStatus status = null;
|
|
FileStatus status = null;
|
|
try {
|
|
try {
|
|
- // get the status or throw an FNFE
|
|
|
|
- status = getFileStatus(path);
|
|
|
|
|
|
+ // get the status or throw an FNFE.
|
|
|
|
+ // when overwriting, there is no need to look for any existing file,
|
|
|
|
+ // and attempting to do so can poison the load balancers with 404
|
|
|
|
+ // entries.
|
|
|
|
+ status = innerGetFileStatus(path, false,
|
|
|
|
+ overwrite
|
|
|
|
+ ? StatusProbeEnum.DIRECTORIES
|
|
|
|
+ : StatusProbeEnum.ALL);
|
|
|
|
|
|
// if the thread reaches here, there is something at the path
|
|
// if the thread reaches here, there is something at the path
|
|
if (status.isDirectory()) {
|
|
if (status.isDirectory()) {
|
|
@@ -1216,7 +1224,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
|
|
|
// get the source file status; this raises a FNFE if there is no source
|
|
// get the source file status; this raises a FNFE if there is no source
|
|
// file.
|
|
// file.
|
|
- S3AFileStatus srcStatus = innerGetFileStatus(src, true);
|
|
|
|
|
|
+ S3AFileStatus srcStatus = innerGetFileStatus(src, true,
|
|
|
|
+ StatusProbeEnum.ALL);
|
|
|
|
|
|
if (srcKey.equals(dstKey)) {
|
|
if (srcKey.equals(dstKey)) {
|
|
LOG.debug("rename: src and dest refer to the same file or directory: {}",
|
|
LOG.debug("rename: src and dest refer to the same file or directory: {}",
|
|
@@ -1228,7 +1237,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
|
|
|
S3AFileStatus dstStatus = null;
|
|
S3AFileStatus dstStatus = null;
|
|
try {
|
|
try {
|
|
- dstStatus = innerGetFileStatus(dst, true);
|
|
|
|
|
|
+ dstStatus = innerGetFileStatus(dst, true, StatusProbeEnum.ALL);
|
|
// if there is no destination entry, an exception is raised.
|
|
// if there is no destination entry, an exception is raised.
|
|
// hence this code sequence can assume that there is something
|
|
// hence this code sequence can assume that there is something
|
|
// at the end of the path; the only detail being what it is and
|
|
// at the end of the path; the only detail being what it is and
|
|
@@ -1261,7 +1270,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
if (!pathToKey(parent).isEmpty()) {
|
|
if (!pathToKey(parent).isEmpty()) {
|
|
try {
|
|
try {
|
|
S3AFileStatus dstParentStatus = innerGetFileStatus(dst.getParent(),
|
|
S3AFileStatus dstParentStatus = innerGetFileStatus(dst.getParent(),
|
|
- false);
|
|
|
|
|
|
+ false, StatusProbeEnum.ALL);
|
|
if (!dstParentStatus.isDirectory()) {
|
|
if (!dstParentStatus.isDirectory()) {
|
|
throw new RenameFailedException(src, dst,
|
|
throw new RenameFailedException(src, dst,
|
|
"destination parent is not a directory");
|
|
"destination parent is not a directory");
|
|
@@ -1660,6 +1669,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
ObjectMetadata meta = changeInvoker.retryUntranslated("GET " + key, true,
|
|
ObjectMetadata meta = changeInvoker.retryUntranslated("GET " + key, true,
|
|
() -> {
|
|
() -> {
|
|
incrementStatistic(OBJECT_METADATA_REQUESTS);
|
|
incrementStatistic(OBJECT_METADATA_REQUESTS);
|
|
|
|
+ LOG.debug("HEAD {} with change tracker {}", key, changeTracker);
|
|
if (changeTracker != null) {
|
|
if (changeTracker != null) {
|
|
changeTracker.maybeApplyConstraint(request);
|
|
changeTracker.maybeApplyConstraint(request);
|
|
}
|
|
}
|
|
@@ -2267,7 +2277,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
entryPoint(INVOCATION_DELETE);
|
|
entryPoint(INVOCATION_DELETE);
|
|
DeleteOperation deleteOperation = new DeleteOperation(
|
|
DeleteOperation deleteOperation = new DeleteOperation(
|
|
createStoreContext(),
|
|
createStoreContext(),
|
|
- innerGetFileStatus(f, true),
|
|
|
|
|
|
+ innerGetFileStatus(f, true, StatusProbeEnum.ALL),
|
|
recursive,
|
|
recursive,
|
|
operationCallbacks,
|
|
operationCallbacks,
|
|
InternalConstants.MAX_ENTRIES_TO_DELETE);
|
|
InternalConstants.MAX_ENTRIES_TO_DELETE);
|
|
@@ -2297,13 +2307,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* Retry policy: retrying; untranslated.
|
|
* Retry policy: retrying; untranslated.
|
|
* @param f path to create
|
|
* @param f path to create
|
|
* @throws IOException IO problem
|
|
* @throws IOException IO problem
|
|
- * @throws AmazonClientException untranslated AWS client problem
|
|
|
|
*/
|
|
*/
|
|
@Retries.RetryTranslated
|
|
@Retries.RetryTranslated
|
|
private void createFakeDirectoryIfNecessary(Path f)
|
|
private void createFakeDirectoryIfNecessary(Path f)
|
|
throws IOException, AmazonClientException {
|
|
throws IOException, AmazonClientException {
|
|
String key = pathToKey(f);
|
|
String key = pathToKey(f);
|
|
- if (!key.isEmpty() && !s3Exists(f)) {
|
|
|
|
|
|
+ // we only make the LIST call; the codepaths to get here should not
|
|
|
|
+ // be reached if there is an empty dir marker -and if they do, it
|
|
|
|
+ // is mostly harmless to create a new one.
|
|
|
|
+ if (!key.isEmpty() && !s3Exists(f, EnumSet.of(StatusProbeEnum.List))) {
|
|
LOG.debug("Creating new fake directory at {}", f);
|
|
LOG.debug("Creating new fake directory at {}", f);
|
|
createFakeDirectory(key);
|
|
createFakeDirectory(key);
|
|
}
|
|
}
|
|
@@ -2314,7 +2326,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* That is: it parent is not the root path and does not yet exist.
|
|
* That is: it parent is not the root path and does not yet exist.
|
|
* @param path whose parent is created if needed.
|
|
* @param path whose parent is created if needed.
|
|
* @throws IOException IO problem
|
|
* @throws IOException IO problem
|
|
- * @throws AmazonClientException untranslated AWS client problem
|
|
|
|
*/
|
|
*/
|
|
@Retries.RetryTranslated
|
|
@Retries.RetryTranslated
|
|
void maybeCreateFakeParentDirectory(Path path)
|
|
void maybeCreateFakeParentDirectory(Path path)
|
|
@@ -2568,14 +2579,23 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
*/
|
|
*/
|
|
@Retries.RetryTranslated
|
|
@Retries.RetryTranslated
|
|
public FileStatus getFileStatus(final Path f) throws IOException {
|
|
public FileStatus getFileStatus(final Path f) throws IOException {
|
|
- return innerGetFileStatus(f, false);
|
|
|
|
|
|
+ entryPoint(INVOCATION_GET_FILE_STATUS);
|
|
|
|
+ return innerGetFileStatus(f, false, StatusProbeEnum.ALL);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
|
|
+ * Get the status of a file or directory, first through S3Guard and then
|
|
|
|
+ * through S3.
|
|
|
|
+ * The S3 probes can leave 404 responses in the S3 load balancers; if
|
|
|
|
+ * a check is only needed for a directory, declaring this saves time and
|
|
|
|
+ * avoids creating one for the object.
|
|
|
|
+ * When only probing for directories, if an entry for a file is found in
|
|
|
|
+ * S3Guard it is returned, but checks for updated values are skipped.
|
|
* Internal version of {@link #getFileStatus(Path)}.
|
|
* Internal version of {@link #getFileStatus(Path)}.
|
|
* @param f The path we want information from
|
|
* @param f The path we want information from
|
|
* @param needEmptyDirectoryFlag if true, implementation will calculate
|
|
* @param needEmptyDirectoryFlag if true, implementation will calculate
|
|
* a TRUE or FALSE value for {@link S3AFileStatus#isEmptyDirectory()}
|
|
* a TRUE or FALSE value for {@link S3AFileStatus#isEmptyDirectory()}
|
|
|
|
+ * @param probes probes to make
|
|
* @return a S3AFileStatus object
|
|
* @return a S3AFileStatus object
|
|
* @throws FileNotFoundException when the path does not exist
|
|
* @throws FileNotFoundException when the path does not exist
|
|
* @throws IOException on other problems.
|
|
* @throws IOException on other problems.
|
|
@@ -2583,9 +2603,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
@Retries.RetryTranslated
|
|
@Retries.RetryTranslated
|
|
S3AFileStatus innerGetFileStatus(final Path f,
|
|
S3AFileStatus innerGetFileStatus(final Path f,
|
|
- boolean needEmptyDirectoryFlag) throws IOException {
|
|
|
|
- entryPoint(INVOCATION_GET_FILE_STATUS);
|
|
|
|
- checkNotClosed();
|
|
|
|
|
|
+ final boolean needEmptyDirectoryFlag,
|
|
|
|
+ final Set<StatusProbeEnum> probes) throws IOException {
|
|
final Path path = qualify(f);
|
|
final Path path = qualify(f);
|
|
String key = pathToKey(path);
|
|
String key = pathToKey(path);
|
|
LOG.debug("Getting path status for {} ({})", path, key);
|
|
LOG.debug("Getting path status for {} ({})", path, key);
|
|
@@ -2602,7 +2621,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
OffsetDateTime deletedAt = OffsetDateTime.ofInstant(
|
|
OffsetDateTime deletedAt = OffsetDateTime.ofInstant(
|
|
Instant.ofEpochMilli(pm.getFileStatus().getModificationTime()),
|
|
Instant.ofEpochMilli(pm.getFileStatus().getModificationTime()),
|
|
ZoneOffset.UTC);
|
|
ZoneOffset.UTC);
|
|
- throw new FileNotFoundException("Path " + f + " is recorded as " +
|
|
|
|
|
|
+ throw new FileNotFoundException("Path " + path + " is recorded as " +
|
|
"deleted by S3Guard at " + deletedAt);
|
|
"deleted by S3Guard at " + deletedAt);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -2612,15 +2631,18 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
// dest is also a directory, there's no difference.
|
|
// dest is also a directory, there's no difference.
|
|
// TODO After HADOOP-16085 the modification detection can be done with
|
|
// TODO After HADOOP-16085 the modification detection can be done with
|
|
// etags or object version instead of modTime
|
|
// etags or object version instead of modTime
|
|
- boolean allowAuthoritative = allowAuthoritative(f);
|
|
|
|
|
|
+ boolean allowAuthoritative = allowAuthoritative(path);
|
|
if (!pm.getFileStatus().isDirectory() &&
|
|
if (!pm.getFileStatus().isDirectory() &&
|
|
- !allowAuthoritative) {
|
|
|
|
|
|
+ !allowAuthoritative &&
|
|
|
|
+ probes.contains(StatusProbeEnum.Head)) {
|
|
|
|
+ // a file has been found in a non-auth path and the caller has not said
|
|
|
|
+ // they only care about directories
|
|
LOG.debug("Metadata for {} found in the non-auth metastore.", path);
|
|
LOG.debug("Metadata for {} found in the non-auth metastore.", path);
|
|
final long msModTime = pm.getFileStatus().getModificationTime();
|
|
final long msModTime = pm.getFileStatus().getModificationTime();
|
|
|
|
|
|
S3AFileStatus s3AFileStatus;
|
|
S3AFileStatus s3AFileStatus;
|
|
try {
|
|
try {
|
|
- s3AFileStatus = s3GetFileStatus(path, key, tombstones);
|
|
|
|
|
|
+ s3AFileStatus = s3GetFileStatus(path, key, probes, tombstones);
|
|
} catch (FileNotFoundException fne) {
|
|
} catch (FileNotFoundException fne) {
|
|
s3AFileStatus = null;
|
|
s3AFileStatus = null;
|
|
}
|
|
}
|
|
@@ -2662,7 +2684,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
// S3 yet, we'll assume the empty directory is true;
|
|
// S3 yet, we'll assume the empty directory is true;
|
|
S3AFileStatus s3FileStatus;
|
|
S3AFileStatus s3FileStatus;
|
|
try {
|
|
try {
|
|
- s3FileStatus = s3GetFileStatus(path, key, tombstones);
|
|
|
|
|
|
+ s3FileStatus = s3GetFileStatus(path, key, probes, tombstones);
|
|
} catch (FileNotFoundException e) {
|
|
} catch (FileNotFoundException e) {
|
|
return S3AFileStatus.fromFileStatus(msStatus, Tristate.TRUE,
|
|
return S3AFileStatus.fromFileStatus(msStatus, Tristate.TRUE,
|
|
null, null);
|
|
null, null);
|
|
@@ -2674,7 +2696,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
// there was no entry in S3Guard
|
|
// there was no entry in S3Guard
|
|
// retrieve the data and update the metadata store in the process.
|
|
// retrieve the data and update the metadata store in the process.
|
|
return S3Guard.putAndReturn(metadataStore,
|
|
return S3Guard.putAndReturn(metadataStore,
|
|
- s3GetFileStatus(path, key, tombstones), instrumentation,
|
|
|
|
|
|
+ s3GetFileStatus(path, key, StatusProbeEnum.ALL, tombstones),
|
|
|
|
+ instrumentation,
|
|
ttlTimeProvider);
|
|
ttlTimeProvider);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -2686,14 +2709,18 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* Retry policy: retry translated.
|
|
* Retry policy: retry translated.
|
|
* @param path Qualified path
|
|
* @param path Qualified path
|
|
* @param key Key string for the path
|
|
* @param key Key string for the path
|
|
|
|
+ * @param probes probes to make
|
|
|
|
+ * @param tombstones tombstones to filter
|
|
* @return Status
|
|
* @return Status
|
|
* @throws FileNotFoundException when the path does not exist
|
|
* @throws FileNotFoundException when the path does not exist
|
|
* @throws IOException on other problems.
|
|
* @throws IOException on other problems.
|
|
*/
|
|
*/
|
|
@Retries.RetryTranslated
|
|
@Retries.RetryTranslated
|
|
- private S3AFileStatus s3GetFileStatus(final Path path, String key,
|
|
|
|
- Set<Path> tombstones) throws IOException {
|
|
|
|
- if (!key.isEmpty()) {
|
|
|
|
|
|
+ private S3AFileStatus s3GetFileStatus(final Path path,
|
|
|
|
+ String key,
|
|
|
|
+ final Set<StatusProbeEnum> probes,
|
|
|
|
+ final Set<Path> tombstones) throws IOException {
|
|
|
|
+ if (!key.isEmpty() && probes.contains(StatusProbeEnum.Head)) {
|
|
try {
|
|
try {
|
|
ObjectMetadata meta = getObjectMetadata(key);
|
|
ObjectMetadata meta = getObjectMetadata(key);
|
|
|
|
|
|
@@ -2711,15 +2738,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
meta.getVersionId());
|
|
meta.getVersionId());
|
|
}
|
|
}
|
|
} catch (AmazonServiceException e) {
|
|
} catch (AmazonServiceException e) {
|
|
- if (e.getStatusCode() != 404) {
|
|
|
|
|
|
+ if (e.getStatusCode() != SC_404) {
|
|
throw translateException("getFileStatus", path, e);
|
|
throw translateException("getFileStatus", path, e);
|
|
}
|
|
}
|
|
} catch (AmazonClientException e) {
|
|
} catch (AmazonClientException e) {
|
|
throw translateException("getFileStatus", path, e);
|
|
throw translateException("getFileStatus", path, e);
|
|
}
|
|
}
|
|
|
|
|
|
- // Necessary?
|
|
|
|
- if (!key.endsWith("/")) {
|
|
|
|
|
|
+ // Look for the dir marker
|
|
|
|
+ if (!key.endsWith("/") && probes.contains(StatusProbeEnum.DirMarker)) {
|
|
String newKey = key + "/";
|
|
String newKey = key + "/";
|
|
try {
|
|
try {
|
|
ObjectMetadata meta = getObjectMetadata(newKey);
|
|
ObjectMetadata meta = getObjectMetadata(newKey);
|
|
@@ -2740,7 +2767,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
meta.getVersionId());
|
|
meta.getVersionId());
|
|
}
|
|
}
|
|
} catch (AmazonServiceException e) {
|
|
} catch (AmazonServiceException e) {
|
|
- if (e.getStatusCode() != 404) {
|
|
|
|
|
|
+ if (e.getStatusCode() != SC_404) {
|
|
throw translateException("getFileStatus", newKey, e);
|
|
throw translateException("getFileStatus", newKey, e);
|
|
}
|
|
}
|
|
} catch (AmazonClientException e) {
|
|
} catch (AmazonClientException e) {
|
|
@@ -2749,39 +2776,42 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- try {
|
|
|
|
- key = maybeAddTrailingSlash(key);
|
|
|
|
- S3ListRequest request = createListObjectsRequest(key, "/", 1);
|
|
|
|
|
|
+ // execute the list
|
|
|
|
+ if (probes.contains(StatusProbeEnum.List)) {
|
|
|
|
+ try {
|
|
|
|
+ key = maybeAddTrailingSlash(key);
|
|
|
|
+ S3ListRequest request = createListObjectsRequest(key, "/", 1);
|
|
|
|
|
|
- S3ListResult objects = listObjects(request);
|
|
|
|
|
|
+ S3ListResult objects = listObjects(request);
|
|
|
|
|
|
- Collection<String> prefixes = objects.getCommonPrefixes();
|
|
|
|
- Collection<S3ObjectSummary> summaries = objects.getObjectSummaries();
|
|
|
|
- if (!isEmptyOfKeys(prefixes, tombstones) ||
|
|
|
|
- !isEmptyOfObjects(summaries, tombstones)) {
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug("Found path as directory (with /): {}/{}",
|
|
|
|
- prefixes.size(), summaries.size());
|
|
|
|
|
|
+ Collection<String> prefixes = objects.getCommonPrefixes();
|
|
|
|
+ Collection<S3ObjectSummary> summaries = objects.getObjectSummaries();
|
|
|
|
+ if (!isEmptyOfKeys(prefixes, tombstones) ||
|
|
|
|
+ !isEmptyOfObjects(summaries, tombstones)) {
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("Found path as directory (with /): {}/{}",
|
|
|
|
+ prefixes.size(), summaries.size());
|
|
|
|
|
|
- for (S3ObjectSummary summary : summaries) {
|
|
|
|
- LOG.debug("Summary: {} {}", summary.getKey(), summary.getSize());
|
|
|
|
- }
|
|
|
|
- for (String prefix : prefixes) {
|
|
|
|
- LOG.debug("Prefix: {}", prefix);
|
|
|
|
|
|
+ for (S3ObjectSummary summary : summaries) {
|
|
|
|
+ LOG.debug("Summary: {} {}", summary.getKey(), summary.getSize());
|
|
|
|
+ }
|
|
|
|
+ for (String prefix : prefixes) {
|
|
|
|
+ LOG.debug("Prefix: {}", prefix);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
- }
|
|
|
|
|
|
|
|
- return new S3AFileStatus(Tristate.FALSE, path, username);
|
|
|
|
- } else if (key.isEmpty()) {
|
|
|
|
- LOG.debug("Found root directory");
|
|
|
|
- return new S3AFileStatus(Tristate.TRUE, path, username);
|
|
|
|
- }
|
|
|
|
- } catch (AmazonServiceException e) {
|
|
|
|
- if (e.getStatusCode() != 404) {
|
|
|
|
|
|
+ return new S3AFileStatus(Tristate.FALSE, path, username);
|
|
|
|
+ } else if (key.isEmpty()) {
|
|
|
|
+ LOG.debug("Found root directory");
|
|
|
|
+ return new S3AFileStatus(Tristate.TRUE, path, username);
|
|
|
|
+ }
|
|
|
|
+ } catch (AmazonServiceException e) {
|
|
|
|
+ if (e.getStatusCode() != SC_404) {
|
|
|
|
+ throw translateException("getFileStatus", path, e);
|
|
|
|
+ }
|
|
|
|
+ } catch (AmazonClientException e) {
|
|
throw translateException("getFileStatus", path, e);
|
|
throw translateException("getFileStatus", path, e);
|
|
}
|
|
}
|
|
- } catch (AmazonClientException e) {
|
|
|
|
- throw translateException("getFileStatus", path, e);
|
|
|
|
}
|
|
}
|
|
|
|
|
|
LOG.debug("Not Found: {}", path);
|
|
LOG.debug("Not Found: {}", path);
|
|
@@ -2834,15 +2864,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* Raw version of {@link FileSystem#exists(Path)} which uses S3 only:
|
|
* Raw version of {@link FileSystem#exists(Path)} which uses S3 only:
|
|
* S3Guard MetadataStore, if any, will be skipped.
|
|
* S3Guard MetadataStore, if any, will be skipped.
|
|
* Retry policy: retrying; translated.
|
|
* Retry policy: retrying; translated.
|
|
|
|
+ * @param path qualified path to look for
|
|
|
|
+ * @param probes probes to make
|
|
* @return true if path exists in S3
|
|
* @return true if path exists in S3
|
|
* @throws IOException IO failure
|
|
* @throws IOException IO failure
|
|
*/
|
|
*/
|
|
@Retries.RetryTranslated
|
|
@Retries.RetryTranslated
|
|
- private boolean s3Exists(final Path f) throws IOException {
|
|
|
|
- Path path = qualify(f);
|
|
|
|
|
|
+ private boolean s3Exists(final Path path, final Set<StatusProbeEnum> probes)
|
|
|
|
+ throws IOException {
|
|
String key = pathToKey(path);
|
|
String key = pathToKey(path);
|
|
try {
|
|
try {
|
|
- s3GetFileStatus(path, key, null);
|
|
|
|
|
|
+ s3GetFileStatus(path, key, probes, null);
|
|
return true;
|
|
return true;
|
|
} catch (FileNotFoundException e) {
|
|
} catch (FileNotFoundException e) {
|
|
return false;
|
|
return false;
|
|
@@ -3160,10 +3192,31 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
String action = "copyFile(" + srcKey + ", " + dstKey + ")";
|
|
String action = "copyFile(" + srcKey + ", " + dstKey + ")";
|
|
Invoker readInvoker = readContext.getReadInvoker();
|
|
Invoker readInvoker = readContext.getReadInvoker();
|
|
|
|
|
|
- ObjectMetadata srcom =
|
|
|
|
- once(action, srcKey,
|
|
|
|
- () ->
|
|
|
|
- getObjectMetadata(srcKey, changeTracker, readInvoker, "copy"));
|
|
|
|
|
|
+ ObjectMetadata srcom;
|
|
|
|
+ try {
|
|
|
|
+ srcom = once(action, srcKey,
|
|
|
|
+ () ->
|
|
|
|
+ getObjectMetadata(srcKey, changeTracker, readInvoker, "copy"));
|
|
|
|
+ } catch (FileNotFoundException e) {
|
|
|
|
+ // if rename fails at this point it means that the expected file was not
|
|
|
|
+ // found.
|
|
|
|
+ // The cause is believed to always be one of
|
|
|
|
+ // - File was deleted since LIST/S3Guard metastore.list.() knew of it.
|
|
|
|
+ // - S3Guard is asking for a specific version and it's been removed by
|
|
|
|
+ // lifecycle rules.
|
|
|
|
+ // - there's a 404 cached in the S3 load balancers.
|
|
|
|
+ LOG.debug("getObjectMetadata({}) failed to find an expected file",
|
|
|
|
+ srcKey, e);
|
|
|
|
+ // We create an exception, but the text depends on the S3Guard state
|
|
|
|
+ String message = hasMetadataStore()
|
|
|
|
+ ? RemoteFileChangedException.FILE_NEVER_FOUND
|
|
|
|
+ : RemoteFileChangedException.FILE_NOT_FOUND_SINGLE_ATTEMPT;
|
|
|
|
+ throw new RemoteFileChangedException(
|
|
|
|
+ keyToQualifiedPath(srcKey).toString(),
|
|
|
|
+ action,
|
|
|
|
+ message,
|
|
|
|
+ e);
|
|
|
|
+ }
|
|
ObjectMetadata dstom = cloneObjectMetadata(srcom);
|
|
ObjectMetadata dstom = cloneObjectMetadata(srcom);
|
|
setOptionalObjectMetadata(dstom);
|
|
setOptionalObjectMetadata(dstom);
|
|
|
|
|