Browse Source

HADOOP-13459. hadoop-azure runs several test cases repeatedly, causing unnecessarily long running time. Contributed by Chris Nauroth.

(cherry picked from commit 4ef1bc0bdb6b2c907431492e4c8395035fa96c84)
Chris Nauroth 8 years ago
parent
commit
8ec3d702a3

+ 66 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AbstractWasbTestBase.java

@@ -0,0 +1,66 @@
+/**
+ * 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.azure;
+
+import static org.junit.Assume.assumeNotNull;
+
+import org.apache.hadoop.fs.FileSystem;
+
+import org.junit.After;
+import org.junit.Before;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract test class that provides basic setup and teardown of testing Azure
+ * Storage account.  Each subclass defines a different set of test cases to run
+ * and overrides {@link #createTestAccount()} to set up the testing account used
+ * to run those tests.  The returned account might integrate with Azure Storage
+ * directly or it might be a mock implementation.
+ */
+abstract class AbstractWasbTestBase {
+
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(AbstractWasbTestBase.class);
+
+  protected FileSystem fs;
+  private AzureBlobStorageTestAccount testAccount;
+
+  @Before
+  public void setUp() throws Exception {
+    testAccount = createTestAccount();
+    if (testAccount != null) {
+      fs = testAccount.getFileSystem();
+    }
+    assumeNotNull(testAccount);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (testAccount != null) {
+      testAccount.cleanup();
+      testAccount = null;
+      fs = null;
+    }
+  }
+
+  protected abstract AzureBlobStorageTestAccount createTestAccount()
+      throws Exception;
+}

+ 2 - 26
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java

@@ -23,7 +23,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.junit.Assume.assumeNotNull;
 
 import java.io.BufferedReader;
 import java.io.BufferedWriter;
@@ -50,8 +49,6 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 import org.apache.hadoop.fs.azure.AzureException;
 import org.apache.hadoop.fs.azure.NativeAzureFileSystem.FolderRenamePending;
@@ -68,34 +65,13 @@ import com.microsoft.azure.storage.blob.CloudBlob;
  * For hand-testing: remove "abstract" keyword and copy in an implementation of createTestAccount
  * from one of the subclasses
  */
-public abstract class NativeAzureFileSystemBaseTest {
+public abstract class NativeAzureFileSystemBaseTest
+    extends AbstractWasbTestBase {
 
-  protected FileSystem fs;
-  private AzureBlobStorageTestAccount testAccount;
   private final long modifiedTimeErrorMargin = 5 * 1000; // Give it +/-5 seconds
 
-  protected abstract AzureBlobStorageTestAccount createTestAccount() throws Exception;
-
   public static final Log LOG = LogFactory.getLog(NativeAzureFileSystemBaseTest.class);
 
-  @Before
-  public void setUp() throws Exception {
-    testAccount = createTestAccount();
-    if (testAccount != null) {
-      fs = testAccount.getFileSystem();
-    }
-    assumeNotNull(testAccount);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (testAccount != null) {
-      testAccount.cleanup();
-      testAccount = null;
-      fs = null;
-    }
-  }
-
   @Test
   public void testCheckingNonExistentOneLetterFile() throws Exception {
     assertFalse(fs.exists(new Path("/a")));

+ 2 - 2
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationExceptionHandling.java

@@ -31,8 +31,8 @@ import org.junit.Assert;
 import org.junit.Test;
 
 
-public class TestFileSystemOperationExceptionHandling extends
-  NativeAzureFileSystemBaseTest {
+public class TestFileSystemOperationExceptionHandling
+    extends AbstractWasbTestBase {
 
   private FSDataInputStream inputStream = null;
 

+ 3 - 3
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationsExceptionHandlingMultiThreaded.java

@@ -29,8 +29,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.junit.After;
 import org.junit.Test;
 
-public class TestFileSystemOperationsExceptionHandlingMultiThreaded extends
-  NativeAzureFileSystemBaseTest {
+public class TestFileSystemOperationsExceptionHandlingMultiThreaded
+    extends AbstractWasbTestBase {
 
   FSDataInputStream inputStream = null;
 
@@ -326,4 +326,4 @@ class DeleteThread implements Runnable {
       // by the other thread
     }
   }
-}
+}

+ 1 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAppend.java

@@ -32,7 +32,7 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-public class TestNativeAzureFileSystemAppend extends NativeAzureFileSystemBaseTest {
+public class TestNativeAzureFileSystemAppend extends AbstractWasbTestBase {
 
   private static final String TEST_FILE = "test.dat";
   private static final Path TEST_PATH = new Path(TEST_FILE);

+ 2 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAtomicRenameDirList.java

@@ -24,7 +24,8 @@ import java.net.URI;
 import org.apache.hadoop.conf.Configuration;
 import org.junit.Test;
 
-public class TestNativeAzureFileSystemAtomicRenameDirList extends NativeAzureFileSystemBaseTest {
+public class TestNativeAzureFileSystemAtomicRenameDirList
+    extends AbstractWasbTestBase {
   private AzureBlobStorageTestAccount testAccount;
 
   // HBase-site config controlling HBase root dir

+ 3 - 3
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemClientLogging.java

@@ -37,8 +37,8 @@ import org.junit.Test;
  * client-side logging.
  *
  */
-public class TestNativeAzureFileSystemClientLogging extends
-    NativeAzureFileSystemBaseTest {
+public class TestNativeAzureFileSystemClientLogging
+    extends AbstractWasbTestBase {
 
   private AzureBlobStorageTestAccount testAccount;
 
@@ -127,4 +127,4 @@ public class TestNativeAzureFileSystemClientLogging extends
     testAccount = AzureBlobStorageTestAccount.create();
     return testAccount;
   }
-}
+}