This is an automated email from the ASF dual-hosted git repository.

gurwls223 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 4569bf63659 [SPARK-40731][DSTREAM] Make `streaming` pass on Java 21
4569bf63659 is described below

commit 4569bf63659f98fa509ae0f7906b8a18bf32f0e9
Author: yangjie01 <[email protected]>
AuthorDate: Mon Jul 3 10:08:34 2023 +0900

    [SPARK-40731][DSTREAM] Make `streaming` pass on Java 21
    
    ### What changes were proposed in this pull request?
    This pr use use a 0 size `ByteBuffer` instead of `mock[ByteBuffer]` for 
Java 17+ in `BatchedWriteAheadLogSuite`  due to mockito 4 can't mock/spy 
`sealed class` now.
    
    ### Why are the changes needed?
    Make `streaming` pass on Java 21
    
    ### Does this PR introduce _any_ user-facing change?
    No
    
    ### How was this patch tested?
    - Pass GitHub Actions
    - manual test
    
    ```
    java -version
    openjdk version "21-ea" 2023-09-19
    OpenJDK Runtime Environment Zulu21+65-CA (build 21-ea+26)
    OpenJDK 64-Bit Server VM Zulu21+65-CA (build 21-ea+26, mixed mode, sharing)
    ```
    
    ```
    build/sbt "streaming/test"
    ```
    
    **Before**
    
    ```
    [info] - BatchedWriteAheadLog - failures in wrappedLog get bubbled up *** 
FAILED *** (3 milliseconds)
    [info]   Expected exception org.apache.spark.SparkException to be thrown, 
but org.mockito.exceptions.base.MockitoException was thrown 
(WriteAheadLogSuite.scala:478)
    [info]   org.scalatest.exceptions.TestFailedException:
    ...
    [info]   Cause: org.mockito.exceptions.base.MockitoException: Cannot 
mock/spy class java.nio.ByteBuffer
    [info] Mockito cannot mock/spy because :
    [info]  - sealed class
    [info]   at 
org.scalatestplus.mockito.MockitoSugar.mock(MockitoSugar.scala:73)
    [info]   at 
org.scalatestplus.mockito.MockitoSugar.mock$(MockitoSugar.scala:72)
    [info]   at 
org.apache.spark.streaming.util.BatchedWriteAheadLogSuite.mock(WriteAheadLogSuite.scala:420)
    [info]   at 
org.apache.spark.streaming.util.BatchedWriteAheadLogSuite.$anonfun$new$38(WriteAheadLogSuite.scala:479)
    ...
    [info] - BatchedWriteAheadLog - name log with the highest timestamp of 
aggregated entries (6 milliseconds)
    [info] - BatchedWriteAheadLog - shutdown properly *** FAILED *** (1 
millisecond)
    [info]   Expected exception java.lang.IllegalStateException to be thrown, 
but org.mockito.exceptions.base.MockitoException was thrown 
(WriteAheadLogSuite.scala:549)
    [info]   org.scalatest.exceptions.TestFailedException:
    ...
    [info]   Cause: org.mockito.exceptions.base.MockitoException: Cannot 
mock/spy class java.nio.ByteBuffer
    [info] Mockito cannot mock/spy because :
    [info]  - sealed class
    [info]   at 
org.scalatestplus.mockito.MockitoSugar.mock(MockitoSugar.scala:73)
    [info]   at 
org.scalatestplus.mockito.MockitoSugar.mock$(MockitoSugar.scala:72)
    [info]   at 
org.apache.spark.streaming.util.BatchedWriteAheadLogSuite.mock(WriteAheadLogSuite.scala:420)
    [info]   at 
org.apache.spark.streaming.util.BatchedWriteAheadLogSuite.$anonfun$new$46(WriteAheadLogSuite.scala:549)
    ...
    [info] Run completed in 4 minutes, 41 seconds.
    [info] Total number of tests run: 340
    [info] Suites: completed 40, aborted 0
    [info] Tests: succeeded 338, failed 2, canceled 0, ignored 1, pending 0
    [info] *** 2 TESTS FAILED ***
    [error] Failed: Total 440, Failed 2, Errors 0, Passed 438, Ignored 1
    [error] Failed tests:
    [error]         org.apache.spark.streaming.util.BatchedWriteAheadLogSuite
    ```
    
    **After**
    
    ```
    [info] Run completed in 4 minutes, 34 seconds.
    [info] Total number of tests run: 340
    [info] Suites: completed 40, aborted 0
    [info] Tests: succeeded 340, failed 0, canceled 0, ignored 1, pending 0
    [info] All tests passed.
    [info] Passed: Total 440, Failed 0, Errors 0, Passed 440, Ignored 1
    ```
    
    Closes #41815 from LuciferYang/SPARK-40731.
    
    Authored-by: yangjie01 <[email protected]>
    Signed-off-by: Hyukjin Kwon <[email protected]>
---
 .../spark/streaming/util/WriteAheadLogSuite.scala      | 18 ++++++++++++++++--
 1 file changed, 16 insertions(+), 2 deletions(-)

diff --git 
a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala
 
b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala
index 60e04403937..4d23230e2ea 100644
--- 
a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala
+++ 
b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala
@@ -28,6 +28,7 @@ import scala.concurrent._
 import scala.concurrent.duration._
 import scala.language.implicitConversions
 
+import org.apache.commons.lang3.{JavaVersion, SystemUtils}
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 import org.mockito.ArgumentCaptor
@@ -476,7 +477,13 @@ class BatchedWriteAheadLogSuite extends 
CommonWriteAheadLogTests(
     val batchedWal = new BatchedWriteAheadLog(wal, sparkConf)
 
     val e = intercept[SparkException] {
-      val buffer = mock[ByteBuffer]
+      val buffer = if (SystemUtils.isJavaVersionAtMost(JavaVersion.JAVA_17)) {
+        mock[ByteBuffer]
+      } else {
+        // SPARK-40731: Use a 0 size `ByteBuffer` instead of `mock[ByteBuffer]`
+        // for Java 17+ due to mockito 4 can't mock/spy sealed class
+        ByteBuffer.allocate(0)
+      }
       batchedWal.write(buffer, 2L)
     }
     assert(e.getCause.getMessage === "Hello!")
@@ -546,7 +553,14 @@ class BatchedWriteAheadLogSuite extends 
CommonWriteAheadLogTests(
     batchedWal.close()
     verify(wal, times(1)).close()
 
-    intercept[IllegalStateException](batchedWal.write(mock[ByteBuffer], 12L))
+    val buffer = if (SystemUtils.isJavaVersionAtMost(JavaVersion.JAVA_17)) {
+      mock[ByteBuffer]
+    } else {
+      // SPARK-40731: Use a 0 size `ByteBuffer` instead of `mock[ByteBuffer]`
+      // for Java 17+ due to mockito 4 can't mock/spy sealed class
+      ByteBuffer.allocate(0)
+    }
+    intercept[IllegalStateException](batchedWal.write(buffer, 12L))
   }
 
   test("BatchedWriteAheadLog - fail everything in queue during shutdown") {


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to