From 9c1b779b1004736d65f8c2811b62a66aa1e9125b Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Wed, 22 May 2024 14:29:12 +0800 Subject: [PATCH] [KYUUBI #6405] Spark engine supports both javax and jakarta ws.rs namespaces MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit # :mag: Description Spark 4.0 upgraded Jersey from 2 to 3, and also migrated from `javax.ws.rs` to `jakarta.ws.rs` in SPARK-47118, this break the Spark SQL engine complication with Spark 4.0 ## Types of changes :bookmark: - [x] Bugfix (non-breaking change which fixes an issue) - [ ] New feature (non-breaking change which adds functionality) - [ ] Breaking change (fix or feature that would cause existing functionality to change) ## Test Plan ๐Ÿงช ``` build/mvn clean install -Pscala-2.13 -Pspark-master \ -pl externals/kyuubi-spark-sql-engine -am -DskipTests ``` before ``` [INFO] --- scala-maven-plugin:4.8.0:compile (scala-compile-first) kyuubi-spark-sql-engine_2.13 --- [INFO] Compiler bridge file: /home/kyuubi/.sbt/1.0/zinc/org.scala-sbt/org.scala-sbt-compiler-bridge_2.13-1.8.0-bin_2.13.8__61.0-1.8.0_20221110T195421.jar [INFO] compiler plugin: BasicArtifact(com.github.ghik,silencer-plugin_2.13.8,1.7.13,null) [INFO] compiling 61 Scala sources to /home/kyuubi/apache-kyuubi/externals/kyuubi-spark-sql-engine/target/scala-2.13/classes ... [ERROR] [Error] /home/kyuubi/apache-kyuubi/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/ExecutePython.scala:27: object ws is not a member of package javax [ERROR] [Error] /home/kyuubi/apache-kyuubi/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/ExecutePython.scala:307: not found: value UriBuilder [ERROR] [Error] /home/kyuubi/apache-kyuubi/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/ExecutePython.scala:320: not found: value UriBuilder ``` after ``` [INFO] --- scala-maven-plugin:4.8.0:compile (scala-compile-first) kyuubi-spark-sql-engine_2.13 --- [INFO] Compiler bridge file: /home/kyuubi/.sbt/1.0/zinc/org.scala-sbt/org.scala-sbt-compiler-bridge_2.13-1.8.0-bin_2.13.8__61.0-1.8.0_20221110T195421.jar [INFO] compiler plugin: BasicArtifact(com.github.ghik,silencer-plugin_2.13.8,1.7.13,null) [INFO] compiling 61 Scala sources to /home/kyuubi/apache-kyuubi/externals/kyuubi-spark-sql-engine/target/scala-2.13/classes ... [INFO] compile in 19.2 s ``` --- # Checklist ๐Ÿ“ - [x] This patch was not authored or co-authored using [Generative Tooling](https://www.apache.org/legal/generative-tooling.html) **Be nice. Be informative.** Closes #6405 from pan3793/jersey. Closes #6405 6cce23b01 [Cheng Pan] SPARK-47118 Jersey Authored-by: Cheng Pan Signed-off-by: Cheng Pan --- .../kyuubi/engine/spark/KyuubiSparkUtil.scala | 40 +++++++++++++++++++ .../spark/operation/ExecutePython.scala | 5 +-- 2 files changed, 42 insertions(+), 3 deletions(-) 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))