HADOOP-15560. ABFS: removed dependency injection and unnecessary dependencies.

Contributed by Da Zhou.
This commit is contained in:
Steve Loughran 2018-07-03 18:55:10 +02:00 committed by Thomas Marquardt
parent f044deedbb
commit a271fd0eca
33 changed files with 462 additions and 1733 deletions

View File

@ -149,12 +149,6 @@
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.threadly</groupId>
<artifactId>threadly</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
@ -185,18 +179,6 @@
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.htrace</groupId>
<artifactId>htrace-core</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>org.apache.htrace</groupId>
<artifactId>htrace-core4</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>com.google.inject</groupId>
<artifactId>guice</artifactId>

View File

@ -43,5 +43,5 @@
<suppressions>
<suppress checks="ParameterNumber|MagicNumber"
files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]services[\\/]AbfsHttpServiceImpl.java"/>
files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]AzureBlobFileSystemStore.java"/>
</suppressions>

View File

@ -39,10 +39,8 @@ import com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.threadly.util.ExceptionUtils;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
import org.apache.commons.lang.ArrayUtils;
import org.apache.hadoop.classification.InterfaceAudience;
@ -58,10 +56,6 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsServiceProvider;
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnhandledException;
@ -70,7 +64,6 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Progressable;
import org.apache.htrace.core.TraceScope;
/**
* A {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on <a
@ -85,10 +78,7 @@ public class AzureBlobFileSystem extends FileSystem {
private UserGroupInformation userGroupInformation;
private String user;
private String primaryUserGroup;
private AbfsServiceProvider abfsServiceProvider;
private TracingService tracingService;
private AbfsHttpService abfsHttpService;
private ConfigurationService configurationService;
private AzureBlobFileSystemStore abfsStore;
private boolean isClosed;
@Override
@ -96,18 +86,8 @@ public class AzureBlobFileSystem extends FileSystem {
throws IOException {
uri = ensureAuthority(uri, configuration);
super.initialize(uri, configuration);
setConf(configuration);
try {
this.abfsServiceProvider = AbfsServiceProviderImpl.create(configuration);
this.tracingService = abfsServiceProvider.get(TracingService.class);
this.abfsHttpService = abfsServiceProvider.get(AbfsHttpService.class);
this.configurationService = abfsServiceProvider.get(ConfigurationService.class);
} catch (AzureBlobFileSystemException exception) {
throw new IOException(exception);
}
this.LOG.debug(
"Initializing AzureBlobFileSystem for {}", uri);
@ -115,13 +95,14 @@ public class AzureBlobFileSystem extends FileSystem {
this.userGroupInformation = UserGroupInformation.getCurrentUser();
this.user = userGroupInformation.getUserName();
this.primaryUserGroup = userGroupInformation.getPrimaryGroupName();
this.abfsStore = new AzureBlobFileSystemStore(uri, this.isSecure(), configuration, userGroupInformation);
this.LOG.debug(
"Initializing NativeAzureFileSystem for {}", uri);
this.setWorkingDirectory(this.getHomeDirectory());
if (this.configurationService.getCreateRemoteFileSystemDuringInitialization()) {
if (abfsStore.getAbfsConfiguration().getCreateRemoteFileSystemDuringInitialization()) {
this.createFileSystem();
}
@ -143,7 +124,7 @@ public class AzureBlobFileSystem extends FileSystem {
"AzureBlobFileSystem.open path: {} bufferSize: {}", path.toString(), bufferSize);
try {
InputStream inputStream = abfsHttpService.openFileForRead(this, makeQualified(path), statistics);
InputStream inputStream = abfsStore.openFileForRead(makeQualified(path), statistics);
return new FSDataInputStream(inputStream);
} catch(AzureBlobFileSystemException ex) {
checkException(path, ex);
@ -162,7 +143,7 @@ public class AzureBlobFileSystem extends FileSystem {
blockSize);
try {
OutputStream outputStream = abfsHttpService.createFile(this, makeQualified(f), overwrite);
OutputStream outputStream = abfsStore.createFile(makeQualified(f), overwrite);
return new FSDataOutputStream(outputStream, statistics);
} catch(AzureBlobFileSystemException ex) {
checkException(f, ex);
@ -221,7 +202,7 @@ public class AzureBlobFileSystem extends FileSystem {
bufferSize);
try {
OutputStream outputStream = abfsHttpService.openFileForWrite(this, makeQualified(f), false);
OutputStream outputStream = abfsStore.openFileForWrite(makeQualified(f), false);
return new FSDataOutputStream(outputStream, statistics);
} catch(AzureBlobFileSystemException ex) {
checkException(f, ex);
@ -251,7 +232,7 @@ public class AzureBlobFileSystem extends FileSystem {
adjustedDst = new Path(dst, sourceFileName);
}
abfsHttpService.rename(this, makeQualified(src), makeQualified(adjustedDst));
abfsStore.rename(makeQualified(src), makeQualified(adjustedDst));
return true;
} catch(AzureBlobFileSystemException ex) {
checkException(
@ -281,7 +262,7 @@ public class AzureBlobFileSystem extends FileSystem {
}
try {
abfsHttpService.delete(this, makeQualified(f), recursive);
abfsStore.delete(makeQualified(f), recursive);
return true;
} catch (AzureBlobFileSystemException ex) {
checkException(f, ex, AzureServiceErrorCode.PATH_NOT_FOUND);
@ -296,7 +277,7 @@ public class AzureBlobFileSystem extends FileSystem {
"AzureBlobFileSystem.listStatus path: {}", f.toString());
try {
FileStatus[] result = abfsHttpService.listStatus(this, makeQualified(f));
FileStatus[] result = abfsStore.listStatus(makeQualified(f));
return result;
} catch (AzureBlobFileSystemException ex) {
checkException(f, ex);
@ -316,7 +297,7 @@ public class AzureBlobFileSystem extends FileSystem {
}
try {
abfsHttpService.createDirectory(this, makeQualified(f));
abfsStore.createDirectory(makeQualified(f));
return true;
} catch (AzureBlobFileSystemException ex) {
checkException(f, ex, AzureServiceErrorCode.PATH_ALREADY_EXISTS);
@ -332,13 +313,7 @@ public class AzureBlobFileSystem extends FileSystem {
super.close();
this.LOG.debug("AzureBlobFileSystem.close");
try {
abfsHttpService.closeFileSystem(this);
} catch (AzureBlobFileSystemException ex) {
checkException(null, ex);
this.isClosed = true;
}
this.isClosed = true;
}
@Override
@ -346,7 +321,7 @@ public class AzureBlobFileSystem extends FileSystem {
this.LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f.toString());
try {
return abfsHttpService.getFileStatus(this, makeQualified(f));
return abfsStore.getFileStatus(makeQualified(f));
} catch(AzureBlobFileSystemException ex) {
checkException(f, ex);
return null;
@ -397,7 +372,7 @@ public class AzureBlobFileSystem extends FileSystem {
if (file.getLen() < start) {
return new BlockLocation[0];
}
final String blobLocationHost = this.configurationService.getAzureBlockLocationHost();
final String blobLocationHost = this.abfsStore.getAbfsConfiguration().getAzureBlockLocationHost();
final String[] name = { blobLocationHost };
final String[] host = { blobLocationHost };
@ -477,12 +452,10 @@ public class AzureBlobFileSystem extends FileSystem {
this.LOG.debug(
"AzureBlobFileSystem.createFileSystem uri: {}", uri);
try {
abfsHttpService.createFilesystem(this);
this.abfsStore.createFilesystem();
} catch (AzureBlobFileSystemException ex) {
checkException(null, ex, AzureServiceErrorCode.FILE_SYSTEM_ALREADY_EXISTS);
}
}
private URI ensureAuthority(URI uri, final Configuration conf) {
@ -540,25 +513,19 @@ public class AzureBlobFileSystem extends FileSystem {
final Callable<T> callableFileOperation,
T defaultResultValue) throws IOException {
final TraceScope traceScope = tracingService.traceBegin(scopeDescription);
try {
final T executionResult = callableFileOperation.call();
return new FileSystemOperation(executionResult, null);
} catch (AbfsRestOperationException abfsRestOperationException) {
return new FileSystemOperation(defaultResultValue, abfsRestOperationException);
} catch (AzureBlobFileSystemException azureBlobFileSystemException) {
tracingService.traceException(traceScope, azureBlobFileSystemException);
throw new IOException(azureBlobFileSystemException);
} catch (Exception exception) {
if (exception instanceof ExecutionException) {
exception = (Exception) ExceptionUtils.getRootCause(exception);
exception = (Exception) getRootCause(exception);
}
final FileSystemOperationUnhandledException fileSystemOperationUnhandledException = new FileSystemOperationUnhandledException(exception);
tracingService.traceException(traceScope, fileSystemOperationUnhandledException);
throw new IOException(fileSystemOperationUnhandledException);
} finally {
tracingService.traceEnd(traceScope);
}
}
@ -590,6 +557,26 @@ public class AzureBlobFileSystem extends FileSystem {
}
}
/**
* Gets the root cause of a provided {@link Throwable}. If there is no cause for the
* {@link Throwable} provided into this function, the original {@link Throwable} is returned.
*
* @param throwable starting {@link Throwable}
* @return root cause {@link Throwable}
*/
private Throwable getRootCause(Throwable throwable) {
if (throwable == null) {
throw new IllegalArgumentException("throwable can not be null");
}
Throwable result = throwable;
while (result.getCause() != null) {
result = result.getCause();
}
return result;
}
@VisibleForTesting
FileSystem.Statistics getFsStatistics() {
return this.statistics;
@ -609,4 +596,9 @@ public class AzureBlobFileSystem extends FileSystem {
return this.exception != null;
}
}
@VisibleForTesting
AzureBlobFileSystemStore getAbfsStore() {
return this.abfsStore;
}
}

View File

@ -15,15 +15,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.azurebfs;
package org.apache.hadoop.fs.azurebfs.services;
import javax.xml.bind.DatatypeConverter;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.MalformedURLException;
import java.net.URI;
import java.net.URISyntaxException;
import java.net.URL;
import java.nio.ByteBuffer;
import java.nio.CharBuffer;
import java.nio.charset.CharacterCodingException;
@ -32,89 +33,110 @@ import java.nio.charset.CharsetDecoder;
import java.nio.charset.CharsetEncoder;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Set;
import java.util.HashSet;
import java.util.Hashtable;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.Set;
import javax.xml.bind.DatatypeConverter;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.inject.Inject;
import com.google.inject.Singleton;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
import org.joda.time.DateTime;
import org.joda.time.format.DateTimeFormat;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TimeoutException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidFileSystemPropertyException;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
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.fs.azurebfs.contracts.exceptions.FileSystemOperationUnhandledException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidFileSystemPropertyException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TimeoutException;
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
import org.apache.hadoop.fs.azurebfs.services.AbfsConfiguration;
import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy;
import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.http.client.utils.URIBuilder;
import org.joda.time.DateTime;
import org.joda.time.format.DateTimeFormat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.hadoop.util.Time.now;
@Singleton
@InterfaceAudience.Private
/**
* Provides the bridging logic between Hadoop's abstract filesystem and Azure Storage
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
final class AbfsHttpServiceImpl implements AbfsHttpService {
public static final Logger LOG = LoggerFactory.getLogger(AbfsHttpService.class);
public class AzureBlobFileSystemStore {
private static final Logger LOG = LoggerFactory.getLogger(AzureBlobFileSystemStore.class);
private AbfsClient client;
private URI uri;
private final UserGroupInformation userGroupInformation;
private static final String DATE_TIME_PATTERN = "E, dd MMM yyyy HH:mm:ss 'GMT'";
private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
private static final int LIST_MAX_RESULTS = 5000;
private static final int DELETE_DIRECTORY_TIMEOUT_MILISECONDS = 180000;
private static final int RENAME_TIMEOUT_MILISECONDS = 180000;
private final AbfsHttpClientFactory abfsHttpClientFactory;
private final ConcurrentHashMap<AzureBlobFileSystem, AbfsClient> clientCache;
private final ConfigurationService configurationService;
private final AbfsConfiguration abfsConfiguration;
private final Set<String> azureAtomicRenameDirSet;
@Inject
AbfsHttpServiceImpl(
final ConfigurationService configurationService,
final AbfsHttpClientFactory abfsHttpClientFactory,
final TracingService tracingService) {
Preconditions.checkNotNull(abfsHttpClientFactory, "abfsHttpClientFactory");
Preconditions.checkNotNull(configurationService, "configurationService");
Preconditions.checkNotNull(tracingService, "tracingService");
this.configurationService = configurationService;
this.clientCache = new ConcurrentHashMap<>();
this.abfsHttpClientFactory = abfsHttpClientFactory;
this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(configurationService.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA)));
public AzureBlobFileSystemStore(URI uri, boolean isSeure, Configuration configuration, UserGroupInformation userGroupInformation)
throws AzureBlobFileSystemException {
this.uri = uri;
try {
this.abfsConfiguration = new AbfsConfiguration(configuration);
} catch (IllegalAccessException exception) {
throw new FileSystemOperationUnhandledException(exception);
}
this.userGroupInformation = userGroupInformation;
this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA)));
initializeClient(uri, isSeure);
}
@Override
public Hashtable<String, String> getFilesystemProperties(final AzureBlobFileSystem azureBlobFileSystem)
throws AzureBlobFileSystemException{
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
@VisibleForTesting
URIBuilder getURIBuilder(final String hostName, boolean isSecure) {
String scheme = isSecure ? FileSystemUriSchemes.HTTPS_SCHEME : FileSystemUriSchemes.HTTP_SCHEME;
final URIBuilder uriBuilder = new URIBuilder();
uriBuilder.setScheme(scheme);
uriBuilder.setHost(hostName);
return uriBuilder;
}
public AbfsConfiguration getAbfsConfiguration() {
return this.abfsConfiguration;
}
public Hashtable<String, String> getFilesystemProperties() throws AzureBlobFileSystemException {
this.LOG.debug(
"getFilesystemProperties for filesystem: {}",
client.getFileSystem());
"getFilesystemProperties for filesystem: {}",
client.getFileSystem());
final Hashtable<String, String> parsedXmsProperties;
@ -126,19 +148,15 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
return parsedXmsProperties;
}
@Override
public void setFilesystemProperties(final AzureBlobFileSystem azureBlobFileSystem, final Hashtable<String, String> properties) throws
AzureBlobFileSystemException {
public void setFilesystemProperties(final Hashtable<String, String> properties) throws AzureBlobFileSystemException {
if (properties == null || properties.size() == 0) {
return;
}
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"setFilesystemProperties for filesystem: {} with properties: {}",
client.getFileSystem(),
properties);
"setFilesystemProperties for filesystem: {} with properties: {}",
client.getFileSystem(),
properties);
final String commaSeparatedProperties;
try {
@ -146,18 +164,15 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
} catch (CharacterCodingException ex) {
throw new InvalidAbfsRestOperationException(ex);
}
client.setFilesystemProperties(commaSeparatedProperties);
}
@Override
public Hashtable<String, String> getPathProperties(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws
AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
public Hashtable<String, String> getPathProperties(final Path path) throws AzureBlobFileSystemException {
this.LOG.debug(
"getPathProperties for filesystem: {} path: {}",
client.getFileSystem(),
path.toString());
"getPathProperties for filesystem: {} path: {}",
client.getFileSystem(),
path.toString());
final Hashtable<String, String> parsedXmsProperties;
final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
@ -169,17 +184,12 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
return parsedXmsProperties;
}
@Override
public void setPathProperties(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final Hashtable<String,
String> properties) throws
AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
public void setPathProperties(final Path path, final Hashtable<String, String> properties) throws AzureBlobFileSystemException {
this.LOG.debug(
"setFilesystemProperties for filesystem: {} path: {} with properties: {}",
client.getFileSystem(),
path.toString(),
properties);
"setFilesystemProperties for filesystem: {} path: {} with properties: {}",
client.getFileSystem(),
path.toString(),
properties);
final String commaSeparatedProperties;
try {
@ -190,71 +200,55 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
client.setPathProperties("/" + getRelativePath(path), commaSeparatedProperties);
}
@Override
public void createFilesystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
public void createFilesystem() throws AzureBlobFileSystemException {
this.LOG.debug(
"createFilesystem for filesystem: {}",
client.getFileSystem());
"createFilesystem for filesystem: {}",
client.getFileSystem());
client.createFilesystem();
}
@Override
public void deleteFilesystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
public void deleteFilesystem() throws AzureBlobFileSystemException {
this.LOG.debug(
"deleteFilesystem for filesystem: {}",
client.getFileSystem());
"deleteFilesystem for filesystem: {}",
client.getFileSystem());
client.deleteFilesystem();
}
@Override
public OutputStream createFile(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean overwrite) throws
AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
public OutputStream createFile(final Path path, final boolean overwrite) throws AzureBlobFileSystemException {
this.LOG.debug(
"createFile filesystem: {} path: {} overwrite: {}",
client.getFileSystem(),
path.toString(),
overwrite);
"createFile filesystem: {} path: {} overwrite: {}",
client.getFileSystem(),
path.toString(),
overwrite);
client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite);
final OutputStream outputStream;
outputStream = new FSDataOutputStream(
new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), 0,
configurationService.getWriteBufferSize()), null);
new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), 0,
abfsConfiguration.getWriteBufferSize()), null);
return outputStream;
}
@Override
public Void createDirectory(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
public Void createDirectory(final Path path) throws AzureBlobFileSystemException {
this.LOG.debug(
"createDirectory filesystem: {} path: {} overwrite: {}",
client.getFileSystem(),
path.toString());
"createDirectory filesystem: {} path: {} overwrite: {}",
client.getFileSystem(),
path.toString());
client.createPath("/" + getRelativePath(path), false, true);
return null;
}
@Override
public InputStream openFileForRead(final AzureBlobFileSystem azureBlobFileSystem, final Path path,
final FileSystem.Statistics statistics) throws AzureBlobFileSystemException {
final AbfsClient client = getOrCreateClient(azureBlobFileSystem);
public InputStream openFileForRead(final Path path, final FileSystem.Statistics statistics) throws AzureBlobFileSystemException {
this.LOG.debug(
"openFileForRead filesystem: {} path: {}",
client.getFileSystem(),
path.toString());
"openFileForRead filesystem: {} path: {}",
client.getFileSystem(),
path.toString());
final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
@ -264,28 +258,25 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
if (parseIsDirectory(resourceType)) {
throw new AbfsRestOperationException(
AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
"openFileForRead must be used with files and not directories",
null);
AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
"openFileForRead must be used with files and not directories",
null);
}
// Add statistics for InputStream
return new FSDataInputStream(
new AbfsInputStream(client, statistics, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength,
configurationService.getReadBufferSize(), configurationService.getReadAheadQueueDepth(), eTag));
new AbfsInputStream(client, statistics, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength,
abfsConfiguration.getReadBufferSize(), abfsConfiguration.getReadAheadQueueDepth(), eTag));
}
@Override
public OutputStream openFileForWrite(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean overwrite) throws
AzureBlobFileSystemException {
final AbfsClient client = getOrCreateClient(azureBlobFileSystem);
public OutputStream openFileForWrite(final Path path, final boolean overwrite) throws
AzureBlobFileSystemException {
this.LOG.debug(
"openFileForWrite filesystem: {} path: {} overwrite: {}",
client.getFileSystem(),
path.toString(),
overwrite);
"openFileForWrite filesystem: {} path: {} overwrite: {}",
client.getFileSystem(),
path.toString(),
overwrite);
final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
@ -294,37 +285,34 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
if (parseIsDirectory(resourceType)) {
throw new AbfsRestOperationException(
AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
"openFileForRead must be used with files and not directories",
null);
AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
"openFileForRead must be used with files and not directories",
null);
}
final long offset = overwrite ? 0 : contentLength;
final OutputStream outputStream;
outputStream = new FSDataOutputStream(
new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path),
offset, configurationService.getWriteBufferSize()), null);
new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path),
offset, abfsConfiguration.getWriteBufferSize()), null);
return outputStream;
}
@Override
public void rename(final AzureBlobFileSystem azureBlobFileSystem, final Path source, final Path destination) throws
AzureBlobFileSystemException {
public void rename(final Path source, final Path destination) throws
AzureBlobFileSystemException {
if (isAtomicRenameKey(source.getName())) {
this.LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename,"
+" create and delete operations are atomic if Namespace is enabled for your Azure Storage account.");
+" create and delete operations are atomic if Namespace is enabled for your Azure Storage account.");
}
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
this.LOG.debug(
"renameAsync filesystem: {} source: {} destination: {}",
client.getFileSystem(),
source.toString(),
destination.toString());
"renameAsync filesystem: {} source: {} destination: {}",
client.getFileSystem(),
source.toString(),
destination.toString());
String continuation = null;
long deadline = now() + RENAME_TIMEOUT_MILISECONDS;
@ -332,30 +320,28 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
do {
if (now() > deadline) {
LOG.debug(
"Rename {} to {} timed out.",
source,
destination);
"Rename {} to {} timed out.",
source,
destination);
throw new TimeoutException("Rename timed out.");
}
AbfsRestOperation op = client.renamePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(source),
AbfsHttpConstants.FORWARD_SLASH + getRelativePath(destination), continuation);
AbfsHttpConstants.FORWARD_SLASH + getRelativePath(destination), continuation);
continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
} while (continuation != null && !continuation.isEmpty());
}
@Override
public void delete(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean recursive) throws
AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
public void delete(final Path path, final boolean recursive) throws
AzureBlobFileSystemException {
this.LOG.debug(
"delete filesystem: {} path: {} recursive: {}",
client.getFileSystem(),
path.toString(),
String.valueOf(recursive));
"delete filesystem: {} path: {} recursive: {}",
client.getFileSystem(),
path.toString(),
String.valueOf(recursive));
String continuation = null;
long deadline = now() + DELETE_DIRECTORY_TIMEOUT_MILISECONDS;
@ -363,7 +349,7 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
do {
if (now() > deadline) {
this.LOG.debug(
"Delete directory {} timed out.", path);
"Delete directory {} timed out.", path);
throw new TimeoutException("Delete directory timed out.");
}
@ -374,60 +360,55 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
} while (continuation != null && !continuation.isEmpty());
}
@Override
public FileStatus getFileStatus(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
public FileStatus getFileStatus(final Path path) throws IOException {
this.LOG.debug(
"getFileStatus filesystem: {} path: {}",
client.getFileSystem(),
path.toString());
"getFileStatus filesystem: {} path: {}",
client.getFileSystem(),
path.toString());
if (path.isRoot()) {
AbfsRestOperation op = client.getFilesystemProperties();
final long blockSize = configurationService.getAzureBlockSize();
final long blockSize = abfsConfiguration.getAzureBlockSize();
final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED);
return new VersionedFileStatus(
azureBlobFileSystem.getOwnerUser(),
azureBlobFileSystem.getOwnerUserPrimaryGroup(),
0,
true,
1,
blockSize,
parseLastModifiedTime(lastModified).getMillis(),
path,
eTag);
userGroupInformation.getUserName(),
userGroupInformation.getPrimaryGroupName(),
0,
true,
1,
blockSize,
parseLastModifiedTime(lastModified).getMillis(),
path,
eTag);
} else {
AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
final long blockSize = configurationService.getAzureBlockSize();
final long blockSize = abfsConfiguration.getAzureBlockSize();
final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED);
final String contentLength = op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH);
final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
return new VersionedFileStatus(
azureBlobFileSystem.getOwnerUser(),
azureBlobFileSystem.getOwnerUserPrimaryGroup(),
parseContentLength(contentLength),
parseIsDirectory(resourceType),
1,
blockSize,
parseLastModifiedTime(lastModified).getMillis(),
path,
eTag);
userGroupInformation.getUserName(),
userGroupInformation.getPrimaryGroupName(),
parseContentLength(contentLength),
parseIsDirectory(resourceType),
1,
blockSize,
parseLastModifiedTime(lastModified).getMillis(),
path,
eTag);
}
}
@Override
public FileStatus[] listStatus(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException {
final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
public FileStatus[] listStatus(final Path path) throws IOException {
this.LOG.debug(
"listStatus filesystem: {} path: {}",
client.getFileSystem(),
path.toString());
"listStatus filesystem: {} path: {}",
client.getFileSystem(),
path.toString());
String relativePath = path.isRoot() ? AbfsHttpConstants.EMPTY_STRING : getRelativePath(path);
String continuation = null;
@ -439,13 +420,13 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
ListResultSchema retrievedSchema = op.getResult().getListResultSchema();
if (retrievedSchema == null) {
throw new AbfsRestOperationException(
AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
"listStatusAsync path not found",
null, op.getResult());
AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
"listStatusAsync path not found",
null, op.getResult());
}
long blockSize = configurationService.getAzureBlockSize();
long blockSize = abfsConfiguration.getAzureBlockSize();
for (ListResultEntrySchema entry : retrievedSchema.paths()) {
long lastModifiedMillis = 0;
@ -453,22 +434,25 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
boolean isDirectory = entry.isDirectory() == null ? false : entry.isDirectory();
if (entry.lastModified() != null && !entry.lastModified().isEmpty()) {
final DateTime dateTime = DateTime.parse(
entry.lastModified(),
DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC());
entry.lastModified(),
DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC());
lastModifiedMillis = dateTime.getMillis();
}
Path entryPath = new Path(File.separator + entry.name());
entryPath = entryPath.makeQualified(this.uri, entryPath);
fileStatuses.add(
new VersionedFileStatus(
azureBlobFileSystem.getOwnerUser(),
azureBlobFileSystem.getOwnerUserPrimaryGroup(),
contentLength,
isDirectory,
1,
blockSize,
lastModifiedMillis,
azureBlobFileSystem.makeQualified(new Path(File.separator + entry.name())),
entry.eTag()));
new VersionedFileStatus(
userGroupInformation.getUserName(),
userGroupInformation.getPrimaryGroupName(),
contentLength,
isDirectory,
1,
blockSize,
lastModifiedMillis,
entryPath,
entry.eTag()));
}
} while (continuation != null && !continuation.isEmpty());
@ -476,16 +460,55 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
return fileStatuses.toArray(new FileStatus[0]);
}
@Override
public synchronized void closeFileSystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException {
this.clientCache.remove(azureBlobFileSystem);
}
@Override
public boolean isAtomicRenameKey(String key) {
return isKeyForDirectorySet(key, azureAtomicRenameDirSet);
}
private void initializeClient(URI uri, boolean isSeure) throws AzureBlobFileSystemException {
if (this.client != null) {
return;
}
final String authority = uri.getRawAuthority();
if (null == authority) {
throw new InvalidUriAuthorityException(uri.toString());
}
if (!authority.contains(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER)) {
throw new InvalidUriAuthorityException(uri.toString());
}
final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2);
if (authorityParts.length < 2 || "".equals(authorityParts[0])) {
final String errMsg = String
.format("URI '%s' has a malformed authority, expected container name. "
+ "Authority takes the form "+ FileSystemUriSchemes.ABFS_SCHEME + "://[<container name>@]<account name>",
uri.toString());
throw new InvalidUriException(errMsg);
}
final String fileSystemName = authorityParts[0];
final String accountName = authorityParts[1];
final URIBuilder uriBuilder = getURIBuilder(accountName, isSeure);
final String url = uriBuilder.toString() + AbfsHttpConstants.FORWARD_SLASH + fileSystemName;
URL baseUrl;
try {
baseUrl = new URL(url);
} catch (MalformedURLException e) {
throw new InvalidUriException(String.format("URI '%s' is malformed", uri.toString()));
}
SharedKeyCredentials creds =
new SharedKeyCredentials(accountName.substring(0, accountName.indexOf(AbfsHttpConstants.DOT)),
this.abfsConfiguration.getStorageAccountKey(accountName));
this.client = new AbfsClient(baseUrl, creds, abfsConfiguration, new ExponentialRetryPolicy());
}
private String getRelativePath(final Path path) {
Preconditions.checkNotNull(path, "path");
final String relativePath = path.toUri().getPath();
@ -505,23 +528,6 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
return relativePath;
}
private synchronized AbfsClient getOrCreateClient(final AzureBlobFileSystem azureBlobFileSystem) throws
AzureBlobFileSystemException {
Preconditions.checkNotNull(azureBlobFileSystem, "azureBlobFileSystem");
AbfsClient client = this.clientCache.get(azureBlobFileSystem);
if (client != null) {
return client;
}
client = abfsHttpClientFactory.create(azureBlobFileSystem);
this.clientCache.put(
azureBlobFileSystem,
client);
return client;
}
private long parseContentLength(final String contentLength) {
if (contentLength == null) {
return -1;
@ -536,12 +542,12 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
private DateTime parseLastModifiedTime(final String lastModifiedTime) {
return DateTime.parse(
lastModifiedTime,
DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC());
lastModifiedTime,
DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC());
}
private String convertXmsPropertiesToCommaSeparatedString(final Hashtable<String, String> properties) throws
CharacterCodingException {
CharacterCodingException {
StringBuilder commaSeparatedProperties = new StringBuilder();
final CharsetEncoder encoder = Charset.forName(XMS_PROPERTIES_ENCODING).newEncoder();
@ -571,7 +577,7 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
}
private Hashtable<String, String> parseCommaSeparatedXmsProperties(String xMsProperties) throws
InvalidFileSystemPropertyException, InvalidAbfsRestOperationException {
InvalidFileSystemPropertyException, InvalidAbfsRestOperationException {
Hashtable<String, String> properties = new Hashtable<>();
final CharsetDecoder decoder = Charset.forName(XMS_PROPERTIES_ENCODING).newDecoder();
@ -633,15 +639,15 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
private final String version;
VersionedFileStatus(
final String owner, final String group,
final long length, final boolean isdir, final int blockReplication,
final long blocksize, final long modificationTime, final Path path,
String version) {
final String owner, final String group,
final long length, final boolean isdir, final int blockReplication,
final long blocksize, final long modificationTime, final Path path,
String version) {
super(length, isdir, blockReplication, blocksize, modificationTime, 0,
new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL),
owner,
group,
path);
new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL),
owner,
group,
path);
this.version = version;
}
@ -690,4 +696,6 @@ final class AbfsHttpServiceImpl implements AbfsHttpService {
return this.version;
}
}
}
}

View File

@ -1,36 +0,0 @@
/**
* 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.contracts.exceptions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
/**
* Thrown a service is either not configured to be injected or the service is not existing.
* For service registration
* @see AbfsServiceProviderImpl
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class ServiceResolutionException extends AzureBlobFileSystemException {
public ServiceResolutionException(String serviceName, Exception innerException) {
super(String.format("%s cannot be resolved.", serviceName), innerException);
}
}

View File

@ -1,39 +0,0 @@
/**
* 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.contracts.services;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
/**
* AbfsClient factory.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface AbfsHttpClientFactory extends InjectableService {
/**
* Creates and configures an instance of new AbfsClient
* @return AbfsClient instance
*/
AbfsClient create(AzureBlobFileSystem fs) throws AzureBlobFileSystemException;
}

View File

@ -1,162 +0,0 @@
/**
* 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.contracts.services;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.Hashtable;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
/**
* File System http service to provide network calls for file system operations.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface AbfsHttpService extends InjectableService {
/**
* Gets filesystem properties on the Azure service.
* @param azureBlobFileSystem filesystem to get the properties.
* @return Hashtable<String, String> hash table containing all the filesystem properties.
*/
Hashtable<String, String> getFilesystemProperties(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
/**
* Sets filesystem properties on the Azure service.
* @param azureBlobFileSystem filesystem to get the properties.
* @param properties file system properties to set.
*/
void setFilesystemProperties(AzureBlobFileSystem azureBlobFileSystem, Hashtable<String, String> properties) throws
AzureBlobFileSystemException;
/**
* Gets path properties on the Azure service.
* @param azureBlobFileSystem filesystem to get the properties of the path.
* @param path path to get properties.
* @return Hashtable<String, String> hash table containing all the path properties.
*/
Hashtable<String, String> getPathProperties(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
/**
* Sets path properties on the Azure service.
* @param azureBlobFileSystem filesystem to get the properties of the path.
* @param path path to set properties.
* @param properties hash table containing all the path properties.
*/
void setPathProperties(AzureBlobFileSystem azureBlobFileSystem, Path path, Hashtable<String, String> properties) throws
AzureBlobFileSystemException;
/**
* Creates filesystem on the Azure service.
* @param azureBlobFileSystem filesystem to be created.
*/
void createFilesystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
/**
* Deletes filesystem on the Azure service.
* @param azureBlobFileSystem filesystem to be deleted.
*/
void deleteFilesystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
/**
* Creates a file on the Azure service.
* @param azureBlobFileSystem filesystem to create file or directory.
* @param path path of the file to be created.
* @param overwrite should overwrite.
* @return OutputStream stream to the file.
*/
OutputStream createFile(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean overwrite) throws AzureBlobFileSystemException;
/**
* Creates a directory on the Azure service.
* @param azureBlobFileSystem filesystem to create file or directory.
* @param path path of the directory to be created.
* @return OutputStream stream to the file.
*/
Void createDirectory(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
/**
* Opens a file to read and returns the stream.
* @param azureBlobFileSystem filesystem to read a file from.
* @param path file path to read.
* @return InputStream a stream to the file to read.
*/
InputStream openFileForRead(AzureBlobFileSystem azureBlobFileSystem, Path path, FileSystem.Statistics statistics) throws AzureBlobFileSystemException;
/**
* Opens a file to write and returns the stream.
* @param azureBlobFileSystem filesystem to write a file to.
* @param path file path to write.
* @param overwrite should overwrite.
* @return OutputStream a stream to the file to write.
*/
OutputStream openFileForWrite(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean overwrite) throws AzureBlobFileSystemException;
/**
* Renames a file or directory from source to destination.
* @param azureBlobFileSystem filesystem to rename a path.
* @param source source path.
* @param destination destination path.
*/
void rename(AzureBlobFileSystem azureBlobFileSystem, Path source, Path destination) throws AzureBlobFileSystemException;
/**
* Deletes a file or directory.
* @param azureBlobFileSystem filesystem to delete the path.
* @param path file path to be deleted.
* @param recursive true if path is a directory and recursive deletion is desired.
*/
void delete(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean recursive) throws AzureBlobFileSystemException;
/**
* Gets path's status under the provided path on the Azure service.
* @param azureBlobFileSystem filesystem to perform the get file status operation.
* @param path path delimiter.
* @return FileStatus FileStatus of the path in the file system.
*/
FileStatus getFileStatus(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
/**
* Lists all the paths under the provided path on the Azure service.
* @param azureBlobFileSystem filesystem to perform the list operation.
* @param path path delimiter.
* @return FileStatus[] list of all paths in the file system.
*/
FileStatus[] listStatus(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException;
/**
* Closes the client to filesystem to Azure service.
* @param azureBlobFileSystem filesystem to perform the list operation.
*/
void closeFileSystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException;
/**
* Checks for the given path if it is marked as atomic rename directory or not.
* @param key
* @return True if the given path is listed under atomic rename property otherwise False.
*/
boolean isAtomicRenameKey(String key);
}

View File

@ -1,40 +0,0 @@
/**
* 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.contracts.services;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ServiceResolutionException;
/**
* Dependency injected Azure Storage services provider interface.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface AbfsServiceProvider {
/**
* Returns an instance of resolved injectable service by class name.
* The injectable service must be configured first to be resolvable.
* @param clazz the injectable service which is expected to be returned.
* @param <T> The type of injectable service.
* @return T instance
* @throws ServiceResolutionException if the service is not resolvable.
*/
<T extends InjectableService> T get(Class<T> clazz) throws ServiceResolutionException;
}

View File

@ -1,143 +0,0 @@
/**
* 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.contracts.services;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
/**
* Configuration service collects required Azure Hadoop configurations and provides it to the consumers.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface ConfigurationService extends InjectableService {
/**
* Checks if ABFS is running from Emulator;
* @return is emulator mode.
*/
boolean isEmulator();
/**
* Retrieves storage secure mode from Hadoop configuration;
* @return storage secure mode;
*/
boolean isSecureMode();
/**
* Retrieves storage account key for provided account name from Hadoop configuration.
* @param accountName the account name to retrieve the key.
* @return storage account key;
*/
String getStorageAccountKey(String accountName) throws ConfigurationPropertyNotFoundException;
/**
* Returns Hadoop configuration.
* @return Hadoop configuration.
*/
Configuration getConfiguration();
/**
* Retrieves configured write buffer size
* @return the size of the write buffer
*/
int getWriteBufferSize();
/**
* Retrieves configured read buffer size
* @return the size of the read buffer
*/
int getReadBufferSize();
/**
* Retrieves configured min backoff interval
* @return min backoff interval
*/
int getMinBackoffIntervalMilliseconds();
/**
* Retrieves configured max backoff interval
* @return max backoff interval
*/
int getMaxBackoffIntervalMilliseconds();
/**
* Retrieves configured backoff interval
* @return backoff interval
*/
int getBackoffIntervalMilliseconds();
/**
* Retrieves configured num of retries
* @return num of retries
*/
int getMaxIoRetries();
/**
* Retrieves configured azure block size
* @return azure block size
*/
long getAzureBlockSize();
/**
* Retrieves configured azure block location host
* @return azure block location host
*/
String getAzureBlockLocationHost();
/**
* Retrieves configured number of concurrent threads
* @return number of concurrent write threads
*/
int getMaxConcurrentWriteThreads();
/**
* Retrieves configured number of concurrent threads
* @return number of concurrent read threads
*/
int getMaxConcurrentReadThreads();
/**
* Retrieves configured boolean for tolerating out of band writes to files
* @return configured boolean for tolerating out of band writes to files
*/
boolean getTolerateOobAppends();
/**
* Retrieves the comma-separated list of directories to receive special treatment so that folder
* rename is made atomic. The default value for this setting is just '/hbase'.
* Example directories list : <value>/hbase,/data</value>
* @see <a href="https://hadoop.apache.org/docs/stable/hadoop-azure/index.html#Configuring_Credentials">AtomicRenameProperty</a>
* @return atomic rename directories
*/
String getAzureAtomicRenameDirs();
/**
* Retrieves configured boolean for creating remote file system during initialization
* @return configured boolean for creating remote file system during initialization
*/
boolean getCreateRemoteFileSystemDuringInitialization();
/**
* Retrieves configured value of read ahead queue
* @return depth of read ahead
*/
int getReadAheadQueueDepth();
}

View File

@ -1,30 +0,0 @@
/**
* 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.contracts.services;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Marker interface for all the injectable services.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface InjectableService {
}

View File

@ -1,66 +0,0 @@
/**
* 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.contracts.services;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.htrace.core.SpanId;
import org.apache.htrace.core.TraceScope;
/**
* Azure Blob File System tracing service.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public interface TracingService extends InjectableService {
/**
* Creates a {@link TraceScope} object with the provided description.
* @param description the trace description.
* @return created traceScope.
*/
TraceScope traceBegin(String description);
/**
* Creates a {@link TraceScope} object with the provided description.
* @param description the trace description.
* @param parentSpanId the span id of the parent trace scope.
* @return create traceScope
*/
TraceScope traceBegin(String description, SpanId parentSpanId);
/**
* Gets current thread latest generated traceScope id.
* @return current thread latest generated traceScope id.
*/
SpanId getCurrentTraceScopeSpanId();
/**
* Appends the provided exception to the trace scope.
* @param traceScope the scope which exception needs to be attached to.
* @param azureBlobFileSystemException the exception to be attached to the scope.
*/
void traceException(TraceScope traceScope, AzureBlobFileSystemException azureBlobFileSystemException);
/**
* Ends the provided traceScope.
* @param traceScope the scope that needs to be ended.
*/
void traceEnd(TraceScope traceScope);
}

View File

@ -28,7 +28,6 @@ import java.util.Locale;
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.services.ConfigurationService;
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
@ -45,17 +44,17 @@ public class AbfsClient {
private final String xMsVersion = "2018-03-28";
private final ExponentialRetryPolicy retryPolicy;
private final String filesystem;
private final ConfigurationService configurationService;
private final AbfsConfiguration abfsConfiguration;
private final String userAgent;
public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials,
final ConfigurationService configurationService,
final AbfsConfiguration abfsConfiguration,
final ExponentialRetryPolicy exponentialRetryPolicy) {
this.baseUrl = baseUrl;
this.sharedKeyCredentials = sharedKeyCredentials;
String baseUrlString = baseUrl.toString();
this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(AbfsHttpConstants.FORWARD_SLASH) + 1);
this.configurationService = configurationService;
this.abfsConfiguration = abfsConfiguration;
this.retryPolicy = exponentialRetryPolicy;
this.userAgent = initializeUserAgent();
}

View File

@ -22,8 +22,6 @@ import java.lang.reflect.Field;
import java.util.Map;
import com.google.common.annotations.VisibleForTesting;
import com.google.inject.Inject;
import com.google.inject.Singleton;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -37,17 +35,18 @@ import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidati
import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.apache.hadoop.fs.azurebfs.diagnostics.Base64StringConfigurationBasicValidator;
import org.apache.hadoop.fs.azurebfs.diagnostics.BooleanConfigurationBasicValidator;
import org.apache.hadoop.fs.azurebfs.diagnostics.IntegerConfigurationBasicValidator;
import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator;
import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator;
@Singleton
/**
* Configuration for Azure Blob FileSystem.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
class ConfigurationServiceImpl implements ConfigurationService {
public class AbfsConfiguration{
private final Configuration configuration;
private final boolean isSecure;
@ -118,8 +117,7 @@ class ConfigurationServiceImpl implements ConfigurationService {
private Map<String, String> storageAccountKeys;
@Inject
ConfigurationServiceImpl(final Configuration configuration) throws IllegalAccessException, InvalidConfigurationValueException {
public AbfsConfiguration(final Configuration configuration) throws IllegalAccessException, InvalidConfigurationValueException {
this.configuration = configuration;
this.isSecure = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_SECURE_MODE, false);
@ -141,17 +139,14 @@ class ConfigurationServiceImpl implements ConfigurationService {
}
}
@Override
public boolean isEmulator() {
return this.getConfiguration().getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false);
}
@Override
public boolean isSecureMode() {
return this.isSecure;
}
@Override
public String getStorageAccountKey(final String accountName) throws ConfigurationPropertyNotFoundException {
String accountKey = this.storageAccountKeys.get(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + accountName);
if (accountKey == null) {
@ -161,77 +156,62 @@ class ConfigurationServiceImpl implements ConfigurationService {
return accountKey;
}
@Override
public Configuration getConfiguration() {
return this.configuration;
}
@Override
public int getWriteBufferSize() {
return this.writeBufferSize;
}
@Override
public int getReadBufferSize() {
return this.readBufferSize;
}
@Override
public int getMinBackoffIntervalMilliseconds() {
return this.minBackoffInterval;
}
@Override
public int getMaxBackoffIntervalMilliseconds() {
return this.maxBackoffInterval;
}
@Override
public int getBackoffIntervalMilliseconds() {
return this.backoffInterval;
}
@Override
public int getMaxIoRetries() {
return this.maxIoRetries;
}
@Override
public long getAzureBlockSize() {
return this.azureBlockSize;
}
@Override
public String getAzureBlockLocationHost() {
return this.azureBlockLocationHost;
}
@Override
public int getMaxConcurrentWriteThreads() {
return this.maxConcurrentWriteThreads;
}
@Override
public int getMaxConcurrentReadThreads() {
return this.maxConcurrentReadThreads;
}
@Override
public boolean getTolerateOobAppends() {
return this.tolerateOobAppends;
}
@Override
public String getAzureAtomicRenameDirs() {
return this.azureAtomicDirs;
}
@Override
public boolean getCreateRemoteFileSystemDuringInitialization() {
return this.createRemoteFileSystemDuringInitialization;
}
@Override
public int getReadAheadQueueDepth() {
return this.readAheadQueueDepth;
}

View File

@ -1,116 +0,0 @@
/**
* 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 java.net.MalformedURLException;
import java.net.URI;
import java.net.URL;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.inject.Inject;
import com.google.inject.Singleton;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.apache.http.client.utils.URIBuilder;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
@Singleton
@InterfaceAudience.Private
@InterfaceStability.Evolving
class AbfsHttpClientFactoryImpl implements AbfsHttpClientFactory {
private final ConfigurationService configurationService;
@Inject
AbfsHttpClientFactoryImpl(
final ConfigurationService configurationService) {
Preconditions.checkNotNull(configurationService, "configurationService");
this.configurationService = configurationService;
}
@VisibleForTesting
URIBuilder getURIBuilder(final String hostName, final FileSystem fs) {
final AzureBlobFileSystem abfs = (AzureBlobFileSystem) fs;
String scheme = FileSystemUriSchemes.HTTP_SCHEME;
if (abfs.isSecure()) {
scheme = FileSystemUriSchemes.HTTPS_SCHEME;
}
final URIBuilder uriBuilder = new URIBuilder();
uriBuilder.setScheme(scheme);
uriBuilder.setHost(hostName);
return uriBuilder;
}
public AbfsClient create(final AzureBlobFileSystem fs) throws AzureBlobFileSystemException {
final URI uri = fs.getUri();
final String authority = uri.getRawAuthority();
if (null == authority) {
throw new InvalidUriAuthorityException(uri.toString());
}
if (!authority.contains(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER)) {
throw new InvalidUriAuthorityException(uri.toString());
}
final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2);
if (authorityParts.length < 2 || "".equals(authorityParts[0])) {
final String errMsg = String
.format("URI '%s' has a malformed authority, expected container name. "
+ "Authority takes the form "+ FileSystemUriSchemes.ABFS_SCHEME + "://[<container name>@]<account name>",
uri.toString());
throw new InvalidUriException(errMsg);
}
final String fileSystemName = authorityParts[0];
final String accountName = authorityParts[1];
final URIBuilder uriBuilder = getURIBuilder(accountName, fs);
final String url = uriBuilder.toString() + AbfsHttpConstants.FORWARD_SLASH + fileSystemName;
URL baseUrl;
try {
baseUrl = new URL(url);
} catch (MalformedURLException e) {
throw new InvalidUriException(String.format("URI '%s' is malformed", uri.toString()));
}
SharedKeyCredentials creds =
new SharedKeyCredentials(accountName.substring(0, accountName.indexOf(AbfsHttpConstants.DOT)),
this.configurationService.getStorageAccountKey(accountName));
return new AbfsClient(baseUrl, creds, configurationService, new ExponentialRetryPolicy());
}
}

View File

@ -1,81 +0,0 @@
/**
* 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 java.util.HashMap;
import java.util.Map;
import com.google.inject.AbstractModule;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
/**
* This class is responsible to configure all the services used by Azure Blob File System.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
class AbfsServiceInjectorImpl extends AbstractModule {
private final Configuration configuration;
private final Map<Class, Class> providers;
private final Map<Class, Object> instances;
AbfsServiceInjectorImpl(Configuration configuration) {
this.providers = new HashMap<>();
this.instances = new HashMap<>();
this.configuration = configuration;
this.instances.put(Configuration.class, this.configuration);
this.providers.put(ConfigurationService.class, ConfigurationServiceImpl.class);
this.providers.put(AbfsHttpService.class, AbfsHttpServiceImpl.class);
this.providers.put(AbfsHttpClientFactory.class, AbfsHttpClientFactoryImpl.class);
this.providers.put(TracingService.class, TracingServiceImpl.class);
}
@Override
protected void configure() {
for (Map.Entry<Class, Object> entrySet : this.instances.entrySet()) {
bind(entrySet.getKey()).toInstance(entrySet.getValue());
}
for (Map.Entry<Class, Class> entrySet : this.providers.entrySet()) {
bind(entrySet.getKey()).to(entrySet.getValue());
}
}
protected Configuration getConfiguration() {
return this.configuration;
}
protected Map<Class, Class> getProviders() {
return this.providers;
}
protected Map<Class, Object> getInstances() {
return this.instances;
}
}

View File

@ -1,96 +0,0 @@
/**
* 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.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.inject.Guice;
import com.google.inject.Injector;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ServiceResolutionException;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsServiceProvider;
import org.apache.hadoop.fs.azurebfs.contracts.services.InjectableService;
/**
* Dependency injected Azure Storage services provider.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class AbfsServiceProviderImpl implements AbfsServiceProvider {
private static AbfsServiceProviderImpl abfsServiceProvider;
private final Injector abfsServiceInjector;
private AbfsServiceProviderImpl(final Configuration configuration) {
this.abfsServiceInjector = Guice.createInjector(new AbfsServiceInjectorImpl(Preconditions.checkNotNull(configuration, "configuration")));
}
@VisibleForTesting
private AbfsServiceProviderImpl(final Injector abfsServiceInjector) {
Preconditions.checkNotNull(abfsServiceInjector, "abfsServiceInjector");
this.abfsServiceInjector = abfsServiceInjector;
}
/**
* Create an instance or returns existing instance of service provider.
* This method must be marked as synchronized to ensure thread-safety.
* @param configuration hadoop configuration.
* @return AbfsServiceProvider the service provider instance.
*/
public static synchronized AbfsServiceProvider create(final Configuration configuration) {
if (abfsServiceProvider == null) {
abfsServiceProvider = new AbfsServiceProviderImpl(configuration);
}
return abfsServiceProvider;
}
/**
* Returns current instance of service provider.
* @return AbfsServiceProvider the service provider instance.
*/
public static AbfsServiceProvider instance() {
return abfsServiceProvider;
}
@VisibleForTesting
static synchronized AbfsServiceProvider create(Injector serviceInjector) {
abfsServiceProvider = new AbfsServiceProviderImpl(serviceInjector);
return abfsServiceProvider;
}
/**
* Returns an instance of resolved injectable service by class name.
* The injectable service must be configured first to be resolvable.
* @param clazz the injectable service which is expected to be returned.
* @param <T> The type of injectable service.
* @return T instance
* @throws ServiceResolutionException if the service is not resolvable.
*/
@Override
public <T extends InjectableService> T get(final Class<T> clazz) throws ServiceResolutionException {
try {
return this.abfsServiceInjector.getInstance(clazz);
} catch (Exception ex) {
throw new ServiceResolutionException(clazz.getSimpleName(), ex);
}
}
}

View File

@ -21,7 +21,10 @@ package org.apache.hadoop.fs.azurebfs.services;
import java.util.Random;
import java.net.HttpURLConnection;
class ExponentialRetryPolicy {
/**
* Retry policy used by AbfsClient.
* */
public class ExponentialRetryPolicy {
/**
* Represents the default number of retry attempts.
*/
@ -83,7 +86,7 @@ class ExponentialRetryPolicy {
/**
* Initializes a new instance of the {@link ExponentialRetryPolicy} class.
*/
ExponentialRetryPolicy() {
public ExponentialRetryPolicy() {
this(DEFAULT_CLIENT_RETRY_COUNT, DEFAULT_MIN_BACKOFF, DEFAULT_MAX_BACKOFF, DEFAULT_CLIENT_BACKOFF);
}
@ -96,7 +99,7 @@ class ExponentialRetryPolicy {
* @param deltaBackoff The value that will be used to calculate a random delta in the exponential delay
* between retries.
*/
ExponentialRetryPolicy(final int retryCount, final int minBackoff, final int maxBackoff, final int deltaBackoff) {
public ExponentialRetryPolicy(final int retryCount, final int minBackoff, final int maxBackoff, final int deltaBackoff) {
this.retryCount = retryCount;
this.minBackoff = minBackoff;
this.maxBackoff = maxBackoff;

View File

@ -1,74 +0,0 @@
/**
* 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 java.io.IOException;
import com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
import org.apache.htrace.core.HTraceConfiguration;
import org.apache.htrace.core.Span;
import org.apache.htrace.core.SpanReceiver;
import org.apache.htrace.fasterxml.jackson.core.JsonProcessingException;
import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper;
import org.apache.htrace.fasterxml.jackson.databind.ObjectWriter;
import org.apache.htrace.fasterxml.jackson.databind.SerializationFeature;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* LoggerSpanReceiver is a layer between HTrace and log4j only used for {@link org.apache.hadoop.fs.azurebfs.contracts.services.TracingService}
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class LoggerSpanReceiver extends SpanReceiver {
private static final ObjectWriter JSON_WRITER =
new ObjectMapper()
.configure(SerializationFeature.INDENT_OUTPUT, true)
.configure(SerializationFeature.WRITE_BIGDECIMAL_AS_PLAIN, true)
.configure(SerializationFeature.WRITE_EMPTY_JSON_ARRAYS, false)
.configure(SerializationFeature.USE_EQUALITY_FOR_OBJECT_ID, false)
.writer();
public LoggerSpanReceiver(HTraceConfiguration hTraceConfiguration) {
Preconditions.checkNotNull(hTraceConfiguration, "hTraceConfiguration");
}
@Override
public void receiveSpan(final Span span) {
String jsonValue;
Logger logger = LoggerFactory.getLogger(AzureBlobFileSystem.class);
try {
jsonValue = JSON_WRITER.writeValueAsString(span);
logger.trace(jsonValue);
} catch (JsonProcessingException e) {
logger.error("Json processing error: " + e.getMessage());
}
}
@Override
public void close() throws IOException {
// No-Op
}
}

View File

@ -1,134 +0,0 @@
/**
* 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 java.io.PrintWriter;
import java.io.StringWriter;
import java.util.Objects;
import com.google.common.base.Preconditions;
import com.google.inject.Inject;
import com.google.inject.Singleton;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
import org.apache.htrace.core.HTraceConfiguration;
import org.apache.htrace.core.Sampler;
import org.apache.htrace.core.SpanId;
import org.apache.htrace.core.TraceScope;
import org.apache.htrace.core.Tracer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@Singleton
@InterfaceAudience.Private
@InterfaceStability.Evolving
final class TracingServiceImpl implements TracingService {
private static final Logger LOG = LoggerFactory.getLogger(TracingService.class);
private final Tracer tracer;
private final ThreadLocal<SpanId> currentScopeId;
@Inject
TracingServiceImpl(
final Configuration configuration) {
Preconditions.checkNotNull(configuration, "configuration");
this.currentScopeId = new ThreadLocal<>();
this.tracer = new Tracer.Builder(TracingService.class.getSimpleName()).
conf(new HTraceConfiguration() {
@Override
public String get(String key) {
if (Objects.equals(key, Tracer.SPAN_RECEIVER_CLASSES_KEY)) {
return LoggerSpanReceiver.class.getName();
}
return null;
}
@Override
public String get(String key, String defaultValue) {
String value = get(key);
if (value != null) {
return value;
}
return defaultValue;
}
}).
build();
this.tracer.addSampler(Sampler.ALWAYS);
}
@Override
public TraceScope traceBegin(String description) {
if (this.LOG.isTraceEnabled()) {
TraceScope traceScope = this.tracer.newScope(description);
this.currentScopeId.set(traceScope.getSpanId());
return traceScope;
}
return null;
}
@Override
public TraceScope traceBegin(String description, SpanId parentSpanId) {
if (this.LOG.isTraceEnabled()) {
TraceScope traceScope = this.tracer.newScope(description, parentSpanId);
this.currentScopeId.set(traceScope.getSpanId());
return traceScope;
}
return null;
}
@Override
public void traceException(TraceScope traceScope, AzureBlobFileSystemException azureBlobFileSystemException) {
if (this.LOG.isTraceEnabled()) {
Preconditions.checkNotNull(traceScope, "traceScope");
Preconditions.checkNotNull(azureBlobFileSystemException, "azureBlobFileSystemException");
StringWriter stringWriter = new StringWriter();
PrintWriter printWriter = new PrintWriter(stringWriter);
azureBlobFileSystemException.printStackTrace(printWriter);
printWriter.flush();
traceScope.addKVAnnotation("Exception", stringWriter.toString());
}
}
@Override
public SpanId getCurrentTraceScopeSpanId() {
return this.currentScopeId.get();
}
@Override
public void traceEnd(TraceScope traceScope) {
if (this.LOG.isTraceEnabled()) {
Preconditions.checkNotNull(traceScope, "traceScope");
SpanId[] parents = traceScope.getSpan().getParents();
this.currentScopeId.set(parents != null && parents.length > 0 ? parents[parents.length - 1] : null);
traceScope.close();
}
}
}

View File

@ -23,11 +23,9 @@ import java.util.UUID;
import java.util.concurrent.Callable;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.mockito.internal.util.MockUtil;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@ -38,12 +36,6 @@ import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.apache.hadoop.fs.azurebfs.services.MockAbfsHttpClientFactoryImpl;
import org.apache.hadoop.fs.azurebfs.services.MockAbfsServiceInjectorImpl;
import org.apache.hadoop.fs.azurebfs.services.MockServiceProviderImpl;
import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@ -54,7 +46,6 @@ import static org.junit.Assume.assumeNotNull;
* Provide dependencies for AzureBlobFileSystem tests.
*/
public abstract class DependencyInjectedTest {
private final MockAbfsServiceInjectorImpl mockServiceInjector;
private final boolean isEmulator;
private NativeAzureFileSystem wasb;
private String abfsScheme;
@ -64,21 +55,19 @@ public abstract class DependencyInjectedTest {
private String accountName;
private String testUrl;
public static final String TEST_CONTAINER_PREFIX = "abfs-testcontainer-";
public DependencyInjectedTest(final boolean secure) {
this(secure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME);
}
public MockAbfsServiceInjectorImpl getMockServiceInjector() {
return this.mockServiceInjector;
}
protected DependencyInjectedTest() {
this(FileSystemUriSchemes.ABFS_SCHEME);
}
private DependencyInjectedTest(final String scheme) {
abfsScheme = scheme;
fileSystemName = UUID.randomUUID().toString();
fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString();
configuration = new Configuration();
configuration.addResource("azure-bfs-test.xml");
@ -98,18 +87,14 @@ public abstract class DependencyInjectedTest {
this.testUrl = defaultUri.toString();
configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
configuration.setBoolean(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
this.mockServiceInjector = new MockAbfsServiceInjectorImpl(configuration);
this.isEmulator = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false);
this.accountName = this.configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME);
}
@Before
public void initialize() throws Exception {
if (this.isEmulator) {
this.mockServiceInjector.replaceProvider(AbfsHttpClientFactory.class, MockAbfsHttpClientFactoryImpl.class);
}
MockServiceProviderImpl.create(this.mockServiceInjector);
//Create filesystem first to make sure getWasbFileSystem() can return an existed filesystem.
this.getFileSystem();
if (!this.isEmulator) {
final URI wasbUri = new URI(abfsUrlToWasbUrl(this.getTestUrl()));
@ -133,28 +118,24 @@ public abstract class DependencyInjectedTest {
FileSystem.closeAll();
final AzureBlobFileSystem fs = this.getFileSystem();
final AbfsHttpService abfsHttpService = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class);
abfsHttpService.deleteFilesystem(fs);
final AzureBlobFileSystemStore abfsStore = fs.getAbfsStore();
abfsStore.deleteFilesystem();
if (!(new MockUtil().isMock(abfsHttpService))) {
AbfsRestOperationException ex = intercept(
AbfsRestOperationException.class,
new Callable<Void>() {
@Override
public Void call() throws Exception {
abfsHttpService.getFilesystemProperties(fs);
return null;
}
});
AbfsRestOperationException ex = intercept(
AbfsRestOperationException.class,
new Callable<Void>() {
@Override
public Void call() throws Exception {
fs.getAbfsStore().getFilesystemProperties();
return null;
}
});
assertEquals(FILE_SYSTEM_NOT_FOUND.getStatusCode(), ex.getStatusCode());
}
assertEquals(FILE_SYSTEM_NOT_FOUND.getStatusCode(), ex.getStatusCode());
}
public AzureBlobFileSystem getFileSystem() throws Exception {
final Configuration configuration = AbfsServiceProviderImpl.instance().get(ConfigurationService.class).getConfiguration();
final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration);
return fs;
return (AzureBlobFileSystem) FileSystem.get(this.configuration);
}
protected NativeAzureFileSystem getWasbFileSystem() {

View File

@ -22,15 +22,12 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.Random;
import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
import org.junit.Test;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
@ -52,8 +49,6 @@ public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest {
super();
Configuration configuration = this.getConfiguration();
configuration.set(ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH, "0");
this.getMockServiceInjector().replaceInstance(Configuration.class, configuration);
}
@Test
@ -82,7 +77,7 @@ public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest {
@Test (expected = IOException.class)
public void testOOBWrites() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
int readBufferSize = AbfsServiceProviderImpl.instance().get(ConfigurationService.class).getReadBufferSize();
int readBufferSize = fs.getAbfsStore().getAbfsConfiguration().getReadBufferSize();
fs.create(TEST_FILE);
FSDataOutputStream writeStream = fs.create(TEST_FILE);

View File

@ -535,17 +535,16 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest {
character = (character == 'z') ? 'a' : (char) ((int) character + 1);
}
System.out.println(("Creating test file {} of size: {} " + TEST_FILE_PATH
+ TEST_FILE_SIZE));
System.out.println(String.format("Creating test file %s of size: %d ", TEST_FILE_PATH, TEST_FILE_SIZE));
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
try(FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) {
try (FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) {
int bytesWritten = 0;
while (bytesWritten < TEST_FILE_SIZE) {
outputStream.write(buffer);
bytesWritten += buffer.length;
}
System.out.println("Closing stream {}" + outputStream);
System.out.println(String.format("Closing stream %s", outputStream));
ContractTestUtils.NanoTimer closeTimer
= new ContractTestUtils.NanoTimer();
outputStream.close();

View File

@ -22,18 +22,10 @@ import java.net.URI;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
import static org.mockito.Matchers.anyObject;
import static org.mockito.Mockito.doReturn;
/**
* Test AzureBlobFileSystem initialization.
@ -41,18 +33,11 @@ import static org.mockito.Mockito.doReturn;
public class ITestFileSystemInitialization extends DependencyInjectedTest {
public ITestFileSystemInitialization() {
super();
this.getMockServiceInjector().removeProvider(AbfsHttpService.class);
this.getMockServiceInjector().replaceInstance(AbfsHttpService.class, Mockito.mock(AbfsHttpService.class));
}
@Test
public void ensureAzureBlobFileSystemIsInitialized() throws Exception {
doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
.when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
.getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
final FileSystem fs = FileSystem.get(this.getConfiguration());
final FileSystem fs = this.getFileSystem();
final String accountName = this.getAccountName();
final String filesystem = this.getFileSystemName();
@ -62,16 +47,12 @@ public class ITestFileSystemInitialization extends DependencyInjectedTest {
@Test
public void ensureSecureAzureBlobFileSystemIsInitialized() throws Exception {
doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
.when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
.getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
final String accountName = this.getAccountName();
final String filesystem = this.getFileSystemName();
final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null);
this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
final FileSystem fs = FileSystem.get(this.getConfiguration());
final FileSystem fs = this.getFileSystem();
Assert.assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null));
Assert.assertNotNull(fs.getWorkingDirectory());
}

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.hadoop.fs.azurebfs.services;
package org.apache.hadoop.fs.azurebfs;
import java.util.Hashtable;
@ -28,19 +28,16 @@ import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
import static org.junit.Assert.assertEquals;
/**
* Test AbfsHttpServiceImpl.
* Test FileSystemProperties.
*/
public class ITestAbfsHttpServiceImpl extends DependencyInjectedTest {
public class ITestFileSystemProperties extends DependencyInjectedTest {
private static final int TEST_DATA = 100;
private static final Path TEST_PATH = new Path("/testfile");
public ITestAbfsHttpServiceImpl() {
public ITestFileSystemProperties() {
super();
}
@ -71,11 +68,11 @@ public class ITestAbfsHttpServiceImpl extends DependencyInjectedTest {
@Ignore("JDK7 doesn't support PATCH, so PUT is used. Fix is applied in latest test tenant")
public void testBase64FileSystemProperties() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final Hashtable<String, String> properties = new Hashtable<>();
properties.put("key", "{ value: value }");
AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setFilesystemProperties(
fs, properties);
Hashtable<String, String> fetchedProperties = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getFilesystemProperties(fs);
fs.getAbfsStore().setFilesystemProperties(properties);
Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties();
Assert.assertEquals(properties, fetchedProperties);
}
@ -86,10 +83,9 @@ public class ITestAbfsHttpServiceImpl extends DependencyInjectedTest {
final Hashtable<String, String> properties = new Hashtable<>();
properties.put("key", "{ value: valueTest }");
fs.create(TEST_PATH);
AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setPathProperties(
fs, TEST_PATH, properties);
fs.getAbfsStore().setPathProperties(TEST_PATH, properties);
Hashtable<String, String> fetchedProperties =
AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getPathProperties(fs, TEST_PATH);
fs.getAbfsStore().getPathProperties(TEST_PATH);
Assert.assertEquals(properties, fetchedProperties);
}
@ -99,9 +95,8 @@ public class ITestAbfsHttpServiceImpl extends DependencyInjectedTest {
final AzureBlobFileSystem fs = this.getFileSystem();
final Hashtable<String, String> properties = new Hashtable<>();
properties.put("key", "{ value: value歲 }");
AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setFilesystemProperties(
fs, properties);
Hashtable<String, String> fetchedProperties = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getFilesystemProperties(fs);
fs.getAbfsStore().setFilesystemProperties(properties);
Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties();
Assert.assertEquals(properties, fetchedProperties);
}
@ -112,10 +107,19 @@ public class ITestAbfsHttpServiceImpl extends DependencyInjectedTest {
final Hashtable<String, String> properties = new Hashtable<>();
properties.put("key", "{ value: valueTest兩 }");
fs.create(TEST_PATH);
AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setPathProperties(
fs, TEST_PATH, properties);
Hashtable<String, String> fetchedProperties =
AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getPathProperties(fs, TEST_PATH);
fs.getAbfsStore().setPathProperties(TEST_PATH, properties);
Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getPathProperties(TEST_PATH);
Assert.assertEquals(properties, fetchedProperties);
}
@Test
public void testSetFileSystemProperties() throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final Hashtable<String, String> properties = new Hashtable<>();
properties.put("containerForDevTest", "true");
fs.getAbfsStore().setFilesystemProperties(properties);
Hashtable<String, String> fetchedProperties = fs.getAbfsStore().getFilesystemProperties();
Assert.assertEquals(properties, fetchedProperties);
}

View File

@ -20,22 +20,14 @@ package org.apache.hadoop.fs.azurebfs;
import java.net.URI;
import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
import org.apache.hadoop.fs.AbstractFileSystem;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService;
import static org.mockito.Matchers.anyObject;
import static org.mockito.Mockito.doReturn;
/**
* Test AzureBlobFileSystem registration.
@ -43,17 +35,10 @@ import static org.mockito.Mockito.doReturn;
public class ITestFileSystemRegistration extends DependencyInjectedTest {
public ITestFileSystemRegistration() throws Exception {
super();
this.getMockServiceInjector().removeProvider(AbfsHttpService.class);
this.getMockServiceInjector().replaceInstance(AbfsHttpService.class, Mockito.mock(AbfsHttpService.class));
}
@Test
public void ensureAzureBlobFileSystemIsDefaultFileSystem() throws Exception {
doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
.when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
.getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
FileSystem fs = FileSystem.get(this.getConfiguration());
Assert.assertTrue(fs instanceof AzureBlobFileSystem);
@ -63,14 +48,10 @@ public class ITestFileSystemRegistration extends DependencyInjectedTest {
@Test
public void ensureSecureAzureBlobFileSystemIsDefaultFileSystem() throws Exception {
doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah")))
.when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class))
.getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject());
final String accountName = this.getAccountName();
final String filesystem = this.getFileSystemName();
final String fileSystemName = this.getFileSystemName();
final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null);
final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, fileSystemName + "@" + accountName, null, null, null);
this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
FileSystem fs = FileSystem.get(this.getConfiguration());

View File

@ -23,6 +23,7 @@ import org.apache.hadoop.fs.FileSystemContractBaseTest;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.junit.After;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
@ -37,7 +38,7 @@ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest {
private final DependencyInjectedContractTest dependencyInjectedContractTest;
public ITestAzureBlobFileSystemBasics() throws Exception {
// If contract tests are running in parallel, some root level tests in this file will fail
// If all contract tests are running in parallel, some root level tests in FileSystemContractBaseTest will fail
// due to the race condition. Hence for this contract test it should be tested in different container
dependencyInjectedContractTest = new DependencyInjectedContractTest(false, false);
}
@ -48,6 +49,14 @@ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest {
fs = this.dependencyInjectedContractTest.getFileSystem();
}
@After
public void testCleanup() throws Exception {
// This contract test is not using existing container for test,
// instead it creates its own temp container for test, hence we need to destroy
// it after the test.
this.dependencyInjectedContractTest.testCleanup();
}
@Test
public void testListOnFolderWithNoChildren() throws IOException {
assertTrue(fs.mkdirs(path("testListStatus/c/1")));

View File

@ -25,7 +25,6 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.junit.Test;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
@ -52,12 +51,11 @@ public class ITestReadWriteAndSeek extends DependencyInjectedTest {
private void testReadWriteAndSeek(int bufferSize) throws Exception {
final AzureBlobFileSystem fs = this.getFileSystem();
final ConfigurationServiceImpl configurationservice = (ConfigurationServiceImpl) AbfsServiceProviderImpl.instance().get(ConfigurationService.class);
final AbfsConfiguration abfsConfiguration = new AbfsConfiguration(this.getConfiguration());
fs.create(TEST_PATH);
configurationservice.setWriteBufferSize(bufferSize);
configurationservice.setReadBufferSize(bufferSize);
abfsConfiguration.setWriteBufferSize(bufferSize);
abfsConfiguration.setReadBufferSize(bufferSize);
final FSDataOutputStream stream = fs.create(TEST_PATH);

View File

@ -1,79 +0,0 @@
/**
* 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 java.util.ArrayList;
import java.util.List;
import org.apache.log4j.AppenderSkeleton;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.apache.log4j.spi.LoggingEvent;
import org.junit.Test;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService;
import org.apache.htrace.core.MilliSpan;
import org.apache.htrace.core.TraceScope;
/**
* Test tracing service.
*/
public class ITestTracingServiceImpl extends DependencyInjectedTest {
private final List<String> messageStorage;
public ITestTracingServiceImpl() {
super();
this.messageStorage = new ArrayList<>();
}
@Test
public void traceSerializationTest() throws Exception {
Logger rootLogger = Logger.getRootLogger();
rootLogger.setLevel(Level.TRACE);
rootLogger.addAppender(new AppenderSkeleton() {
@Override
protected void append(LoggingEvent event) {
if (event.getLogger().getName().indexOf("AzureBlobFileSystem") != -1) {
messageStorage.add(event.getRenderedMessage());
}
}
@Override
public void close() {
}
@Override
public boolean requiresLayout() {
return false;
}
});
TracingService tracingService = new TracingServiceImpl(new Configuration());
TraceScope traceScope = tracingService.traceBegin("Test Scope");
traceScope.addTimelineAnnotation("Timeline Annotations");
traceScope.addKVAnnotation("key", "value");
traceScope.close();
// Should not throw exception.
MilliSpan.fromJson(messageStorage.get(0));
}
}

View File

@ -1,69 +0,0 @@
/**
* 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.annotations.VisibleForTesting;
import com.google.inject.Inject;
import com.google.inject.Singleton;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService;
import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
import org.apache.http.client.utils.URIBuilder;
/**
* Mock AbfsHttpClientFactoryImpl.
*/
@Singleton
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class MockAbfsHttpClientFactoryImpl extends AbfsHttpClientFactoryImpl {
private final ConfigurationService configurationService;
@Inject
MockAbfsHttpClientFactoryImpl(
final ConfigurationService configurationService) {
super(configurationService);
this.configurationService = configurationService;
}
@VisibleForTesting
URIBuilder getURIBuilder(final String hostName, final FileSystem fs) {
final URIBuilder uriBuilder = new URIBuilder();
final String testHost = this.configurationService.getConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_HOST_NAME);
final Integer testHostPort = this.configurationService.getConfiguration().getInt(TestConfigurationKeys.FS_AZURE_TEST_HOST_PORT, 80);
final String testAccount = this.configurationService.getConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME);
String scheme = FileSystemUriSchemes.HTTP_SCHEME;
uriBuilder.setScheme(scheme);
uriBuilder.setHost(testHost);
uriBuilder.setPort(testHostPort);
uriBuilder.setPath("/" + UriUtils.extractRawAccountFromAccountName(testAccount) + "/");
return uriBuilder;
}
}

View File

@ -1,50 +0,0 @@
/**
* 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.conf.Configuration;
/**
* Mock AbfsServiceInjectorImpl.
*/
public class MockAbfsServiceInjectorImpl extends AbfsServiceInjectorImpl {
public MockAbfsServiceInjectorImpl(Configuration configuration) {
super(configuration);
}
public <T> void replaceInstance(Class<T> tInterface, Object object) {
this.removeInstance(tInterface);
this.removeProvider(tInterface);
this.getInstances().put(tInterface, object);
}
public <T> void removeInstance(Class<T> tInterface) {
this.getInstances().remove(tInterface);
}
public <T> void replaceProvider(Class<T> tInterface, Class<? extends T> tClazz) {
this.removeInstance(tInterface);
this.removeProvider(tInterface);
this.getProviders().put(tInterface, tClazz);
}
public <T> void removeProvider(Class<T> tInterface) {
this.getProviders().remove(tInterface);
}
}

View File

@ -1,36 +0,0 @@
/**
* 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.inject.Guice;
import com.google.inject.Injector;
/**
* Mock ABFS ServiceProviderImpl.
*/
public final class MockServiceProviderImpl {
public static void create(MockAbfsServiceInjectorImpl abfsServiceInjector) {
Injector injector = Guice.createInjector(abfsServiceInjector);
AbfsServiceProviderImpl.create(injector);
}
private MockServiceProviderImpl() {
// no-op
}
}

View File

@ -47,8 +47,8 @@ import org.junit.Test;
/**
* Test ConfigurationServiceFieldsValidation.
*/
public class TestConfigurationServiceFieldsValidation {
private ConfigurationServiceImpl configService;
public class TestAbfsConfigurationFieldsValidation {
private AbfsConfiguration abfsConfiguration;
private static final String INT_KEY= "intKey";
private static final String LONG_KEY= "longKey";
@ -88,7 +88,7 @@ public class TestConfigurationServiceFieldsValidation {
DefaultValue = false)
private boolean boolField;
public TestConfigurationServiceFieldsValidation() throws Exception {
public TestAbfsConfigurationFieldsValidation() throws Exception {
super();
Base64 base64 = new Base64();
this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
@ -101,7 +101,7 @@ public class TestConfigurationServiceFieldsValidation {
configuration.set(BASE64_KEY, encodedString);
configuration.set(BOOLEAN_KEY, "true");
configuration.set(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + "testaccount1.blob.core.windows.net", this.encodedAccountKey);
configService = new ConfigurationServiceImpl(configuration);
abfsConfiguration = new AbfsConfiguration(configuration);
}
@Test
@ -110,15 +110,15 @@ public class TestConfigurationServiceFieldsValidation {
for (Field field : fields) {
field.setAccessible(true);
if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) {
assertEquals(TEST_INT, configService.validateInt(field));
assertEquals(TEST_INT, abfsConfiguration.validateInt(field));
} else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) {
assertEquals(DEFAULT_LONG, configService.validateLong(field));
assertEquals(DEFAULT_LONG, abfsConfiguration.validateLong(field));
} else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) {
assertEquals("stringValue", configService.validateString(field));
assertEquals("stringValue", abfsConfiguration.validateString(field));
} else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) {
assertEquals(this.encodedString, configService.validateBase64String(field));
assertEquals(this.encodedString, abfsConfiguration.validateBase64String(field));
} else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) {
assertEquals(true, configService.validateBoolean(field));
assertEquals(true, abfsConfiguration.validateBoolean(field));
}
}
}
@ -126,24 +126,24 @@ public class TestConfigurationServiceFieldsValidation {
@Test
public void testConfigServiceImplAnnotatedFieldsInitialized() throws Exception {
// test that all the ConfigurationServiceImpl annotated fields have been initialized in the constructor
assertEquals(DEFAULT_WRITE_BUFFER_SIZE, configService.getWriteBufferSize());
assertEquals(DEFAULT_READ_BUFFER_SIZE, configService.getReadBufferSize());
assertEquals(DEFAULT_MIN_BACKOFF_INTERVAL, configService.getMinBackoffIntervalMilliseconds());
assertEquals(DEFAULT_MAX_BACKOFF_INTERVAL, configService.getMaxBackoffIntervalMilliseconds());
assertEquals(DEFAULT_BACKOFF_INTERVAL, configService.getBackoffIntervalMilliseconds());
assertEquals(DEFAULT_MAX_RETRY_ATTEMPTS, configService.getMaxIoRetries());
assertEquals(MAX_AZURE_BLOCK_SIZE, configService.getAzureBlockSize());
assertEquals(AZURE_BLOCK_LOCATION_HOST_DEFAULT, configService.getAzureBlockLocationHost());
assertEquals(DEFAULT_WRITE_BUFFER_SIZE, abfsConfiguration.getWriteBufferSize());
assertEquals(DEFAULT_READ_BUFFER_SIZE, abfsConfiguration.getReadBufferSize());
assertEquals(DEFAULT_MIN_BACKOFF_INTERVAL, abfsConfiguration.getMinBackoffIntervalMilliseconds());
assertEquals(DEFAULT_MAX_BACKOFF_INTERVAL, abfsConfiguration.getMaxBackoffIntervalMilliseconds());
assertEquals(DEFAULT_BACKOFF_INTERVAL, abfsConfiguration.getBackoffIntervalMilliseconds());
assertEquals(DEFAULT_MAX_RETRY_ATTEMPTS, abfsConfiguration.getMaxIoRetries());
assertEquals(MAX_AZURE_BLOCK_SIZE, abfsConfiguration.getAzureBlockSize());
assertEquals(AZURE_BLOCK_LOCATION_HOST_DEFAULT, abfsConfiguration.getAzureBlockLocationHost());
}
@Test
public void testGetAccountKey() throws Exception {
String accountKey = configService.getStorageAccountKey("testaccount1.blob.core.windows.net");
String accountKey = abfsConfiguration.getStorageAccountKey("testaccount1.blob.core.windows.net");
assertEquals(this.encodedAccountKey, accountKey);
}
@Test (expected = ConfigurationPropertyNotFoundException.class)
public void testGetAccountKeyWithNonExistingAccountName() throws Exception {
configService.getStorageAccountKey("bogusAccountName");
abfsConfiguration.getStorageAccountKey("bogusAccountName");
}
}

View File

@ -0,0 +1,68 @@
/*
* 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.utils;
import com.microsoft.azure.storage.CloudStorageAccount;
import com.microsoft.azure.storage.blob.CloudBlobClient;
import com.microsoft.azure.storage.blob.CloudBlobContainer;
import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
import org.junit.Test;
import static org.apache.hadoop.fs.azurebfs.DependencyInjectedTest.TEST_CONTAINER_PREFIX;
/**
* If unit tests were interrupted and crushed accidentally, the test containers won't be deleted.
* In that case, dev can use this tool to list and delete all test containers.
* By default, all test container used in E2E tests sharing same prefix: "abfs-testcontainer-"
*/
public class CleanUpAbfsTestContainer {
@Test
public void testEnumContainers() throws Throwable {
int count = 0;
CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount();
CloudBlobClient blobClient = storageAccount.createCloudBlobClient();
Iterable<CloudBlobContainer> containers
= blobClient.listContainers(TEST_CONTAINER_PREFIX);
for (CloudBlobContainer container : containers) {
count++;
System.out.println(String.format("Container %s URI %s",
container.getName(),
container.getUri()));
}
System.out.println(String.format("Found %d test containers", count));
}
@Test
public void testDeleteContainers() throws Throwable {
int count = 0;
CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount();
CloudBlobClient blobClient = storageAccount.createCloudBlobClient();
Iterable<CloudBlobContainer> containers
= blobClient.listContainers(TEST_CONTAINER_PREFIX);
for (CloudBlobContainer container : containers) {
System.out.println(String.format("Container %s URI %s",
container.getName(),
container.getUri()));
if (container.deleteIfExists()) {
count++;
}
}
System.out.println(String.format("Deleted %s test containers", count));
}
}