Kaynağa Gözat

HADOOP-19618. Replace AssumptionViolatedException with TestAbortedException. (#7800)

Signed-off-by: Shilun Fan <slfan1989@apache.org>
slfan1989 1 gün önce
ebeveyn
işleme
a5ed640849

+ 2 - 2
hadoop-cloud-storage-project/hadoop-cos/src/test/java/org/apache/hadoop/fs/cosn/CosNTestUtils.java

@@ -23,7 +23,7 @@ import java.net.URI;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.junit.internal.AssumptionViolatedException;
+import org.opentest4j.TestAbortedException;
 
 /**
  * Utilities for the CosN tests.
@@ -52,7 +52,7 @@ public final class CosNTestUtils {
       testUri = URI.create(fsName);
       liveTest = testUri.getScheme().equals(CosNFileSystem.SCHEME);
     } else {
-      throw new AssumptionViolatedException("no test file system in " +
+      throw new TestAbortedException("no test file system in " +
           fsName);
     }
 

+ 4 - 4
hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestUtils.java

@@ -21,7 +21,7 @@ package org.apache.hadoop.fs.obs;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.junit.internal.AssumptionViolatedException;
+import org.opentest4j.TestAbortedException;
 
 import java.io.IOException;
 import java.net.URI;
@@ -45,7 +45,7 @@ public final class OBSTestUtils {
    * @param conf configuration
    * @return the FS
    * @throws IOException                 IO Problems
-   * @throws AssumptionViolatedException if the FS is not named
+   * @throws TestAbortedException if the FS is not named
    */
   public static OBSFileSystem createTestFileSystem(Configuration conf)
       throws IOException {
@@ -62,7 +62,7 @@ public final class OBSTestUtils {
    * @param purge flag to enable Multipart purging
    * @return the FS
    * @throws IOException                 IO Problems
-   * @throws AssumptionViolatedException if the FS is not named
+   * @throws TestAbortedException if the FS is not named
    */
   @SuppressWarnings("deprecation")
   public static OBSFileSystem createTestFileSystem(Configuration conf,
@@ -80,7 +80,7 @@ public final class OBSTestUtils {
     if (!liveTest) {
       // This doesn't work with our JUnit 3 style test cases, so instead we'll
       // make this whole class not run by default
-      throw new AssumptionViolatedException(
+      throw new TestAbortedException(
           "No test filesystem in " + TEST_FS_OBS_NAME);
     }
     OBSFileSystem fs1 = new OBSFileSystem();

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractCreateTest.java

@@ -28,7 +28,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StreamCapabilities;
 
 import org.junit.jupiter.api.Test;
-import org.junit.AssumptionViolatedException;
+import org.opentest4j.TestAbortedException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -172,7 +172,7 @@ public abstract class AbstractContractCreateTest extends
     } catch (AssertionError failure) {
       if (isSupported(CREATE_OVERWRITES_DIRECTORY)) {
         // file/directory hack surfaces here
-        throw new AssumptionViolatedException(failure.toString(), failure);
+        throw new TestAbortedException(failure.toString(), failure);
       }
       // else: rethrow
       throw failure;

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java

@@ -28,8 +28,8 @@ import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Timeout;
-import org.junit.AssumptionViolatedException;
 import org.junit.jupiter.api.extension.RegisterExtension;
+import org.opentest4j.TestAbortedException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -149,7 +149,7 @@ public abstract class AbstractFSContractTestBase extends Assertions
    */
   protected void assumeEnabled() {
     if (!contract.isEnabled())
-      throw new AssumptionViolatedException("test cases disabled for " + contract);
+      throw new TestAbortedException("test cases disabled for " + contract);
   }
 
   /**

+ 7 - 7
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java

@@ -37,7 +37,7 @@ import org.apache.hadoop.util.functional.RemoteIterators;
 import org.apache.hadoop.util.functional.FutureIO;
 
 import org.junit.jupiter.api.Assertions;
-import org.junit.AssumptionViolatedException;
+import org.opentest4j.TestAbortedException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -565,19 +565,19 @@ public class ContractTestUtils extends Assertions {
    * exception for the Junit test runner to mark as failed.
    * @param message text message
    * @param failure what failed
-   * @throws AssumptionViolatedException always
+   * @throws TestAbortedException always
    */
   public static void downgrade(String message, Throwable failure) {
     LOG.warn("Downgrading test " + message, failure);
-    AssumptionViolatedException ave =
-        new AssumptionViolatedException(failure, null);
+    TestAbortedException ave =
+        new TestAbortedException(null, failure);
     throw ave;
   }
 
   /**
    * report an overridden test as unsupported.
    * @param message message to use in the text
-   * @throws AssumptionViolatedException always
+   * @throws TestAbortedException always
    */
   public static void unsupported(String message) {
     skip(message);
@@ -586,11 +586,11 @@ public class ContractTestUtils extends Assertions {
   /**
    * report a test has been skipped for some reason.
    * @param message message to use in the text
-   * @throws AssumptionViolatedException always
+   * @throws TestAbortedException always
    */
   public static void skip(String message) {
     LOG.info("Skipping: {}", message);
-    throw new AssumptionViolatedException(message);
+    throw new TestAbortedException(message);
   }
 
   /**

+ 3 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/PlatformAssumptions.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.test;
 
-import org.junit.AssumptionViolatedException;
+import org.opentest4j.TestAbortedException;
 
 /**
  * JUnit assumptions for the environment (OS).
@@ -34,13 +34,13 @@ public final class PlatformAssumptions {
 
   public static void assumeNotWindows(String message) {
     if (WINDOWS) {
-      throw new AssumptionViolatedException(message);
+      throw new TestAbortedException(message);
     }
   }
 
   public static void assumeWindows() {
     if (!WINDOWS) {
-      throw new AssumptionViolatedException(
+      throw new TestAbortedException(
           "Expected Windows platform but got " + OS_NAME);
     }
   }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataDirs.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.util.Shell;
-import org.junit.AssumptionViolatedException;
+import org.opentest4j.TestAbortedException;
 import org.junit.Test;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
@@ -104,7 +104,7 @@ public class TestDataDirs {
   @Test
   public void testDataDirFileSystem() throws Exception {
     if (Shell.MAC) {
-      throw new AssumptionViolatedException("Not supported on MAC OS");
+      throw new TestAbortedException("Not supported on MAC OS");
     }
     Configuration conf = new Configuration();
     String archiveDir = "/home";

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTimeouts.java

@@ -52,8 +52,8 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.jupiter.api.AfterEach;
-import org.junit.AssumptionViolatedException;
 import org.junit.jupiter.api.Timeout;
+import org.opentest4j.TestAbortedException;
 
 /**
  * This test suite checks that WebHdfsFileSystem sets connection timeouts and
@@ -408,7 +408,7 @@ public class TestWebHdfsTimeouts {
 
   private void assumeBacklogConsumed() {
     if (failedToConsumeBacklog) {
-      throw new AssumptionViolatedException(
+      throw new TestAbortedException(
           "failed to fill up connection backlog.");
     }
   }

+ 2 - 2
hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSTestUtils.java

@@ -21,7 +21,7 @@ package org.apache.hadoop.fs.aliyun.oss;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
-import org.junit.internal.AssumptionViolatedException;
+import org.opentest4j.TestAbortedException;
 
 import java.io.IOException;
 import java.net.URI;
@@ -68,7 +68,7 @@ public final class AliyunOSSTestUtils {
     }
 
     if (!liveTest) {
-      throw new AssumptionViolatedException("No test filesystem in "
+      throw new TestAbortedException("No test filesystem in "
           + TestAliyunOSSFileSystemContract.TEST_FS_OSS_NAME);
     }
     return testURI;

+ 4 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java

@@ -71,7 +71,7 @@ import org.apache.hadoop.util.functional.FutureIO;
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assumptions;
 import org.junit.Assume;
-import org.junit.AssumptionViolatedException;
+import org.opentest4j.TestAbortedException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
@@ -201,7 +201,7 @@ public final class S3ATestUtils {
    * @param conf configuration
    * @return the FS
    * @throws IOException IO Problems
-   * @throws AssumptionViolatedException if the FS is not named
+   * @throws TestAbortedException if the FS is not named
    */
   public static S3AFileSystem createTestFileSystem(Configuration conf)
       throws IOException {
@@ -1470,7 +1470,7 @@ public final class S3ATestUtils {
 
   /**
    * Assume that a condition is met. If not: log at WARN and
-   * then throw an {@link AssumptionViolatedException}.
+   * then throw an {@link TestAbortedException}.
    * @param message
    * @param condition
    */
@@ -1488,7 +1488,7 @@ public final class S3ATestUtils {
    * @param t thrown exception.
    */
   public static void raiseAsAssumption(Throwable t) {
-    throw new AssumptionViolatedException(t.toString(), t);
+    throw new TestAbortedException(t.toString(), t);
   }
 
   /**

+ 6 - 6
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestUtils.java

@@ -28,7 +28,7 @@ import java.util.List;
 
 import org.junit.jupiter.api.Assertions;
 import org.junit.Assume;
-import org.junit.internal.AssumptionViolatedException;
+import org.opentest4j.TestAbortedException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -75,7 +75,7 @@ public final class AzureTestUtils extends Assertions {
    * @param conf configuration
    * @return the FS
    * @throws IOException IO Problems
-   * @throws AssumptionViolatedException if the FS is not named
+   * @throws TestAbortedException if the FS is not named
    */
   public static NativeAzureFileSystem createTestFileSystem(Configuration conf)
       throws IOException {
@@ -90,7 +90,7 @@ public final class AzureTestUtils extends Assertions {
     }
     if (!liveTest) {
       // Skip the test
-      throw new AssumptionViolatedException(
+      throw new TestAbortedException(
           "No test filesystem in " + TEST_FS_WASB_NAME);
     }
     NativeAzureFileSystem fs1 = new NativeAzureFileSystem();
@@ -108,7 +108,7 @@ public final class AzureTestUtils extends Assertions {
    * @param conf configuration
    * @return the FS
    * @throws IOException IO Problems
-   * @throws AssumptionViolatedException if the FS is not named
+   * @throws TestAbortedException if the FS is not named
    */
   public static FileContext createTestFileContext(Configuration conf)
       throws IOException {
@@ -123,7 +123,7 @@ public final class AzureTestUtils extends Assertions {
     if (!liveTest) {
       // This doesn't work with our JUnit 3 style test cases, so instead we'll
       // make this whole class not run by default
-      throw new AssumptionViolatedException("No test filesystem in "
+      throw new TestAbortedException("No test filesystem in "
           + TEST_FS_WASB_NAME);
     }
     FileContext fc = FileContext.getFileContext(testURI, conf);
@@ -386,7 +386,7 @@ public final class AzureTestUtils extends Assertions {
 
   /**
    * Assume that a condition is met. If not: log at WARN and
-   * then throw an {@link AssumptionViolatedException}.
+   * then throw an {@link TestAbortedException}.
    * @param message message in an assumption
    * @param condition condition to probe
    */

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

@@ -28,7 +28,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.assertj.core.api.Assertions;
 import org.junit.Assume;
-import org.junit.AssumptionViolatedException;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -47,6 +46,7 @@ import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Preconditions;
+import org.opentest4j.TestAbortedException;
 
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_CORRELATIONID;
@@ -181,7 +181,7 @@ public class TestTracingContext extends AbstractAbfsIntegrationTest {
         testClasses.get(testClass).invoke(testClass);
         testClass.teardown();
       } catch (InvocationTargetException e) {
-        if (!(e.getCause() instanceof AssumptionViolatedException)) {
+        if (!(e.getCause() instanceof TestAbortedException)) {
           throw new IOException(testClasses.get(testClass).getName()
               + " failed tracing context validation test");
         }