You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by tm...@apache.org on 2018/08/11 05:37:51 UTC

[47/50] [abbrv] hadoop git commit: HADOOP-15560. ABFS: removed dependency injection and unnecessary dependencies. Contributed by Da Zhou.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
index e148a05..9f3b4a7 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
+++ b/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")));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java
deleted file mode 100644
index 588df20..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java
+++ /dev/null
@@ -1,122 +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.Hashtable;
-
-import org.junit.Assert;
-import org.junit.Ignore;
-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.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.
- */
-public class ITestAbfsHttpServiceImpl extends DependencyInjectedTest {
-  private static final int TEST_DATA = 100;
-  private static final Path TEST_PATH = new Path("/testfile");
-  public ITestAbfsHttpServiceImpl() {
-    super();
-  }
-
-  @Test
-  public void testReadWriteBytesToFileAndEnsureThreadPoolCleanup() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    testWriteOneByteToFileAndEnsureThreadPoolCleanup();
-
-    FSDataInputStream inputStream = fs.open(TEST_PATH, 4 * 1024 * 1024);
-    int i = inputStream.read();
-
-    assertEquals(TEST_DATA, i);
-  }
-
-  @Test
-  public void testWriteOneByteToFileAndEnsureThreadPoolCleanup() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    FSDataOutputStream stream = fs.create(TEST_PATH);
-
-    stream.write(TEST_DATA);
-    stream.close();
-
-    FileStatus fileStatus = fs.getFileStatus(TEST_PATH);
-    assertEquals(1, fileStatus.getLen());
-  }
-
-  @Test
-  @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);
-
-    Assert.assertEquals(properties, fetchedProperties);
-  }
-
-  @Test
-  public void testBase64PathProperties() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    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);
-
-    Assert.assertEquals(properties, fetchedProperties);
-  }
-
-  @Test (expected = Exception.class)
-  public void testBase64InvalidFileSystemProperties() 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);
-
-    Assert.assertEquals(properties, fetchedProperties);
-  }
-
-  @Test (expected = Exception.class)
-  public void testBase64InvalidPathProperties() throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    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);
-
-    Assert.assertEquals(properties, fetchedProperties);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
index 1db93cb..cabe049 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
+++ b/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);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java
deleted file mode 100644
index e90eb22..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java
+++ /dev/null
@@ -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));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java
deleted file mode 100644
index 2f27b16..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java
+++ /dev/null
@@ -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;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java
deleted file mode 100644
index ff2fb2a..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java
+++ /dev/null
@@ -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);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java
deleted file mode 100644
index 5992f75..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java
+++ /dev/null
@@ -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
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
new file mode 100644
index 0000000..441a35a
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
@@ -0,0 +1,149 @@
+/**
+ * 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.lang.reflect.Field;
+
+import org.apache.commons.codec.Charsets;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT;
+
+import org.apache.commons.codec.binary.Base64;
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+
+/**
+ * Test ConfigurationServiceFieldsValidation.
+ */
+public class TestAbfsConfigurationFieldsValidation {
+  private AbfsConfiguration abfsConfiguration;
+
+  private static final String INT_KEY= "intKey";
+  private static final String LONG_KEY= "longKey";
+  private static final String STRING_KEY= "stringKey";
+  private static final String BASE64_KEY= "base64Key";
+  private static final String BOOLEAN_KEY= "booleanKey";
+  private static final int DEFAULT_INT = 4194304;
+  private static final int DEFAULT_LONG = 4194304;
+
+  private static final int TEST_INT = 1234565;
+  private static final int TEST_LONG = 4194304;
+
+  private final String encodedString;
+  private final String encodedAccountKey;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = INT_KEY,
+      MinValue = Integer.MIN_VALUE,
+      MaxValue = Integer.MAX_VALUE,
+      DefaultValue = DEFAULT_INT)
+  private int intField;
+
+  @LongConfigurationValidatorAnnotation(ConfigurationKey = LONG_KEY,
+      MinValue = Long.MIN_VALUE,
+      MaxValue = Long.MAX_VALUE,
+      DefaultValue = DEFAULT_LONG)
+  private int longField;
+
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = STRING_KEY,
+  DefaultValue = "default")
+  private String stringField;
+
+  @Base64StringConfigurationValidatorAnnotation(ConfigurationKey = BASE64_KEY,
+  DefaultValue = "base64")
+  private String base64Field;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = BOOLEAN_KEY,
+  DefaultValue = false)
+  private boolean boolField;
+
+  public TestAbfsConfigurationFieldsValidation() throws Exception {
+    super();
+    Base64 base64 = new Base64();
+    this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
+    this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
+    Configuration configuration = new Configuration();
+    configuration.addResource("azure-bfs-test.xml");
+    configuration.set(INT_KEY, "1234565");
+    configuration.set(LONG_KEY, "4194304");
+    configuration.set(STRING_KEY, "stringValue");
+    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);
+    abfsConfiguration = new AbfsConfiguration(configuration);
+  }
+
+  @Test
+  public void testValidateFunctionsInConfigServiceImpl() throws Exception {
+    Field[] fields = this.getClass().getDeclaredFields();
+    for (Field field : fields) {
+      field.setAccessible(true);
+      if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) {
+        assertEquals(TEST_INT, abfsConfiguration.validateInt(field));
+      } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) {
+        assertEquals(DEFAULT_LONG, abfsConfiguration.validateLong(field));
+      } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) {
+        assertEquals("stringValue", abfsConfiguration.validateString(field));
+      } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) {
+        assertEquals(this.encodedString, abfsConfiguration.validateBase64String(field));
+      } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) {
+        assertEquals(true, abfsConfiguration.validateBoolean(field));
+      }
+    }
+  }
+
+  @Test
+  public void testConfigServiceImplAnnotatedFieldsInitialized() throws Exception {
+    // test that all the ConfigurationServiceImpl annotated fields have been initialized in the constructor
+    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 = abfsConfiguration.getStorageAccountKey("testaccount1.blob.core.windows.net");
+    assertEquals(this.encodedAccountKey, accountKey);
+  }
+
+  @Test (expected = ConfigurationPropertyNotFoundException.class)
+  public void testGetAccountKeyWithNonExistingAccountName() throws Exception {
+    abfsConfiguration.getStorageAccountKey("bogusAccountName");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java
deleted file mode 100644
index 688c522..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java
+++ /dev/null
@@ -1,149 +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.lang.reflect.Field;
-
-import org.apache.commons.codec.Charsets;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
-
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT;
-
-import org.apache.commons.codec.binary.Base64;
-import static org.junit.Assert.assertEquals;
-
-import org.junit.Test;
-
-/**
- * Test ConfigurationServiceFieldsValidation.
- */
-public class TestConfigurationServiceFieldsValidation  {
-  private ConfigurationServiceImpl configService;
-
-  private static final String INT_KEY= "intKey";
-  private static final String LONG_KEY= "longKey";
-  private static final String STRING_KEY= "stringKey";
-  private static final String BASE64_KEY= "base64Key";
-  private static final String BOOLEAN_KEY= "booleanKey";
-  private static final int DEFAULT_INT = 4194304;
-  private static final int DEFAULT_LONG = 4194304;
-
-  private static final int TEST_INT = 1234565;
-  private static final int TEST_LONG = 4194304;
-
-  private final String encodedString;
-  private final String encodedAccountKey;
-
-  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = INT_KEY,
-      MinValue = Integer.MIN_VALUE,
-      MaxValue = Integer.MAX_VALUE,
-      DefaultValue = DEFAULT_INT)
-  private int intField;
-
-  @LongConfigurationValidatorAnnotation(ConfigurationKey = LONG_KEY,
-      MinValue = Long.MIN_VALUE,
-      MaxValue = Long.MAX_VALUE,
-      DefaultValue = DEFAULT_LONG)
-  private int longField;
-
-  @StringConfigurationValidatorAnnotation(ConfigurationKey = STRING_KEY,
-  DefaultValue = "default")
-  private String stringField;
-
-  @Base64StringConfigurationValidatorAnnotation(ConfigurationKey = BASE64_KEY,
-  DefaultValue = "base64")
-  private String base64Field;
-
-  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = BOOLEAN_KEY,
-  DefaultValue = false)
-  private boolean boolField;
-
-  public TestConfigurationServiceFieldsValidation() throws Exception {
-    super();
-    Base64 base64 = new Base64();
-    this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
-    this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
-    Configuration configuration = new Configuration();
-    configuration.addResource("azure-bfs-test.xml");
-    configuration.set(INT_KEY, "1234565");
-    configuration.set(LONG_KEY, "4194304");
-    configuration.set(STRING_KEY, "stringValue");
-    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);
-  }
-
-  @Test
-  public void testValidateFunctionsInConfigServiceImpl() throws Exception {
-    Field[] fields = this.getClass().getDeclaredFields();
-    for (Field field : fields) {
-      field.setAccessible(true);
-      if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) {
-        assertEquals(TEST_INT, configService.validateInt(field));
-      } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) {
-        assertEquals(DEFAULT_LONG, configService.validateLong(field));
-      } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) {
-        assertEquals("stringValue", configService.validateString(field));
-      } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) {
-        assertEquals(this.encodedString, configService.validateBase64String(field));
-      } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) {
-        assertEquals(true, configService.validateBoolean(field));
-      }
-    }
-  }
-
-  @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());
-  }
-
-  @Test
-  public void testGetAccountKey() throws Exception {
-    String accountKey = configService.getStorageAccountKey("testaccount1.blob.core.windows.net");
-    assertEquals(this.encodedAccountKey, accountKey);
-  }
-
-  @Test (expected = ConfigurationPropertyNotFoundException.class)
-  public void testGetAccountKeyWithNonExistingAccountName() throws Exception {
-    configService.getStorageAccountKey("bogusAccountName");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62ea8ad9/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java
new file mode 100644
index 0000000..2716bff
--- /dev/null
+++ b/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));
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org