diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/KyuubiSparkUtil.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/KyuubiSparkUtil.scala index 2e33d8ce6..6192cd0fc 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/KyuubiSparkUtil.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/KyuubiSparkUtil.scala @@ -17,6 +17,7 @@ package org.apache.kyuubi.engine.spark +import java.net.URI import java.time.{Instant, LocalDateTime, ZoneId} import scala.annotation.meta.getter @@ -28,6 +29,7 @@ import org.apache.spark.util.kvstore.KVIndex import org.apache.kyuubi.Logging import org.apache.kyuubi.config.ConfigEntry import org.apache.kyuubi.util.SemanticVersion +import org.apache.kyuubi.util.reflect.DynMethods object KyuubiSparkUtil extends Logging { @@ -113,4 +115,42 @@ object KyuubiSparkUtil extends Logging { SparkSQLEngine.kyuubiConf.get(configEntry) } } + + // SPARK-47118 (4.0.0) upgrades Jersey from 2 to 3 which also changes javax.ws.rs to + // jakarta.ws.rs, this is an equivalent implementation using reflection of the following + // plain invocation: + // {javax|jakarta}.ws.rs.core.UriBuilder.fromUri(uri).fragment(fragment).build() + def buildURI(uri: URI, fragment: String): URI = { + if (SPARK_ENGINE_RUNTIME_VERSION >= "4.0") { + var uriBuilder = DynMethods.builder("fromUri") + .impl("jakarta.ws.rs.core.UriBuilder", classOf[URI]) + .build() + .invoke[AnyRef](uri) + + uriBuilder = DynMethods.builder("fragment") + .impl("jakarta.ws.rs.core.UriBuilder", classOf[String]) + .build(uriBuilder) + .invoke[AnyRef](fragment) + + DynMethods.builder("build") + .impl("jakarta.ws.rs.core.UriBuilder") + .build(uriBuilder) + .invoke[URI]() + } else { + var uriBuilder = DynMethods.builder("fromUri") + .impl("javax.ws.rs.core.UriBuilder", classOf[URI]) + .build() + .invoke[AnyRef](uri) + + uriBuilder = DynMethods.builder("fragment") + .impl("javax.ws.rs.core.UriBuilder", classOf[String]) + .build(uriBuilder) + .invoke[AnyRef](fragment) + + DynMethods.builder("build") + .impl("javax.ws.rs.core.UriBuilder") + .build(uriBuilder) + .invoke[URI]() + } + } } diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/ExecutePython.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/ExecutePython.scala index f60b1d4c8..e350232c8 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/ExecutePython.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/ExecutePython.scala @@ -24,7 +24,6 @@ import java.nio.file.{Files, Path, Paths} import java.util.concurrent.RejectedExecutionException import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.locks.ReentrantLock -import javax.ws.rs.core.UriBuilder import scala.collection.JavaConverters._ @@ -304,7 +303,7 @@ object ExecutePython extends Logging { archive => var uri = new URI(archive) if (uri.getFragment == null) { - uri = UriBuilder.fromUri(uri).fragment(DEFAULT_SPARK_PYTHON_ENV_ARCHIVE_FRAGMENT).build() + uri = buildURI(uri, DEFAULT_SPARK_PYTHON_ENV_ARCHIVE_FRAGMENT) } spark.sparkContext.addArchive(uri.toString) Paths.get(SparkFiles.get(uri.getFragment), pythonEnvExecPath) @@ -317,7 +316,7 @@ object ExecutePython extends Logging { archive => var uri = new URI(archive) if (uri.getFragment == null) { - uri = UriBuilder.fromUri(uri).fragment(DEFAULT_SPARK_PYTHON_HOME_ARCHIVE_FRAGMENT).build() + uri = buildURI(uri, DEFAULT_SPARK_PYTHON_HOME_ARCHIVE_FRAGMENT) } spark.sparkContext.addArchive(uri.toString) Paths.get(SparkFiles.get(uri.getFragment))