Pārlūkot izejas kodu

HADOOP-15560. ABFS: removed dependency injection and unnecessary dependencies.
Contributed by Da Zhou.

Steve Loughran 6 gadi atpakaļ
vecāks
revīzija
a271fd0eca
33 mainītis faili ar 461 papildinājumiem un 1732 dzēšanām
  1. 0 18
      hadoop-tools/hadoop-azure/pom.xml
  2. 1 1
      hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml
  3. 40 48
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
  4. 254 246
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
  5. 0 36
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java
  6. 0 39
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java
  7. 0 162
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java
  8. 0 40
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java
  9. 0 143
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java
  10. 0 30
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java
  11. 0 66
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java
  12. 3 4
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
  13. 5 25
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java
  14. 0 116
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java
  15. 0 81
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java
  16. 0 96
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java
  17. 6 3
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
  18. 0 74
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java
  19. 0 134
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java
  20. 20 39
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java
  21. 1 6
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
  22. 3 4
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
  23. 2 21
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
  24. 24 20
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java
  25. 2 21
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
  26. 10 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
  27. 3 5
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
  28. 0 79
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java
  29. 0 69
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java
  30. 0 50
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java
  31. 0 36
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java
  32. 19 19
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
  33. 68 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java

+ 0 - 18
hadoop-tools/hadoop-azure/pom.xml

@@ -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>

+ 1 - 1
hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml

@@ -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>

+ 40 - 48
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java

@@ -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;
+  }
 }

+ 254 - 246
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java → hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java

@@ -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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+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.TimeoutException;
+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.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.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.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.fs.azurebfs.constants.HttpHeaderConfigurations;
+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);
+  }
+
+  @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;
   }
 
-  @Override
-  public Hashtable<String, String> getFilesystemProperties(final AzureBlobFileSystem azureBlobFileSystem)
-      throws AzureBlobFileSystemException{
-    final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem);
+  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;
     }
   }
-}
+
+
+}

+ 0 - 36
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java

@@ -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);
-  }
-}

+ 0 - 39
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java

@@ -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;
-}

+ 0 - 162
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java

@@ -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);
-}

+ 0 - 40
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java

@@ -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;
-}

+ 0 - 143
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java

@@ -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();
-}

+ 0 - 30
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java

@@ -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 {
-}

+ 0 - 66
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java

@@ -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);
-}

+ 3 - 4
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java

@@ -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();
   }

+ 5 - 25
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java → hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java

@@ -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;
   }

+ 0 - 116
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java

@@ -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());
-  }
-}

+ 0 - 81
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java

@@ -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;
-  }
-}

+ 0 - 96
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java

@@ -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);
-    }
-  }
-}

+ 6 - 3
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java

@@ -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;

+ 0 - 74
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java

@@ -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
-  }
-}

+ 0 - 134
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java

@@ -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();
-    }
-  }
-}

+ 20 - 39
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java

@@ -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);
-
-    if (!(new MockUtil().isMock(abfsHttpService))) {
-      AbfsRestOperationException ex = intercept(
-          AbfsRestOperationException.class,
-          new Callable<Void>() {
-            @Override
-            public Void call() throws Exception {
-              abfsHttpService.getFilesystemProperties(fs);
-              return null;
-            }
-          });
-
-      assertEquals(FILE_SYSTEM_NOT_FOUND.getStatusCode(), ex.getStatusCode());
-    }
+    final AzureBlobFileSystemStore abfsStore = fs.getAbfsStore();
+    abfsStore.deleteFilesystem();
+
+    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());
   }
 
   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() {

+ 1 - 6
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java

@@ -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);

+ 3 - 4
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java

@@ -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();

+ 2 - 21
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java

@@ -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());
   }

+ 24 - 20
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java → hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java

@@ -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);
   }

+ 2 - 21
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java

@@ -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());

+ 10 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java

@@ -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")));

+ 3 - 5
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java

@@ -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);
 

+ 0 - 79
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java

@@ -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));
-  }
-}

+ 0 - 69
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java

@@ -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;
-  }
-}

+ 0 - 50
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java

@@ -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);
-  }
-}

+ 0 - 36
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java

@@ -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
-  }
-}

+ 19 - 19
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java → hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java

@@ -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");
   }
 }

+ 68 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java

@@ -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));
+  }
+}