Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -328,6 +328,10 @@ public class AbfsConfiguration{
FS_AZURE_ENABLE_ABFS_LIST_ITERATOR, DefaultValue = DEFAULT_ENABLE_ABFS_LIST_ITERATOR)
private boolean enableAbfsListIterator;

@BooleanConfigurationValidatorAnnotation(ConfigurationKey =
FS_AZURE_ABFS_RENAME_RESILIENCE, DefaultValue = DEFAULT_ENABLE_ABFS_RENAME_RESILIENCE)
private boolean renameResilience;

public AbfsConfiguration(final Configuration rawConfig, String accountName)
throws IllegalAccessException, InvalidConfigurationValueException, IOException {
this.rawConfig = ProviderUtils.excludeIncompatibleCredentialProviders(
Expand Down Expand Up @@ -1130,4 +1134,7 @@ public void setEnableAbfsListIterator(boolean enableAbfsListIterator) {
this.enableAbfsListIterator = enableAbfsListIterator;
}

public boolean getRenameResilience() {
return renameResilience;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -233,6 +233,9 @@ public final class ConfigurationKeys {
/** Key for rate limit capacity, as used by IO operations which try to throttle themselves. */
public static final String FS_AZURE_ABFS_IO_RATE_LIMIT = "fs.azure.io.rate.limit";

/** Add extra resilience to rename failures, at the expense of performance. */
public static final String FS_AZURE_ABFS_RENAME_RESILIENCE = "fs.azure.enable.rename.resilience";

public static String accountProperty(String property, String account) {
return property + "." + account;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,7 @@ public final class FileSystemConfigurations {

public static final int STREAM_ID_LEN = 12;
public static final boolean DEFAULT_ENABLE_ABFS_LIST_ITERATOR = true;
public static final boolean DEFAULT_ENABLE_ABFS_RENAME_RESILIENCE = true;

/**
* Limit of queued block upload operations before writes
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import java.util.concurrent.TimeUnit;

import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

import seem to be in the wrong place here.

import org.apache.hadoop.fs.store.LogExactlyOnce;
import org.apache.hadoop.util.Preconditions;
import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
Expand Down Expand Up @@ -68,6 +69,7 @@
import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
import org.apache.hadoop.util.concurrent.HadoopExecutors;

import static org.apache.commons.lang3.StringUtils.isEmpty;
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;
Expand All @@ -77,7 +79,9 @@
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.HttpQueryParams.*;
import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.PATH_NOT_FOUND;
import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND;
import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND;

/**
* AbfsClient.
Expand Down Expand Up @@ -105,6 +109,11 @@ public class AbfsClient implements Closeable {

private final ListeningScheduledExecutorService executorService;

/**
* Enable resilient rename.
*/
private final boolean renameResilience;

/** logging the rename failure if metadata is in an incomplete state. */
private static final LogExactlyOnce ABFS_METADATA_INCOMPLETE_RENAME_FAILURE =
new LogExactlyOnce(LOG);
Expand Down Expand Up @@ -157,6 +166,9 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden
new ThreadFactoryBuilder().setNameFormat("AbfsClient Lease Ops").setDaemon(true).build();
this.executorService = MoreExecutors.listeningDecorator(
HadoopExecutors.newScheduledThreadPool(this.abfsConfiguration.getNumLeaseThreads(), tf));
// rename resilience
renameResilience = abfsConfiguration.getRenameResilience();
LOG.debug("Rename resilience is {}",renameResilience);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: space after ",".

}

public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials,
Expand Down Expand Up @@ -519,11 +531,38 @@ public AbfsClientRenameResult renamePath(
final String destination,
final String continuation,
final TracingContext tracingContext,
final String sourceEtag,
String sourceEtag,
boolean isMetadataIncompleteState)
throws AzureBlobFileSystemException {
final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();

// etag passed in, so source is a file
final boolean hasEtag = !isEmpty(sourceEtag);
boolean isDir = !hasEtag;
if (!hasEtag && renameResilience) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sourceEtag should be passed down only for HNS account as eTag does not remain the same when its FNS (as rename = copy to destination and delete source).

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yeah, but AFAIK there's no way to check in the client whether it is available.

it can only get passed in through the manifest committer, and as ABFS.createResilientCommitSupport() requires etag preservation, it won't be doing it on a non-HNS store.

// no etag was passed in and rename resilience is enabled, so
// get the value

try {
final AbfsRestOperation srcStatusOp = getPathStatus(source,
false, tracingContext);
final AbfsHttpOperation result = srcStatusOp.getResult();

sourceEtag = extractEtagHeader(result);
// and update the directory status.
final String resourceType =
result.getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
isDir = AbfsHttpConstants.DIRECTORY.equalsIgnoreCase(resourceType);
LOG.debug("Retrieved etag of source for rename recovery: {}; isDir={}", sourceEtag, isDir);
} catch (AbfsRestOperationException e) {
// switch file not found to source not found
if (PATH_NOT_FOUND.equals(e.getErrorCode())) {
throw new AbfsRestOperationException(e.getStatusCode(), SOURCE_PATH_NOT_FOUND.getErrorCode(),
e.getMessage(), e);
}
}
}

String encodedRenameSource = urlEncode(FORWARD_SLASH + this.getFileSystem() + source);
if (authType == AuthType.SAS) {
final AbfsUriQueryBuilder srcQueryBuilder = new AbfsUriQueryBuilder();
Expand All @@ -540,12 +579,7 @@ public AbfsClientRenameResult renamePath(
appendSASTokenToQuery(destination, SASTokenProvider.RENAME_DESTINATION_OPERATION, abfsUriQueryBuilder);

final URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString());
final AbfsRestOperation op = new AbfsRestOperation(
AbfsRestOperationType.RenamePath,
this,
HTTP_METHOD_PUT,
url,
requestHeaders);
final AbfsRestOperation op = createRenameRestOperation(url, requestHeaders);
try {
incrementAbfsRenamePath();
op.execute(tracingContext);
Expand All @@ -560,6 +594,7 @@ public AbfsClientRenameResult renamePath(
if (!op.hasResult()) {
throw e;
}
LOG.debug("Rename of {} to {} failed, attempting recovery", source, destination, e);

// ref: HADOOP-18242. Rename failure occurring due to a rare case of
// tracking metadata being in incomplete state.
Expand Down Expand Up @@ -588,7 +623,7 @@ public AbfsClientRenameResult renamePath(

boolean etagCheckSucceeded = renameIdempotencyCheckOp(
source,
sourceEtag, op, destination, tracingContext);
sourceEtag, op, destination, tracingContext, isDir);
if (!etagCheckSucceeded) {
// idempotency did not return different result
// throw back the exception
Expand All @@ -598,6 +633,17 @@ public AbfsClientRenameResult renamePath(
}
}

@VisibleForTesting
AbfsRestOperation createRenameRestOperation(URL url, List<AbfsHttpHeader> requestHeaders) {
AbfsRestOperation op = new AbfsRestOperation(
AbfsRestOperationType.RenamePath,
this,
HTTP_METHOD_PUT,
url,
requestHeaders);
return op;
}

private void incrementAbfsRenamePath() {
abfsCounters.incrementCounter(RENAME_PATH_ATTEMPTS, 1);
}
Expand All @@ -613,40 +659,60 @@ private void incrementAbfsRenamePath() {
* Exceptions raised in the probe of the destination are swallowed,
* so that they do not interfere with the original rename failures.
* @param source source path
* @param sourceEtag etag of source file. may be null or empty
* @param op Rename request REST operation response with non-null HTTP response
* @param destination rename destination path
* @param sourceEtag etag of source file. may be null or empty
* @param tracingContext Tracks identifiers for request header
* @param isDir is the source a file or directory
* @return true if the file was successfully copied
*/
public boolean renameIdempotencyCheckOp(
final String source,
final String sourceEtag,
final AbfsRestOperation op,
final String destination,
TracingContext tracingContext) {
TracingContext tracingContext,
final boolean isDir) {
Preconditions.checkArgument(op.hasResult(), "Operations has null HTTP response");

if ((op.isARetriedRequest())
&& (op.getResult().getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND)
&& isNotEmpty(sourceEtag)) {
LOG.debug("rename({}, {}) failure {}; retry={} isDir {} etag {}",
source, destination, op.getResult().getStatusCode(), op.isARetriedRequest(), isDir, sourceEtag);
if (!(op.isARetriedRequest()
&& (op.getResult().getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND))) {
// only attempt recovery if the failure was a 404 on a retried rename request.
return false;
}

// Server has returned HTTP 404, which means rename source no longer
// exists. Check on destination status and if its etag matches
// that of the source, consider it to be a success.
LOG.debug("rename {} to {} failed, checking etag of destination",
if (isDir) {
// directory recovery is not supported.
// log and fail.
LOG.info("rename directory {} to {} failed; unable to recover",
source, destination);
return false;
}
if (isNotEmpty(sourceEtag)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As per code in renamePath(),

isDir = !isNotEmpty(sourceEtag)

Should we use the same relation in this method to reduce confusion, instead of having a new argument isDir.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, but I want to log slightly differently. though actually it is confusion as isDir is true if resilience is off. will cut

// Server has returned HTTP 404, we have an etag, so see
// if the rename has actually taken place,
LOG.info("rename {} to {} failed, checking etag of destination",
source, destination);

try {
final AbfsRestOperation destStatusOp = getPathStatus(destination,
false, tracingContext);
final AbfsHttpOperation result = destStatusOp.getResult();

return result.getStatusCode() == HttpURLConnection.HTTP_OK
final boolean recovered = result.getStatusCode() == HttpURLConnection.HTTP_OK
&& sourceEtag.equals(extractEtagHeader(result));
} catch (AzureBlobFileSystemException ignored) {
LOG.info("File rename has taken place: recovery {}",
recovered ? "succeeded" : "failed");
return recovered;
} catch (AzureBlobFileSystemException ex) {
// GetFileStatus on the destination failed, the rename did not take place
// or some other failure. log and swallow.
LOG.debug("Failed to get status of path {}", destination, ex);
}
} else {
LOG.debug("No source etag; unable to probe for the operation's success");
}
return false;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,4 +58,13 @@ public boolean isRenameRecovered() {
public boolean isIncompleteMetadataState() {
return isIncompleteMetadataState;
}

@Override
public String toString() {
return "AbfsClientRenameResult{" +
"op=" + op +
", renameRecovered=" + renameRecovered +
", isIncompleteMetadataState=" + isIncompleteMetadataState +
'}';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -264,26 +264,7 @@ private boolean executeHttpOperation(final int retryCount,
incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1);
tracingContext.constructHeader(httpOperation, failureReason);

switch(client.getAuthType()) {
case Custom:
case OAuth:
LOG.debug("Authenticating request with OAuth2 access token");
httpOperation.getConnection().setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
client.getAccessToken());
break;
case SAS:
// do nothing; the SAS token should already be appended to the query string
httpOperation.setMaskForSAS(); //mask sig/oid from url for logs
break;
case SharedKey:
// sign the HTTP request
LOG.debug("Signing request with shared key");
// sign the HTTP request
client.getSharedKeyCredentials().signRequest(
httpOperation.getConnection(),
hasRequestBody ? bufferLength : 0);
break;
}
signRequest(httpOperation, hasRequestBody ? bufferLength : 0);
} catch (IOException e) {
LOG.debug("Auth failure: {}, {}", method, url);
throw new AbfsRestOperationException(-1, null,
Expand Down Expand Up @@ -351,6 +332,36 @@ private boolean executeHttpOperation(final int retryCount,
return true;
}

/**
* Sign an operation.
* @param httpOperation operation to sign
* @param bytesToSign how many bytes to sign for shared key auth.
* @throws IOException failure
*/
@VisibleForTesting
public void signRequest(final AbfsHttpOperation httpOperation, int bytesToSign) throws IOException {
switch(client.getAuthType()) {
case Custom:
case OAuth:
LOG.debug("Authenticating request with OAuth2 access token");
httpOperation.getConnection().setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
client.getAccessToken());
break;
case SAS:
// do nothing; the SAS token should already be appended to the query string
httpOperation.setMaskForSAS(); //mask sig/oid from url for logs
break;
case SharedKey:
// sign the HTTP request
LOG.debug("Signing request with shared key");
// sign the HTTP request
client.getSharedKeyCredentials().signRequest(
httpOperation.getConnection(),
bytesToSign);
break;
}
}

/**
* Creates new object of {@link AbfsHttpOperation} with the url, method, and
* requestHeaders fields of the AbfsRestOperation object.
Expand Down
Original file line number Diff line number Diff line change
@@ -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.azurebfs.contract;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;

/**
* Contract test for rename operation with rename resilience disabled.
* This is critical to ensure that adding resilience does not cause
* any regressions when disabled.
*/
public class ITestAbfsContractRenameWithoutResilience
extends ITestAbfsFileSystemContractRename {

public ITestAbfsContractRenameWithoutResilience() throws Exception {
}

@Override
protected Configuration createConfiguration() {
final Configuration conf = super.createConfiguration();
conf.setBoolean(ConfigurationKeys.FS_AZURE_ABFS_RENAME_RESILIENCE, false);
return conf;
}

}
Loading