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

chengpan 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 cd01c4c6f646 [SPARK-55682][CORE][SQL] ServiceLoader returned iterator 
may throw `NoClassDefFoundError` on `hasNext()`
cd01c4c6f646 is described below

commit cd01c4c6f6462dc558ed42473559be4011347afa
Author: Cheng Pan <[email protected]>
AuthorDate: Wed Feb 25 22:18:56 2026 +0800

    [SPARK-55682][CORE][SQL] ServiceLoader returned iterator may throw 
`NoClassDefFoundError` on `hasNext()`
    
    ### What changes were proposed in this pull request?
    
    JDK 24 introduces a behavior change on `ServiceLoader`'s returned iterator, 
`iterator.hasNext()` might throw
    `NoClassDefFoundError`, which previously only happened on 
`iterator.next()`. See more details at OpenJDK
    bug report [JDK-8350481](https://bugs.openjdk.org/browse/JDK-8350481)
    
    Another issue is even the 
[Javadoc](https://docs.oracle.com/en/java/javase/25/docs/api//java.base/java/util/ServiceLoader.html#iterator())
 of `ServiceLoader` say
    
    > Its hasNext and next methods can therefore throw a 
ServiceConfigurationError for any of the reasons specified
    > in the Errors section above. To write robust code it is only necessary to 
catch ServiceConfigurationError when
    > using the iterator. If an error is thrown then subsequent invocations of 
the iterator will make a best effort
    > to locate and instantiate the next available provider, but in general 
such recovery cannot be guaranteed.
    
    but it actually might throw `NoClassDefFoundError`. See more details at 
OpenJDK bug report [JDK-8196182](https://bugs.openjdk.org/browse/JDK-8196182)
    
    ### Why are the changes needed?
    
    Enable Java 25 support.
    
    ### Does this PR introduce _any_ user-facing change?
    
    No.
    
    ### How was this patch tested?
    
    Changes in `HadoopDelegationTokenManager` are covered by UT, other 2 places 
are not, need to review carefully.
    
    ```
    $ export JAVA_HOME=/path/of/openjdk-25
    $ build/sbt -Phive "hive/testOnly *HiveHadoopDelegationTokenManagerSuite"
    ```
    
    Before
    ```
    [info]   Cause: java.lang.ClassNotFoundException: 
org.apache.hadoop.hive.conf.HiveConf
    [info]   at 
org.apache.spark.sql.hive.security.HiveHadoopDelegationTokenManagerSuite$$anon$1.loadClass(HiveHadoopDelegationTokenManagerSuite.scala:51)
    [info]   at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:490)
    [info]   at java.base/java.lang.Class.getDeclaredConstructors0(Native 
Method)
    [info]   at 
java.base/java.lang.Class.privateGetDeclaredConstructors(Class.java:2985)
    [info]   at java.base/java.lang.Class.getConstructor0(Class.java:3180)
    [info]   at java.base/java.lang.Class.getConstructor(Class.java:2199)
    [info]   at 
java.base/java.util.ServiceLoader.getConstructor(ServiceLoader.java:623)
    [info]   at 
java.base/java.util.ServiceLoader$LazyClassPathLookupIterator.hasNextService(ServiceLoader.java:1111)
    [info]   at 
java.base/java.util.ServiceLoader$LazyClassPathLookupIterator.hasNext(ServiceLoader.java:1142)
    [info]   at 
java.base/java.util.ServiceLoader$1.hasNext(ServiceLoader.java:1164)
    [info]   at 
java.base/java.util.ServiceLoader$2.hasNext(ServiceLoader.java:1246)
    [info]   at 
org.apache.spark.deploy.security.HadoopDelegationTokenManager.loadProviders(HadoopDelegationTokenManager.scala:273)
    [info]   at 
org.apache.spark.deploy.security.HadoopDelegationTokenManager.<init>(HadoopDelegationTokenManager.scala:79)
    ...
    [info] Run completed in 1 second, 186 milliseconds.
    [info] Total number of tests run: 3
    [info] Suites: completed 1, aborted 0
    [info] Tests: succeeded 2, failed 1, canceled 0, ignored 0, pending 0
    [info] *** 1 TEST FAILED ***
    [error] Failed tests:
    [error]         
org.apache.spark.sql.hive.security.HiveHadoopDelegationTokenManagerSuite
    [error] (hive / Test / testOnly) sbt.TestsFailedException: Tests 
unsuccessful
    [error] Total time: 192 s (0:03:12.0), completed Feb 25, 2026, 4:40:21 PM
    ```
    
    After
    ```
    [info] HiveHadoopDelegationTokenManagerSuite:
    [info] - default configuration (44 milliseconds)
    16:41:38.695 WARN 
org.apache.spark.deploy.security.HadoopDelegationTokenManager: 
spark.yarn.security.credentials.hive.enabled is deprecated. Please use 
spark.security.credentials.hive.enabled instead.
    [info] - using deprecated configurations (6 milliseconds)
    [info] - SPARK-23209: obtain tokens when Hive classes are not available 
(284 milliseconds)
    [info] Run completed in 1 second, 200 milliseconds.
    [info] Total number of tests run: 3
    [info] Suites: completed 1, aborted 0
    [info] Tests: succeeded 3, failed 0, canceled 0, ignored 0, pending 0
    [info] All tests passed.
    [success] Total time: 33 s, completed Feb 25, 2026, 4:41:43 PM
    ```
    
    A similar issue was fixed in Hadoop too, see HADOOP-19821. I created [a 
simple project](https://github.com/pan3793/HADOOP-19821) to help reviewers 
better understand this issue and solution.
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    No.
    
    Closes #54476 from pan3793/SPARK-55682.
    
    Authored-by: Cheng Pan <[email protected]>
    Signed-off-by: Cheng Pan <[email protected]>
---
 .../deploy/security/HadoopDelegationTokenManager.scala     |  9 +++++++--
 .../scala/org/apache/spark/sql/classic/SparkSession.scala  | 14 ++++++++++----
 .../spark/sql/execution/datasources/DataSource.scala       |  8 ++++++++
 .../datasources/jdbc/connection/ConnectionProvider.scala   | 13 +++++++++----
 4 files changed, 34 insertions(+), 10 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
 
b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
index bc7715941547..58bbd7477bd0 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
@@ -270,9 +270,14 @@ private[spark] class HadoopDelegationTokenManager(
     val providers = mutable.ArrayBuffer[HadoopDelegationTokenProvider]()
 
     val iterator = loader.iterator
-    while (iterator.hasNext) {
+    var keepLoading = true
+    while (keepLoading) {
       try {
-        providers += iterator.next
+        if (iterator.hasNext) {
+          providers += iterator.next()
+        } else {
+          keepLoading = false
+        }
       } catch {
         case t: Throwable =>
           logDebug(s"Failed to load built in provider.", t)
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala
index d1c7c406544c..fef62d8b2755 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala
@@ -1281,12 +1281,18 @@ object SparkSession extends SparkSessionCompanion with 
Logging {
       Utils.getContextOrSparkClassLoader)
     val loadedExts = loader.iterator()
 
-    while (loadedExts.hasNext) {
+    var keepLoading = true
+    while (keepLoading) {
       try {
-        val ext = loadedExts.next()
-        ext(extensions)
+        if (loadedExts.hasNext) {
+          val ext = loadedExts.next()
+          ext(extensions)
+        } else {
+          keepLoading = false
+        }
       } catch {
-        case e: Throwable => logWarning("Failed to load session extension", e)
+        case e: Throwable =>
+          logWarning("Failed to load session extension", e)
       }
     }
   }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
index a48c71ad0d36..0c234e3fcf18 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
@@ -734,6 +734,14 @@ object DataSource extends Logging {
         } else {
           throw e
         }
+      case e: NoClassDefFoundError =>
+        // NoClassDefFoundError's class name uses "/" rather than "." for 
packages
+        val className = e.getMessage.replaceAll("/", ".")
+        if (spark2RemovedClasses.contains(className)) {
+          throw QueryExecutionErrors.incompatibleDataSourceRegisterError(e)
+        } else {
+          throw e
+        }
     }
   }
 
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala
index 7342f701e345..03257140b8ee 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala
@@ -39,11 +39,16 @@ protected abstract class ConnectionProviderBase extends 
Logging {
     val providers = mutable.ArrayBuffer[JdbcConnectionProvider]()
 
     val iterator = loader.iterator
-    while (iterator.hasNext) {
+    var keepLoading = true
+    while (keepLoading) {
       try {
-        val provider = iterator.next
-        logDebug(s"Loaded built-in provider: $provider")
-        providers += provider
+        if (iterator.hasNext) {
+          val provider = iterator.next()
+          logDebug(s"Loaded built-in provider: $provider")
+          providers += provider
+        } else {
+          keepLoading = false
+        }
       } catch {
         case t: Throwable =>
           logError("Failed to load built-in provider.")


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

Reply via email to