|
@@ -38,6 +38,7 @@ import java.util.concurrent.ThreadFactory;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import org.apache.hadoop.classification.VisibleForTesting;
|
|
import org.apache.hadoop.classification.VisibleForTesting;
|
|
|
|
+import org.apache.hadoop.fs.store.LogExactlyOnce;
|
|
import org.apache.hadoop.util.Preconditions;
|
|
import org.apache.hadoop.util.Preconditions;
|
|
import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
|
|
import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
|
|
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback;
|
|
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback;
|
|
@@ -51,7 +52,6 @@ import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFact
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
-import org.apache.commons.lang3.tuple.Pair;
|
|
|
|
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
|
|
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
|
|
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
|
|
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
|
|
import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
|
|
import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
|
|
@@ -69,6 +69,7 @@ import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
|
|
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
|
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
|
|
|
|
|
import static org.apache.commons.lang3.StringUtils.isNotEmpty;
|
|
import static org.apache.commons.lang3.StringUtils.isNotEmpty;
|
|
|
|
+import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.RENAME_PATH_ATTEMPTS;
|
|
import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.extractEtagHeader;
|
|
import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.extractEtagHeader;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_DELETE_CONSIDERED_IDEMPOTENT;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_DELETE_CONSIDERED_IDEMPOTENT;
|
|
@@ -76,6 +77,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.S
|
|
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.*;
|
|
import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.*;
|
|
|
|
+import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND;
|
|
|
|
|
|
/**
|
|
/**
|
|
* AbfsClient.
|
|
* AbfsClient.
|
|
@@ -102,6 +104,10 @@ public class AbfsClient implements Closeable {
|
|
|
|
|
|
private final ListeningScheduledExecutorService executorService;
|
|
private final ListeningScheduledExecutorService executorService;
|
|
|
|
|
|
|
|
+ /** logging the rename failure if metadata is in an incomplete state. */
|
|
|
|
+ private static final LogExactlyOnce ABFS_METADATA_INCOMPLETE_RENAME_FAILURE =
|
|
|
|
+ new LogExactlyOnce(LOG);
|
|
|
|
+
|
|
private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials,
|
|
private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials,
|
|
final AbfsConfiguration abfsConfiguration,
|
|
final AbfsConfiguration abfsConfiguration,
|
|
final AbfsClientContext abfsClientContext)
|
|
final AbfsClientContext abfsClientContext)
|
|
@@ -496,15 +502,19 @@ public class AbfsClient implements Closeable {
|
|
* @param continuation continuation.
|
|
* @param continuation continuation.
|
|
* @param tracingContext trace context
|
|
* @param tracingContext trace context
|
|
* @param sourceEtag etag of source file. may be null or empty
|
|
* @param sourceEtag etag of source file. may be null or empty
|
|
- * @return pair of (the rename operation, flag indicating recovery took place)
|
|
|
|
|
|
+ * @param isMetadataIncompleteState was there a rename failure due to
|
|
|
|
+ * incomplete metadata state?
|
|
|
|
+ * @return AbfsClientRenameResult result of rename operation indicating the
|
|
|
|
+ * AbfsRest operation, rename recovery and incomplete metadata state failure.
|
|
* @throws AzureBlobFileSystemException failure, excluding any recovery from overload failures.
|
|
* @throws AzureBlobFileSystemException failure, excluding any recovery from overload failures.
|
|
*/
|
|
*/
|
|
- public Pair<AbfsRestOperation, Boolean> renamePath(
|
|
|
|
|
|
+ public AbfsClientRenameResult renamePath(
|
|
final String source,
|
|
final String source,
|
|
final String destination,
|
|
final String destination,
|
|
final String continuation,
|
|
final String continuation,
|
|
final TracingContext tracingContext,
|
|
final TracingContext tracingContext,
|
|
- final String sourceEtag)
|
|
|
|
|
|
+ final String sourceEtag,
|
|
|
|
+ boolean isMetadataIncompleteState)
|
|
throws AzureBlobFileSystemException {
|
|
throws AzureBlobFileSystemException {
|
|
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
|
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
|
|
|
|
|
|
@@ -531,13 +541,45 @@ public class AbfsClient implements Closeable {
|
|
url,
|
|
url,
|
|
requestHeaders);
|
|
requestHeaders);
|
|
try {
|
|
try {
|
|
|
|
+ incrementAbfsRenamePath();
|
|
op.execute(tracingContext);
|
|
op.execute(tracingContext);
|
|
- return Pair.of(op, false);
|
|
|
|
|
|
+ // AbfsClientResult contains the AbfsOperation, If recovery happened or
|
|
|
|
+ // not, and the incompleteMetaDataState is true or false.
|
|
|
|
+ // If we successfully rename a path and isMetadataIncompleteState was
|
|
|
|
+ // true, then rename was recovered, else it didn't, this is why
|
|
|
|
+ // isMetadataIncompleteState is used for renameRecovery(as the 2nd param).
|
|
|
|
+ return new AbfsClientRenameResult(op, isMetadataIncompleteState, isMetadataIncompleteState);
|
|
} catch (AzureBlobFileSystemException e) {
|
|
} catch (AzureBlobFileSystemException e) {
|
|
// If we have no HTTP response, throw the original exception.
|
|
// If we have no HTTP response, throw the original exception.
|
|
if (!op.hasResult()) {
|
|
if (!op.hasResult()) {
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ // ref: HADOOP-18242. Rename failure occurring due to a rare case of
|
|
|
|
+ // tracking metadata being in incomplete state.
|
|
|
|
+ if (op.getResult().getStorageErrorCode()
|
|
|
|
+ .equals(RENAME_DESTINATION_PARENT_PATH_NOT_FOUND.getErrorCode())
|
|
|
|
+ && !isMetadataIncompleteState) {
|
|
|
|
+ //Logging
|
|
|
|
+ ABFS_METADATA_INCOMPLETE_RENAME_FAILURE
|
|
|
|
+ .info("Rename Failure attempting to resolve tracking metadata state and retrying.");
|
|
|
|
+
|
|
|
|
+ // Doing a HEAD call resolves the incomplete metadata state and
|
|
|
|
+ // then we can retry the rename operation.
|
|
|
|
+ AbfsRestOperation sourceStatusOp = getPathStatus(source, false,
|
|
|
|
+ tracingContext);
|
|
|
|
+ isMetadataIncompleteState = true;
|
|
|
|
+ // Extract the sourceEtag, using the status Op, and set it
|
|
|
|
+ // for future rename recovery.
|
|
|
|
+ AbfsHttpOperation sourceStatusResult = sourceStatusOp.getResult();
|
|
|
|
+ String sourceEtagAfterFailure = extractEtagHeader(sourceStatusResult);
|
|
|
|
+ renamePath(source, destination, continuation, tracingContext,
|
|
|
|
+ sourceEtagAfterFailure, isMetadataIncompleteState);
|
|
|
|
+ }
|
|
|
|
+ // if we get out of the condition without a successful rename, then
|
|
|
|
+ // it isn't metadata incomplete state issue.
|
|
|
|
+ isMetadataIncompleteState = false;
|
|
|
|
+
|
|
boolean etagCheckSucceeded = renameIdempotencyCheckOp(
|
|
boolean etagCheckSucceeded = renameIdempotencyCheckOp(
|
|
source,
|
|
source,
|
|
sourceEtag, op, destination, tracingContext);
|
|
sourceEtag, op, destination, tracingContext);
|
|
@@ -546,10 +588,14 @@ public class AbfsClient implements Closeable {
|
|
// throw back the exception
|
|
// throw back the exception
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
- return Pair.of(op, true);
|
|
|
|
|
|
+ return new AbfsClientRenameResult(op, true, isMetadataIncompleteState);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private void incrementAbfsRenamePath() {
|
|
|
|
+ abfsCounters.incrementCounter(RENAME_PATH_ATTEMPTS, 1);
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Check if the rename request failure is post a retry and if earlier rename
|
|
* Check if the rename request failure is post a retry and if earlier rename
|
|
* request might have succeeded at back-end.
|
|
* request might have succeeded at back-end.
|