diff --git a/common/network-common/src/test/java/org/apache/spark/network/protocol/EncryptedMessageWithHeaderSuite.java b/common/network-common/src/test/java/org/apache/spark/network/protocol/EncryptedMessageWithHeaderSuite.java index 7478fa1db7113..2865d411bf673 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/protocol/EncryptedMessageWithHeaderSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/protocol/EncryptedMessageWithHeaderSuite.java @@ -116,7 +116,7 @@ public void testChunkedStream() throws Exception { // Validate we read data correctly assertEquals(bodyResult.readableBytes(), chunkSize); - assert(bodyResult.readableBytes() < (randomData.length - readIndex)); + assertTrue(bodyResult.readableBytes() < (randomData.length - readIndex)); while (bodyResult.readableBytes() > 0) { assertEquals(bodyResult.readByte(), randomData[readIndex++]); } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockTransferorSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockTransferorSuite.java index 3725973ae7333..84c8b1b3353f2 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockTransferorSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockTransferorSuite.java @@ -288,7 +288,7 @@ public void testRetryOnSaslTimeout() throws IOException, InterruptedException { verify(listener, timeout(5000)).onBlockTransferSuccess("b0", block0); verify(listener).getTransferType(); verifyNoMoreInteractions(listener); - assert(_retryingBlockTransferor.getRetryCount() == 0); + assertEquals(0, _retryingBlockTransferor.getRetryCount()); } @Test @@ -310,7 +310,7 @@ public void testRepeatedSaslRetryFailures() throws IOException, InterruptedExcep verify(listener, timeout(5000)).onBlockTransferFailure("b0", saslTimeoutException); verify(listener, times(3)).getTransferType(); verifyNoMoreInteractions(listener); - assert(_retryingBlockTransferor.getRetryCount() == MAX_RETRIES); + assertEquals(MAX_RETRIES, _retryingBlockTransferor.getRetryCount()); } @Test @@ -339,7 +339,7 @@ public void testBlockTransferFailureAfterSasl() throws IOException, InterruptedE // This should be equal to 1 because after the SASL exception is retried, // retryCount should be set back to 0. Then after that b1 encounters an // exception that is retried. - assert(_retryingBlockTransferor.getRetryCount() == 1); + assertEquals(1, _retryingBlockTransferor.getRetryCount()); } @Test @@ -368,7 +368,7 @@ public void testIOExceptionFailsConnectionEvenWithSaslException() verify(listener, timeout(5000)).onBlockTransferFailure("b0", saslExceptionFinal); verify(listener, atLeastOnce()).getTransferType(); verifyNoMoreInteractions(listener); - assert(_retryingBlockTransferor.getRetryCount() == MAX_RETRIES); + assertEquals(MAX_RETRIES, _retryingBlockTransferor.getRetryCount()); } @Test diff --git a/common/utils/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java b/common/utils/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java index 46bfe3415080d..0869f9827324d 100644 --- a/common/utils/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java +++ b/common/utils/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java @@ -30,6 +30,8 @@ import org.apache.spark.internal.LogKeys; import org.apache.spark.internal.MDC; +import static org.junit.jupiter.api.Assertions.assertTrue; + public abstract class SparkLoggerSuiteBase { abstract SparkLogger logger(); @@ -104,8 +106,8 @@ public void testBasicMsgLogger() { Pair.of(Level.DEBUG, debugFn), Pair.of(Level.TRACE, traceFn)).forEach(pair -> { try { - assert (captureLogOutput(pair.getRight()).matches( - expectedPatternForBasicMsg(pair.getLeft()))); + assertTrue(captureLogOutput(pair.getRight()).matches( + expectedPatternForBasicMsg(pair.getLeft()))); } catch (IOException e) { throw new RuntimeException(e); } @@ -127,8 +129,8 @@ public void testBasicLoggerWithException() { Pair.of(Level.DEBUG, debugFn), Pair.of(Level.TRACE, traceFn)).forEach(pair -> { try { - assert (captureLogOutput(pair.getRight()).matches( - expectedPatternForBasicMsgWithException(pair.getLeft()))); + assertTrue(captureLogOutput(pair.getRight()).matches( + expectedPatternForBasicMsgWithException(pair.getLeft()))); } catch (IOException e) { throw new RuntimeException(e); } @@ -147,8 +149,8 @@ public void testLoggerWithMDC() { Pair.of(Level.WARN, warnFn), Pair.of(Level.INFO, infoFn)).forEach(pair -> { try { - assert (captureLogOutput(pair.getRight()).matches( - expectedPatternForMsgWithMDC(pair.getLeft()))); + assertTrue(captureLogOutput(pair.getRight()).matches( + expectedPatternForMsgWithMDC(pair.getLeft()))); } catch (IOException e) { throw new RuntimeException(e); } @@ -165,8 +167,8 @@ public void testLoggerWithMDCs() { Pair.of(Level.WARN, warnFn), Pair.of(Level.INFO, infoFn)).forEach(pair -> { try { - assert (captureLogOutput(pair.getRight()).matches( - expectedPatternForMsgWithMDCs(pair.getLeft()))); + assertTrue(captureLogOutput(pair.getRight()).matches( + expectedPatternForMsgWithMDCs(pair.getLeft()))); } catch (IOException e) { throw new RuntimeException(e); } @@ -184,8 +186,8 @@ public void testLoggerWithMDCsAndException() { Pair.of(Level.WARN, warnFn), Pair.of(Level.INFO, infoFn)).forEach(pair -> { try { - assert (captureLogOutput(pair.getRight()).matches( - expectedPatternForMsgWithMDCsAndException(pair.getLeft()))); + assertTrue(captureLogOutput(pair.getRight()).matches( + expectedPatternForMsgWithMDCsAndException(pair.getLeft()))); } catch (IOException e) { throw new RuntimeException(e); } @@ -202,8 +204,8 @@ public void testLoggerWithMDCValueIsNull() { Pair.of(Level.WARN, warnFn), Pair.of(Level.INFO, infoFn)).forEach(pair -> { try { - assert (captureLogOutput(pair.getRight()).matches( - expectedPatternForMsgWithMDCValueIsNull(pair.getLeft()))); + assertTrue(captureLogOutput(pair.getRight()).matches( + expectedPatternForMsgWithMDCValueIsNull(pair.getLeft()))); } catch (IOException e) { throw new RuntimeException(e); } @@ -220,8 +222,8 @@ public void testLoggerWithExternalSystemCustomLogKey() { Pair.of(Level.WARN, warnFn), Pair.of(Level.INFO, infoFn)).forEach(pair -> { try { - assert (captureLogOutput(pair.getRight()).matches( - expectedPatternForExternalSystemCustomLogKey(pair.getLeft()))); + assertTrue(captureLogOutput(pair.getRight()).matches( + expectedPatternForExternalSystemCustomLogKey(pair.getLeft()))); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/TestStatefulProcessor.java b/sql/core/src/test/java/test/org/apache/spark/sql/TestStatefulProcessor.java index e53e977da1494..b9841ee0f9735 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/TestStatefulProcessor.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/TestStatefulProcessor.java @@ -24,6 +24,8 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.streaming.*; +import static org.junit.jupiter.api.Assertions.*; + /** * A test stateful processor used with transformWithState arbitrary stateful operator in * Structured Streaming. The processor primarily aims to test various functionality of the Java API @@ -74,7 +76,7 @@ public scala.collection.Iterator handleInputRows( } else { keyCountMap.updateValue(value, 1L); } - assert(keyCountMap.containsKey(value)); + assertTrue(keyCountMap.containsKey(value)); keysList.appendValue(value); sb.append(value); } @@ -82,13 +84,13 @@ public scala.collection.Iterator handleInputRows( scala.collection.Iterator keys = keysList.get(); while (keys.hasNext()) { String keyVal = keys.next(); - assert(keyCountMap.containsKey(keyVal)); - assert(keyCountMap.getValue(keyVal) > 0); + assertTrue(keyCountMap.containsKey(keyVal)); + assertTrue(keyCountMap.getValue(keyVal) > 0); } count += numRows; countState.update(count); - assert (countState.get() == count); + assertEquals(count, (long) countState.get()); result.add(sb.toString()); } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/TestStatefulProcessorWithInitialState.java b/sql/core/src/test/java/test/org/apache/spark/sql/TestStatefulProcessorWithInitialState.java index bfa542e81e354..55046a7c0d3df 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/TestStatefulProcessorWithInitialState.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/TestStatefulProcessorWithInitialState.java @@ -24,6 +24,8 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.streaming.*; +import static org.junit.jupiter.api.Assertions.assertFalse; + /** * A test stateful processor concatenates all input rows for a key and emits the result. * Primarily used for testing the Java API for arbitrary stateful operator in structured streaming @@ -71,7 +73,7 @@ public scala.collection.Iterator handleInputRows( } testState.clear(); - assert(testState.exists() == false); + assertFalse(testState.exists()); result.add(sb.toString()); } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaAdvancedDataSourceV2WithV2Filter.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaAdvancedDataSourceV2WithV2Filter.java index 0e3f6aed3b681..07bef16cdf2da 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaAdvancedDataSourceV2WithV2Filter.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaAdvancedDataSourceV2WithV2Filter.java @@ -34,6 +34,8 @@ import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; + public class JavaAdvancedDataSourceV2WithV2Filter implements TestingV2Source { @Override @@ -66,9 +68,9 @@ public StructType readSchema() { public Predicate[] pushPredicates(Predicate[] predicates) { Predicate[] supported = Arrays.stream(predicates).filter(f -> { if (f.name().equals(">")) { - assert(f.children()[0] instanceof FieldReference); + assertInstanceOf(FieldReference.class, f.children()[0]); FieldReference column = (FieldReference) f.children()[0]; - assert(f.children()[1] instanceof LiteralValue); + assertInstanceOf(LiteralValue.class, f.children()[1]); Literal value = (Literal) f.children()[1]; return column.describe().equals("i") && value.value() instanceof Integer; } else { @@ -78,9 +80,9 @@ public Predicate[] pushPredicates(Predicate[] predicates) { Predicate[] unsupported = Arrays.stream(predicates).filter(f -> { if (f.name().equals(">")) { - assert(f.children()[0] instanceof FieldReference); + assertInstanceOf(FieldReference.class, f.children()[0]); FieldReference column = (FieldReference) f.children()[0]; - assert(f.children()[1] instanceof LiteralValue); + assertInstanceOf(LiteralValue.class, f.children()[1]); Literal value = (LiteralValue) f.children()[1]; return !column.describe().equals("i") || !(value.value() instanceof Integer); } else { @@ -125,9 +127,9 @@ public InputPartition[] planInputPartitions() { Integer lowerBound = null; for (Predicate predicate : predicates) { if (predicate.name().equals(">")) { - assert(predicate.children()[0] instanceof FieldReference); + assertInstanceOf(FieldReference.class, predicate.children()[0]); FieldReference column = (FieldReference) predicate.children()[0]; - assert(predicate.children()[1] instanceof LiteralValue); + assertInstanceOf(LiteralValue.class, predicate.children()[1]); Literal value = (Literal) predicate.children()[1]; if ("i".equals(column.describe()) && value.value() instanceof Integer integer) { lowerBound = integer;