From 5a2ef896128b2aecf5ff2c9b83281d1d517628db Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Wed, 25 Mar 2020 13:24:15 -0400 Subject: [PATCH 01/18] HADOOP-16948. Support single writer dirs. --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 13 + .../fs/azurebfs/AzureBlobFileSystem.java | 49 ++++ .../fs/azurebfs/AzureBlobFileSystemStore.java | 96 ++++++- .../azurebfs/constants/AbfsHttpConstants.java | 5 + .../azurebfs/constants/ConfigurationKeys.java | 3 + .../constants/FileSystemConfigurations.java | 1 + .../constants/HttpHeaderConfigurations.java | 5 + .../services/AppendRequestParameters.java | 8 +- .../fs/azurebfs/services/AbfsClient.java | 86 +++++- .../fs/azurebfs/services/AbfsErrors.java | 36 +++ .../azurebfs/services/AbfsOutputStream.java | 50 +++- .../services/AbfsOutputStreamContext.java | 11 + .../azurebfs/services/AbfsRestOperation.java | 1 + .../services/AbfsRestOperationType.java | 3 +- .../azurebfs/services/SelfRenewingLease.java | 199 ++++++++++++++ .../ITestAzureBlobFileSystemLease.java | 247 ++++++++++++++++++ .../services/TestAbfsOutputStream.java | 63 ++--- 17 files changed, 831 insertions(+), 45 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index f36cc7d5bfde0..55b6a308c8fbf 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -208,6 +208,10 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_FS_AZURE_APPEND_BLOB_DIRECTORIES) private String azureAppendBlobDirs; + @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_SINGLE_WRITER_KEY, + DefaultValue = DEFAULT_FS_AZURE_SINGLE_WRITER_DIRECTORIES) + private String azureSingleWriterDirs; + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, DefaultValue = DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION) private boolean createRemoteFileSystemDuringInitialization; @@ -634,6 +638,10 @@ public String getAppendBlobDirs() { return this.azureAppendBlobDirs; } + public String getAzureSingleWriterDirs() { + return this.azureSingleWriterDirs; + } + public boolean getCreateRemoteFileSystemDuringInitialization() { // we do not support creating the filesystem when AuthType is SAS return this.createRemoteFileSystemDuringInitialization @@ -933,6 +941,11 @@ void setDisableOutputStreamFlush(boolean disableOutputStreamFlush) { this.disableOutputStreamFlush = disableOutputStreamFlush; } + @VisibleForTesting + void setAzureSingleWriterDirs(String dirs) { + this.azureSingleWriterDirs = dirs; + } + @VisibleForTesting void setListMaxResults(int listMaxResults) { this.listMaxResults = listMaxResults; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index d8a2ed7bcd542..faa1fdc922044 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -505,6 +505,55 @@ public FileStatus getFileStatus(final Path f) throws IOException { } } + public String acquireLease(final Path f, final int duration) throws IOException { + LOG.debug("AzureBlobFileSystem.acquireLease path: {}", f); + + Path qualifiedPath = makeQualified(f); + + try { + return abfsStore.acquireLease(qualifiedPath, duration); + } catch(AzureBlobFileSystemException ex) { + checkException(f, ex); + return null; + } + } + + public void renewLease(final Path f, final String leaseId) throws IOException { + LOG.debug("AzureBlobFileSystem.renewLease path: {} id: {}", f, leaseId); + + Path qualifiedPath = makeQualified(f); + + try { + abfsStore.renewLease(qualifiedPath, leaseId); + } catch(AzureBlobFileSystemException ex) { + checkException(f, ex); + } + } + + public void releaseLease(final Path f, final String leaseId) throws IOException { + LOG.debug("AzureBlobFileSystem.releaseLease path: {} id: {}", f, leaseId); + + Path qualifiedPath = makeQualified(f); + + try { + abfsStore.releaseLease(qualifiedPath, leaseId); + } catch(AzureBlobFileSystemException ex) { + checkException(f, ex); + } + } + + public void breakLease(final Path f) throws IOException { + LOG.debug("AzureBlobFileSystem.breakLease path: {}", f); + + Path qualifiedPath = makeQualified(f); + + try { + abfsStore.breakLease(qualifiedPath); + } catch(AzureBlobFileSystemException ex) { + checkException(f, ex); + } + } + /** * Qualify a path to one which uses this FileSystem and, if relative, * made absolute. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 75419c26dd214..59304efda8968 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -39,6 +39,7 @@ import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.HashSet; @@ -48,6 +49,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.WeakHashMap; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -100,6 +102,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy; +import org.apache.hadoop.fs.azurebfs.services.SelfRenewingLease; import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials; import org.apache.hadoop.fs.azurebfs.services.AbfsPerfTracker; import org.apache.hadoop.fs.azurebfs.services.AbfsPerfInfo; @@ -145,8 +148,11 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport { private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1"; private static final int GET_SET_AGGREGATE_COUNT = 2; + private final Map leaseRefs; + private final AbfsConfiguration abfsConfiguration; private final Set azureAtomicRenameDirSet; + private Set azureSingleWriterDirSet; private Trilean isNamespaceEnabled; private final AuthType authType; private final UserGroupInformation userGroupInformation; @@ -167,6 +173,8 @@ public AzureBlobFileSystemStore(URI uri, boolean isSecureScheme, final String fileSystemName = authorityParts[0]; final String accountName = authorityParts[1]; + leaseRefs = Collections.synchronizedMap(new WeakHashMap<>()); + try { this.abfsConfiguration = new AbfsConfiguration(configuration, accountName); } catch (IllegalAccessException exception) { @@ -195,6 +203,7 @@ public AzureBlobFileSystemStore(URI uri, boolean isSecureScheme, this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList( abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA))); + updateSingleWriterDirs(); this.authType = abfsConfiguration.getAuthType(accountName); boolean usingOauth = (authType == AuthType.OAuth); boolean useHttps = (usingOauth || abfsConfiguration.isHttpsAlwaysUsed()) ? true : isSecureScheme; @@ -246,6 +255,16 @@ public String getPrimaryGroup() { @Override public void close() throws IOException { + for (SelfRenewingLease lease : leaseRefs.keySet()) { + if (lease == null) { + continue; + } + try { + lease.free(); + } catch (Exception e) { + LOG.debug("Got exception freeing lease {}", lease.getLeaseID(), e); + } + } IOUtils.cleanupWithLogger(LOG, client); } @@ -454,13 +473,16 @@ public OutputStream createFile(final Path path, final FsPermission umask) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(); - LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}", + boolean enableSingleWriter = isSingleWriterKey(path.toString()); + LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} " + + "isNamespaceEnabled: {} enableSingleWriter: {}", client.getFileSystem(), path, overwrite, permission, umask, - isNamespaceEnabled); + isNamespaceEnabled, + enableSingleWriter); String relativePath = getRelativePath(path); boolean isAppendBlob = false; @@ -501,7 +523,8 @@ public OutputStream createFile(final Path path, statistics, relativePath, 0, - populateAbfsOutputStreamContext(isAppendBlob)); + leaseRefs, + populateAbfsOutputStreamContext(isAppendBlob, enableSingleWriter)); } } @@ -573,7 +596,8 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa return op; } - private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppendBlob) { + private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppendBlob, + boolean enableSingleWriter) { int bufferSize = abfsConfiguration.getWriteBufferSize(); if (isAppendBlob && bufferSize > FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE) { bufferSize = FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE; @@ -587,6 +611,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppend .withAppendBlob(isAppendBlob) .withWriteMaxConcurrentRequestCount(abfsConfiguration.getWriteMaxConcurrentRequestCount()) .withMaxWriteRequestsToQueue(abfsConfiguration.getMaxWriteRequestsToQueue()) + .withSingleWriterEnabled(enableSingleWriter) .build(); } @@ -675,10 +700,12 @@ private AbfsInputStreamContext populateAbfsInputStreamContext( public OutputStream openFileForWrite(final Path path, final FileSystem.Statistics statistics, final boolean overwrite) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("openFileForWrite", "getPathStatus")) { - LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}", + boolean enableSingleWriter = isSingleWriterKey(path.toString()); + LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {} enableSingleWriter: {}", client.getFileSystem(), path, - overwrite); + overwrite, + enableSingleWriter); String relativePath = getRelativePath(path); @@ -710,10 +737,41 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic statistics, relativePath, offset, - populateAbfsOutputStreamContext(isAppendBlob)); + leaseRefs, + populateAbfsOutputStreamContext(isAppendBlob, enableSingleWriter)); } } + public String acquireLease(final Path path, final int duration) throws AzureBlobFileSystemException { + LOG.debug("lease path: {}", path); + + final AbfsRestOperation op = + client.acquireLease(getRelativePath(path), duration); + + return op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID); + } + + public void renewLease(final Path path, final String leaseId) throws AzureBlobFileSystemException { + LOG.debug("lease path: {}, renew lease id: {}", path, leaseId); + + final AbfsRestOperation op = + client.renewLease(getRelativePath(path), leaseId); + } + + public void releaseLease(final Path path, final String leaseId) throws AzureBlobFileSystemException { + LOG.debug("lease path: {}, release lease id: {}", path, leaseId); + + final AbfsRestOperation op = + client.releaseLease(getRelativePath(path), leaseId); + } + + public void breakLease(final Path path) throws AzureBlobFileSystemException { + LOG.debug("lease path: {}", path); + + final AbfsRestOperation op = + client.breakLease(getRelativePath(path)); + } + public void rename(final Path source, final Path destination) throws AzureBlobFileSystemException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); @@ -1347,6 +1405,13 @@ public boolean isAtomicRenameKey(String key) { return isKeyForDirectorySet(key, azureAtomicRenameDirSet); } + public boolean isSingleWriterKey(String key) { + if (azureSingleWriterDirSet.isEmpty()) { + return false; + } + return isKeyForDirectorySet(key, azureSingleWriterDirSet); + } + /** * A on-off operation to initialize AbfsClient for AzureBlobFileSystem * Operations. @@ -1636,4 +1701,21 @@ void setNamespaceEnabled(Trilean isNamespaceEnabled){ this.isNamespaceEnabled = isNamespaceEnabled; } + void updateSingleWriterDirs() { + this.azureSingleWriterDirSet = new HashSet<>(Arrays.asList( + abfsConfiguration.getAzureSingleWriterDirs().split(AbfsHttpConstants.COMMA))); + // remove the empty string, since isKeyForDirectory returns true for empty strings + // and we don't want to default to enabling single writer dirs + this.azureSingleWriterDirSet.remove(""); + } + + @VisibleForTesting + boolean areLeasesFreed() { + for (SelfRenewingLease lease : leaseRefs.keySet()) { + if (lease != null && !lease.isFreed()) { + return false; + } + } + return true; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java index 184657e7d66ad..5cf7ec565b59e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java @@ -39,6 +39,11 @@ public final class AbfsHttpConstants { public static final String GET_ACCESS_CONTROL = "getAccessControl"; public static final String CHECK_ACCESS = "checkAccess"; public static final String GET_STATUS = "getStatus"; + public static final String ACQUIRE_LEASE_ACTION = "acquire"; + public static final String BREAK_LEASE_ACTION = "break"; + public static final String RELEASE_LEASE_ACTION = "release"; + public static final String RENEW_LEASE_ACTION = "renew"; + public static final String DEFAULT_LEASE_BREAK_PERIOD = "0"; public static final String DEFAULT_TIMEOUT = "90"; public static final String APPEND_BLOB_TYPE = "appendblob"; public static final String TOKEN_VERSION = "2"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 02b143cd61bd3..d2b31ca69f41d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -87,6 +87,9 @@ public final class ConfigurationKeys { /** Provides a config to provide comma separated path prefixes on which Appendblob based files are created * Default is empty. **/ public static final String FS_AZURE_APPEND_BLOB_KEY = "fs.azure.appendblob.directories"; + /** Provides a config to provide comma separated path prefixes which support single writers. + * Default is empty. **/ + public static final String FS_AZURE_SINGLE_WRITER_KEY = "fs.azure.singlewriter.directories"; public static final String FS_AZURE_READ_AHEAD_QUEUE_DEPTH = "fs.azure.readaheadqueue.depth"; public static final String FS_AZURE_ALWAYS_READ_BUFFER_SIZE = "fs.azure.read.alwaysReadBufferSize"; public static final String FS_AZURE_READ_AHEAD_BLOCK_SIZE = "fs.azure.read.readahead.blocksize"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index d90f525712af7..9353609a0f2a8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -78,6 +78,7 @@ public final class FileSystemConfigurations { public static final boolean DEFAULT_FS_AZURE_ENABLE_CONDITIONAL_CREATE_OVERWRITE = true; public static final boolean DEFAULT_FS_AZURE_ENABLE_MKDIR_OVERWRITE = true; public static final String DEFAULT_FS_AZURE_APPEND_BLOB_DIRECTORIES = ""; + public static final String DEFAULT_FS_AZURE_SINGLE_WRITER_DIRECTORIES = ""; public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = -1; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java index 27ddcee695aaa..232553844fcf3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java @@ -60,6 +60,11 @@ public final class HttpHeaderConfigurations { public static final String X_MS_UMASK = "x-ms-umask"; public static final String X_MS_NAMESPACE_ENABLED = "x-ms-namespace-enabled"; public static final String X_MS_ABFS_CLIENT_LATENCY = "x-ms-abfs-client-latency"; + public static final String X_MS_LEASE_ACTION = "x-ms-lease-action"; + public static final String X_MS_LEASE_DURATION = "x-ms-lease-duration"; + public static final String X_MS_LEASE_ID = "x-ms-lease-id"; + public static final String X_MS_PROPOSED_LEASE_ID = "x-ms-proposed-lease-id"; + public static final String X_MS_LEASE_BREAK_PERIOD = "x-ms-lease-break-period"; private HttpHeaderConfigurations() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java index fb4d29f87949a..7369bfaf56422 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java @@ -33,17 +33,20 @@ public enum Mode { private final int length; private final Mode mode; private final boolean isAppendBlob; + private final String leaseId; public AppendRequestParameters(final long position, final int offset, final int length, final Mode mode, - final boolean isAppendBlob) { + final boolean isAppendBlob, + final String leaseId) { this.position = position; this.offset = offset; this.length = length; this.mode = mode; this.isAppendBlob = isAppendBlob; + this.leaseId = leaseId; } public long getPosition() { @@ -66,4 +69,7 @@ public boolean isAppendBlob() { return this.isAppendBlob; } + public String getLeaseId() { + return this.leaseId; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 92b24f0dda212..db92b496f4b40 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -29,6 +29,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Locale; +import java.util.UUID; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; @@ -317,6 +318,83 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, fin return op; } + public AbfsRestOperation acquireLease(final String path, int duration) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, ACQUIRE_LEASE_ACTION)); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_DURATION, Integer.toString(duration))); + requestHeaders.add(new AbfsHttpHeader(X_MS_PROPOSED_LEASE_ID, UUID.randomUUID().toString())); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.LeasePath, + this, + HTTP_METHOD_POST, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation renewLease(final String path, final String leaseId) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, RENEW_LEASE_ACTION)); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.LeasePath, + this, + HTTP_METHOD_POST, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation releaseLease(final String path, final String leaseId) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, RELEASE_LEASE_ACTION)); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.LeasePath, + this, + HTTP_METHOD_POST, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation breakLease(final String path) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, BREAK_LEASE_ACTION)); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_BREAK_PERIOD, DEFAULT_LEASE_BREAK_PERIOD)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.LeasePath, + this, + HTTP_METHOD_POST, + url, + requestHeaders); + op.execute(); + return op; + } + public AbfsRestOperation renamePath(String source, final String destination, final String continuation) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); @@ -416,6 +494,9 @@ public AbfsRestOperation append(final String path, final byte[] buffer, // PUT and specify the real method in the X-Http-Method-Override header. requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); + if (reqParams.getLeaseId() != null) { + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, reqParams.getLeaseId())); + } final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION); @@ -492,13 +573,16 @@ public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path, } public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData, - boolean isClose, final String cachedSasToken) + boolean isClose, final String cachedSasToken, final String leaseId) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); + if (leaseId != null) { + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId)); + } final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, FLUSH_ACTION); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java new file mode 100644 index 0000000000000..e066f1f3fab38 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java @@ -0,0 +1,36 @@ +/** + * 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.services; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * ABFS error constants. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class AbfsErrors { + public static final String ERR_WRITE_WITHOUT_LEASE = "Attempted to write to file without lease"; + public static final String ERR_LEASE_EXPIRED = "A lease ID was specified, but the lease for " + + "the resource has expired."; + public static final String ERR_PARALLEL_ACCESS_DETECTED = "Parallel access to the create path " + + "detected. Failing request to honor single writer semantics"; + public static final String ERR_ACQUIRING_LEASE = "Unable to acquire lease"; + public static final String ERR_LEASE_PRESENT = "There is already a lease present."; +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 2d02019ab11c6..c66ed4dd7164e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -24,6 +24,7 @@ import java.io.OutputStream; import java.net.HttpURLConnection; import java.nio.ByteBuffer; +import java.util.Map; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ExecutorCompletionService; @@ -37,6 +38,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; @@ -50,9 +52,11 @@ import org.apache.hadoop.io.ElasticByteBufferPool; import org.apache.hadoop.fs.FileSystem.Statistics; import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.Syncable; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_WRITE_WITHOUT_LEASE; import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable; import static org.apache.hadoop.io.IOUtils.wrapException; import static org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode.APPEND_MODE; @@ -92,6 +96,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable, // SAS tokens can be re-used until they expire private CachedSASToken cachedSasToken; + private SelfRenewingLease lease; + private String leaseId; + /** * Queue storing buffers with the size of the Azure block ready for * reuse. The pool allows reusing the blocks instead of allocating new @@ -113,7 +120,8 @@ public AbfsOutputStream( final Statistics statistics, final String path, final long position, - AbfsOutputStreamContext abfsOutputStreamContext) { + final Map leaseRefs, + AbfsOutputStreamContext abfsOutputStreamContext) throws AzureBlobFileSystemException { this.client = client; this.statistics = statistics; this.path = path; @@ -142,6 +150,15 @@ public AbfsOutputStream( } this.maxRequestsThatCanBeQueued = abfsOutputStreamContext .getMaxWriteRequestsToQueue(); + + if (abfsOutputStreamContext.isEnableSingleWriter()) { + lease = new SelfRenewingLease(client, new Path(path)); + this.leaseId = lease.getLeaseID(); + if (leaseRefs != null) { + leaseRefs.put(lease, null); + } + } + this.threadExecutor = new ThreadPoolExecutor(maxConcurrentRequestCount, maxConcurrentRequestCount, @@ -203,6 +220,10 @@ public synchronized void write(final byte[] data, final int off, final int lengt throw new IndexOutOfBoundsException(); } + if (lease != null && lease.isFreed()) { + throw new PathIOException(path, ERR_WRITE_WITHOUT_LEASE); + } + int currentOffset = off; int writableBytes = bufferSize - bufferIndex; int numberOfBytesToWrite = length; @@ -306,6 +327,13 @@ public synchronized void close() throws IOException { // See HADOOP-16785 throw wrapException(path, e.getMessage(), e); } finally { + if (lease != null) { + if (LOG.isDebugEnabled()) { + LOG.info("Freeing lease {}", leaseId); + } + lease.free(); + lease = null; + } lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED); buffer = null; bufferIndex = 0; @@ -372,7 +400,7 @@ private void writeAppendBlobCurrentBufferToService() throws IOException { try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "writeCurrentBufferToService", "append")) { AppendRequestParameters reqParams = new AppendRequestParameters(offset, 0, - bytesLength, APPEND_MODE, true); + bytesLength, APPEND_MODE, true, leaseId); AbfsRestOperation op = client.append(path, bytes, reqParams, cachedSasToken.get()); cachedSasToken.update(op.getSasToken()); if (outputStreamStatistics != null) { @@ -448,7 +476,7 @@ private synchronized void writeCurrentBufferToService(boolean isFlush, boolean i mode = FLUSH_MODE; } AppendRequestParameters reqParams = new AppendRequestParameters( - offset, 0, bytesLength, mode, false); + offset, 0, bytesLength, mode, false, leaseId); AbfsRestOperation op = client.append(path, bytes, reqParams, cachedSasToken.get()); cachedSasToken.update(op.getSasToken()); @@ -517,7 +545,8 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "flushWrittenBytesToServiceInternal", "flush")) { - AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose, cachedSasToken.get()); + AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose, + cachedSasToken.get(), leaseId); cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()).registerSuccess(true); } catch (AzureBlobFileSystemException ex) { @@ -637,6 +666,19 @@ public IOStatistics getIOStatistics() { return ioStatistics; } + @VisibleForTesting + public boolean isLeaseFreed() { + if (lease == null) { + return true; + } + return lease.isFreed(); + } + + @VisibleForTesting + public boolean hasLease() { + return lease != null; + } + /** * Appending AbfsOutputStream statistics to base toString(). * diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java index 925cd4f7b5646..301954f7a73db 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java @@ -39,6 +39,8 @@ public class AbfsOutputStreamContext extends AbfsStreamContext { private int maxWriteRequestsToQueue; + private boolean enableSingleWriter; + public AbfsOutputStreamContext(final long sasTokenRenewPeriodForStreamsInSeconds) { super(sasTokenRenewPeriodForStreamsInSeconds); } @@ -94,6 +96,11 @@ public AbfsOutputStreamContext withMaxWriteRequestsToQueue( return this; } + public AbfsOutputStreamContext withSingleWriterEnabled(final boolean enableSingleWriter) { + this.enableSingleWriter = enableSingleWriter; + return this; + } + public int getWriteBufferSize() { return writeBufferSize; } @@ -125,4 +132,8 @@ public int getMaxWriteRequestsToQueue() { public boolean isEnableSmallWriteOptimization() { return this.enableSmallWriteOptimization; } + + public boolean isEnableSingleWriter() { + return this.enableSingleWriter; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java index 24ec2926647e6..1174892fd762a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java @@ -132,6 +132,7 @@ String getSasToken() { this.url = url; this.requestHeaders = requestHeaders; this.hasRequestBody = (AbfsHttpConstants.HTTP_METHOD_PUT.equals(method) + || AbfsHttpConstants.HTTP_METHOD_POST.equals(method) || AbfsHttpConstants.HTTP_METHOD_PATCH.equals(method)); this.sasToken = sasToken; this.abfsCounters = client.getAbfsCounters(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.java index d3031860dd1c2..830297f381b91 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.java @@ -40,5 +40,6 @@ public enum AbfsRestOperationType { Flush, ReadFile, DeletePath, - CheckAccess + CheckAccess, + LeasePath, } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java new file mode 100644 index 0000000000000..43de22a0ad00b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java @@ -0,0 +1,199 @@ +/** + * 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.services; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_ACQUIRING_LEASE; + +/** + * An Azure blob lease that automatically renews itself indefinitely + * using a background thread. Use it to synchronize distributed processes, + * or to prevent writes to the blob by other processes that don't + * have the lease. + * + * Creating a new Lease object blocks the caller until the Azure blob lease is + * acquired. + * + * Call free() to release the Lease. + * + * You can use this Lease like a distributed lock. If the holder process + * dies, the lease will time out since it won't be renewed. + * + * See also {@link org.apache.hadoop.fs.azure.SelfRenewingLease}. + */ +public class SelfRenewingLease { + + private final AbfsClient client; + private final Path path; + private Thread renewer; + private volatile boolean leaseFreed; + private String leaseID = null; + private static final int LEASE_TIMEOUT = 60; // Lease timeout in seconds + + // Time to wait to renew lease in milliseconds + public static final int LEASE_RENEWAL_PERIOD = 40000; + public static final Logger LOG = LoggerFactory.getLogger(SelfRenewingLease.class); + + // Used to allocate thread serial numbers in thread name + private static AtomicInteger threadNumber = new AtomicInteger(0); + + + // Time to wait to retry getting the lease in milliseconds + static final int LEASE_ACQUIRE_RETRY_INTERVAL = 2000; + static final int LEASE_MAX_RETRIES = 5; + + public static class LeaseException extends AzureBlobFileSystemException { + public LeaseException(Exception innerException) { + super(ERR_ACQUIRING_LEASE, innerException); + } + } + + public SelfRenewingLease(AbfsClient client, Path path) throws AzureBlobFileSystemException { + + this.leaseFreed = false; + this.client = client; + this.path = path; + + // Try to get the lease a specified number of times, else throw an error + int numRetries = 0; + while (leaseID == null && numRetries < LEASE_MAX_RETRIES) { + numRetries++; + try { + LOG.debug("lease path: {}", path); + final AbfsRestOperation op = + client.acquireLease(getRelativePath(path), + LEASE_TIMEOUT); + + leaseID = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID); + } catch (IOException e) { + if (numRetries < LEASE_MAX_RETRIES) { + LOG.info("Caught exception when trying to acquire lease on blob {}, retrying: {}", path, + e.getMessage()); + LOG.debug("Exception acquiring lease", e); + } else { + throw new LeaseException(e); + } + } + if (leaseID == null) { + try { + Thread.sleep(LEASE_ACQUIRE_RETRY_INTERVAL); + } catch (InterruptedException e) { + + // Restore the interrupted status + Thread.currentThread().interrupt(); + } + } + } + renewer = new Thread(new Renewer()); + + // A Renewer running should not keep JVM from exiting, so make it a daemon. + renewer.setDaemon(true); + renewer.setName("AzureBFSLeaseRenewer-" + threadNumber.getAndIncrement()); + renewer.start(); + LOG.debug("Acquired lease {} on {} managed by thread {}", leaseID, path, renewer.getName()); + } + + /** + * Free the lease and stop the keep-alive thread. + */ + public void free() { + try { + LOG.debug("lease path: {}, release lease id: {}", path, leaseID); + client.releaseLease(getRelativePath(path), leaseID); + } catch (IOException e) { + LOG.info("Exception when trying to release lease {} on {}. Lease will be left to expire: {}", + leaseID, path, e.getMessage()); + LOG.debug("Exception releasing lease", e); + } finally { + + // Even if releasing the lease fails (e.g. because the file was deleted), + // make sure to record that we freed the lease, to terminate the + // keep-alive thread. + leaseFreed = true; + LOG.debug("Freed lease {} on {} managed by thread {}", leaseID, path, renewer.getName()); + } + } + + public boolean isFreed() { + return leaseFreed; + } + + public String getLeaseID() { + return leaseID; + } + + private class Renewer implements Runnable { + + /** + * Start a keep-alive thread that will continue to renew + * the lease until it is freed or the process dies. + */ + @Override + public void run() { + LOG.debug("Starting lease keep-alive thread."); + + while (!leaseFreed) { + try { + Thread.sleep(LEASE_RENEWAL_PERIOD); + } catch (InterruptedException e) { + LOG.debug("Keep-alive thread for lease {} interrupted", leaseID); + + // Restore the interrupted status + Thread.currentThread().interrupt(); + } + try { + if (!leaseFreed) { + LOG.debug("lease path: {}, renew lease id: {}", path, leaseID); + client.renewLease(getRelativePath(path), leaseID); + + // It'll be very rare to renew the lease (most will be short) + // so log that we did it, to help with system debugging. + LOG.info("Renewed lease {} on {}", leaseID, path); + } + } catch (IOException e) { + if (!leaseFreed) { + + // Free the lease so we don't leave this thread running forever. + leaseFreed = true; + + // Normally leases should be freed and there should be no + // exceptions, so log a warning. + LOG.warn("Attempt to renew lease {} on {} failed, stopping renewal thread: ", + leaseID, path, e.getMessage()); + LOG.debug("Exception renewing lease", e); + } + } + } + } + } + + private String getRelativePath(final Path path) { + Preconditions.checkNotNull(path, "path"); + return path.toUri().getPath(); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java new file mode 100644 index 0000000000000..8022115704906 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -0,0 +1,247 @@ +/** + * 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; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_ACQUIRING_LEASE; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_EXPIRED; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_PARALLEL_ACCESS_DETECTED; + +/** + * Test lease operations. + */ +public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest { + private static final String TEST_FILE = "testfile"; + + public ITestAzureBlobFileSystemLease() throws Exception { + super(); + } + + @Test + public void testNoSingleWriter() throws IOException { + final AzureBlobFileSystem fs = getFileSystem(); + final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + fs.mkdirs(testFilePath.getParent()); + try (FSDataOutputStream out = fs.create(testFilePath)) { + Assert.assertFalse("Output stream should not have lease", + ((AbfsOutputStream)out.getWrappedStream()).hasLease()); + } + Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + } + + @Test + public void testOneWriter() throws IOException { + final AzureBlobFileSystem fs = getFileSystem(); + final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + fs.mkdirs(testFilePath.getParent()); + fs.getAbfsStore().getAbfsConfiguration() + .setAzureSingleWriterDirs(testFilePath.getParent().toString()); + fs.getAbfsStore().updateSingleWriterDirs(); + + FSDataOutputStream out = fs.create(testFilePath); + Assert.assertTrue("Output stream should have lease", + ((AbfsOutputStream)out.getWrappedStream()).hasLease()); + out.close(); + Assert.assertFalse("Output stream should not have lease", + ((AbfsOutputStream)out.getWrappedStream()).hasLease()); + Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + } + + @Test + public void testSubDir() throws IOException { + final AzureBlobFileSystem fs = getFileSystem(); + final Path testFilePath = new Path(new Path(path(methodName.getMethodName()), "subdir"), + TEST_FILE); + fs.mkdirs(testFilePath.getParent().getParent()); + fs.getAbfsStore().getAbfsConfiguration() + .setAzureSingleWriterDirs(testFilePath.getParent().getParent().toString()); + fs.getAbfsStore().updateSingleWriterDirs(); + + FSDataOutputStream out = fs.create(testFilePath); + Assert.assertTrue("Output stream should have lease", + ((AbfsOutputStream)out.getWrappedStream()).hasLease()); + out.close(); + Assert.assertFalse("Output stream should not have lease", + ((AbfsOutputStream)out.getWrappedStream()).hasLease()); + Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + } + + @Test + public void testTwoCreate() throws IOException { + final AzureBlobFileSystem fs = getFileSystem(); + final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + fs.mkdirs(testFilePath.getParent()); + fs.getAbfsStore().getAbfsConfiguration() + .setAzureSingleWriterDirs(testFilePath.getParent().toString()); + fs.getAbfsStore().updateSingleWriterDirs(); + + try (FSDataOutputStream out = fs.create(testFilePath)) { + try (FSDataOutputStream out2 = fs.create(testFilePath)) { + Assert.fail("Second create succeeded"); + } catch (IOException e) { + Assert.assertTrue("Unexpected error message: " + e.getMessage(), + e.getMessage().contains(ERR_PARALLEL_ACCESS_DETECTED)); + } + } + Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + } + + private void twoWriters(AzureBlobFileSystem fs, Path testFilePath, boolean expectException) throws Exception { + try (FSDataOutputStream out = fs.create(testFilePath)) { + try (FSDataOutputStream out2 = fs.append(testFilePath)) { + out2.writeInt(2); + out2.hsync(); + } catch (IOException e) { + if (expectException) { + Assert.assertTrue("Unexpected error message: " + e.getMessage(), + e.getMessage().contains(ERR_ACQUIRING_LEASE)); + } else { + Assert.fail("Unexpected exception " + e.getMessage()); + } + } + out.writeInt(1); + out.hsync(); + } + + Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + } + + @Test(timeout = 30 * 1000) + public void testTwoWritersCreateAppendNoSingleWriter() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + fs.mkdirs(testFilePath.getParent()); + + twoWriters(fs, testFilePath, false); + } + + @Test(timeout = 30 * 1000) + public void testTwoWritersCreateAppendWithSingleWriterEnabled() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + fs.mkdirs(testFilePath.getParent()); + fs.getAbfsStore().getAbfsConfiguration() + .setAzureSingleWriterDirs(testFilePath.getParent().toString()); + fs.getAbfsStore().updateSingleWriterDirs(); + + twoWriters(fs, testFilePath, true); + } + + @Test(timeout = 30 * 1000) + public void testLeaseFreedOnClose() throws IOException { + final AzureBlobFileSystem fs = getFileSystem(); + final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + fs.mkdirs(testFilePath.getParent()); + fs.getAbfsStore().getAbfsConfiguration() + .setAzureSingleWriterDirs(testFilePath.getParent().toString()); + fs.getAbfsStore().updateSingleWriterDirs(); + + FSDataOutputStream out; + out = fs.create(testFilePath); + out.write(0); + Assert.assertTrue("Output stream should have lease", + ((AbfsOutputStream)out.getWrappedStream()).hasLease()); + out.close(); + Assert.assertFalse("Output stream should not have lease after close", + ((AbfsOutputStream)out.getWrappedStream()).hasLease()); + Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + } + + @Test(timeout = 30 * 1000) + public void testWriteAfterBreakLease() throws IOException { + final AzureBlobFileSystem fs = getFileSystem(); + final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + fs.mkdirs(testFilePath.getParent()); + fs.getAbfsStore().getAbfsConfiguration() + .setAzureSingleWriterDirs(testFilePath.getParent().toString()); + fs.getAbfsStore().updateSingleWriterDirs(); + + FSDataOutputStream out; + out = fs.create(testFilePath); + out.write(0); + out.hsync(); + + fs.breakLease(testFilePath); + try { + out.write(1); + out.hsync(); + Assert.fail("Expected exception on write after lease break"); + } catch (IOException e) { + Assert.assertTrue("Unexpected error message: " + e.getMessage(), + e.getMessage().contains(ERR_LEASE_EXPIRED)); + } + try { + out.close(); + Assert.fail("Expected exception on close after lease break"); + } catch (IOException e) { + Assert.assertTrue("Unexpected error message: " + e.getMessage(), + e.getMessage().contains(ERR_LEASE_EXPIRED)); + } + + Assert.assertTrue(((AbfsOutputStream)out.getWrappedStream()).isLeaseFreed()); + + try (FSDataOutputStream out2 = fs.append(testFilePath)) { + out2.write(2); + out2.hsync(); + } + + Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + } + + @Test(timeout = 90 * 1000) + public void testLeaseFreedAfterBreak() throws IOException { + final AzureBlobFileSystem fs = getFileSystem(); + final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + fs.mkdirs(testFilePath.getParent()); + fs.getAbfsStore().getAbfsConfiguration() + .setAzureSingleWriterDirs(testFilePath.getParent().toString()); + fs.getAbfsStore().updateSingleWriterDirs(); + + FSDataOutputStream out = null; + try { + out = fs.create(testFilePath); + out.write(0); + + fs.breakLease(testFilePath); + while (!((AbfsOutputStream) out.getWrappedStream()).isLeaseFreed()) { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + } + } + } finally { + try { + if (out != null) { + out.close(); + } + // exception might or might not occur + } catch (IOException e) { + Assert.assertTrue("Unexpected error message: " + e.getMessage(), + e.getMessage().contains(ERR_LEASE_EXPIRED)); + } + } + Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 1e6b8efe6d9d2..081f2e9350efd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.isNull; import static org.mockito.ArgumentMatchers.refEq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -86,9 +87,9 @@ public void verifyShortWriteRequest() throws Exception { AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any())).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, null, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[WRITE_SIZE]; new Random().nextBytes(b); @@ -104,9 +105,9 @@ public void verifyShortWriteRequest() throws Exception { out.hsync(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, WRITE_SIZE, APPEND_MODE, false); + 0, 0, WRITE_SIZE, APPEND_MODE, false, null); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - WRITE_SIZE, 0, 2 * WRITE_SIZE, APPEND_MODE, false); + WRITE_SIZE, 0, 2 * WRITE_SIZE, APPEND_MODE, false, null); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any()); @@ -133,9 +134,9 @@ public void verifyWriteRequest() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any())).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, null, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[WRITE_SIZE]; new Random().nextBytes(b); @@ -146,9 +147,9 @@ public void verifyWriteRequest() throws Exception { out.close(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, 5*WRITE_SIZE-BUFFER_SIZE, APPEND_MODE, false); + BUFFER_SIZE, 0, 5*WRITE_SIZE-BUFFER_SIZE, APPEND_MODE, false, null); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any()); @@ -165,7 +166,7 @@ public void verifyWriteRequest() throws Exception { ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); verify(client, times(1)).flush(acFlushPath.capture(), acFlushPosition.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture()); + acFlushSASToken.capture(), isNull()); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushPath.getAllValues()); assertThat(Arrays.asList(Long.valueOf(5*WRITE_SIZE))).describedAs("position").isEqualTo(acFlushPosition.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -189,11 +190,11 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any())).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op); when(op.getSasToken()).thenReturn("testToken"); when(op.getResult()).thenReturn(httpOp); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, null, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -204,9 +205,9 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { out.close(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any()); @@ -223,7 +224,7 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); verify(client, times(1)).flush(acFlushPath.capture(), acFlushPosition.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture()); + acFlushSASToken.capture(), isNull()); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushPath.getAllValues()); assertThat(Arrays.asList(Long.valueOf(2*BUFFER_SIZE))).describedAs("position").isEqualTo(acFlushPosition.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -247,11 +248,11 @@ public void verifyWriteRequestOfBufferSize() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any())).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op); when(op.getSasToken()).thenReturn("testToken"); when(op.getResult()).thenReturn(httpOp); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, null, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -262,9 +263,9 @@ public void verifyWriteRequestOfBufferSize() throws Exception { Thread.sleep(1000); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any()); @@ -291,9 +292,9 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any())).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, null, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, true)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -304,9 +305,9 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { Thread.sleep(1000); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, true); + 0, 0, BUFFER_SIZE, APPEND_MODE, true, null); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, true); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, true, null); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any()); @@ -334,9 +335,9 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any())).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, null, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -347,9 +348,9 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { out.hflush(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any()); @@ -366,7 +367,7 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); verify(client, times(1)).flush(acFlushPath.capture(), acFlushPosition.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture()); + acFlushSASToken.capture(), isNull()); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushPath.getAllValues()); assertThat(Arrays.asList(Long.valueOf(2*BUFFER_SIZE))).describedAs("position").isEqualTo(acFlushPosition.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -388,9 +389,9 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any())).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, null, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -403,9 +404,9 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { Thread.sleep(1000); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any()); From 86246dc3243ff290766d1a760d128e0194f26a13 Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Wed, 28 Oct 2020 12:38:27 -0400 Subject: [PATCH 02/18] HADOOP-16948. Fix findbugs and checkstyle problems. --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 15 ++++------ .../fs/azurebfs/services/AbfsErrors.java | 16 +++++------ .../azurebfs/services/AbfsOutputStream.java | 2 +- .../azurebfs/services/SelfRenewingLease.java | 2 +- .../ITestAzureBlobFileSystemLease.java | 28 ++++++++++--------- 5 files changed, 31 insertions(+), 32 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 59304efda8968..3144614e47cf3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -148,7 +148,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport { private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1"; private static final int GET_SET_AGGREGATE_COUNT = 2; - private final Map leaseRefs; + private final Map leaseRefs; private final AbfsConfiguration abfsConfiguration; private final Set azureAtomicRenameDirSet; @@ -474,8 +474,8 @@ public OutputStream createFile(final Path path, try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(); boolean enableSingleWriter = isSingleWriterKey(path.toString()); - LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} " + - "isNamespaceEnabled: {} enableSingleWriter: {}", + LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} " + + "isNamespaceEnabled: {} enableSingleWriter: {}", client.getFileSystem(), path, overwrite, @@ -754,22 +754,19 @@ public String acquireLease(final Path path, final int duration) throws AzureBlob public void renewLease(final Path path, final String leaseId) throws AzureBlobFileSystemException { LOG.debug("lease path: {}, renew lease id: {}", path, leaseId); - final AbfsRestOperation op = - client.renewLease(getRelativePath(path), leaseId); + client.renewLease(getRelativePath(path), leaseId); } public void releaseLease(final Path path, final String leaseId) throws AzureBlobFileSystemException { LOG.debug("lease path: {}, release lease id: {}", path, leaseId); - final AbfsRestOperation op = - client.releaseLease(getRelativePath(path), leaseId); + client.releaseLease(getRelativePath(path), leaseId); } public void breakLease(final Path path) throws AzureBlobFileSystemException { LOG.debug("lease path: {}", path); - final AbfsRestOperation op = - client.breakLease(getRelativePath(path)); + client.breakLease(getRelativePath(path)); } public void rename(final Path source, final Path destination) throws diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java index e066f1f3fab38..19001abb051a6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java @@ -25,12 +25,12 @@ */ @InterfaceAudience.Public @InterfaceStability.Evolving -public final class AbfsErrors { - public static final String ERR_WRITE_WITHOUT_LEASE = "Attempted to write to file without lease"; - public static final String ERR_LEASE_EXPIRED = "A lease ID was specified, but the lease for " + - "the resource has expired."; - public static final String ERR_PARALLEL_ACCESS_DETECTED = "Parallel access to the create path " + - "detected. Failing request to honor single writer semantics"; - public static final String ERR_ACQUIRING_LEASE = "Unable to acquire lease"; - public static final String ERR_LEASE_PRESENT = "There is already a lease present."; +public interface AbfsErrors { + String ERR_WRITE_WITHOUT_LEASE = "Attempted to write to file without lease"; + String ERR_LEASE_EXPIRED = "A lease ID was specified, but the lease for the resource has " + + "expired."; + String ERR_PARALLEL_ACCESS_DETECTED = "Parallel access to the create path detected. Failing " + + "request to honor single writer semantics"; + String ERR_ACQUIRING_LEASE = "Unable to acquire lease"; + String ERR_LEASE_PRESENT = "There is already a lease present."; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index c66ed4dd7164e..88926f691c466 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -120,7 +120,7 @@ public AbfsOutputStream( final Statistics statistics, final String path, final long position, - final Map leaseRefs, + final Map leaseRefs, AbfsOutputStreamContext abfsOutputStreamContext) throws AzureBlobFileSystemException { this.client = client; this.statistics = statistics; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java index 43de22a0ad00b..6cec797459826 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java @@ -46,7 +46,7 @@ * * See also {@link org.apache.hadoop.fs.azure.SelfRenewingLease}. */ -public class SelfRenewingLease { +final public class SelfRenewingLease { private final AbfsClient client; private final Path path; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index 8022115704906..8eebcda0df022 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -33,6 +33,8 @@ * Test lease operations. */ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest { + private static final int TEST_EXECUTION_TIMEOUT = 30 * 1000; + private static final int LONG_TEST_EXECUTION_TIMEOUT = 90 * 1000; private static final String TEST_FILE = "testfile"; public ITestAzureBlobFileSystemLease() throws Exception { @@ -46,7 +48,7 @@ public void testNoSingleWriter() throws IOException { fs.mkdirs(testFilePath.getParent()); try (FSDataOutputStream out = fs.create(testFilePath)) { Assert.assertFalse("Output stream should not have lease", - ((AbfsOutputStream)out.getWrappedStream()).hasLease()); + ((AbfsOutputStream) out.getWrappedStream()).hasLease()); } Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); } @@ -62,10 +64,10 @@ public void testOneWriter() throws IOException { FSDataOutputStream out = fs.create(testFilePath); Assert.assertTrue("Output stream should have lease", - ((AbfsOutputStream)out.getWrappedStream()).hasLease()); + ((AbfsOutputStream) out.getWrappedStream()).hasLease()); out.close(); Assert.assertFalse("Output stream should not have lease", - ((AbfsOutputStream)out.getWrappedStream()).hasLease()); + ((AbfsOutputStream) out.getWrappedStream()).hasLease()); Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); } @@ -81,10 +83,10 @@ public void testSubDir() throws IOException { FSDataOutputStream out = fs.create(testFilePath); Assert.assertTrue("Output stream should have lease", - ((AbfsOutputStream)out.getWrappedStream()).hasLease()); + ((AbfsOutputStream) out.getWrappedStream()).hasLease()); out.close(); Assert.assertFalse("Output stream should not have lease", - ((AbfsOutputStream)out.getWrappedStream()).hasLease()); + ((AbfsOutputStream) out.getWrappedStream()).hasLease()); Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); } @@ -128,7 +130,7 @@ private void twoWriters(AzureBlobFileSystem fs, Path testFilePath, boolean expec Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); } - @Test(timeout = 30 * 1000) + @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testTwoWritersCreateAppendNoSingleWriter() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); @@ -137,7 +139,7 @@ public void testTwoWritersCreateAppendNoSingleWriter() throws Exception { twoWriters(fs, testFilePath, false); } - @Test(timeout = 30 * 1000) + @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testTwoWritersCreateAppendWithSingleWriterEnabled() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); @@ -149,7 +151,7 @@ public void testTwoWritersCreateAppendWithSingleWriterEnabled() throws Exception twoWriters(fs, testFilePath, true); } - @Test(timeout = 30 * 1000) + @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testLeaseFreedOnClose() throws IOException { final AzureBlobFileSystem fs = getFileSystem(); final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); @@ -162,14 +164,14 @@ public void testLeaseFreedOnClose() throws IOException { out = fs.create(testFilePath); out.write(0); Assert.assertTrue("Output stream should have lease", - ((AbfsOutputStream)out.getWrappedStream()).hasLease()); + ((AbfsOutputStream) out.getWrappedStream()).hasLease()); out.close(); Assert.assertFalse("Output stream should not have lease after close", - ((AbfsOutputStream)out.getWrappedStream()).hasLease()); + ((AbfsOutputStream) out.getWrappedStream()).hasLease()); Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); } - @Test(timeout = 30 * 1000) + @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testWriteAfterBreakLease() throws IOException { final AzureBlobFileSystem fs = getFileSystem(); final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); @@ -200,7 +202,7 @@ public void testWriteAfterBreakLease() throws IOException { e.getMessage().contains(ERR_LEASE_EXPIRED)); } - Assert.assertTrue(((AbfsOutputStream)out.getWrappedStream()).isLeaseFreed()); + Assert.assertTrue(((AbfsOutputStream) out.getWrappedStream()).isLeaseFreed()); try (FSDataOutputStream out2 = fs.append(testFilePath)) { out2.write(2); @@ -210,7 +212,7 @@ public void testWriteAfterBreakLease() throws IOException { Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); } - @Test(timeout = 90 * 1000) + @Test(timeout = LONG_TEST_EXECUTION_TIMEOUT) public void testLeaseFreedAfterBreak() throws IOException { final AzureBlobFileSystem fs = getFileSystem(); final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); From fe3dd135a060400cb761799f8fd1d189deadcd7e Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Wed, 28 Oct 2020 14:01:51 -0400 Subject: [PATCH 03/18] HADOOP-16948. Fix remaining checkstyle problems. --- .../fs/azurebfs/services/AbfsErrors.java | 18 ++++++++++-------- .../azurebfs/services/SelfRenewingLease.java | 2 +- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java index 19001abb051a6..1786d86cf19b4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java @@ -25,12 +25,14 @@ */ @InterfaceAudience.Public @InterfaceStability.Evolving -public interface AbfsErrors { - String ERR_WRITE_WITHOUT_LEASE = "Attempted to write to file without lease"; - String ERR_LEASE_EXPIRED = "A lease ID was specified, but the lease for the resource has " - + "expired."; - String ERR_PARALLEL_ACCESS_DETECTED = "Parallel access to the create path detected. Failing " - + "request to honor single writer semantics"; - String ERR_ACQUIRING_LEASE = "Unable to acquire lease"; - String ERR_LEASE_PRESENT = "There is already a lease present."; +public final class AbfsErrors { + public static final String ERR_WRITE_WITHOUT_LEASE = "Attempted to write to file without lease"; + public static final String ERR_LEASE_EXPIRED = "A lease ID was specified, but the lease for the" + + " resource has expired."; + public static final String ERR_PARALLEL_ACCESS_DETECTED = "Parallel access to the create path " + + "detected. Failing request to honor single writer semantics"; + public static final String ERR_ACQUIRING_LEASE = "Unable to acquire lease"; + public static final String ERR_LEASE_PRESENT = "There is already a lease present."; + + private AbfsErrors() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java index 6cec797459826..5da174901479a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java @@ -46,7 +46,7 @@ * * See also {@link org.apache.hadoop.fs.azure.SelfRenewingLease}. */ -final public class SelfRenewingLease { +public final class SelfRenewingLease { private final AbfsClient client; private final Path path; From 1cd1a927f0b63b8c5153a4d8c8c30c10d7e4bb79 Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Mon, 4 Jan 2021 14:20:22 -0500 Subject: [PATCH 04/18] HADOOP-16948. Add DurationInfo, retry policy for acquiring lease, and javadocs --- .../fs/azurebfs/AzureBlobFileSystem.java | 13 +++-- .../fs/azurebfs/AzureBlobFileSystemStore.java | 28 ++++++++++ .../azurebfs/services/SelfRenewingLease.java | 52 ++++++++++++------- .../ITestAzureBlobFileSystemLease.java | 1 + 4 files changed, 72 insertions(+), 22 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index faa1fdc922044..d83c7fff68939 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -43,6 +43,7 @@ import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.apache.hadoop.util.DurationInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -510,7 +511,8 @@ public String acquireLease(final Path f, final int duration) throws IOException Path qualifiedPath = makeQualified(f); - try { + try (DurationInfo ignored = new DurationInfo(LOG, false, "Acquire lease for %s", + qualifiedPath)) { return abfsStore.acquireLease(qualifiedPath, duration); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); @@ -523,7 +525,8 @@ public void renewLease(final Path f, final String leaseId) throws IOException { Path qualifiedPath = makeQualified(f); - try { + try (DurationInfo ignored = new DurationInfo(LOG, false, "Renew lease for %s", + qualifiedPath)) { abfsStore.renewLease(qualifiedPath, leaseId); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); @@ -535,7 +538,8 @@ public void releaseLease(final Path f, final String leaseId) throws IOException Path qualifiedPath = makeQualified(f); - try { + try (DurationInfo ignored = new DurationInfo(LOG, false, "Release lease for %s", + qualifiedPath)) { abfsStore.releaseLease(qualifiedPath, leaseId); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); @@ -547,7 +551,8 @@ public void breakLease(final Path f) throws IOException { Path qualifiedPath = makeQualified(f); - try { + try (DurationInfo ignored = new DurationInfo(LOG, false, "Break lease for %s", + qualifiedPath)) { abfsStore.breakLease(qualifiedPath); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 3144614e47cf3..3e2e7e650e284 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -742,6 +742,14 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic } } + /** + * Acquire a lease on an ABFS file for a specified duration. This requires the file to exist. + * + * @param path file name + * @param duration time lease will be held before expiring + * @return the acquired lease ID + * @throws AzureBlobFileSystemException + */ public String acquireLease(final Path path, final int duration) throws AzureBlobFileSystemException { LOG.debug("lease path: {}", path); @@ -751,18 +759,38 @@ public String acquireLease(final Path path, final int duration) throws AzureBlob return op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID); } + /** + * Renew an existing lease on an ABFS file. + * + * @param path file name + * @param leaseId lease ID to renew + * @throws AzureBlobFileSystemException + */ public void renewLease(final Path path, final String leaseId) throws AzureBlobFileSystemException { LOG.debug("lease path: {}, renew lease id: {}", path, leaseId); client.renewLease(getRelativePath(path), leaseId); } + /** + * Release an existing lease on an ABFS file. + * + * @param path file name + * @param leaseId lease ID to release + * @throws AzureBlobFileSystemException + */ public void releaseLease(final Path path, final String leaseId) throws AzureBlobFileSystemException { LOG.debug("lease path: {}, release lease id: {}", path, leaseId); client.releaseLease(getRelativePath(path), leaseId); } + /** + * Break any current lease on an ABFS file. + * + * @param path file name + * @throws AzureBlobFileSystemException + */ public void breakLease(final Path path) throws AzureBlobFileSystemException { LOG.debug("lease path: {}", path); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java index 5da174901479a..ce97610c0847f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java @@ -18,14 +18,17 @@ package org.apache.hadoop.fs.azurebfs.services; -import com.google.common.base.Preconditions; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.io.retry.RetryPolicy; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_ACQUIRING_LEASE; @@ -71,6 +74,10 @@ public static class LeaseException extends AzureBlobFileSystemException { public LeaseException(Exception innerException) { super(ERR_ACQUIRING_LEASE, innerException); } + + public LeaseException() { + super(ERR_ACQUIRING_LEASE); + } } public SelfRenewingLease(AbfsClient client, Path path) throws AzureBlobFileSystemException { @@ -80,33 +87,42 @@ public SelfRenewingLease(AbfsClient client, Path path) throws AzureBlobFileSyste this.path = path; // Try to get the lease a specified number of times, else throw an error + RetryPolicy retryPolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep(LEASE_MAX_RETRIES, + LEASE_ACQUIRE_RETRY_INTERVAL, TimeUnit.MILLISECONDS); int numRetries = 0; - while (leaseID == null && numRetries < LEASE_MAX_RETRIES) { - numRetries++; + Exception lastException = null; + while (leaseID == null) { try { + if (RetryPolicy.RetryAction.RetryDecision.RETRY != + retryPolicy.shouldRetry(null, numRetries, 0, true).action) { + LOG.error("Exceeded maximum number of retries for acquiring lease on blob {}", path); + if (lastException != null) { + throw new LeaseException(lastException); + } + throw new LeaseException(); + } + LOG.debug("lease path: {}", path); final AbfsRestOperation op = client.acquireLease(getRelativePath(path), LEASE_TIMEOUT); leaseID = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID); - } catch (IOException e) { - if (numRetries < LEASE_MAX_RETRIES) { - LOG.info("Caught exception when trying to acquire lease on blob {}, retrying: {}", path, - e.getMessage()); - LOG.debug("Exception acquiring lease", e); - } else { - throw new LeaseException(e); - } + } catch (Exception e) { + lastException = e; + } finally { + numRetries++; } - if (leaseID == null) { - try { - Thread.sleep(LEASE_ACQUIRE_RETRY_INTERVAL); - } catch (InterruptedException e) { - // Restore the interrupted status - Thread.currentThread().interrupt(); - } + if (leaseID != null) { + break; + } + + try { + Thread.sleep(LEASE_ACQUIRE_RETRY_INTERVAL); + } catch (InterruptedException e) { + // Restore the interrupted status + Thread.currentThread().interrupt(); } } renewer = new Thread(new Renewer()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index 8eebcda0df022..be20461295d75 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -239,6 +239,7 @@ public void testLeaseFreedAfterBreak() throws IOException { out.close(); } // exception might or might not occur + Assert.fail("No exception on close after broken lease"); } catch (IOException e) { Assert.assertTrue("Unexpected error message: " + e.getMessage(), e.getMessage().contains(ERR_LEASE_EXPIRED)); From eeb4ba0f9bc6986d84ea176d8f02ef73cc2fb0e3 Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Tue, 12 Jan 2021 16:49:23 -0500 Subject: [PATCH 05/18] HADOOP-16948. Convert ABFS client to use an executor for lease ops --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 14 +- .../fs/azurebfs/AzureBlobFileSystemStore.java | 24 +- .../azurebfs/constants/ConfigurationKeys.java | 3 + .../constants/FileSystemConfigurations.java | 2 + .../fs/azurebfs/services/AbfsClient.java | 32 +++ .../fs/azurebfs/services/AbfsErrors.java | 8 +- .../azurebfs/services/AbfsOutputStream.java | 3 - .../azurebfs/services/SelfRenewingLease.java | 242 +++++++++--------- .../ITestAzureBlobFileSystemLease.java | 119 +++++---- 9 files changed, 265 insertions(+), 182 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 55b6a308c8fbf..41f18389ceb0e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -212,6 +212,11 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_FS_AZURE_SINGLE_WRITER_DIRECTORIES) private String azureSingleWriterDirs; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_LEASE_THREADS, + MinValue = MIN_LEASE_THREADS, + DefaultValue = DEFAULT_LEASE_THREADS) + private int numLeaseThreads; + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, DefaultValue = DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION) private boolean createRemoteFileSystemDuringInitialization; @@ -642,6 +647,10 @@ public String getAzureSingleWriterDirs() { return this.azureSingleWriterDirs; } + public int getNumLeaseThreads() { + return this.numLeaseThreads; + } + public boolean getCreateRemoteFileSystemDuringInitialization() { // we do not support creating the filesystem when AuthType is SAS return this.createRemoteFileSystemDuringInitialization @@ -941,11 +950,6 @@ void setDisableOutputStreamFlush(boolean disableOutputStreamFlush) { this.disableOutputStreamFlush = disableOutputStreamFlush; } - @VisibleForTesting - void setAzureSingleWriterDirs(String dirs) { - this.azureSingleWriterDirs = dirs; - } - @VisibleForTesting void setListMaxResults(int listMaxResults) { this.listMaxResults = listMaxResults; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 3e2e7e650e284..db21dac81c6f1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -50,10 +50,13 @@ import java.util.Optional; import java.util.Set; import java.util.WeakHashMap; +import java.util.concurrent.ExecutionException; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -255,17 +258,24 @@ public String getPrimaryGroup() { @Override public void close() throws IOException { + List> futures = new ArrayList<>(); for (SelfRenewingLease lease : leaseRefs.keySet()) { if (lease == null) { continue; } - try { - lease.free(); - } catch (Exception e) { - LOG.debug("Got exception freeing lease {}", lease.getLeaseID(), e); - } + ListenableFuture future = client.submit(() -> lease.free()); + futures.add(future); + } + try { + Futures.allAsList(futures).get(); + } catch (InterruptedException e) { + LOG.error("Interrupted freeing leases", e); + Thread.currentThread().interrupt(); + } catch (ExecutionException e) { + LOG.error("Error freeing leases", e); + } finally { + IOUtils.cleanupWithLogger(LOG, client); } - IOUtils.cleanupWithLogger(LOG, client); } byte[] encodeAttribute(String value) throws UnsupportedEncodingException { @@ -1726,7 +1736,7 @@ void setNamespaceEnabled(Trilean isNamespaceEnabled){ this.isNamespaceEnabled = isNamespaceEnabled; } - void updateSingleWriterDirs() { + private void updateSingleWriterDirs() { this.azureSingleWriterDirSet = new HashSet<>(Arrays.asList( abfsConfiguration.getAzureSingleWriterDirs().split(AbfsHttpConstants.COMMA))); // remove the empty string, since isKeyForDirectory returns true for empty strings diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index d2b31ca69f41d..e30387fde1f85 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -90,6 +90,9 @@ public final class ConfigurationKeys { /** Provides a config to provide comma separated path prefixes which support single writers. * Default is empty. **/ public static final String FS_AZURE_SINGLE_WRITER_KEY = "fs.azure.singlewriter.directories"; + /** Provides a number of threads to use for lease operations for single writer directories. + * Must be set to a minimum of 1 if single writer directories are to be used. Default is 0. **/ + public static final String FS_AZURE_LEASE_THREADS = "fs.azure.lease.threads"; public static final String FS_AZURE_READ_AHEAD_QUEUE_DEPTH = "fs.azure.readaheadqueue.depth"; public static final String FS_AZURE_ALWAYS_READ_BUFFER_SIZE = "fs.azure.read.alwaysReadBufferSize"; public static final String FS_AZURE_READ_AHEAD_BLOCK_SIZE = "fs.azure.read.readahead.blocksize"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 9353609a0f2a8..3908020275b1c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -79,6 +79,8 @@ public final class FileSystemConfigurations { public static final boolean DEFAULT_FS_AZURE_ENABLE_MKDIR_OVERWRITE = true; public static final String DEFAULT_FS_AZURE_APPEND_BLOB_DIRECTORIES = ""; public static final String DEFAULT_FS_AZURE_SINGLE_WRITER_DIRECTORIES = ""; + public static final int DEFAULT_LEASE_THREADS = 0; + public static final int MIN_LEASE_THREADS = 0; public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = -1; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index db92b496f4b40..66a4b2993b735 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -30,6 +30,9 @@ import java.util.List; import java.util.Locale; import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; @@ -37,6 +40,12 @@ import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableScheduledFuture; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningScheduledExecutorService; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -77,6 +86,8 @@ public class AbfsClient implements Closeable { private SASTokenProvider sasTokenProvider; private final AbfsCounters abfsCounters; + private final ListeningScheduledExecutorService executorService; + private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, final AbfsConfiguration abfsConfiguration, final AbfsClientContext abfsClientContext) { @@ -107,6 +118,9 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden this.userAgent = initializeUserAgent(abfsConfiguration, sslProviderName); this.abfsPerfTracker = abfsClientContext.getAbfsPerfTracker(); this.abfsCounters = abfsClientContext.getAbfsCounters(); + + this.executorService = MoreExecutors.listeningDecorator( + Executors.newScheduledThreadPool(this.abfsConfiguration.getNumLeaseThreads())); } public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, @@ -130,6 +144,7 @@ public void close() throws IOException { if (tokenProvider instanceof Closeable) { IOUtils.cleanupWithLogger(LOG, (Closeable) tokenProvider); } + executorService.shutdownNow(); } public String getFileSystem() { @@ -1087,4 +1102,21 @@ public SASTokenProvider getSasTokenProvider() { protected AbfsCounters getAbfsCounters() { return abfsCounters; } + + public int getNumLeaseThreads() { + return abfsConfiguration.getNumLeaseThreads(); + } + + public ListenableScheduledFuture schedule(Callable callable, long delay, + TimeUnit timeUnit) { + return executorService.schedule(callable, delay, timeUnit); + } + + public ListenableFuture submit(Runnable runnable) { + return executorService.submit(runnable); + } + + public void addCallback(ListenableFuture future, FutureCallback callback) { + Futures.addCallback(future, callback, executorService); + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java index 1786d86cf19b4..7905565d1204d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java @@ -20,6 +20,8 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS; + /** * ABFS error constants. */ @@ -32,7 +34,11 @@ public final class AbfsErrors { public static final String ERR_PARALLEL_ACCESS_DETECTED = "Parallel access to the create path " + "detected. Failing request to honor single writer semantics"; public static final String ERR_ACQUIRING_LEASE = "Unable to acquire lease"; - public static final String ERR_LEASE_PRESENT = "There is already a lease present."; + public static final String ERR_LEASE_NOT_PRESENT = "There is currently no lease on the resource"; + public static final String ERR_LEASE_FUTURE_EXISTS = "There is already an existing lease " + + "operation"; + public static final String ERR_NO_LEASE_THREADS = "Lease desired but no lease threads " + + "configured, set " + FS_AZURE_LEASE_THREADS; private AbfsErrors() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 88926f691c466..77643d5b6c2fb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -328,9 +328,6 @@ public synchronized void close() throws IOException { throw wrapException(path, e.getMessage(), e); } finally { if (lease != null) { - if (LOG.isDebugEnabled()) { - LOG.info("Freeing lease {}", leaseId); - } lease.free(); lease = null; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java index ce97610c0847f..73700abd7f821 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java @@ -20,138 +20,185 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableScheduledFuture; +import org.apache.hadoop.thirdparty.org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.concurrent.CancellationException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; +import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_ACQUIRING_LEASE; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_FUTURE_EXISTS; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_NO_LEASE_THREADS; /** - * An Azure blob lease that automatically renews itself indefinitely - * using a background thread. Use it to synchronize distributed processes, - * or to prevent writes to the blob by other processes that don't - * have the lease. + * An Azure blob lease that automatically renews itself indefinitely by scheduling lease + * operations through the ABFS client. Use it to prevent writes to the blob by other processes + * that don't have the lease. * - * Creating a new Lease object blocks the caller until the Azure blob lease is - * acquired. + * Creating a new Lease object blocks the caller until the Azure blob lease is acquired. It will + * retry a fixed number of times before failing if there is a problem acquiring the lease. * - * Call free() to release the Lease. - * - * You can use this Lease like a distributed lock. If the holder process - * dies, the lease will time out since it won't be renewed. - * - * See also {@link org.apache.hadoop.fs.azure.SelfRenewingLease}. + * Call free() to release the Lease. If the holder process dies, the lease will time out since it + * won't be renewed. */ public final class SelfRenewingLease { + private static final Logger LOG = LoggerFactory.getLogger(SelfRenewingLease.class); - private final AbfsClient client; - private final Path path; - private Thread renewer; - private volatile boolean leaseFreed; - private String leaseID = null; - private static final int LEASE_TIMEOUT = 60; // Lease timeout in seconds - - // Time to wait to renew lease in milliseconds - public static final int LEASE_RENEWAL_PERIOD = 40000; - public static final Logger LOG = LoggerFactory.getLogger(SelfRenewingLease.class); + static final int LEASE_DURATION = 60; // Lease duration in seconds + static final int LEASE_RENEWAL_PERIOD = 40; // Lease renewal interval in seconds - // Used to allocate thread serial numbers in thread name - private static AtomicInteger threadNumber = new AtomicInteger(0); + static final int LEASE_ACQUIRE_RETRY_INTERVAL = 10; // Retry interval for acquiring lease in secs + static final int LEASE_ACQUIRE_MAX_RETRIES = 7; // Number of retries for acquiring lease + private final AbfsClient client; + private final String path; - // Time to wait to retry getting the lease in milliseconds - static final int LEASE_ACQUIRE_RETRY_INTERVAL = 2000; - static final int LEASE_MAX_RETRIES = 5; + // Lease status variables + private volatile boolean leaseFreed; + private volatile String leaseID = null; + private volatile Throwable exception = null; + private volatile ListenableScheduledFuture future = null; public static class LeaseException extends AzureBlobFileSystemException { - public LeaseException(Exception innerException) { - super(ERR_ACQUIRING_LEASE, innerException); + public LeaseException(Throwable t) { + super(ERR_ACQUIRING_LEASE + ": " + t.getMessage()); } - public LeaseException() { - super(ERR_ACQUIRING_LEASE); + public LeaseException(String s) { + super(s); } } public SelfRenewingLease(AbfsClient client, Path path) throws AzureBlobFileSystemException { - this.leaseFreed = false; this.client = client; - this.path = path; + this.path = getRelativePath(path); + + if (client.getNumLeaseThreads() < 1) { + throw new LeaseException(ERR_NO_LEASE_THREADS); + } // Try to get the lease a specified number of times, else throw an error - RetryPolicy retryPolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep(LEASE_MAX_RETRIES, - LEASE_ACQUIRE_RETRY_INTERVAL, TimeUnit.MILLISECONDS); - int numRetries = 0; - Exception lastException = null; - while (leaseID == null) { - try { - if (RetryPolicy.RetryAction.RetryDecision.RETRY != - retryPolicy.shouldRetry(null, numRetries, 0, true).action) { - LOG.error("Exceeded maximum number of retries for acquiring lease on blob {}", path); - if (lastException != null) { - throw new LeaseException(lastException); - } - throw new LeaseException(); - } + RetryPolicy retryPolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep( + LEASE_ACQUIRE_MAX_RETRIES, LEASE_ACQUIRE_RETRY_INTERVAL, TimeUnit.SECONDS); + acquireLease(retryPolicy, 0, 0); - LOG.debug("lease path: {}", path); - final AbfsRestOperation op = - client.acquireLease(getRelativePath(path), - LEASE_TIMEOUT); + while (leaseID == null && exception == null) { + } + if (exception != null) { + LOG.error("Failed to acquire lease on {}", path); + throw new LeaseException(exception); + } + + renewLease(LEASE_RENEWAL_PERIOD); + LOG.debug("Acquired lease {} on {}", leaseID, path); + } + + private void acquireLease(RetryPolicy retryPolicy, int numRetries, long delay) + throws LeaseException { + LOG.debug("Attempting to acquire lease on {}, retry {}", path, numRetries); + if (future != null && !future.isDone()) { + throw new LeaseException(ERR_LEASE_FUTURE_EXISTS); + } + future = client.schedule(() -> client.acquireLease(path, LEASE_DURATION), + delay, TimeUnit.SECONDS); + client.addCallback(future, new FutureCallback() { + @Override + public void onSuccess(@Nullable AbfsRestOperation op) { leaseID = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID); - } catch (Exception e) { - lastException = e; - } finally { - numRetries++; + LOG.debug("Acquired lease {} on {}", leaseID, path); } - if (leaseID != null) { - break; + @Override + public void onFailure(Throwable throwable) { + try { + if (RetryPolicy.RetryAction.RetryDecision.RETRY == + retryPolicy.shouldRetry(null, numRetries, 0, true).action) { + LOG.debug("Failed acquire lease on {}, retrying: {}", path, throwable); + acquireLease(retryPolicy, numRetries + 1, LEASE_ACQUIRE_RETRY_INTERVAL); + } else { + exception = throwable; + } + } catch (Exception e) { + exception = throwable; + } } + }); + } - try { - Thread.sleep(LEASE_ACQUIRE_RETRY_INTERVAL); - } catch (InterruptedException e) { - // Restore the interrupted status - Thread.currentThread().interrupt(); - } + private void renewLease(long delay) { + LOG.debug("Attempting to renew lease on {}, renew lease id {}, delay {}", path, leaseID, delay); + if (future != null && !future.isDone()) { + LOG.warn("Unexpected new lease renewal operation occurred while operation already existed. " + + "Not initiating new renewal"); + return; } - renewer = new Thread(new Renewer()); + future = client.schedule(() -> client.renewLease(path, leaseID), delay, + TimeUnit.SECONDS); + client.addCallback(future, new FutureCallback() { + @Override + public void onSuccess(@Nullable AbfsRestOperation op) { + LOG.debug("Renewed lease {} on {}", leaseID, path); + renewLease(delay); + } + + @Override + public void onFailure(Throwable throwable) { + if (throwable instanceof CancellationException) { + LOG.info("Stopping renewal due to cancellation"); + free(); + return; + } else if (throwable instanceof AbfsRestOperationException) { + AbfsRestOperationException opEx = ((AbfsRestOperationException) throwable); + if (opEx.getStatusCode() < HTTP_INTERNAL_ERROR) { + // error in 400 range indicates a type of error that should not result in a retry + // such as the lease being broken or a different lease being present + LOG.info("Stopping renewal due to {}: {}, {}", opEx.getStatusCode(), + opEx.getErrorCode(), opEx.getErrorMessage()); + free(); + return; + } + } - // A Renewer running should not keep JVM from exiting, so make it a daemon. - renewer.setDaemon(true); - renewer.setName("AzureBFSLeaseRenewer-" + threadNumber.getAndIncrement()); - renewer.start(); - LOG.debug("Acquired lease {} on {} managed by thread {}", leaseID, path, renewer.getName()); + LOG.debug("Failed to renew lease on {}, renew lease id {}, retrying: {}", path, leaseID, + throwable); + renewLease(0); + } + }); } /** - * Free the lease and stop the keep-alive thread. + * Cancel renewal and free the lease. If an exception occurs, this method assumes the lease + * will expire after the lease duration. */ public void free() { try { - LOG.debug("lease path: {}, release lease id: {}", path, leaseID); - client.releaseLease(getRelativePath(path), leaseID); + LOG.debug("Freeing lease: path {}, lease id {}", path, leaseID); + if (future != null && !future.isDone()) { + future.cancel(true); + } + client.releaseLease(path, leaseID); } catch (IOException e) { LOG.info("Exception when trying to release lease {} on {}. Lease will be left to expire: {}", leaseID, path, e.getMessage()); - LOG.debug("Exception releasing lease", e); } finally { // Even if releasing the lease fails (e.g. because the file was deleted), // make sure to record that we freed the lease, to terminate the // keep-alive thread. leaseFreed = true; - LOG.debug("Freed lease {} on {} managed by thread {}", leaseID, path, renewer.getName()); + LOG.debug("Freed lease {} on {}", leaseID, path); } } @@ -163,51 +210,6 @@ public String getLeaseID() { return leaseID; } - private class Renewer implements Runnable { - - /** - * Start a keep-alive thread that will continue to renew - * the lease until it is freed or the process dies. - */ - @Override - public void run() { - LOG.debug("Starting lease keep-alive thread."); - - while (!leaseFreed) { - try { - Thread.sleep(LEASE_RENEWAL_PERIOD); - } catch (InterruptedException e) { - LOG.debug("Keep-alive thread for lease {} interrupted", leaseID); - - // Restore the interrupted status - Thread.currentThread().interrupt(); - } - try { - if (!leaseFreed) { - LOG.debug("lease path: {}, renew lease id: {}", path, leaseID); - client.renewLease(getRelativePath(path), leaseID); - - // It'll be very rare to renew the lease (most will be short) - // so log that we did it, to help with system debugging. - LOG.info("Renewed lease {} on {}", leaseID, path); - } - } catch (IOException e) { - if (!leaseFreed) { - - // Free the lease so we don't leave this thread running forever. - leaseFreed = true; - - // Normally leases should be freed and there should be no - // exceptions, so log a warning. - LOG.warn("Attempt to renew lease {} on {} failed, stopping renewal thread: ", - leaseID, path, e.getMessage()); - LOG.debug("Exception renewing lease", e); - } - } - } - } - } - private String getRelativePath(final Path path) { Preconditions.checkNotNull(path, "path"); return path.toUri().getPath(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index be20461295d75..f1aa34a341483 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -17,16 +17,23 @@ */ package org.apache.hadoop.fs.azurebfs; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; +import org.apache.hadoop.test.GenericTestUtils; import org.junit.Assert; import org.junit.Test; import java.io.IOException; +import java.util.concurrent.RejectedExecutionException; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SINGLE_WRITER_KEY; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_ACQUIRING_LEASE; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_EXPIRED; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_NOT_PRESENT; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_NO_LEASE_THREADS; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_PARALLEL_ACCESS_DETECTED; /** @@ -41,10 +48,19 @@ public ITestAzureBlobFileSystemLease() throws Exception { super(); } + private AzureBlobFileSystem getCustomFileSystem(String singleWriterDirs, int numLeaseThreads) + throws Exception { + Configuration conf = getRawConfiguration(); + conf.setBoolean(String.format("fs.%s.impl.disable.cache", getAbfsScheme()), true); + conf.set(FS_AZURE_SINGLE_WRITER_KEY, singleWriterDirs); + conf.setInt(FS_AZURE_LEASE_THREADS, numLeaseThreads); + return getFileSystem(conf); + } + @Test public void testNoSingleWriter() throws IOException { - final AzureBlobFileSystem fs = getFileSystem(); final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + final AzureBlobFileSystem fs = getFileSystem(); fs.mkdirs(testFilePath.getParent()); try (FSDataOutputStream out = fs.create(testFilePath)) { Assert.assertFalse("Output stream should not have lease", @@ -54,13 +70,22 @@ public void testNoSingleWriter() throws IOException { } @Test - public void testOneWriter() throws IOException { - final AzureBlobFileSystem fs = getFileSystem(); + public void testNoLeaseThreads() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 0); + fs.mkdirs(testFilePath.getParent()); + try (FSDataOutputStream out = fs.create(testFilePath)) { + Assert.fail("No failure when lease requested with 0 lease threads"); + } catch (Exception e) { + GenericTestUtils.assertExceptionContains(ERR_NO_LEASE_THREADS, e); + } + } + + @Test + public void testOneWriter() throws Exception { + final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 1); fs.mkdirs(testFilePath.getParent()); - fs.getAbfsStore().getAbfsConfiguration() - .setAzureSingleWriterDirs(testFilePath.getParent().toString()); - fs.getAbfsStore().updateSingleWriterDirs(); FSDataOutputStream out = fs.create(testFilePath); Assert.assertTrue("Output stream should have lease", @@ -72,14 +97,12 @@ public void testOneWriter() throws IOException { } @Test - public void testSubDir() throws IOException { - final AzureBlobFileSystem fs = getFileSystem(); + public void testSubDir() throws Exception { final Path testFilePath = new Path(new Path(path(methodName.getMethodName()), "subdir"), TEST_FILE); + final AzureBlobFileSystem fs = + getCustomFileSystem(testFilePath.getParent().getParent().toString(), 1); fs.mkdirs(testFilePath.getParent().getParent()); - fs.getAbfsStore().getAbfsConfiguration() - .setAzureSingleWriterDirs(testFilePath.getParent().getParent().toString()); - fs.getAbfsStore().updateSingleWriterDirs(); FSDataOutputStream out = fs.create(testFilePath); Assert.assertTrue("Output stream should have lease", @@ -91,20 +114,16 @@ public void testSubDir() throws IOException { } @Test - public void testTwoCreate() throws IOException { - final AzureBlobFileSystem fs = getFileSystem(); + public void testTwoCreate() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 1); fs.mkdirs(testFilePath.getParent()); - fs.getAbfsStore().getAbfsConfiguration() - .setAzureSingleWriterDirs(testFilePath.getParent().toString()); - fs.getAbfsStore().updateSingleWriterDirs(); try (FSDataOutputStream out = fs.create(testFilePath)) { try (FSDataOutputStream out2 = fs.create(testFilePath)) { Assert.fail("Second create succeeded"); } catch (IOException e) { - Assert.assertTrue("Unexpected error message: " + e.getMessage(), - e.getMessage().contains(ERR_PARALLEL_ACCESS_DETECTED)); + GenericTestUtils.assertExceptionContains(ERR_PARALLEL_ACCESS_DETECTED, e); } } Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); @@ -132,33 +151,27 @@ private void twoWriters(AzureBlobFileSystem fs, Path testFilePath, boolean expec @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testTwoWritersCreateAppendNoSingleWriter() throws Exception { - final AzureBlobFileSystem fs = getFileSystem(); final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + final AzureBlobFileSystem fs = getFileSystem(); fs.mkdirs(testFilePath.getParent()); twoWriters(fs, testFilePath, false); } - @Test(timeout = TEST_EXECUTION_TIMEOUT) + @Test(timeout = LONG_TEST_EXECUTION_TIMEOUT) public void testTwoWritersCreateAppendWithSingleWriterEnabled() throws Exception { - final AzureBlobFileSystem fs = getFileSystem(); final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 1); fs.mkdirs(testFilePath.getParent()); - fs.getAbfsStore().getAbfsConfiguration() - .setAzureSingleWriterDirs(testFilePath.getParent().toString()); - fs.getAbfsStore().updateSingleWriterDirs(); twoWriters(fs, testFilePath, true); } @Test(timeout = TEST_EXECUTION_TIMEOUT) - public void testLeaseFreedOnClose() throws IOException { - final AzureBlobFileSystem fs = getFileSystem(); + public void testLeaseFreedOnClose() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 1); fs.mkdirs(testFilePath.getParent()); - fs.getAbfsStore().getAbfsConfiguration() - .setAzureSingleWriterDirs(testFilePath.getParent().toString()); - fs.getAbfsStore().updateSingleWriterDirs(); FSDataOutputStream out; out = fs.create(testFilePath); @@ -172,13 +185,10 @@ public void testLeaseFreedOnClose() throws IOException { } @Test(timeout = TEST_EXECUTION_TIMEOUT) - public void testWriteAfterBreakLease() throws IOException { - final AzureBlobFileSystem fs = getFileSystem(); + public void testWriteAfterBreakLease() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 1); fs.mkdirs(testFilePath.getParent()); - fs.getAbfsStore().getAbfsConfiguration() - .setAzureSingleWriterDirs(testFilePath.getParent().toString()); - fs.getAbfsStore().updateSingleWriterDirs(); FSDataOutputStream out; out = fs.create(testFilePath); @@ -191,15 +201,13 @@ public void testWriteAfterBreakLease() throws IOException { out.hsync(); Assert.fail("Expected exception on write after lease break"); } catch (IOException e) { - Assert.assertTrue("Unexpected error message: " + e.getMessage(), - e.getMessage().contains(ERR_LEASE_EXPIRED)); + GenericTestUtils.assertExceptionContains(ERR_LEASE_EXPIRED, e); } try { out.close(); Assert.fail("Expected exception on close after lease break"); } catch (IOException e) { - Assert.assertTrue("Unexpected error message: " + e.getMessage(), - e.getMessage().contains(ERR_LEASE_EXPIRED)); + GenericTestUtils.assertExceptionContains(ERR_LEASE_EXPIRED, e); } Assert.assertTrue(((AbfsOutputStream) out.getWrappedStream()).isLeaseFreed()); @@ -213,13 +221,10 @@ public void testWriteAfterBreakLease() throws IOException { } @Test(timeout = LONG_TEST_EXECUTION_TIMEOUT) - public void testLeaseFreedAfterBreak() throws IOException { - final AzureBlobFileSystem fs = getFileSystem(); + public void testLeaseFreedAfterBreak() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 1); fs.mkdirs(testFilePath.getParent()); - fs.getAbfsStore().getAbfsConfiguration() - .setAzureSingleWriterDirs(testFilePath.getParent().toString()); - fs.getAbfsStore().updateSingleWriterDirs(); FSDataOutputStream out = null; try { @@ -238,13 +243,35 @@ public void testLeaseFreedAfterBreak() throws IOException { if (out != null) { out.close(); } - // exception might or might not occur Assert.fail("No exception on close after broken lease"); } catch (IOException e) { - Assert.assertTrue("Unexpected error message: " + e.getMessage(), - e.getMessage().contains(ERR_LEASE_EXPIRED)); + GenericTestUtils.assertExceptionContains(ERR_LEASE_NOT_PRESENT, e); } } Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); } + + @Test(timeout = TEST_EXECUTION_TIMEOUT) + public void testFileSystemClose() throws Exception { + final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 1); + fs.mkdirs(testFilePath.getParent()); + + FSDataOutputStream out = fs.create(testFilePath); + out.write(0); + Assert.assertFalse(fs.getAbfsStore().areLeasesFreed()); + fs.close(); + Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + try { + out.close(); + Assert.fail("No exception on close after closed FS"); + } catch (IOException e) { + GenericTestUtils.assertExceptionContains(ERR_LEASE_NOT_PRESENT, e); + } + + try (FSDataOutputStream out2 = fs.append(testFilePath)) { + Assert.fail("Expected an error on operation after closed FS"); + } catch (RejectedExecutionException e) { + } + } } From 8e91f6165f8bf599a72bc4e826ef2c19474b1b60 Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Thu, 14 Jan 2021 12:21:26 -0500 Subject: [PATCH 06/18] HADOOP-16948. Fix ABFS lease test for non-HNS --- .../fs/azurebfs/services/AbfsErrors.java | 4 +++- .../ITestAzureBlobFileSystemLease.java | 24 ++++++++++++++++--- 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java index 7905565d1204d..9f83a8299d664 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java @@ -30,7 +30,9 @@ public final class AbfsErrors { public static final String ERR_WRITE_WITHOUT_LEASE = "Attempted to write to file without lease"; public static final String ERR_LEASE_EXPIRED = "A lease ID was specified, but the lease for the" - + " resource has expired."; + + " resource has expired"; + public static final String ERR_NO_LEASE_ID_SPECIFIED = "There is currently a lease on the " + + "resource and no lease ID was specified in the request"; public static final String ERR_PARALLEL_ACCESS_DETECTED = "Parallel access to the create path " + "detected. Failing request to honor single writer semantics"; public static final String ERR_ACQUIRING_LEASE = "Unable to acquire lease"; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index f1aa34a341483..347bd45facf8c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -30,9 +30,11 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SINGLE_WRITER_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_ACQUIRING_LEASE; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_EXPIRED; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_NOT_PRESENT; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_NO_LEASE_ID_SPECIFIED; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_NO_LEASE_THREADS; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_PARALLEL_ACCESS_DETECTED; @@ -43,9 +45,13 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest { private static final int TEST_EXECUTION_TIMEOUT = 30 * 1000; private static final int LONG_TEST_EXECUTION_TIMEOUT = 90 * 1000; private static final String TEST_FILE = "testfile"; + private final boolean isHNSEnabled; public ITestAzureBlobFileSystemLease() throws Exception { super(); + + this.isHNSEnabled = getConfiguration() + .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false); } private AzureBlobFileSystem getCustomFileSystem(String singleWriterDirs, int numLeaseThreads) @@ -123,7 +129,11 @@ public void testTwoCreate() throws Exception { try (FSDataOutputStream out2 = fs.create(testFilePath)) { Assert.fail("Second create succeeded"); } catch (IOException e) { - GenericTestUtils.assertExceptionContains(ERR_PARALLEL_ACCESS_DETECTED, e); + if (isHNSEnabled) { + GenericTestUtils.assertExceptionContains(ERR_PARALLEL_ACCESS_DETECTED, e); + } else { + GenericTestUtils.assertExceptionContains(ERR_NO_LEASE_ID_SPECIFIED, e); + } } } Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); @@ -245,7 +255,11 @@ public void testLeaseFreedAfterBreak() throws Exception { } Assert.fail("No exception on close after broken lease"); } catch (IOException e) { - GenericTestUtils.assertExceptionContains(ERR_LEASE_NOT_PRESENT, e); + if (isHNSEnabled) { + GenericTestUtils.assertExceptionContains(ERR_LEASE_NOT_PRESENT, e); + } else { + GenericTestUtils.assertExceptionContains(ERR_LEASE_EXPIRED, e); + } } } Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); @@ -266,7 +280,11 @@ public void testFileSystemClose() throws Exception { out.close(); Assert.fail("No exception on close after closed FS"); } catch (IOException e) { - GenericTestUtils.assertExceptionContains(ERR_LEASE_NOT_PRESENT, e); + if (isHNSEnabled) { + GenericTestUtils.assertExceptionContains(ERR_LEASE_NOT_PRESENT, e); + } else { + GenericTestUtils.assertExceptionContains(ERR_LEASE_EXPIRED, e); + } } try (FSDataOutputStream out2 = fs.append(testFilePath)) { From 8eceabb9052daaac60a4fac203ee85073b7ac118 Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Thu, 14 Jan 2021 12:35:51 -0500 Subject: [PATCH 07/18] HADOOP-16948. Fix checkstyle and javadoc --- .../hadoop/fs/azurebfs/AzureBlobFileSystemStore.java | 8 ++++---- .../hadoop/fs/azurebfs/services/SelfRenewingLease.java | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index db21dac81c6f1..1d0e7bd7a7802 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -758,7 +758,7 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic * @param path file name * @param duration time lease will be held before expiring * @return the acquired lease ID - * @throws AzureBlobFileSystemException + * @throws AzureBlobFileSystemException on any exception while acquiring the lease */ public String acquireLease(final Path path, final int duration) throws AzureBlobFileSystemException { LOG.debug("lease path: {}", path); @@ -774,7 +774,7 @@ public String acquireLease(final Path path, final int duration) throws AzureBlob * * @param path file name * @param leaseId lease ID to renew - * @throws AzureBlobFileSystemException + * @throws AzureBlobFileSystemException on any exception while renewing the lease */ public void renewLease(final Path path, final String leaseId) throws AzureBlobFileSystemException { LOG.debug("lease path: {}, renew lease id: {}", path, leaseId); @@ -787,7 +787,7 @@ public void renewLease(final Path path, final String leaseId) throws AzureBlobFi * * @param path file name * @param leaseId lease ID to release - * @throws AzureBlobFileSystemException + * @throws AzureBlobFileSystemException on any exception while releasing the lease */ public void releaseLease(final Path path, final String leaseId) throws AzureBlobFileSystemException { LOG.debug("lease path: {}, release lease id: {}", path, leaseId); @@ -799,7 +799,7 @@ public void releaseLease(final Path path, final String leaseId) throws AzureBlob * Break any current lease on an ABFS file. * * @param path file name - * @throws AzureBlobFileSystemException + * @throws AzureBlobFileSystemException on any exception while breaking the lease */ public void breakLease(final Path path) throws AzureBlobFileSystemException { LOG.debug("lease path: {}", path); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java index 73700abd7f821..b4bc5a4f330e9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java @@ -123,8 +123,8 @@ public void onSuccess(@Nullable AbfsRestOperation op) { @Override public void onFailure(Throwable throwable) { try { - if (RetryPolicy.RetryAction.RetryDecision.RETRY == - retryPolicy.shouldRetry(null, numRetries, 0, true).action) { + if (RetryPolicy.RetryAction.RetryDecision.RETRY + == retryPolicy.shouldRetry(null, numRetries, 0, true).action) { LOG.debug("Failed acquire lease on {}, retrying: {}", path, throwable); acquireLease(retryPolicy, numRetries + 1, LEASE_ACQUIRE_RETRY_INTERVAL); } else { From 42f68c2a835ed74fd788e8b87ed19231ce0eded1 Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Sun, 17 Jan 2021 14:51:04 -0500 Subject: [PATCH 08/18] HADOOP-16948. Address review comments --- .../fs/azurebfs/AzureBlobFileSystem.java | 39 ++++- .../fs/azurebfs/AzureBlobFileSystemStore.java | 3 +- .../AzureBlobFileSystemException.java | 4 + .../fs/azurebfs/services/AbfsClient.java | 14 +- .../fs/azurebfs/services/AbfsErrors.java | 7 + .../azurebfs/services/AbfsOutputStream.java | 10 +- .../services/AbfsOutputStreamContext.java | 15 ++ .../azurebfs/services/SelfRenewingLease.java | 32 ++-- .../hadoop-azure/src/site/markdown/abfs.md | 15 ++ .../ITestAzureBlobFileSystemLease.java | 148 +++++++++++------- .../services/TestAbfsOutputStream.java | 14 +- 11 files changed, 203 insertions(+), 98 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index d83c7fff68939..d99c8dc9abc6d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -43,7 +43,6 @@ import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; -import org.apache.hadoop.util.DurationInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,6 +87,7 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.functional.RemoteIterators; +import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.LambdaUtils; import org.apache.hadoop.util.Progressable; @@ -506,6 +506,20 @@ public FileStatus getFileStatus(final Path f) throws IOException { } } + /** + * Acquire a lease on an ABFS file for a specified duration. This requires the file to exist. + * Writes will not be able to be performed while the lease is held because there is not + * currently a way to provide a previously acquired lease ID to AbfsOutputStream. + * + * If the file is in a single writer directory, the client will automatically acquire, renew, + * and release a lease and this method does not need to be used. The output stream holding the + * lease will be able to write to the file in that case. + * + * @param f file name + * @param duration time lease will be held before expiring + * @return the acquired lease ID + * @throws IOException on any exception while acquiring the lease + */ public String acquireLease(final Path f, final int duration) throws IOException { LOG.debug("AzureBlobFileSystem.acquireLease path: {}", f); @@ -520,6 +534,15 @@ public String acquireLease(final Path f, final int duration) throws IOException } } + /** + * Renew an existing lease on an ABFS file. If a lease expires without being renewed, a + * different lease may be acquired. A lease can still be renewed after expiring if a different + * lease has not been acquired. + * + * @param f file name + * @param leaseId lease ID to renew + * @throws IOException on any exception while renewing the lease + */ public void renewLease(final Path f, final String leaseId) throws IOException { LOG.debug("AzureBlobFileSystem.renewLease path: {} id: {}", f, leaseId); @@ -533,6 +556,13 @@ public void renewLease(final Path f, final String leaseId) throws IOException { } } + /** + * Release an existing lease on an ABFS file. + * + * @param f file name + * @param leaseId lease ID to release + * @throws IOException on any exception while releasing the lease + */ public void releaseLease(final Path f, final String leaseId) throws IOException { LOG.debug("AzureBlobFileSystem.releaseLease path: {} id: {}", f, leaseId); @@ -546,6 +576,13 @@ public void releaseLease(final Path f, final String leaseId) throws IOException } } + /** + * Break the current lease on an ABFS file if it exists. A lease that is broken cannot be + * renewed. A new lease may be obtained on the file immediately. + * + * @param f file name + * @throws IOException on any exception while breaking the lease + */ public void breakLease(final Path f) throws IOException { LOG.debug("AzureBlobFileSystem.breakLease path: {}", f); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 1d0e7bd7a7802..f50a6dd9d1467 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -533,7 +533,6 @@ public OutputStream createFile(final Path path, statistics, relativePath, 0, - leaseRefs, populateAbfsOutputStreamContext(isAppendBlob, enableSingleWriter)); } } @@ -622,6 +621,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppend .withWriteMaxConcurrentRequestCount(abfsConfiguration.getWriteMaxConcurrentRequestCount()) .withMaxWriteRequestsToQueue(abfsConfiguration.getMaxWriteRequestsToQueue()) .withSingleWriterEnabled(enableSingleWriter) + .withLeaseRefs(leaseRefs) .build(); } @@ -747,7 +747,6 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic statistics, relativePath, offset, - leaseRefs, populateAbfsOutputStreamContext(isAppendBlob, enableSingleWriter)); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java index 9b1bead886e6c..d829c5ac6779c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java @@ -37,6 +37,10 @@ public AzureBlobFileSystemException(final String message, final Exception innerE super(message, innerException); } + public AzureBlobFileSystemException(final String message, final Throwable innerThrowable) { + super(message, innerThrowable); + } + @Override public String toString() { if (this.getMessage() == null && this.getCause() == null) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 66a4b2993b735..7e3b6dc878e09 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -31,15 +31,10 @@ import java.util.Locale; import java.util.UUID; import java.util.concurrent.Callable; -import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; -import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; -import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; -import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture; @@ -49,6 +44,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.SASTokenProviderException; @@ -59,6 +57,8 @@ import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.fs.azurebfs.utils.DateTimeUtils; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; +import org.apache.hadoop.util.concurrent.HadoopExecutors; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_DELETE_CONSIDERED_IDEMPOTENT; @@ -120,7 +120,7 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden this.abfsCounters = abfsClientContext.getAbfsCounters(); this.executorService = MoreExecutors.listeningDecorator( - Executors.newScheduledThreadPool(this.abfsConfiguration.getNumLeaseThreads())); + HadoopExecutors.newScheduledThreadPool(this.abfsConfiguration.getNumLeaseThreads())); } public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, @@ -144,7 +144,7 @@ public void close() throws IOException { if (tokenProvider instanceof Closeable) { IOUtils.cleanupWithLogger(LOG, (Closeable) tokenProvider); } - executorService.shutdownNow(); + HadoopExecutors.shutdown(executorService, LOG, 0, TimeUnit.SECONDS); } public String getFileSystem() { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java index 9f83a8299d664..e15795efee68d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java @@ -36,7 +36,14 @@ public final class AbfsErrors { public static final String ERR_PARALLEL_ACCESS_DETECTED = "Parallel access to the create path " + "detected. Failing request to honor single writer semantics"; public static final String ERR_ACQUIRING_LEASE = "Unable to acquire lease"; + public static final String ERR_LEASE_ALREADY_PRESENT = "There is already a lease present"; public static final String ERR_LEASE_NOT_PRESENT = "There is currently no lease on the resource"; + public static final String ERR_LEASE_ID_NOT_PRESENT = "The lease ID is not present with the " + + "specified lease operation"; + public static final String ERR_LEASE_DID_NOT_MATCH = "The lease ID specified did not match the " + + "lease ID for the resource with the specified lease operation"; + public static final String ERR_LEASE_BROKEN = "The lease ID matched, but the lease has been " + + "broken explicitly and cannot be renewed"; public static final String ERR_LEASE_FUTURE_EXISTS = "There is already an existing lease " + "operation"; public static final String ERR_NO_LEASE_THREADS = "Lease desired but no lease threads " diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 77643d5b6c2fb..8864cbcf6ac39 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -24,7 +24,6 @@ import java.io.OutputStream; import java.net.HttpURLConnection; import java.nio.ByteBuffer; -import java.util.Map; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ExecutorCompletionService; @@ -120,7 +119,6 @@ public AbfsOutputStream( final Statistics statistics, final String path, final long position, - final Map leaseRefs, AbfsOutputStreamContext abfsOutputStreamContext) throws AzureBlobFileSystemException { this.client = client; this.statistics = statistics; @@ -154,9 +152,7 @@ public AbfsOutputStream( if (abfsOutputStreamContext.isEnableSingleWriter()) { lease = new SelfRenewingLease(client, new Path(path)); this.leaseId = lease.getLeaseID(); - if (leaseRefs != null) { - leaseRefs.put(lease, null); - } + abfsOutputStreamContext.addLease(lease); } this.threadExecutor @@ -220,7 +216,7 @@ public synchronized void write(final byte[] data, final int off, final int lengt throw new IndexOutOfBoundsException(); } - if (lease != null && lease.isFreed()) { + if (hasLease() && isLeaseFreed()) { throw new PathIOException(path, ERR_WRITE_WITHOUT_LEASE); } @@ -327,7 +323,7 @@ public synchronized void close() throws IOException { // See HADOOP-16785 throw wrapException(path, e.getMessage(), e); } finally { - if (lease != null) { + if (hasLease()) { lease.free(); lease = null; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java index 301954f7a73db..a293a33dfb090 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java @@ -18,6 +18,8 @@ package org.apache.hadoop.fs.azurebfs.services; +import java.util.Map; + /** * Class to hold extra output stream configs. */ @@ -41,6 +43,8 @@ public class AbfsOutputStreamContext extends AbfsStreamContext { private boolean enableSingleWriter; + private Map leaseRefs; + public AbfsOutputStreamContext(final long sasTokenRenewPeriodForStreamsInSeconds) { super(sasTokenRenewPeriodForStreamsInSeconds); } @@ -101,6 +105,11 @@ public AbfsOutputStreamContext withSingleWriterEnabled(final boolean enableSingl return this; } + public AbfsOutputStreamContext withLeaseRefs(final Map leaseRefs) { + this.leaseRefs = leaseRefs; + return this; + } + public int getWriteBufferSize() { return writeBufferSize; } @@ -136,4 +145,10 @@ public boolean isEnableSmallWriteOptimization() { public boolean isEnableSingleWriter() { return this.enableSingleWriter; } + + public void addLease(SelfRenewingLease lease) { + if (this.leaseRefs != null) { + this.leaseRefs.put(lease, null); + } + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java index b4bc5a4f330e9..12649aafe74a1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java @@ -18,12 +18,11 @@ package org.apache.hadoop.fs.azurebfs.services; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; -import org.apache.hadoop.io.retry.RetryPolicies; -import org.apache.hadoop.io.retry.RetryPolicy; +import java.io.IOException; +import java.net.HttpURLConnection; +import java.util.concurrent.CancellationException; +import java.util.concurrent.TimeUnit; + import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableScheduledFuture; @@ -31,11 +30,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.concurrent.CancellationException; -import java.util.concurrent.TimeUnit; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.io.retry.RetryPolicy; -import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_ACQUIRING_LEASE; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_FUTURE_EXISTS; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_NO_LEASE_THREADS; @@ -71,7 +72,7 @@ public final class SelfRenewingLease { public static class LeaseException extends AzureBlobFileSystemException { public LeaseException(Throwable t) { - super(ERR_ACQUIRING_LEASE + ": " + t.getMessage()); + super(ERR_ACQUIRING_LEASE, t); } public LeaseException(String s) { @@ -161,7 +162,7 @@ public void onFailure(Throwable throwable) { return; } else if (throwable instanceof AbfsRestOperationException) { AbfsRestOperationException opEx = ((AbfsRestOperationException) throwable); - if (opEx.getStatusCode() < HTTP_INTERNAL_ERROR) { + if (opEx.getStatusCode() < HttpURLConnection.HTTP_INTERNAL_ERROR) { // error in 400 range indicates a type of error that should not result in a retry // such as the lease being broken or a different lease being present LOG.info("Stopping renewal due to {}: {}, {}", opEx.getStatusCode(), @@ -183,6 +184,9 @@ public void onFailure(Throwable throwable) { * will expire after the lease duration. */ public void free() { + if (leaseFreed) { + return; + } try { LOG.debug("Freeing lease: path {}, lease id {}", path, leaseID); if (future != null && !future.isDone()) { @@ -193,10 +197,8 @@ public void free() { LOG.info("Exception when trying to release lease {} on {}. Lease will be left to expire: {}", leaseID, path, e.getMessage()); } finally { - // Even if releasing the lease fails (e.g. because the file was deleted), - // make sure to record that we freed the lease, to terminate the - // keep-alive thread. + // make sure to record that we freed the lease leaseFreed = true; LOG.debug("Freed lease {} on {}", leaseID, path); } diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index 33d4a0fa428a0..e33a48427c520 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -887,6 +887,21 @@ enabled for your Azure Storage account." The directories can be specified as comma separated values. By default the value is "/hbase" +### Single Writer Options +`fs.azure.singlewriter.directories`: Directories for single writer support +can be specified comma separated in this config. By default, multiple +clients will be able to write to the same file simultaneously. When writing +to files contained within the directories specified in this config, the +client will obtain a lease on the file that will prevent any other clients +from writing to the file. The lease will be renewed by the client until the +output stream is closed, after which it will be released. To revoke a client's +write access for a file, the AzureBlobFilesystem breakLease method may be + called. + +`fs.azure.lease.threads`: This is the size of the thread pool that will be +used for lease operations for single writer directories. By default the value +is 0, so it must be set to at least 1 to support single writer directories. + ### Perf Options #### 1. HTTP Request Tracking Options diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index 347bd45facf8c..a2b0469c223f4 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -17,22 +17,28 @@ */ package org.apache.hadoop.fs.azurebfs; +import java.io.IOException; +import java.util.concurrent.RejectedExecutionException; + +import org.junit.Assert; +import org.junit.Test; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.util.concurrent.RejectedExecutionException; +import org.apache.hadoop.test.LambdaTestUtils; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SINGLE_WRITER_KEY; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_ACQUIRING_LEASE; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_ALREADY_PRESENT; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_BROKEN; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_DID_NOT_MATCH; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_EXPIRED; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_ID_NOT_PRESENT; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_NOT_PRESENT; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_NO_LEASE_ID_SPECIFIED; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_NO_LEASE_THREADS; @@ -44,6 +50,7 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest { private static final int TEST_EXECUTION_TIMEOUT = 30 * 1000; private static final int LONG_TEST_EXECUTION_TIMEOUT = 90 * 1000; + private static final int TEST_LEASE_DURATION = 15; private static final String TEST_FILE = "testfile"; private final boolean isHNSEnabled; @@ -126,15 +133,12 @@ public void testTwoCreate() throws Exception { fs.mkdirs(testFilePath.getParent()); try (FSDataOutputStream out = fs.create(testFilePath)) { - try (FSDataOutputStream out2 = fs.create(testFilePath)) { - Assert.fail("Second create succeeded"); - } catch (IOException e) { - if (isHNSEnabled) { - GenericTestUtils.assertExceptionContains(ERR_PARALLEL_ACCESS_DETECTED, e); - } else { - GenericTestUtils.assertExceptionContains(ERR_NO_LEASE_ID_SPECIFIED, e); + LambdaTestUtils.intercept(IOException.class, isHNSEnabled ? ERR_PARALLEL_ACCESS_DETECTED : + ERR_NO_LEASE_ID_SPECIFIED, () -> { + try (FSDataOutputStream out2 = fs.create(testFilePath)) { } - } + return "Expected second create on single writer dir to fail"; + }); } Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); } @@ -146,8 +150,7 @@ private void twoWriters(AzureBlobFileSystem fs, Path testFilePath, boolean expec out2.hsync(); } catch (IOException e) { if (expectException) { - Assert.assertTrue("Unexpected error message: " + e.getMessage(), - e.getMessage().contains(ERR_ACQUIRING_LEASE)); + GenericTestUtils.assertExceptionContains(ERR_ACQUIRING_LEASE, e); } else { Assert.fail("Unexpected exception " + e.getMessage()); } @@ -206,19 +209,17 @@ public void testWriteAfterBreakLease() throws Exception { out.hsync(); fs.breakLease(testFilePath); - try { + + LambdaTestUtils.intercept(IOException.class, ERR_LEASE_EXPIRED, () -> { out.write(1); out.hsync(); - Assert.fail("Expected exception on write after lease break"); - } catch (IOException e) { - GenericTestUtils.assertExceptionContains(ERR_LEASE_EXPIRED, e); - } - try { + return "Expected exception on write after lease break but got " + out; + }); + + LambdaTestUtils.intercept(IOException.class, ERR_LEASE_EXPIRED, () -> { out.close(); - Assert.fail("Expected exception on close after lease break"); - } catch (IOException e) { - GenericTestUtils.assertExceptionContains(ERR_LEASE_EXPIRED, e); - } + return "Expected exception on close after lease break but got " + out; + }); Assert.assertTrue(((AbfsOutputStream) out.getWrappedStream()).isLeaseFreed()); @@ -236,32 +237,24 @@ public void testLeaseFreedAfterBreak() throws Exception { final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 1); fs.mkdirs(testFilePath.getParent()); - FSDataOutputStream out = null; - try { - out = fs.create(testFilePath); - out.write(0); + FSDataOutputStream out = fs.create(testFilePath); + out.write(0); + + fs.breakLease(testFilePath); - fs.breakLease(testFilePath); - while (!((AbfsOutputStream) out.getWrappedStream()).isLeaseFreed()) { - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - } - } - } finally { + while (!((AbfsOutputStream) out.getWrappedStream()).isLeaseFreed()) { try { - if (out != null) { - out.close(); - } - Assert.fail("No exception on close after broken lease"); - } catch (IOException e) { - if (isHNSEnabled) { - GenericTestUtils.assertExceptionContains(ERR_LEASE_NOT_PRESENT, e); - } else { - GenericTestUtils.assertExceptionContains(ERR_LEASE_EXPIRED, e); - } + Thread.sleep(1000); + } catch (InterruptedException e) { } } + + LambdaTestUtils.intercept(IOException.class, isHNSEnabled ? ERR_LEASE_NOT_PRESENT : + ERR_LEASE_EXPIRED, () -> { + out.close(); + return "Expected exception on close after lease break but got " + out; + }); + Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); } @@ -276,20 +269,57 @@ public void testFileSystemClose() throws Exception { Assert.assertFalse(fs.getAbfsStore().areLeasesFreed()); fs.close(); Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); - try { + + LambdaTestUtils.intercept(IOException.class, isHNSEnabled ? ERR_LEASE_NOT_PRESENT : + ERR_LEASE_EXPIRED, () -> { out.close(); - Assert.fail("No exception on close after closed FS"); - } catch (IOException e) { - if (isHNSEnabled) { - GenericTestUtils.assertExceptionContains(ERR_LEASE_NOT_PRESENT, e); - } else { - GenericTestUtils.assertExceptionContains(ERR_LEASE_EXPIRED, e); + return "Expected exception on close after closed FS but got " + out; + }); + + LambdaTestUtils.intercept(RejectedExecutionException.class, () -> { + try (FSDataOutputStream out2 = fs.append(testFilePath)) { } - } + return "Expected exception on new append after closed FS"; + }); + } - try (FSDataOutputStream out2 = fs.append(testFilePath)) { - Assert.fail("Expected an error on operation after closed FS"); - } catch (RejectedExecutionException e) { - } + @Test(timeout = TEST_EXECUTION_TIMEOUT) + public void testFileSystemLeaseOps() throws Exception { + final Path testDir = path(methodName.getMethodName()); + final AzureBlobFileSystem fs = getFileSystem(); + fs.mkdirs(testDir); + + LambdaTestUtils.intercept(IOException.class, ERR_LEASE_ID_NOT_PRESENT, () -> { + fs.breakLease(testDir); + return "Expected exception on break lease when no lease is held"; + }); + + String leaseId = fs.acquireLease(testDir, TEST_LEASE_DURATION); + LambdaTestUtils.intercept(IOException.class, ERR_LEASE_ALREADY_PRESENT, () -> { + fs.acquireLease(testDir, TEST_LEASE_DURATION); + return "Expected exception on acquire lease when lease is already held"; + }); + fs.renewLease(testDir, leaseId); + fs.releaseLease(testDir, leaseId); + LambdaTestUtils.intercept(IOException.class, ERR_LEASE_NOT_PRESENT, () -> { + fs.renewLease(testDir, leaseId); + return "Expected exception on renew lease after lease has been released"; + }); + + String leaseId2 = fs.acquireLease(testDir, TEST_LEASE_DURATION); + LambdaTestUtils.intercept(IOException.class, ERR_LEASE_DID_NOT_MATCH, () -> { + fs.renewLease(testDir, leaseId); + return "Expected exception on renew wrong lease ID"; + }); + LambdaTestUtils.intercept(IOException.class, ERR_LEASE_DID_NOT_MATCH, () -> { + fs.releaseLease(testDir, leaseId); + return "Expected exception on release wrong lease ID"; + }); + fs.breakLease(testDir); + LambdaTestUtils.intercept(IOException.class, ERR_LEASE_BROKEN, () -> { + fs.renewLease(testDir, leaseId2); + return "Expected exception on renewing broken lease"; + }); + fs.releaseLease(testDir, leaseId2); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 081f2e9350efd..f4243bc7e287b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -89,7 +89,7 @@ public void verifyShortWriteRequest() throws Exception { when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, null, + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[WRITE_SIZE]; new Random().nextBytes(b); @@ -136,7 +136,7 @@ public void verifyWriteRequest() throws Exception { when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, null, + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[WRITE_SIZE]; new Random().nextBytes(b); @@ -194,7 +194,7 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { when(op.getSasToken()).thenReturn("testToken"); when(op.getResult()).thenReturn(httpOp); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, null, + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -252,7 +252,7 @@ public void verifyWriteRequestOfBufferSize() throws Exception { when(op.getSasToken()).thenReturn("testToken"); when(op.getResult()).thenReturn(httpOp); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, null, + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -294,7 +294,7 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, null, + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, true)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -337,7 +337,7 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, null, + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -391,7 +391,7 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any())).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull())).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, null, + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); From 9f3d6897f7cd1cc43061e144abe73c8da18de93d Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Fri, 29 Jan 2021 09:07:18 -0500 Subject: [PATCH 09/18] HADOOP-16948. Use daemon threads for ABFS lease ops --- .../org/apache/hadoop/fs/azurebfs/services/AbfsClient.java | 7 ++++++- .../hadoop/fs/azurebfs/services/SelfRenewingLease.java | 2 +- .../hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java | 2 +- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 7e3b6dc878e09..7c8a2112bfa46 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -31,6 +31,7 @@ import java.util.Locale; import java.util.UUID; import java.util.concurrent.Callable; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; @@ -41,6 +42,8 @@ import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableScheduledFuture; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningScheduledExecutorService; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -119,8 +122,10 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden this.abfsPerfTracker = abfsClientContext.getAbfsPerfTracker(); this.abfsCounters = abfsClientContext.getAbfsCounters(); + ThreadFactory tf = + new ThreadFactoryBuilder().setNameFormat("AbfsClient Lease Ops").setDaemon(true).build(); this.executorService = MoreExecutors.listeningDecorator( - HadoopExecutors.newScheduledThreadPool(this.abfsConfiguration.getNumLeaseThreads())); + HadoopExecutors.newScheduledThreadPool(this.abfsConfiguration.getNumLeaseThreads(), tf)); } public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java index 12649aafe74a1..c366d747caa8c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java @@ -72,7 +72,7 @@ public final class SelfRenewingLease { public static class LeaseException extends AzureBlobFileSystemException { public LeaseException(Throwable t) { - super(ERR_ACQUIRING_LEASE, t); + super(ERR_ACQUIRING_LEASE + ": " + t, t); } public LeaseException(String s) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index a2b0469c223f4..1eeab62ff0b69 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -152,7 +152,7 @@ private void twoWriters(AzureBlobFileSystem fs, Path testFilePath, boolean expec if (expectException) { GenericTestUtils.assertExceptionContains(ERR_ACQUIRING_LEASE, e); } else { - Assert.fail("Unexpected exception " + e.getMessage()); + throw e; } } out.writeInt(1); From d67882f80615f8467716a3ed37c10e38249f2afa Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Tue, 2 Mar 2021 18:20:38 -0500 Subject: [PATCH 10/18] HADOOP-16948. Make lease duration configurable --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 29 +++++++ .../fs/azurebfs/AzureBlobFileSystemStore.java | 31 ++++--- .../azurebfs/constants/ConfigurationKeys.java | 4 + .../constants/FileSystemConfigurations.java | 4 + .../ConfigurationValidationAnnotations.java | 16 ++++ .../IntegerConfigurationBasicValidator.java | 13 ++- .../azurebfs/services/AbfsOutputStream.java | 8 +- .../services/AbfsOutputStreamContext.java | 26 +++--- .../azurebfs/services/SelfRenewingLease.java | 25 +++--- .../ITestAzureBlobFileSystemLease.java | 81 +++++++++++++------ .../TestConfigurationValidators.java | 29 ++++++- 11 files changed, 194 insertions(+), 72 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 41f18389ceb0e..febf0b996174c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.AuthConfigurations; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation; +import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerWithOutlierConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation; @@ -217,6 +218,13 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_LEASE_THREADS) private int numLeaseThreads; + @IntegerWithOutlierConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_LEASE_DURATION, + OutlierValue = INFINITE_LEASE_DURATION, + MinValue = MIN_LEASE_DURATION, + MaxValue = MAX_LEASE_DURATION, + DefaultValue = DEFAULT_LEASE_DURATION) + private int leaseDuration; + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, DefaultValue = DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION) private boolean createRemoteFileSystemDuringInitialization; @@ -305,6 +313,8 @@ public AbfsConfiguration(final Configuration rawConfig, String accountName) field.setAccessible(true); if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) { field.set(this, validateInt(field)); + } else if (field.isAnnotationPresent(IntegerWithOutlierConfigurationValidatorAnnotation.class)) { + field.set(this, validateIntWithOutlier(field)); } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) { field.set(this, validateLong(field)); } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) { @@ -651,6 +661,10 @@ public int getNumLeaseThreads() { return this.numLeaseThreads; } + public int getLeaseDuration() { + return this.leaseDuration; + } + public boolean getCreateRemoteFileSystemDuringInitialization() { // we do not support creating the filesystem when AuthType is SAS return this.createRemoteFileSystemDuringInitialization @@ -860,6 +874,21 @@ int validateInt(Field field) throws IllegalAccessException, InvalidConfiguration validator.ThrowIfInvalid()).validate(value); } + int validateIntWithOutlier(Field field) throws IllegalAccessException, InvalidConfigurationValueException { + IntegerWithOutlierConfigurationValidatorAnnotation validator = + field.getAnnotation(IntegerWithOutlierConfigurationValidatorAnnotation.class); + String value = get(validator.ConfigurationKey()); + + // validate + return new IntegerConfigurationBasicValidator( + validator.OutlierValue(), + validator.MinValue(), + validator.MaxValue(), + validator.DefaultValue(), + validator.ConfigurationKey(), + validator.ThrowIfInvalid()).validate(value); + } + long validateLong(Field field) throws IllegalAccessException, InvalidConfigurationValueException { LongConfigurationValidatorAnnotation validator = field.getAnnotation(LongConfigurationValidatorAnnotation.class); String value = rawConfig.get(validator.ConfigurationKey()); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index f50a6dd9d1467..ee7932440c1d5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -483,16 +483,14 @@ public OutputStream createFile(final Path path, final FsPermission umask) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(); - boolean enableSingleWriter = isSingleWriterKey(path.toString()); LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} " - + "isNamespaceEnabled: {} enableSingleWriter: {}", + + "isNamespaceEnabled: {}", client.getFileSystem(), path, overwrite, permission, umask, - isNamespaceEnabled, - enableSingleWriter); + isNamespaceEnabled); String relativePath = getRelativePath(path); boolean isAppendBlob = false; @@ -528,12 +526,14 @@ public OutputStream createFile(final Path path, } perfInfo.registerResult(op.getResult()).registerSuccess(true); + SelfRenewingLease lease = maybeCreateLease(relativePath); + return new AbfsOutputStream( client, statistics, relativePath, 0, - populateAbfsOutputStreamContext(isAppendBlob, enableSingleWriter)); + populateAbfsOutputStreamContext(isAppendBlob, lease)); } } @@ -606,7 +606,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa } private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppendBlob, - boolean enableSingleWriter) { + SelfRenewingLease lease) { int bufferSize = abfsConfiguration.getWriteBufferSize(); if (isAppendBlob && bufferSize > FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE) { bufferSize = FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE; @@ -620,8 +620,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppend .withAppendBlob(isAppendBlob) .withWriteMaxConcurrentRequestCount(abfsConfiguration.getWriteMaxConcurrentRequestCount()) .withMaxWriteRequestsToQueue(abfsConfiguration.getMaxWriteRequestsToQueue()) - .withSingleWriterEnabled(enableSingleWriter) - .withLeaseRefs(leaseRefs) + .withLease(lease) .build(); } @@ -742,12 +741,14 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic isAppendBlob = true; } + SelfRenewingLease lease = maybeCreateLease(relativePath); + return new AbfsOutputStream( client, statistics, relativePath, offset, - populateAbfsOutputStreamContext(isAppendBlob, enableSingleWriter)); + populateAbfsOutputStreamContext(isAppendBlob, lease)); } } @@ -1743,6 +1744,18 @@ private void updateSingleWriterDirs() { this.azureSingleWriterDirSet.remove(""); } + private SelfRenewingLease maybeCreateLease(String relativePath) + throws AzureBlobFileSystemException { + boolean enableSingleWriter = isSingleWriterKey(relativePath); + if (!enableSingleWriter) { + return null; + } + SelfRenewingLease lease = new SelfRenewingLease(client, relativePath, + abfsConfiguration.getLeaseDuration()); + leaseRefs.put(lease, null); + return lease; + } + @VisibleForTesting boolean areLeasesFreed() { for (SelfRenewingLease lease : leaseRefs.keySet()) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index e30387fde1f85..7628b261ff908 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -93,6 +93,10 @@ public final class ConfigurationKeys { /** Provides a number of threads to use for lease operations for single writer directories. * Must be set to a minimum of 1 if single writer directories are to be used. Default is 0. **/ public static final String FS_AZURE_LEASE_THREADS = "fs.azure.lease.threads"; + /** Provides a lease duration in seconds. Must be between 15 and 60 seconds or -1 for infinite + * lease duration. For finite durations, leases will be renewed automatically for single + * writer directories. **/ + public static final String FS_AZURE_LEASE_DURATION = "fs.azure.lease.duration"; public static final String FS_AZURE_READ_AHEAD_QUEUE_DEPTH = "fs.azure.readaheadqueue.depth"; public static final String FS_AZURE_ALWAYS_READ_BUFFER_SIZE = "fs.azure.read.alwaysReadBufferSize"; public static final String FS_AZURE_READ_AHEAD_BLOCK_SIZE = "fs.azure.read.readahead.blocksize"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 3908020275b1c..612bafdc9432a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -81,6 +81,10 @@ public final class FileSystemConfigurations { public static final String DEFAULT_FS_AZURE_SINGLE_WRITER_DIRECTORIES = ""; public static final int DEFAULT_LEASE_THREADS = 0; public static final int MIN_LEASE_THREADS = 0; + public static final int DEFAULT_LEASE_DURATION = 60; + public static final int INFINITE_LEASE_DURATION = -1; + public static final int MIN_LEASE_DURATION = 15; + public static final int MAX_LEASE_DURATION = 60; public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = -1; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java index 82c571a3b03b3..9fbe5a22cdf77 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java @@ -46,6 +46,22 @@ public class ConfigurationValidationAnnotations { boolean ThrowIfInvalid() default false; } + @Target({ ElementType.FIELD }) + @Retention(RetentionPolicy.RUNTIME) + public @interface IntegerWithOutlierConfigurationValidatorAnnotation { + String ConfigurationKey(); + + int MaxValue() default Integer.MAX_VALUE; + + int MinValue() default Integer.MIN_VALUE; + + int OutlierValue() default Integer.MIN_VALUE; + + int DefaultValue(); + + boolean ThrowIfInvalid() default false; + } + /** * Describes the requirements when validating the annotated long field. */ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java index 26c7d2f0ac19c..9d4beb74bbe3f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java @@ -31,11 +31,18 @@ public class IntegerConfigurationBasicValidator extends ConfigurationBasicValidator implements ConfigurationValidator { private final int min; private final int max; + private final int outlier; public IntegerConfigurationBasicValidator(final int min, final int max, final int defaultVal, final String configKey, final boolean throwIfInvalid) { + this(min, min, max, defaultVal, configKey, throwIfInvalid); + } + + public IntegerConfigurationBasicValidator(final int outlier, final int min, final int max, + final int defaultVal, final String configKey, final boolean throwIfInvalid) { super(configKey, defaultVal, throwIfInvalid); this.min = min; this.max = max; + this.outlier = outlier; } public Integer validate(final String configValue) throws InvalidConfigurationValueException { @@ -47,10 +54,14 @@ public Integer validate(final String configValue) throws InvalidConfigurationVal try { result = Integer.parseInt(configValue); // throw an exception if a 'within bounds' value is missing - if (getThrowIfInvalid() && (result < this.min || result > this.max)) { + if (getThrowIfInvalid() && (result != outlier) && (result < this.min || result > this.max)) { throw new InvalidConfigurationValueException(getConfigKey()); } + if (result == outlier) { + return result; + } + // set the value to the nearest bound if it's out of bounds if (result < this.min) { return this.min; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 8864cbcf6ac39..1ca92404b51ab 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -51,7 +51,6 @@ import org.apache.hadoop.io.ElasticByteBufferPool; import org.apache.hadoop.fs.FileSystem.Statistics; import org.apache.hadoop.fs.FSExceptionMessages; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.Syncable; @@ -149,11 +148,8 @@ public AbfsOutputStream( this.maxRequestsThatCanBeQueued = abfsOutputStreamContext .getMaxWriteRequestsToQueue(); - if (abfsOutputStreamContext.isEnableSingleWriter()) { - lease = new SelfRenewingLease(client, new Path(path)); - this.leaseId = lease.getLeaseID(); - abfsOutputStreamContext.addLease(lease); - } + this.lease = abfsOutputStreamContext.getLease(); + this.leaseId = abfsOutputStreamContext.getLeaseId(); this.threadExecutor = new ThreadPoolExecutor(maxConcurrentRequestCount, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java index a293a33dfb090..d8a44e80d640e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java @@ -18,8 +18,6 @@ package org.apache.hadoop.fs.azurebfs.services; -import java.util.Map; - /** * Class to hold extra output stream configs. */ @@ -41,9 +39,7 @@ public class AbfsOutputStreamContext extends AbfsStreamContext { private int maxWriteRequestsToQueue; - private boolean enableSingleWriter; - - private Map leaseRefs; + private SelfRenewingLease lease; public AbfsOutputStreamContext(final long sasTokenRenewPeriodForStreamsInSeconds) { super(sasTokenRenewPeriodForStreamsInSeconds); @@ -100,13 +96,8 @@ public AbfsOutputStreamContext withMaxWriteRequestsToQueue( return this; } - public AbfsOutputStreamContext withSingleWriterEnabled(final boolean enableSingleWriter) { - this.enableSingleWriter = enableSingleWriter; - return this; - } - - public AbfsOutputStreamContext withLeaseRefs(final Map leaseRefs) { - this.leaseRefs = leaseRefs; + public AbfsOutputStreamContext withLease(final SelfRenewingLease lease) { + this.lease = lease; return this; } @@ -142,13 +133,14 @@ public boolean isEnableSmallWriteOptimization() { return this.enableSmallWriteOptimization; } - public boolean isEnableSingleWriter() { - return this.enableSingleWriter; + public SelfRenewingLease getLease() { + return this.lease; } - public void addLease(SelfRenewingLease lease) { - if (this.leaseRefs != null) { - this.leaseRefs.put(lease, null); + public String getLeaseId() { + if (this.lease == null) { + return null; } + return this.lease.getLeaseID(); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java index c366d747caa8c..3ad45719fb31f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java @@ -23,20 +23,19 @@ import java.util.concurrent.CancellationException; import java.util.concurrent.TimeUnit; -import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableScheduledFuture; import org.apache.hadoop.thirdparty.org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryPolicy; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.INFINITE_LEASE_DURATION; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_ACQUIRING_LEASE; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_FUTURE_EXISTS; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_NO_LEASE_THREADS; @@ -55,14 +54,15 @@ public final class SelfRenewingLease { private static final Logger LOG = LoggerFactory.getLogger(SelfRenewingLease.class); - static final int LEASE_DURATION = 60; // Lease duration in seconds - static final int LEASE_RENEWAL_PERIOD = 40; // Lease renewal interval in seconds + static final float LEASE_RENEWAL_PERCENT_OF_DURATION = 0.67f; // Lease renewal percent of duration static final int LEASE_ACQUIRE_RETRY_INTERVAL = 10; // Retry interval for acquiring lease in secs static final int LEASE_ACQUIRE_MAX_RETRIES = 7; // Number of retries for acquiring lease private final AbfsClient client; private final String path; + private final int duration; + private final int renewalPeriod; // Lease status variables private volatile boolean leaseFreed; @@ -80,10 +80,12 @@ public LeaseException(String s) { } } - public SelfRenewingLease(AbfsClient client, Path path) throws AzureBlobFileSystemException { + public SelfRenewingLease(AbfsClient client, String path, int duration) throws AzureBlobFileSystemException { this.leaseFreed = false; this.client = client; - this.path = getRelativePath(path); + this.path = path; + this.duration = duration; + this.renewalPeriod = (int) (LEASE_RENEWAL_PERCENT_OF_DURATION * this.duration); if (client.getNumLeaseThreads() < 1) { throw new LeaseException(ERR_NO_LEASE_THREADS); @@ -101,7 +103,9 @@ public SelfRenewingLease(AbfsClient client, Path path) throws AzureBlobFileSyste throw new LeaseException(exception); } - renewLease(LEASE_RENEWAL_PERIOD); + if (duration != INFINITE_LEASE_DURATION) { + renewLease(renewalPeriod); + } LOG.debug("Acquired lease {} on {}", leaseID, path); } @@ -112,7 +116,7 @@ private void acquireLease(RetryPolicy retryPolicy, int numRetries, long delay) if (future != null && !future.isDone()) { throw new LeaseException(ERR_LEASE_FUTURE_EXISTS); } - future = client.schedule(() -> client.acquireLease(path, LEASE_DURATION), + future = client.schedule(() -> client.acquireLease(path, duration), delay, TimeUnit.SECONDS); client.addCallback(future, new FutureCallback() { @Override @@ -211,9 +215,4 @@ public boolean isFreed() { public String getLeaseID() { return leaseID; } - - private String getRelativePath(final Path path) { - Preconditions.checkNotNull(path, "path"); - return path.toUri().getPath(); - } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index 1eeab62ff0b69..44704995fac3b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -30,8 +30,10 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_DURATION; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SINGLE_WRITER_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.INFINITE_LEASE_DURATION; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_ACQUIRING_LEASE; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_ALREADY_PRESENT; @@ -61,16 +63,17 @@ public ITestAzureBlobFileSystemLease() throws Exception { .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false); } - private AzureBlobFileSystem getCustomFileSystem(String singleWriterDirs, int numLeaseThreads) - throws Exception { + private AzureBlobFileSystem getCustomFileSystem(Path singleWriterDirs, int numLeaseThreads, + int leaseDuration) throws Exception { Configuration conf = getRawConfiguration(); conf.setBoolean(String.format("fs.%s.impl.disable.cache", getAbfsScheme()), true); - conf.set(FS_AZURE_SINGLE_WRITER_KEY, singleWriterDirs); + conf.set(FS_AZURE_SINGLE_WRITER_KEY, singleWriterDirs.toUri().getPath()); conf.setInt(FS_AZURE_LEASE_THREADS, numLeaseThreads); + conf.setInt(FS_AZURE_LEASE_DURATION, leaseDuration); return getFileSystem(conf); } - @Test + @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testNoSingleWriter() throws IOException { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getFileSystem(); @@ -82,22 +85,24 @@ public void testNoSingleWriter() throws IOException { Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); } - @Test + @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testNoLeaseThreads() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 0); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 0, + TEST_LEASE_DURATION); fs.mkdirs(testFilePath.getParent()); - try (FSDataOutputStream out = fs.create(testFilePath)) { - Assert.fail("No failure when lease requested with 0 lease threads"); - } catch (Exception e) { - GenericTestUtils.assertExceptionContains(ERR_NO_LEASE_THREADS, e); - } + LambdaTestUtils.intercept(IOException.class, ERR_NO_LEASE_THREADS, () -> { + try (FSDataOutputStream out = fs.create(testFilePath)) { + } + return "No failure when lease requested with 0 lease threads"; + }); } - @Test + @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testOneWriter() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 1); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, + TEST_LEASE_DURATION); fs.mkdirs(testFilePath.getParent()); FSDataOutputStream out = fs.create(testFilePath); @@ -109,12 +114,13 @@ public void testOneWriter() throws Exception { Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); } - @Test + @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testSubDir() throws Exception { final Path testFilePath = new Path(new Path(path(methodName.getMethodName()), "subdir"), TEST_FILE); final AzureBlobFileSystem fs = - getCustomFileSystem(testFilePath.getParent().getParent().toString(), 1); + getCustomFileSystem(testFilePath.getParent().getParent(), 1, + TEST_LEASE_DURATION); fs.mkdirs(testFilePath.getParent().getParent()); FSDataOutputStream out = fs.create(testFilePath); @@ -126,10 +132,11 @@ public void testSubDir() throws Exception { Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); } - @Test + @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testTwoCreate() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 1); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, + TEST_LEASE_DURATION); fs.mkdirs(testFilePath.getParent()); try (FSDataOutputStream out = fs.create(testFilePath)) { @@ -174,7 +181,8 @@ public void testTwoWritersCreateAppendNoSingleWriter() throws Exception { @Test(timeout = LONG_TEST_EXECUTION_TIMEOUT) public void testTwoWritersCreateAppendWithSingleWriterEnabled() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 1); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, + TEST_LEASE_DURATION); fs.mkdirs(testFilePath.getParent()); twoWriters(fs, testFilePath, true); @@ -183,7 +191,8 @@ public void testTwoWritersCreateAppendWithSingleWriterEnabled() throws Exception @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testLeaseFreedOnClose() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 1); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, + TEST_LEASE_DURATION); fs.mkdirs(testFilePath.getParent()); FSDataOutputStream out; @@ -200,7 +209,8 @@ public void testLeaseFreedOnClose() throws Exception { @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testWriteAfterBreakLease() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 1); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, + TEST_LEASE_DURATION); fs.mkdirs(testFilePath.getParent()); FSDataOutputStream out; @@ -234,7 +244,8 @@ public void testWriteAfterBreakLease() throws Exception { @Test(timeout = LONG_TEST_EXECUTION_TIMEOUT) public void testLeaseFreedAfterBreak() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 1); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, + TEST_LEASE_DURATION); fs.mkdirs(testFilePath.getParent()); FSDataOutputStream out = fs.create(testFilePath); @@ -258,10 +269,33 @@ public void testLeaseFreedAfterBreak() throws Exception { Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); } + @Test(timeout = TEST_EXECUTION_TIMEOUT) + public void testInfiniteLease() throws Exception { + final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, + INFINITE_LEASE_DURATION); + fs.mkdirs(testFilePath.getParent()); + + try (FSDataOutputStream out = fs.create(testFilePath)) { + Assert.assertTrue("Output stream should have lease", + ((AbfsOutputStream) out.getWrappedStream()).hasLease()); + out.write(0); + } + Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + + try (FSDataOutputStream out = fs.append(testFilePath)) { + Assert.assertTrue("Output stream should have lease", + ((AbfsOutputStream) out.getWrappedStream()).hasLease()); + out.write(1); + } + Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + } + @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testFileSystemClose() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent().toString(), 1); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, + TEST_LEASE_DURATION); fs.mkdirs(testFilePath.getParent()); FSDataOutputStream out = fs.create(testFilePath); @@ -301,7 +335,8 @@ public void testFileSystemLeaseOps() throws Exception { }); fs.renewLease(testDir, leaseId); fs.releaseLease(testDir, leaseId); - LambdaTestUtils.intercept(IOException.class, ERR_LEASE_NOT_PRESENT, () -> { + LambdaTestUtils.intercept(IOException.class, isHNSEnabled ? ERR_LEASE_NOT_PRESENT : + ERR_LEASE_DID_NOT_MATCH, () -> { fs.renewLease(testDir, leaseId); return "Expected exception on renew lease after lease has been released"; }); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java index f02eadc9a0491..6a02435fc6e5e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java @@ -24,11 +24,14 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.apache.hadoop.fs.azurebfs.utils.Base64; -import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; -import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_LEASE_DURATION; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE; - +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.INFINITE_LEASE_DURATION; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_LEASE_DURATION; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_LEASE_DURATION; /** * Test configuration validators. @@ -58,6 +61,26 @@ public void testIntegerConfigValidatorThrowsIfMissingValidValue() throws Excepti integerConfigurationValidator.validate("3072"); } + @Test + public void testIntegerWithOutlierConfigValidator() throws Exception { + IntegerConfigurationBasicValidator integerConfigurationValidator = new IntegerConfigurationBasicValidator( + INFINITE_LEASE_DURATION, MIN_LEASE_DURATION, MAX_LEASE_DURATION, DEFAULT_LEASE_DURATION, FAKE_KEY, + false); + + assertEquals(INFINITE_LEASE_DURATION, (int) integerConfigurationValidator.validate("-1")); + assertEquals(DEFAULT_LEASE_DURATION, (int) integerConfigurationValidator.validate(null)); + assertEquals(MIN_LEASE_DURATION, (int) integerConfigurationValidator.validate("15")); + assertEquals(MAX_LEASE_DURATION, (int) integerConfigurationValidator.validate("60")); + } + + @Test(expected = InvalidConfigurationValueException.class) + public void testIntegerWithOutlierConfigValidatorThrowsIfMissingValidValue() throws Exception { + IntegerConfigurationBasicValidator integerConfigurationValidator = new IntegerConfigurationBasicValidator( + INFINITE_LEASE_DURATION, MIN_LEASE_DURATION, MAX_LEASE_DURATION, DEFAULT_LEASE_DURATION, FAKE_KEY, + true); + integerConfigurationValidator.validate("14"); + } + @Test public void testLongConfigValidator() throws Exception { LongConfigurationBasicValidator longConfigurationValidator = new LongConfigurationBasicValidator( From f00c1459e99b6cdd88ffc55af66fdb44267837bc Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Tue, 2 Mar 2021 18:59:32 -0500 Subject: [PATCH 11/18] HADOOP-16948. Add error messages to test assertions --- .../ITestAzureBlobFileSystemLease.java | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index 44704995fac3b..745b18dd3cddc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -82,7 +82,7 @@ public void testNoSingleWriter() throws IOException { Assert.assertFalse("Output stream should not have lease", ((AbfsOutputStream) out.getWrappedStream()).hasLease()); } - Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); } @Test(timeout = TEST_EXECUTION_TIMEOUT) @@ -111,7 +111,7 @@ public void testOneWriter() throws Exception { out.close(); Assert.assertFalse("Output stream should not have lease", ((AbfsOutputStream) out.getWrappedStream()).hasLease()); - Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); } @Test(timeout = TEST_EXECUTION_TIMEOUT) @@ -129,7 +129,7 @@ public void testSubDir() throws Exception { out.close(); Assert.assertFalse("Output stream should not have lease", ((AbfsOutputStream) out.getWrappedStream()).hasLease()); - Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); } @Test(timeout = TEST_EXECUTION_TIMEOUT) @@ -147,7 +147,7 @@ public void testTwoCreate() throws Exception { return "Expected second create on single writer dir to fail"; }); } - Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); } private void twoWriters(AzureBlobFileSystem fs, Path testFilePath, boolean expectException) throws Exception { @@ -166,7 +166,7 @@ private void twoWriters(AzureBlobFileSystem fs, Path testFilePath, boolean expec out.hsync(); } - Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); } @Test(timeout = TEST_EXECUTION_TIMEOUT) @@ -203,7 +203,7 @@ public void testLeaseFreedOnClose() throws Exception { out.close(); Assert.assertFalse("Output stream should not have lease after close", ((AbfsOutputStream) out.getWrappedStream()).hasLease()); - Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); } @Test(timeout = TEST_EXECUTION_TIMEOUT) @@ -231,14 +231,15 @@ public void testWriteAfterBreakLease() throws Exception { return "Expected exception on close after lease break but got " + out; }); - Assert.assertTrue(((AbfsOutputStream) out.getWrappedStream()).isLeaseFreed()); + Assert.assertTrue("Output stream lease should be freed", + ((AbfsOutputStream) out.getWrappedStream()).isLeaseFreed()); try (FSDataOutputStream out2 = fs.append(testFilePath)) { out2.write(2); out2.hsync(); } - Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); } @Test(timeout = LONG_TEST_EXECUTION_TIMEOUT) @@ -266,7 +267,7 @@ public void testLeaseFreedAfterBreak() throws Exception { return "Expected exception on close after lease break but got " + out; }); - Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); } @Test(timeout = TEST_EXECUTION_TIMEOUT) @@ -288,7 +289,7 @@ public void testInfiniteLease() throws Exception { ((AbfsOutputStream) out.getWrappedStream()).hasLease()); out.write(1); } - Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); } @Test(timeout = TEST_EXECUTION_TIMEOUT) @@ -300,9 +301,9 @@ public void testFileSystemClose() throws Exception { FSDataOutputStream out = fs.create(testFilePath); out.write(0); - Assert.assertFalse(fs.getAbfsStore().areLeasesFreed()); + Assert.assertFalse("Store leases should exist", fs.getAbfsStore().areLeasesFreed()); fs.close(); - Assert.assertTrue(fs.getAbfsStore().areLeasesFreed()); + Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); LambdaTestUtils.intercept(IOException.class, isHNSEnabled ? ERR_LEASE_NOT_PRESENT : ERR_LEASE_EXPIRED, () -> { From f00178355c66810ce7664e21a366cd24fc4c1965 Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Wed, 3 Mar 2021 12:13:56 -0500 Subject: [PATCH 12/18] HADOOP-16948. Remove extra isSingleWriterKey call --- .../hadoop/fs/azurebfs/AzureBlobFileSystemStore.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index ee7932440c1d5..d707d7ad47a01 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -483,8 +483,7 @@ public OutputStream createFile(final Path path, final FsPermission umask) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(); - LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} " - + "isNamespaceEnabled: {}", + LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}", client.getFileSystem(), path, overwrite, @@ -709,12 +708,10 @@ private AbfsInputStreamContext populateAbfsInputStreamContext( public OutputStream openFileForWrite(final Path path, final FileSystem.Statistics statistics, final boolean overwrite) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("openFileForWrite", "getPathStatus")) { - boolean enableSingleWriter = isSingleWriterKey(path.toString()); - LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {} enableSingleWriter: {}", + LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}", client.getFileSystem(), path, - overwrite, - enableSingleWriter); + overwrite); String relativePath = getRelativePath(path); From 92e7343b26eb67cf7f2a6ebc885fdd6eb9c9b551 Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Mon, 15 Mar 2021 15:24:48 -0400 Subject: [PATCH 13/18] HADOOP-16948. Use only infinite lease duration due to cost of renewal ops --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 11 --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 19 +++-- .../azurebfs/constants/ConfigurationKeys.java | 4 - .../constants/FileSystemConfigurations.java | 2 +- ...{SelfRenewingLease.java => AbfsLease.java} | 79 +++---------------- .../azurebfs/services/AbfsOutputStream.java | 2 +- .../services/AbfsOutputStreamContext.java | 6 +- .../hadoop-azure/src/site/markdown/abfs.md | 9 ++- .../ITestAzureBlobFileSystemLease.java | 49 ++++-------- 9 files changed, 47 insertions(+), 134 deletions(-) rename hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/{SelfRenewingLease.java => AbfsLease.java} (65%) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index febf0b996174c..7b72819696c2e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -218,13 +218,6 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_LEASE_THREADS) private int numLeaseThreads; - @IntegerWithOutlierConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_LEASE_DURATION, - OutlierValue = INFINITE_LEASE_DURATION, - MinValue = MIN_LEASE_DURATION, - MaxValue = MAX_LEASE_DURATION, - DefaultValue = DEFAULT_LEASE_DURATION) - private int leaseDuration; - @BooleanConfigurationValidatorAnnotation(ConfigurationKey = AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, DefaultValue = DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION) private boolean createRemoteFileSystemDuringInitialization; @@ -661,10 +654,6 @@ public int getNumLeaseThreads() { return this.numLeaseThreads; } - public int getLeaseDuration() { - return this.leaseDuration; - } - public boolean getCreateRemoteFileSystemDuringInitialization() { // we do not support creating the filesystem when AuthType is SAS return this.createRemoteFileSystemDuringInitialization diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index d707d7ad47a01..e154aed8f514a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -105,7 +105,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy; -import org.apache.hadoop.fs.azurebfs.services.SelfRenewingLease; +import org.apache.hadoop.fs.azurebfs.services.AbfsLease; import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials; import org.apache.hadoop.fs.azurebfs.services.AbfsPerfTracker; import org.apache.hadoop.fs.azurebfs.services.AbfsPerfInfo; @@ -151,7 +151,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport { private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1"; private static final int GET_SET_AGGREGATE_COUNT = 2; - private final Map leaseRefs; + private final Map leaseRefs; private final AbfsConfiguration abfsConfiguration; private final Set azureAtomicRenameDirSet; @@ -259,7 +259,7 @@ public String getPrimaryGroup() { @Override public void close() throws IOException { List> futures = new ArrayList<>(); - for (SelfRenewingLease lease : leaseRefs.keySet()) { + for (AbfsLease lease : leaseRefs.keySet()) { if (lease == null) { continue; } @@ -525,7 +525,7 @@ public OutputStream createFile(final Path path, } perfInfo.registerResult(op.getResult()).registerSuccess(true); - SelfRenewingLease lease = maybeCreateLease(relativePath); + AbfsLease lease = maybeCreateLease(relativePath); return new AbfsOutputStream( client, @@ -605,7 +605,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa } private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppendBlob, - SelfRenewingLease lease) { + AbfsLease lease) { int bufferSize = abfsConfiguration.getWriteBufferSize(); if (isAppendBlob && bufferSize > FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE) { bufferSize = FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE; @@ -738,7 +738,7 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic isAppendBlob = true; } - SelfRenewingLease lease = maybeCreateLease(relativePath); + AbfsLease lease = maybeCreateLease(relativePath); return new AbfsOutputStream( client, @@ -1741,21 +1741,20 @@ private void updateSingleWriterDirs() { this.azureSingleWriterDirSet.remove(""); } - private SelfRenewingLease maybeCreateLease(String relativePath) + private AbfsLease maybeCreateLease(String relativePath) throws AzureBlobFileSystemException { boolean enableSingleWriter = isSingleWriterKey(relativePath); if (!enableSingleWriter) { return null; } - SelfRenewingLease lease = new SelfRenewingLease(client, relativePath, - abfsConfiguration.getLeaseDuration()); + AbfsLease lease = new AbfsLease(client, relativePath); leaseRefs.put(lease, null); return lease; } @VisibleForTesting boolean areLeasesFreed() { - for (SelfRenewingLease lease : leaseRefs.keySet()) { + for (AbfsLease lease : leaseRefs.keySet()) { if (lease != null && !lease.isFreed()) { return false; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 7628b261ff908..e30387fde1f85 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -93,10 +93,6 @@ public final class ConfigurationKeys { /** Provides a number of threads to use for lease operations for single writer directories. * Must be set to a minimum of 1 if single writer directories are to be used. Default is 0. **/ public static final String FS_AZURE_LEASE_THREADS = "fs.azure.lease.threads"; - /** Provides a lease duration in seconds. Must be between 15 and 60 seconds or -1 for infinite - * lease duration. For finite durations, leases will be renewed automatically for single - * writer directories. **/ - public static final String FS_AZURE_LEASE_DURATION = "fs.azure.lease.duration"; public static final String FS_AZURE_READ_AHEAD_QUEUE_DEPTH = "fs.azure.readaheadqueue.depth"; public static final String FS_AZURE_ALWAYS_READ_BUFFER_SIZE = "fs.azure.read.alwaysReadBufferSize"; public static final String FS_AZURE_READ_AHEAD_BLOCK_SIZE = "fs.azure.read.readahead.blocksize"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 612bafdc9432a..5672e1d849237 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -81,7 +81,7 @@ public final class FileSystemConfigurations { public static final String DEFAULT_FS_AZURE_SINGLE_WRITER_DIRECTORIES = ""; public static final int DEFAULT_LEASE_THREADS = 0; public static final int MIN_LEASE_THREADS = 0; - public static final int DEFAULT_LEASE_DURATION = 60; + public static final int DEFAULT_LEASE_DURATION = -1; public static final int INFINITE_LEASE_DURATION = -1; public static final int MIN_LEASE_DURATION = 15; public static final int MAX_LEASE_DURATION = 60; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java similarity index 65% rename from hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java rename to hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java index 3ad45719fb31f..15bd2b43fc3bd 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SelfRenewingLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java @@ -19,8 +19,6 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; -import java.net.HttpURLConnection; -import java.util.concurrent.CancellationException; import java.util.concurrent.TimeUnit; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; @@ -30,7 +28,6 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryPolicy; @@ -41,28 +38,24 @@ import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_NO_LEASE_THREADS; /** - * An Azure blob lease that automatically renews itself indefinitely by scheduling lease - * operations through the ABFS client. Use it to prevent writes to the blob by other processes - * that don't have the lease. + * AbfsLease manages an Azure blob lease. It acquires an infinite lease on instantiation and + * releases the lease when free() is called. Use it to prevent writes to the blob by other + * processes that don't have the lease. * * Creating a new Lease object blocks the caller until the Azure blob lease is acquired. It will * retry a fixed number of times before failing if there is a problem acquiring the lease. * - * Call free() to release the Lease. If the holder process dies, the lease will time out since it - * won't be renewed. + * Call free() to release the Lease. If the holder process dies, AzureBlobFileSystem breakLease + * will need to be called before another client will be able to write to the file. */ -public final class SelfRenewingLease { - private static final Logger LOG = LoggerFactory.getLogger(SelfRenewingLease.class); - - static final float LEASE_RENEWAL_PERCENT_OF_DURATION = 0.67f; // Lease renewal percent of duration +public final class AbfsLease { + private static final Logger LOG = LoggerFactory.getLogger(AbfsLease.class); static final int LEASE_ACQUIRE_RETRY_INTERVAL = 10; // Retry interval for acquiring lease in secs static final int LEASE_ACQUIRE_MAX_RETRIES = 7; // Number of retries for acquiring lease private final AbfsClient client; private final String path; - private final int duration; - private final int renewalPeriod; // Lease status variables private volatile boolean leaseFreed; @@ -80,12 +73,10 @@ public LeaseException(String s) { } } - public SelfRenewingLease(AbfsClient client, String path, int duration) throws AzureBlobFileSystemException { + public AbfsLease(AbfsClient client, String path) throws AzureBlobFileSystemException { this.leaseFreed = false; this.client = client; this.path = path; - this.duration = duration; - this.renewalPeriod = (int) (LEASE_RENEWAL_PERCENT_OF_DURATION * this.duration); if (client.getNumLeaseThreads() < 1) { throw new LeaseException(ERR_NO_LEASE_THREADS); @@ -103,10 +94,6 @@ public SelfRenewingLease(AbfsClient client, String path, int duration) throws Az throw new LeaseException(exception); } - if (duration != INFINITE_LEASE_DURATION) { - renewLease(renewalPeriod); - } - LOG.debug("Acquired lease {} on {}", leaseID, path); } @@ -116,7 +103,7 @@ private void acquireLease(RetryPolicy retryPolicy, int numRetries, long delay) if (future != null && !future.isDone()) { throw new LeaseException(ERR_LEASE_FUTURE_EXISTS); } - future = client.schedule(() -> client.acquireLease(path, duration), + future = client.schedule(() -> client.acquireLease(path, INFINITE_LEASE_DURATION), delay, TimeUnit.SECONDS); client.addCallback(future, new FutureCallback() { @Override @@ -142,50 +129,10 @@ public void onFailure(Throwable throwable) { }); } - private void renewLease(long delay) { - LOG.debug("Attempting to renew lease on {}, renew lease id {}, delay {}", path, leaseID, delay); - if (future != null && !future.isDone()) { - LOG.warn("Unexpected new lease renewal operation occurred while operation already existed. " - + "Not initiating new renewal"); - return; - } - future = client.schedule(() -> client.renewLease(path, leaseID), delay, - TimeUnit.SECONDS); - client.addCallback(future, new FutureCallback() { - @Override - public void onSuccess(@Nullable AbfsRestOperation op) { - LOG.debug("Renewed lease {} on {}", leaseID, path); - renewLease(delay); - } - - @Override - public void onFailure(Throwable throwable) { - if (throwable instanceof CancellationException) { - LOG.info("Stopping renewal due to cancellation"); - free(); - return; - } else if (throwable instanceof AbfsRestOperationException) { - AbfsRestOperationException opEx = ((AbfsRestOperationException) throwable); - if (opEx.getStatusCode() < HttpURLConnection.HTTP_INTERNAL_ERROR) { - // error in 400 range indicates a type of error that should not result in a retry - // such as the lease being broken or a different lease being present - LOG.info("Stopping renewal due to {}: {}, {}", opEx.getStatusCode(), - opEx.getErrorCode(), opEx.getErrorMessage()); - free(); - return; - } - } - - LOG.debug("Failed to renew lease on {}, renew lease id {}, retrying: {}", path, leaseID, - throwable); - renewLease(0); - } - }); - } - /** - * Cancel renewal and free the lease. If an exception occurs, this method assumes the lease - * will expire after the lease duration. + * Cancel future and free the lease. If an exception occurs while releasing the lease, the error + * will be logged. If the lease cannot be released, AzureBlobFileSystem breakLease will need to + * be called before another client will be able to write to the file. */ public void free() { if (leaseFreed) { @@ -198,7 +145,7 @@ public void free() { } client.releaseLease(path, leaseID); } catch (IOException e) { - LOG.info("Exception when trying to release lease {} on {}. Lease will be left to expire: {}", + LOG.info("Exception when trying to release lease {} on {}. Lease will need to be broken: {}", leaseID, path, e.getMessage()); } finally { // Even if releasing the lease fails (e.g. because the file was deleted), diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 1ca92404b51ab..1cd83759b6b46 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -94,7 +94,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable, // SAS tokens can be re-used until they expire private CachedSASToken cachedSasToken; - private SelfRenewingLease lease; + private AbfsLease lease; private String leaseId; /** diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java index d8a44e80d640e..48f6f54081053 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java @@ -39,7 +39,7 @@ public class AbfsOutputStreamContext extends AbfsStreamContext { private int maxWriteRequestsToQueue; - private SelfRenewingLease lease; + private AbfsLease lease; public AbfsOutputStreamContext(final long sasTokenRenewPeriodForStreamsInSeconds) { super(sasTokenRenewPeriodForStreamsInSeconds); @@ -96,7 +96,7 @@ public AbfsOutputStreamContext withMaxWriteRequestsToQueue( return this; } - public AbfsOutputStreamContext withLease(final SelfRenewingLease lease) { + public AbfsOutputStreamContext withLease(final AbfsLease lease) { this.lease = lease; return this; } @@ -133,7 +133,7 @@ public boolean isEnableSmallWriteOptimization() { return this.enableSmallWriteOptimization; } - public SelfRenewingLease getLease() { + public AbfsLease getLease() { return this.lease; } diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index e33a48427c520..2f94236931b9b 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -893,10 +893,11 @@ can be specified comma separated in this config. By default, multiple clients will be able to write to the same file simultaneously. When writing to files contained within the directories specified in this config, the client will obtain a lease on the file that will prevent any other clients -from writing to the file. The lease will be renewed by the client until the -output stream is closed, after which it will be released. To revoke a client's -write access for a file, the AzureBlobFilesystem breakLease method may be - called. +from writing to the file. When the output stream is closed, the lease will be +released. To revoke a client's write access for a file, the +AzureBlobFilesystem breakLease method may be called. If the client dies +before the file can be closed and the lease released, breakLease will need to +be called before another client will be able to write to the file. `fs.azure.lease.threads`: This is the size of the thread pool that will be used for lease operations for single writer directories. By default the value diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index 745b18dd3cddc..062a5d69d08e0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -30,10 +30,8 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_DURATION; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SINGLE_WRITER_KEY; -import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.INFINITE_LEASE_DURATION; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_ACQUIRING_LEASE; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_ALREADY_PRESENT; @@ -63,13 +61,11 @@ public ITestAzureBlobFileSystemLease() throws Exception { .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false); } - private AzureBlobFileSystem getCustomFileSystem(Path singleWriterDirs, int numLeaseThreads, - int leaseDuration) throws Exception { + private AzureBlobFileSystem getCustomFileSystem(Path singleWriterDirs, int numLeaseThreads) throws Exception { Configuration conf = getRawConfiguration(); conf.setBoolean(String.format("fs.%s.impl.disable.cache", getAbfsScheme()), true); conf.set(FS_AZURE_SINGLE_WRITER_KEY, singleWriterDirs.toUri().getPath()); conf.setInt(FS_AZURE_LEASE_THREADS, numLeaseThreads); - conf.setInt(FS_AZURE_LEASE_DURATION, leaseDuration); return getFileSystem(conf); } @@ -88,8 +84,7 @@ public void testNoSingleWriter() throws IOException { @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testNoLeaseThreads() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 0, - TEST_LEASE_DURATION); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 0); fs.mkdirs(testFilePath.getParent()); LambdaTestUtils.intercept(IOException.class, ERR_NO_LEASE_THREADS, () -> { try (FSDataOutputStream out = fs.create(testFilePath)) { @@ -101,8 +96,7 @@ public void testNoLeaseThreads() throws Exception { @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testOneWriter() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, - TEST_LEASE_DURATION); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); fs.mkdirs(testFilePath.getParent()); FSDataOutputStream out = fs.create(testFilePath); @@ -119,8 +113,7 @@ public void testSubDir() throws Exception { final Path testFilePath = new Path(new Path(path(methodName.getMethodName()), "subdir"), TEST_FILE); final AzureBlobFileSystem fs = - getCustomFileSystem(testFilePath.getParent().getParent(), 1, - TEST_LEASE_DURATION); + getCustomFileSystem(testFilePath.getParent().getParent(), 1); fs.mkdirs(testFilePath.getParent().getParent()); FSDataOutputStream out = fs.create(testFilePath); @@ -135,8 +128,7 @@ public void testSubDir() throws Exception { @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testTwoCreate() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, - TEST_LEASE_DURATION); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); fs.mkdirs(testFilePath.getParent()); try (FSDataOutputStream out = fs.create(testFilePath)) { @@ -181,8 +173,7 @@ public void testTwoWritersCreateAppendNoSingleWriter() throws Exception { @Test(timeout = LONG_TEST_EXECUTION_TIMEOUT) public void testTwoWritersCreateAppendWithSingleWriterEnabled() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, - TEST_LEASE_DURATION); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); fs.mkdirs(testFilePath.getParent()); twoWriters(fs, testFilePath, true); @@ -191,8 +182,7 @@ public void testTwoWritersCreateAppendWithSingleWriterEnabled() throws Exception @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testLeaseFreedOnClose() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, - TEST_LEASE_DURATION); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); fs.mkdirs(testFilePath.getParent()); FSDataOutputStream out; @@ -209,8 +199,7 @@ public void testLeaseFreedOnClose() throws Exception { @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testWriteAfterBreakLease() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, - TEST_LEASE_DURATION); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); fs.mkdirs(testFilePath.getParent()); FSDataOutputStream out; @@ -245,8 +234,7 @@ public void testWriteAfterBreakLease() throws Exception { @Test(timeout = LONG_TEST_EXECUTION_TIMEOUT) public void testLeaseFreedAfterBreak() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, - TEST_LEASE_DURATION); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); fs.mkdirs(testFilePath.getParent()); FSDataOutputStream out = fs.create(testFilePath); @@ -254,27 +242,21 @@ public void testLeaseFreedAfterBreak() throws Exception { fs.breakLease(testFilePath); - while (!((AbfsOutputStream) out.getWrappedStream()).isLeaseFreed()) { - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - } - } - - LambdaTestUtils.intercept(IOException.class, isHNSEnabled ? ERR_LEASE_NOT_PRESENT : - ERR_LEASE_EXPIRED, () -> { + LambdaTestUtils.intercept(IOException.class, ERR_LEASE_EXPIRED, () -> { out.close(); return "Expected exception on close after lease break but got " + out; }); + Assert.assertTrue("Output stream lease should be freed", + ((AbfsOutputStream) out.getWrappedStream()).isLeaseFreed()); + Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); } @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testInfiniteLease() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, - INFINITE_LEASE_DURATION); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); fs.mkdirs(testFilePath.getParent()); try (FSDataOutputStream out = fs.create(testFilePath)) { @@ -295,8 +277,7 @@ public void testInfiniteLease() throws Exception { @Test(timeout = TEST_EXECUTION_TIMEOUT) public void testFileSystemClose() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); - final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1, - TEST_LEASE_DURATION); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); fs.mkdirs(testFilePath.getParent()); FSDataOutputStream out = fs.create(testFilePath); From eca41b4d4ad9aab9ea5caf0df348ddaa824797bc Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Tue, 16 Mar 2021 14:07:56 -0400 Subject: [PATCH 14/18] HADOOP-16948. Remove acquire/renew/release lease methods --- .../fs/azurebfs/AzureBlobFileSystem.java | 70 ------------------- .../fs/azurebfs/AzureBlobFileSystemStore.java | 43 ------------ .../ITestAzureBlobFileSystemLease.java | 41 ----------- 3 files changed, 154 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index d99c8dc9abc6d..30108ed1e2fb3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -506,76 +506,6 @@ public FileStatus getFileStatus(final Path f) throws IOException { } } - /** - * Acquire a lease on an ABFS file for a specified duration. This requires the file to exist. - * Writes will not be able to be performed while the lease is held because there is not - * currently a way to provide a previously acquired lease ID to AbfsOutputStream. - * - * If the file is in a single writer directory, the client will automatically acquire, renew, - * and release a lease and this method does not need to be used. The output stream holding the - * lease will be able to write to the file in that case. - * - * @param f file name - * @param duration time lease will be held before expiring - * @return the acquired lease ID - * @throws IOException on any exception while acquiring the lease - */ - public String acquireLease(final Path f, final int duration) throws IOException { - LOG.debug("AzureBlobFileSystem.acquireLease path: {}", f); - - Path qualifiedPath = makeQualified(f); - - try (DurationInfo ignored = new DurationInfo(LOG, false, "Acquire lease for %s", - qualifiedPath)) { - return abfsStore.acquireLease(qualifiedPath, duration); - } catch(AzureBlobFileSystemException ex) { - checkException(f, ex); - return null; - } - } - - /** - * Renew an existing lease on an ABFS file. If a lease expires without being renewed, a - * different lease may be acquired. A lease can still be renewed after expiring if a different - * lease has not been acquired. - * - * @param f file name - * @param leaseId lease ID to renew - * @throws IOException on any exception while renewing the lease - */ - public void renewLease(final Path f, final String leaseId) throws IOException { - LOG.debug("AzureBlobFileSystem.renewLease path: {} id: {}", f, leaseId); - - Path qualifiedPath = makeQualified(f); - - try (DurationInfo ignored = new DurationInfo(LOG, false, "Renew lease for %s", - qualifiedPath)) { - abfsStore.renewLease(qualifiedPath, leaseId); - } catch(AzureBlobFileSystemException ex) { - checkException(f, ex); - } - } - - /** - * Release an existing lease on an ABFS file. - * - * @param f file name - * @param leaseId lease ID to release - * @throws IOException on any exception while releasing the lease - */ - public void releaseLease(final Path f, final String leaseId) throws IOException { - LOG.debug("AzureBlobFileSystem.releaseLease path: {} id: {}", f, leaseId); - - Path qualifiedPath = makeQualified(f); - - try (DurationInfo ignored = new DurationInfo(LOG, false, "Release lease for %s", - qualifiedPath)) { - abfsStore.releaseLease(qualifiedPath, leaseId); - } catch(AzureBlobFileSystemException ex) { - checkException(f, ex); - } - } - /** * Break the current lease on an ABFS file if it exists. A lease that is broken cannot be * renewed. A new lease may be obtained on the file immediately. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index e154aed8f514a..4b39acdce850b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -749,49 +749,6 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic } } - /** - * Acquire a lease on an ABFS file for a specified duration. This requires the file to exist. - * - * @param path file name - * @param duration time lease will be held before expiring - * @return the acquired lease ID - * @throws AzureBlobFileSystemException on any exception while acquiring the lease - */ - public String acquireLease(final Path path, final int duration) throws AzureBlobFileSystemException { - LOG.debug("lease path: {}", path); - - final AbfsRestOperation op = - client.acquireLease(getRelativePath(path), duration); - - return op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID); - } - - /** - * Renew an existing lease on an ABFS file. - * - * @param path file name - * @param leaseId lease ID to renew - * @throws AzureBlobFileSystemException on any exception while renewing the lease - */ - public void renewLease(final Path path, final String leaseId) throws AzureBlobFileSystemException { - LOG.debug("lease path: {}, renew lease id: {}", path, leaseId); - - client.renewLease(getRelativePath(path), leaseId); - } - - /** - * Release an existing lease on an ABFS file. - * - * @param path file name - * @param leaseId lease ID to release - * @throws AzureBlobFileSystemException on any exception while releasing the lease - */ - public void releaseLease(final Path path, final String leaseId) throws AzureBlobFileSystemException { - LOG.debug("lease path: {}, release lease id: {}", path, leaseId); - - client.releaseLease(getRelativePath(path), leaseId); - } - /** * Break any current lease on an ABFS file. * diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index 062a5d69d08e0..942d361b6ffbd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -298,45 +298,4 @@ public void testFileSystemClose() throws Exception { return "Expected exception on new append after closed FS"; }); } - - @Test(timeout = TEST_EXECUTION_TIMEOUT) - public void testFileSystemLeaseOps() throws Exception { - final Path testDir = path(methodName.getMethodName()); - final AzureBlobFileSystem fs = getFileSystem(); - fs.mkdirs(testDir); - - LambdaTestUtils.intercept(IOException.class, ERR_LEASE_ID_NOT_PRESENT, () -> { - fs.breakLease(testDir); - return "Expected exception on break lease when no lease is held"; - }); - - String leaseId = fs.acquireLease(testDir, TEST_LEASE_DURATION); - LambdaTestUtils.intercept(IOException.class, ERR_LEASE_ALREADY_PRESENT, () -> { - fs.acquireLease(testDir, TEST_LEASE_DURATION); - return "Expected exception on acquire lease when lease is already held"; - }); - fs.renewLease(testDir, leaseId); - fs.releaseLease(testDir, leaseId); - LambdaTestUtils.intercept(IOException.class, isHNSEnabled ? ERR_LEASE_NOT_PRESENT : - ERR_LEASE_DID_NOT_MATCH, () -> { - fs.renewLease(testDir, leaseId); - return "Expected exception on renew lease after lease has been released"; - }); - - String leaseId2 = fs.acquireLease(testDir, TEST_LEASE_DURATION); - LambdaTestUtils.intercept(IOException.class, ERR_LEASE_DID_NOT_MATCH, () -> { - fs.renewLease(testDir, leaseId); - return "Expected exception on renew wrong lease ID"; - }); - LambdaTestUtils.intercept(IOException.class, ERR_LEASE_DID_NOT_MATCH, () -> { - fs.releaseLease(testDir, leaseId); - return "Expected exception on release wrong lease ID"; - }); - fs.breakLease(testDir); - LambdaTestUtils.intercept(IOException.class, ERR_LEASE_BROKEN, () -> { - fs.renewLease(testDir, leaseId2); - return "Expected exception on renewing broken lease"; - }); - fs.releaseLease(testDir, leaseId2); - } } From 822615efbf93f3c4056ae62284c631f4bc46554c Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Tue, 16 Mar 2021 14:19:46 -0400 Subject: [PATCH 15/18] HADOOP-16948. Rename single writer dirs to infinite lease dirs --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 10 ++++---- .../fs/azurebfs/AzureBlobFileSystemStore.java | 24 +++++++++---------- .../azurebfs/constants/ConfigurationKeys.java | 13 ++++++---- .../constants/FileSystemConfigurations.java | 2 +- .../fs/azurebfs/services/AbfsLease.java | 2 +- .../azurebfs/services/AbfsOutputStream.java | 2 +- .../hadoop-azure/src/site/markdown/abfs.md | 8 +++---- .../ITestAzureBlobFileSystemLease.java | 19 ++++++--------- 8 files changed, 39 insertions(+), 41 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 7b72819696c2e..0a8224aaaeb58 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -209,9 +209,9 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_FS_AZURE_APPEND_BLOB_DIRECTORIES) private String azureAppendBlobDirs; - @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_SINGLE_WRITER_KEY, - DefaultValue = DEFAULT_FS_AZURE_SINGLE_WRITER_DIRECTORIES) - private String azureSingleWriterDirs; + @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_INFINITE_LEASE_KEY, + DefaultValue = DEFAULT_FS_AZURE_INFINITE_LEASE_DIRECTORIES) + private String azureInfiniteLeaseDirs; @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_LEASE_THREADS, MinValue = MIN_LEASE_THREADS, @@ -646,8 +646,8 @@ public String getAppendBlobDirs() { return this.azureAppendBlobDirs; } - public String getAzureSingleWriterDirs() { - return this.azureSingleWriterDirs; + public String getAzureInfiniteLeaseDirs() { + return this.azureInfiniteLeaseDirs; } public int getNumLeaseThreads() { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 4b39acdce850b..fa7e12bc80e28 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -155,7 +155,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport { private final AbfsConfiguration abfsConfiguration; private final Set azureAtomicRenameDirSet; - private Set azureSingleWriterDirSet; + private Set azureInfiniteLeaseDirSet; private Trilean isNamespaceEnabled; private final AuthType authType; private final UserGroupInformation userGroupInformation; @@ -206,7 +206,7 @@ public AzureBlobFileSystemStore(URI uri, boolean isSecureScheme, this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList( abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA))); - updateSingleWriterDirs(); + updateInfiniteLeaseDirs(); this.authType = abfsConfiguration.getAuthType(accountName); boolean usingOauth = (authType == AuthType.OAuth); boolean useHttps = (usingOauth || abfsConfiguration.isHttpsAlwaysUsed()) ? true : isSecureScheme; @@ -1394,11 +1394,11 @@ public boolean isAtomicRenameKey(String key) { return isKeyForDirectorySet(key, azureAtomicRenameDirSet); } - public boolean isSingleWriterKey(String key) { - if (azureSingleWriterDirSet.isEmpty()) { + public boolean isInfiniteLeaseKey(String key) { + if (azureInfiniteLeaseDirSet.isEmpty()) { return false; } - return isKeyForDirectorySet(key, azureSingleWriterDirSet); + return isKeyForDirectorySet(key, azureInfiniteLeaseDirSet); } /** @@ -1690,18 +1690,18 @@ void setNamespaceEnabled(Trilean isNamespaceEnabled){ this.isNamespaceEnabled = isNamespaceEnabled; } - private void updateSingleWriterDirs() { - this.azureSingleWriterDirSet = new HashSet<>(Arrays.asList( - abfsConfiguration.getAzureSingleWriterDirs().split(AbfsHttpConstants.COMMA))); + private void updateInfiniteLeaseDirs() { + this.azureInfiniteLeaseDirSet = new HashSet<>(Arrays.asList( + abfsConfiguration.getAzureInfiniteLeaseDirs().split(AbfsHttpConstants.COMMA))); // remove the empty string, since isKeyForDirectory returns true for empty strings - // and we don't want to default to enabling single writer dirs - this.azureSingleWriterDirSet.remove(""); + // and we don't want to default to enabling infinite lease dirs + this.azureInfiniteLeaseDirSet.remove(""); } private AbfsLease maybeCreateLease(String relativePath) throws AzureBlobFileSystemException { - boolean enableSingleWriter = isSingleWriterKey(relativePath); - if (!enableSingleWriter) { + boolean enableInfiniteLease = isInfiniteLeaseKey(relativePath); + if (!enableInfiniteLease) { return null; } AbfsLease lease = new AbfsLease(client, relativePath); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index e30387fde1f85..4fe1d1c276db5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -87,11 +87,14 @@ public final class ConfigurationKeys { /** Provides a config to provide comma separated path prefixes on which Appendblob based files are created * Default is empty. **/ public static final String FS_AZURE_APPEND_BLOB_KEY = "fs.azure.appendblob.directories"; - /** Provides a config to provide comma separated path prefixes which support single writers. - * Default is empty. **/ - public static final String FS_AZURE_SINGLE_WRITER_KEY = "fs.azure.singlewriter.directories"; - /** Provides a number of threads to use for lease operations for single writer directories. - * Must be set to a minimum of 1 if single writer directories are to be used. Default is 0. **/ + /** Provides a config to provide comma separated path prefixes which support infinite leases. + * Files under these paths will be leased when created or opened for writing and the lease will + * be released when the file is closed. The lease may be broken with the breakLease method on + * AzureBlobFileSystem. Default is empty. + * **/ + public static final String FS_AZURE_INFINITE_LEASE_KEY = "fs.azure.infinite-lease.directories"; + /** Provides a number of threads to use for lease operations for infinite lease directories. + * Must be set to a minimum of 1 if infinite lease directories are to be used. Default is 0. **/ public static final String FS_AZURE_LEASE_THREADS = "fs.azure.lease.threads"; public static final String FS_AZURE_READ_AHEAD_QUEUE_DEPTH = "fs.azure.readaheadqueue.depth"; public static final String FS_AZURE_ALWAYS_READ_BUFFER_SIZE = "fs.azure.read.alwaysReadBufferSize"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 5672e1d849237..040b18ae4c281 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -78,7 +78,7 @@ public final class FileSystemConfigurations { public static final boolean DEFAULT_FS_AZURE_ENABLE_CONDITIONAL_CREATE_OVERWRITE = true; public static final boolean DEFAULT_FS_AZURE_ENABLE_MKDIR_OVERWRITE = true; public static final String DEFAULT_FS_AZURE_APPEND_BLOB_DIRECTORIES = ""; - public static final String DEFAULT_FS_AZURE_SINGLE_WRITER_DIRECTORIES = ""; + public static final String DEFAULT_FS_AZURE_INFINITE_LEASE_DIRECTORIES = ""; public static final int DEFAULT_LEASE_THREADS = 0; public static final int MIN_LEASE_THREADS = 0; public static final int DEFAULT_LEASE_DURATION = -1; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java index 15bd2b43fc3bd..814879332d2cf 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java @@ -145,7 +145,7 @@ public void free() { } client.releaseLease(path, leaseID); } catch (IOException e) { - LOG.info("Exception when trying to release lease {} on {}. Lease will need to be broken: {}", + LOG.warn("Exception when trying to release lease {} on {}. Lease will need to be broken: {}", leaseID, path, e.getMessage()); } finally { // Even if releasing the lease fails (e.g. because the file was deleted), diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 1cd83759b6b46..80b35ee4d3a91 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -118,7 +118,7 @@ public AbfsOutputStream( final Statistics statistics, final String path, final long position, - AbfsOutputStreamContext abfsOutputStreamContext) throws AzureBlobFileSystemException { + AbfsOutputStreamContext abfsOutputStreamContext) { this.client = client; this.statistics = statistics; this.path = path; diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index 2f94236931b9b..6be5952b03aa6 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -887,8 +887,8 @@ enabled for your Azure Storage account." The directories can be specified as comma separated values. By default the value is "/hbase" -### Single Writer Options -`fs.azure.singlewriter.directories`: Directories for single writer support +### Infinite Lease Options +`fs.azure.infinite-lease.directories`: Directories for infinite lease support can be specified comma separated in this config. By default, multiple clients will be able to write to the same file simultaneously. When writing to files contained within the directories specified in this config, the @@ -900,8 +900,8 @@ before the file can be closed and the lease released, breakLease will need to be called before another client will be able to write to the file. `fs.azure.lease.threads`: This is the size of the thread pool that will be -used for lease operations for single writer directories. By default the value -is 0, so it must be set to at least 1 to support single writer directories. +used for lease operations for infinite lease directories. By default the value +is 0, so it must be set to at least 1 to support infinite lease directories. ### Perf Options diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index 942d361b6ffbd..d7f27ccb26c9f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -30,15 +30,11 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_INFINITE_LEASE_KEY; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SINGLE_WRITER_KEY; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_ACQUIRING_LEASE; -import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_ALREADY_PRESENT; -import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_BROKEN; -import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_DID_NOT_MATCH; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_EXPIRED; -import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_ID_NOT_PRESENT; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_NOT_PRESENT; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_NO_LEASE_ID_SPECIFIED; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_NO_LEASE_THREADS; @@ -50,7 +46,6 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest { private static final int TEST_EXECUTION_TIMEOUT = 30 * 1000; private static final int LONG_TEST_EXECUTION_TIMEOUT = 90 * 1000; - private static final int TEST_LEASE_DURATION = 15; private static final String TEST_FILE = "testfile"; private final boolean isHNSEnabled; @@ -61,16 +56,16 @@ public ITestAzureBlobFileSystemLease() throws Exception { .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false); } - private AzureBlobFileSystem getCustomFileSystem(Path singleWriterDirs, int numLeaseThreads) throws Exception { + private AzureBlobFileSystem getCustomFileSystem(Path infiniteLeaseDirs, int numLeaseThreads) throws Exception { Configuration conf = getRawConfiguration(); conf.setBoolean(String.format("fs.%s.impl.disable.cache", getAbfsScheme()), true); - conf.set(FS_AZURE_SINGLE_WRITER_KEY, singleWriterDirs.toUri().getPath()); + conf.set(FS_AZURE_INFINITE_LEASE_KEY, infiniteLeaseDirs.toUri().getPath()); conf.setInt(FS_AZURE_LEASE_THREADS, numLeaseThreads); return getFileSystem(conf); } @Test(timeout = TEST_EXECUTION_TIMEOUT) - public void testNoSingleWriter() throws IOException { + public void testNoInfiniteLease() throws IOException { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getFileSystem(); fs.mkdirs(testFilePath.getParent()); @@ -136,7 +131,7 @@ public void testTwoCreate() throws Exception { ERR_NO_LEASE_ID_SPECIFIED, () -> { try (FSDataOutputStream out2 = fs.create(testFilePath)) { } - return "Expected second create on single writer dir to fail"; + return "Expected second create on infinite lease dir to fail"; }); } Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); @@ -162,7 +157,7 @@ private void twoWriters(AzureBlobFileSystem fs, Path testFilePath, boolean expec } @Test(timeout = TEST_EXECUTION_TIMEOUT) - public void testTwoWritersCreateAppendNoSingleWriter() throws Exception { + public void testTwoWritersCreateAppendNoInfiniteLease() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getFileSystem(); fs.mkdirs(testFilePath.getParent()); @@ -171,7 +166,7 @@ public void testTwoWritersCreateAppendNoSingleWriter() throws Exception { } @Test(timeout = LONG_TEST_EXECUTION_TIMEOUT) - public void testTwoWritersCreateAppendWithSingleWriterEnabled() throws Exception { + public void testTwoWritersCreateAppendWithInfiniteLeaseEnabled() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); fs.mkdirs(testFilePath.getParent()); From 9fc4f08bbf794eab83ffaed74a79e0a4ff4aec22 Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Wed, 17 Mar 2021 00:44:31 -0400 Subject: [PATCH 16/18] HADOOP-16948. Fix checkstyle --- .../hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index d7f27ccb26c9f..a92f3bd7bfa59 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -127,8 +127,8 @@ public void testTwoCreate() throws Exception { fs.mkdirs(testFilePath.getParent()); try (FSDataOutputStream out = fs.create(testFilePath)) { - LambdaTestUtils.intercept(IOException.class, isHNSEnabled ? ERR_PARALLEL_ACCESS_DETECTED : - ERR_NO_LEASE_ID_SPECIFIED, () -> { + LambdaTestUtils.intercept(IOException.class, isHNSEnabled ? ERR_PARALLEL_ACCESS_DETECTED + : ERR_NO_LEASE_ID_SPECIFIED, () -> { try (FSDataOutputStream out2 = fs.create(testFilePath)) { } return "Expected second create on infinite lease dir to fail"; @@ -281,8 +281,8 @@ public void testFileSystemClose() throws Exception { fs.close(); Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); - LambdaTestUtils.intercept(IOException.class, isHNSEnabled ? ERR_LEASE_NOT_PRESENT : - ERR_LEASE_EXPIRED, () -> { + LambdaTestUtils.intercept(IOException.class, isHNSEnabled ? ERR_LEASE_NOT_PRESENT + : ERR_LEASE_EXPIRED, () -> { out.close(); return "Expected exception on close after closed FS but got " + out; }); From b6803cfa25040895401bc7b6fdeccc2a96e2da7b Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Mon, 22 Mar 2021 10:26:15 -0400 Subject: [PATCH 17/18] HADOOP-16948. Wait for acquire lease future --- .../org/apache/hadoop/fs/azurebfs/services/AbfsLease.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java index 814879332d2cf..4185bcb5dcd40 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java @@ -88,6 +88,12 @@ public AbfsLease(AbfsClient client, String path) throws AzureBlobFileSystemExcep acquireLease(retryPolicy, 0, 0); while (leaseID == null && exception == null) { + try { + future.get(); + } catch (Exception e) { + LOG.debug("Got exception waiting for acquire lease future. Checking if lease ID or " + + "exception have been set", e); + } } if (exception != null) { LOG.error("Failed to acquire lease on {}", path); @@ -117,7 +123,7 @@ public void onFailure(Throwable throwable) { try { if (RetryPolicy.RetryAction.RetryDecision.RETRY == retryPolicy.shouldRetry(null, numRetries, 0, true).action) { - LOG.debug("Failed acquire lease on {}, retrying: {}", path, throwable); + LOG.debug("Failed to acquire lease on {}, retrying: {}", path, throwable); acquireLease(retryPolicy, numRetries + 1, LEASE_ACQUIRE_RETRY_INTERVAL); } else { exception = throwable; From 4fdfc08089b7a126a8c455ba663637f247ae4d29 Mon Sep 17 00:00:00 2001 From: Billie Rinaldi Date: Mon, 22 Mar 2021 22:01:24 -0400 Subject: [PATCH 18/18] HADOOP-16948. Add unit test for acquire lease failure --- .../fs/azurebfs/services/AbfsLease.java | 29 +++++++++++--- .../ITestAzureBlobFileSystemLease.java | 40 +++++++++++++++++++ 2 files changed, 63 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java index 4185bcb5dcd40..97a8b0228a5b3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java @@ -21,9 +21,11 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableScheduledFuture; import org.apache.hadoop.thirdparty.org.checkerframework.checker.nullness.qual.Nullable; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,8 +53,10 @@ public final class AbfsLease { private static final Logger LOG = LoggerFactory.getLogger(AbfsLease.class); - static final int LEASE_ACQUIRE_RETRY_INTERVAL = 10; // Retry interval for acquiring lease in secs - static final int LEASE_ACQUIRE_MAX_RETRIES = 7; // Number of retries for acquiring lease + // Number of retries for acquiring lease + static final int DEFAULT_LEASE_ACQUIRE_MAX_RETRIES = 7; + // Retry interval for acquiring lease in secs + static final int DEFAULT_LEASE_ACQUIRE_RETRY_INTERVAL = 10; private final AbfsClient client; private final String path; @@ -61,6 +65,7 @@ public final class AbfsLease { private volatile boolean leaseFreed; private volatile String leaseID = null; private volatile Throwable exception = null; + private volatile int acquireRetryCount = 0; private volatile ListenableScheduledFuture future = null; public static class LeaseException extends AzureBlobFileSystemException { @@ -74,6 +79,12 @@ public LeaseException(String s) { } public AbfsLease(AbfsClient client, String path) throws AzureBlobFileSystemException { + this(client, path, DEFAULT_LEASE_ACQUIRE_MAX_RETRIES, DEFAULT_LEASE_ACQUIRE_RETRY_INTERVAL); + } + + @VisibleForTesting + public AbfsLease(AbfsClient client, String path, int acquireMaxRetries, + int acquireRetryInterval) throws AzureBlobFileSystemException { this.leaseFreed = false; this.client = client; this.path = path; @@ -84,8 +95,8 @@ public AbfsLease(AbfsClient client, String path) throws AzureBlobFileSystemExcep // Try to get the lease a specified number of times, else throw an error RetryPolicy retryPolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep( - LEASE_ACQUIRE_MAX_RETRIES, LEASE_ACQUIRE_RETRY_INTERVAL, TimeUnit.SECONDS); - acquireLease(retryPolicy, 0, 0); + acquireMaxRetries, acquireRetryInterval, TimeUnit.SECONDS); + acquireLease(retryPolicy, 0, acquireRetryInterval, 0); while (leaseID == null && exception == null) { try { @@ -103,7 +114,7 @@ public AbfsLease(AbfsClient client, String path) throws AzureBlobFileSystemExcep LOG.debug("Acquired lease {} on {}", leaseID, path); } - private void acquireLease(RetryPolicy retryPolicy, int numRetries, long delay) + private void acquireLease(RetryPolicy retryPolicy, int numRetries, int retryInterval, long delay) throws LeaseException { LOG.debug("Attempting to acquire lease on {}, retry {}", path, numRetries); if (future != null && !future.isDone()) { @@ -124,7 +135,8 @@ public void onFailure(Throwable throwable) { if (RetryPolicy.RetryAction.RetryDecision.RETRY == retryPolicy.shouldRetry(null, numRetries, 0, true).action) { LOG.debug("Failed to acquire lease on {}, retrying: {}", path, throwable); - acquireLease(retryPolicy, numRetries + 1, LEASE_ACQUIRE_RETRY_INTERVAL); + acquireRetryCount++; + acquireLease(retryPolicy, numRetries + 1, retryInterval, retryInterval); } else { exception = throwable; } @@ -168,4 +180,9 @@ public boolean isFreed() { public String getLeaseID() { return leaseID; } + + @VisibleForTesting + public int getAcquireRetryCount() { + return acquireRetryCount; + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index a92f3bd7bfa59..9857da8957e22 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -26,10 +26,18 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsLease; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.spy; + import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_INFINITE_LEASE_KEY; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT; @@ -293,4 +301,36 @@ public void testFileSystemClose() throws Exception { return "Expected exception on new append after closed FS"; }); } + + @Test(timeout = TEST_EXECUTION_TIMEOUT) + public void testAcquireRetry() throws Exception { + final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); + fs.mkdirs(testFilePath.getParent()); + fs.createNewFile(testFilePath); + + AbfsLease lease = new AbfsLease(fs.getAbfsClient(), testFilePath.toUri().getPath()); + Assert.assertNotNull("Did not successfully lease file", lease.getLeaseID()); + lease.free(); + Assert.assertEquals("Unexpected acquire retry count", 0, lease.getAcquireRetryCount()); + + AbfsClient mockClient = spy(fs.getAbfsClient()); + + doThrow(new AbfsLease.LeaseException("failed to acquire 1")) + .doThrow(new AbfsLease.LeaseException("failed to acquire 2")) + .doCallRealMethod() + .when(mockClient).acquireLease(anyString(), anyInt()); + + lease = new AbfsLease(mockClient, testFilePath.toUri().getPath(), 5, 1); + Assert.assertNotNull("Acquire lease should have retried", lease.getLeaseID()); + lease.free(); + Assert.assertEquals("Unexpected acquire retry count", 2, lease.getAcquireRetryCount()); + + doThrow(new AbfsLease.LeaseException("failed to acquire")) + .when(mockClient).acquireLease(anyString(), anyInt()); + + LambdaTestUtils.intercept(AzureBlobFileSystemException.class, () -> { + new AbfsLease(mockClient, testFilePath.toUri().getPath(), 5, 1); + }); + } }