-
Notifications
You must be signed in to change notification settings - Fork 28.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-43186][SQL][HIVE] Remove workaround for FileSinkDesc #40848
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
Merged to master, thanks! |
@pan3793 I found an interesting thing, after this one merged, when I run the following commands:
there are 4 test failed as following:
but if I revert this one, the the failure will disappear. |
Tests pass if enable
Seems reasonable |
No, these test can passed without -Phive without this pr |
The error stack in server side as follows:
|
…nnect-client-jvm` without -Phive ### What changes were proposed in this pull request? This pr aims to added a cleaning action for the `$sparkHome/sql/hive/target/$scalaDir/classes` and `$sparkHome/sql/hive/target/$scalaDir/test-classes` directories before `SimpleSparkConnectService` starts when running test cases that inherit `RemoteSparkSession` without `-Phive` to avoid to unexpected loading of `sql/hive/target/scala-2.12/classes/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister` by `ServiceLoader`. ### Why are the changes needed? When we run the test cases that inherit `RemoteSparkSession`, the classpath used to launch `SimpleSparkConnectService` will at least include the following directory, both maven and sbt: ``` $sparkHome/conf/ $sparkHome/common/kvstore/target/scala-2.12/classes/ $sparkHome/common/network-common/target/scala-2.12/classes/ $sparkHome/common/network-shuffle/target/scala-2.12/classes/ $sparkHome/common/network-yarn/target/scala-2.12/classes $sparkHome/common/sketch/target/scala-2.12/classes/ $sparkHome/common/tags/target/scala-2.12/classes/ $sparkHome/common/unsafe/target/scala-2.12/classes/ $sparkHome/core/target/scala-2.12/classes/ $sparkHome/examples/target/scala-2.12/classes/ $sparkHome/graphx/target/scala-2.12/classes/ $sparkHome/launcher/target/scala-2.12/classes/ $sparkHome/mllib/target/scala-2.12/classes/ $sparkHome/repl/target/scala-2.12/classes/ $sparkHome/resource-managers/mesos/target/scala-2.12/classes $sparkHome/resource-managers/yarn/target/scala-2.12/classes $sparkHome/sql/catalyst/target/scala-2.12/classes/ $sparkHome/sql/core/target/scala-2.12/classes/ $sparkHome/sql/hive/target/scala-2.12/classes/ $sparkHome/sql/hive-thriftserver/target/scala-2.12/classes/ $sparkHome/streaming/target/scala-2.12/classes/ $sparkHome/common/kvstore/target/scala-2.12/test-classes $sparkHome/common/network-common/target/scala-2.12/test-classes/ $sparkHome/common/network-shuffle/target/scala-2.12/test-classes/ $sparkHome/common/network-yarn/target/scala-2.12/test-classes $sparkHome/common/sketch/target/scala-2.12/test-classes $sparkHome/common/tags/target/scala-2.12/test-classes/ $sparkHome/common/unsafe/target/scala-2.12/test-classes $sparkHome/core/target/scala-2.12/test-classes/ $sparkHome/examples/target/scala-2.12/test-classes $sparkHome/graphx/target/scala-2.12/test-classes $sparkHome/launcher/target/scala-2.12/test-classes/ $sparkHome/mllib/target/scala-2.12/test-classes $sparkHome/repl/target/scala-2.12/test-classes $sparkHome/resource-managers/mesos/target/scala-2.12/test-classes $sparkHome/resource-managers/yarn/target/scala-2.12/test-classes $sparkHome/sql/catalyst/target/scala-2.12/test-classes/ $sparkHome/sql/core/target/scala-2.12/test-classes $sparkHome/sql/hive/target/scala-2.12/test-classes $sparkHome/sql/hive-thriftserver/target/scala-2.12/test-classes $sparkHome/streaming/target/scala-2.12/test-classes $sparkHome/connector/connect/client/jvm/target/scala-2.12/test-classes/ $sparkHome/connector/connect/common/target/scala-2.12/test-classes/ ... ``` So if the test case need calls `DataSource#lookupDataSource` and the `hive` module is compiled, `sql/hive/target/scala-2.12/classes/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister` will be loaded by `ServiceLoader`. After SPARK-43186 | #40848 merged, `org.apache.spark.sql.hive.execution.HiveFileFormat` changed to use `org.apache.hadoop.hive.ql.plan.FileSinkDesc` instead of `org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc`, it has a strong dependence on `hive-exec`. But when there is no hive related jars under `assembly/target/$scalaDir/jars/`, it will cause initialization fail of `org.apache.spark.sql.hive.execution.HiveFileFormat` and test fail. For example, when we run the following commands to test `connect-client-jvm` without `-Phive`: ``` build/mvn clean install -DskipTests build/mvn test -pl connector/connect/client/jvm ``` Then hive related jars will not be copied to `assembly/target/$scalaDir/jars/`, there will be test error as: **Client side** ``` - read and write *** FAILED *** io.grpc.StatusRuntimeException: INTERNAL: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.sql.hive.execution.HiveFileFormat could not be instantiated at io.grpc.Status.asRuntimeException(Status.java:535) at io.grpc.stub.ClientCalls$BlockingResponseStream.hasNext(ClientCalls.java:660) at scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:45) at scala.collection.Iterator.toStream(Iterator.scala:1417) at scala.collection.Iterator.toStream$(Iterator.scala:1416) at scala.collection.AbstractIterator.toStream(Iterator.scala:1431) at scala.collection.TraversableOnce.toSeq(TraversableOnce.scala:354) at scala.collection.TraversableOnce.toSeq$(TraversableOnce.scala:354) at scala.collection.AbstractIterator.toSeq(Iterator.scala:1431) at org.apache.spark.sql.SparkSession.execute(SparkSession.scala:489) ... ``` **Server side** ``` java.util.ServiceConfigurationError: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.sql.hive.execution.HiveFileFormat could not be instantiated at java.util.ServiceLoader.fail(ServiceLoader.java:232) at java.util.ServiceLoader.access$100(ServiceLoader.java:185) at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384) at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404) at java.util.ServiceLoader$1.next(ServiceLoader.java:480) at scala.collection.convert.Wrappers$JIteratorWrapper.next(Wrappers.scala:46) at scala.collection.Iterator.foreach(Iterator.scala:943) at scala.collection.Iterator.foreach$(Iterator.scala:943) at scala.collection.AbstractIterator.foreach(Iterator.scala:1431) at scala.collection.IterableLike.foreach(IterableLike.scala:74) at scala.collection.IterableLike.foreach$(IterableLike.scala:73) at scala.collection.AbstractIterable.foreach(Iterable.scala:56) at scala.collection.TraversableLike.filterImpl(TraversableLike.scala:303) at scala.collection.TraversableLike.filterImpl$(TraversableLike.scala:297) at scala.collection.AbstractTraversable.filterImpl(Traversable.scala:108) at scala.collection.TraversableLike.filter(TraversableLike.scala:395) at scala.collection.TraversableLike.filter$(TraversableLike.scala:395) at scala.collection.AbstractTraversable.filter(Traversable.scala:108) at org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSource(DataSource.scala:629) at org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSourceV2(DataSource.scala:697) at org.apache.spark.sql.DataFrameWriter.lookupV2Provider(DataFrameWriter.scala:860) at org.apache.spark.sql.DataFrameWriter.saveAsTable(DataFrameWriter.scala:559) at org.apache.spark.sql.connect.planner.SparkConnectPlanner.handleWriteOperation(SparkConnectPlanner.scala:2326) at org.apache.spark.sql.connect.planner.SparkConnectPlanner.process(SparkConnectPlanner.scala:2091) at org.apache.spark.sql.connect.service.SparkConnectStreamHandler.handleCommand(SparkConnectStreamHandler.scala:120) at org.apache.spark.sql.connect.service.SparkConnectStreamHandler.$anonfun$handle$2(SparkConnectStreamHandler.scala:86) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:825) at org.apache.spark.sql.connect.service.SparkConnectStreamHandler.$anonfun$handle$1(SparkConnectStreamHandler.scala:53) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) at org.apache.spark.util.Utils$.withContextClassLoader(Utils.scala:209) at org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager$.withArtifactClassLoader(SparkConnectArtifactManager.scala:178) at org.apache.spark.sql.connect.service.SparkConnectStreamHandler.handle(SparkConnectStreamHandler.scala:48) at org.apache.spark.sql.connect.service.SparkConnectService.executePlan(SparkConnectService.scala:166) at org.apache.spark.connect.proto.SparkConnectServiceGrpc$MethodHandlers.invoke(SparkConnectServiceGrpc.java:611) at org.sparkproject.connect.grpc.io.grpc.stub.ServerCalls$UnaryServerCallHandler$UnaryServerCallListener.onHalfClose(ServerCalls.java:182) at org.sparkproject.connect.grpc.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.halfClosed(ServerCallImpl.java:352) at org.sparkproject.connect.grpc.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1HalfClosed.runInContext(ServerImpl.java:866) at org.sparkproject.connect.grpc.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37) at org.sparkproject.connect.grpc.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:750) Caused by: java.lang.NoClassDefFoundError: org/apache/hadoop/hive/ql/plan/FileSinkDesc at java.lang.Class.getDeclaredConstructors0(Native Method) at java.lang.Class.privateGetDeclaredConstructors(Class.java:2671) at java.lang.Class.getConstructor0(Class.java:3075) at java.lang.Class.newInstance(Class.java:412) at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:380) ... 40 more Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.ql.plan.FileSinkDesc at java.net.URLClassLoader.findClass(URLClassLoader.java:387) at java.lang.ClassLoader.loadClass(ClassLoader.java:419) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:352) at java.lang.ClassLoader.loadClass(ClassLoader.java:352) ... 45 more ``` So this PR proposal takes the initiative to clean up the `$sparkHome/sql/hive/target/$scalaDir/classes` and `$sparkHome/sql/hive/target/$scalaDir/test-classes` directories when `IntegrationTestUtils#isSparkHiveJarAvailable` is false to protect the above scenario. ### Does this PR introduce _any_ user-facing change? No, just for test. ### How was this patch tested? - Pass Github Actions - Manual test: The following command can reproduce the problem without this pr Maven ``` build/mvn clean install -DskipTests build/mvn test -pl connector/connect/client/jvm ``` SBT ``` build/sbt package build/sbt "connect-client-jvm/test" ``` **Before** Maven There are 13 test cases with similar failures ``` - recoverPartitions *** FAILED *** io.grpc.StatusRuntimeException: INTERNAL: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.sql.hive.execution.HiveFileFormat could not be instantiated at io.grpc.Status.asRuntimeException(Status.java:535) at io.grpc.stub.ClientCalls$BlockingResponseStream.hasNext(ClientCalls.java:660) at scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:45) at scala.collection.Iterator.toStream(Iterator.scala:1417) at scala.collection.Iterator.toStream$(Iterator.scala:1416) at scala.collection.AbstractIterator.toStream(Iterator.scala:1431) at scala.collection.TraversableOnce.toSeq(TraversableOnce.scala:354) at scala.collection.TraversableOnce.toSeq$(TraversableOnce.scala:354) at scala.collection.AbstractIterator.toSeq(Iterator.scala:1431) at org.apache.spark.sql.SparkSession.execute(SparkSession.scala:489) ... ``` SBT There are similar errors of sbt, and the test will unexpectedly aborted. **After** Both Maven and SBT no longer have similar test failures Closes #41282 from LuciferYang/SPARK-43647. Authored-by: yangjie01 <yangjie01@baidu.com> Signed-off-by: yangjie01 <yangjie01@baidu.com>
…nnect-client-jvm` without -Phive ### What changes were proposed in this pull request? This pr aims to added a cleaning action for the `$sparkHome/sql/hive/target/$scalaDir/classes` and `$sparkHome/sql/hive/target/$scalaDir/test-classes` directories before `SimpleSparkConnectService` starts when running test cases that inherit `RemoteSparkSession` without `-Phive` to avoid to unexpected loading of `sql/hive/target/scala-2.12/classes/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister` by `ServiceLoader`. ### Why are the changes needed? When we run the test cases that inherit `RemoteSparkSession`, the classpath used to launch `SimpleSparkConnectService` will at least include the following directory, both maven and sbt: ``` $sparkHome/conf/ $sparkHome/common/kvstore/target/scala-2.12/classes/ $sparkHome/common/network-common/target/scala-2.12/classes/ $sparkHome/common/network-shuffle/target/scala-2.12/classes/ $sparkHome/common/network-yarn/target/scala-2.12/classes $sparkHome/common/sketch/target/scala-2.12/classes/ $sparkHome/common/tags/target/scala-2.12/classes/ $sparkHome/common/unsafe/target/scala-2.12/classes/ $sparkHome/core/target/scala-2.12/classes/ $sparkHome/examples/target/scala-2.12/classes/ $sparkHome/graphx/target/scala-2.12/classes/ $sparkHome/launcher/target/scala-2.12/classes/ $sparkHome/mllib/target/scala-2.12/classes/ $sparkHome/repl/target/scala-2.12/classes/ $sparkHome/resource-managers/mesos/target/scala-2.12/classes $sparkHome/resource-managers/yarn/target/scala-2.12/classes $sparkHome/sql/catalyst/target/scala-2.12/classes/ $sparkHome/sql/core/target/scala-2.12/classes/ $sparkHome/sql/hive/target/scala-2.12/classes/ $sparkHome/sql/hive-thriftserver/target/scala-2.12/classes/ $sparkHome/streaming/target/scala-2.12/classes/ $sparkHome/common/kvstore/target/scala-2.12/test-classes $sparkHome/common/network-common/target/scala-2.12/test-classes/ $sparkHome/common/network-shuffle/target/scala-2.12/test-classes/ $sparkHome/common/network-yarn/target/scala-2.12/test-classes $sparkHome/common/sketch/target/scala-2.12/test-classes $sparkHome/common/tags/target/scala-2.12/test-classes/ $sparkHome/common/unsafe/target/scala-2.12/test-classes $sparkHome/core/target/scala-2.12/test-classes/ $sparkHome/examples/target/scala-2.12/test-classes $sparkHome/graphx/target/scala-2.12/test-classes $sparkHome/launcher/target/scala-2.12/test-classes/ $sparkHome/mllib/target/scala-2.12/test-classes $sparkHome/repl/target/scala-2.12/test-classes $sparkHome/resource-managers/mesos/target/scala-2.12/test-classes $sparkHome/resource-managers/yarn/target/scala-2.12/test-classes $sparkHome/sql/catalyst/target/scala-2.12/test-classes/ $sparkHome/sql/core/target/scala-2.12/test-classes $sparkHome/sql/hive/target/scala-2.12/test-classes $sparkHome/sql/hive-thriftserver/target/scala-2.12/test-classes $sparkHome/streaming/target/scala-2.12/test-classes $sparkHome/connector/connect/client/jvm/target/scala-2.12/test-classes/ $sparkHome/connector/connect/common/target/scala-2.12/test-classes/ ... ``` So if the test case need calls `DataSource#lookupDataSource` and the `hive` module is compiled, `sql/hive/target/scala-2.12/classes/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister` will be loaded by `ServiceLoader`. After SPARK-43186 | apache#40848 merged, `org.apache.spark.sql.hive.execution.HiveFileFormat` changed to use `org.apache.hadoop.hive.ql.plan.FileSinkDesc` instead of `org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc`, it has a strong dependence on `hive-exec`. But when there is no hive related jars under `assembly/target/$scalaDir/jars/`, it will cause initialization fail of `org.apache.spark.sql.hive.execution.HiveFileFormat` and test fail. For example, when we run the following commands to test `connect-client-jvm` without `-Phive`: ``` build/mvn clean install -DskipTests build/mvn test -pl connector/connect/client/jvm ``` Then hive related jars will not be copied to `assembly/target/$scalaDir/jars/`, there will be test error as: **Client side** ``` - read and write *** FAILED *** io.grpc.StatusRuntimeException: INTERNAL: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.sql.hive.execution.HiveFileFormat could not be instantiated at io.grpc.Status.asRuntimeException(Status.java:535) at io.grpc.stub.ClientCalls$BlockingResponseStream.hasNext(ClientCalls.java:660) at scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:45) at scala.collection.Iterator.toStream(Iterator.scala:1417) at scala.collection.Iterator.toStream$(Iterator.scala:1416) at scala.collection.AbstractIterator.toStream(Iterator.scala:1431) at scala.collection.TraversableOnce.toSeq(TraversableOnce.scala:354) at scala.collection.TraversableOnce.toSeq$(TraversableOnce.scala:354) at scala.collection.AbstractIterator.toSeq(Iterator.scala:1431) at org.apache.spark.sql.SparkSession.execute(SparkSession.scala:489) ... ``` **Server side** ``` java.util.ServiceConfigurationError: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.sql.hive.execution.HiveFileFormat could not be instantiated at java.util.ServiceLoader.fail(ServiceLoader.java:232) at java.util.ServiceLoader.access$100(ServiceLoader.java:185) at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384) at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404) at java.util.ServiceLoader$1.next(ServiceLoader.java:480) at scala.collection.convert.Wrappers$JIteratorWrapper.next(Wrappers.scala:46) at scala.collection.Iterator.foreach(Iterator.scala:943) at scala.collection.Iterator.foreach$(Iterator.scala:943) at scala.collection.AbstractIterator.foreach(Iterator.scala:1431) at scala.collection.IterableLike.foreach(IterableLike.scala:74) at scala.collection.IterableLike.foreach$(IterableLike.scala:73) at scala.collection.AbstractIterable.foreach(Iterable.scala:56) at scala.collection.TraversableLike.filterImpl(TraversableLike.scala:303) at scala.collection.TraversableLike.filterImpl$(TraversableLike.scala:297) at scala.collection.AbstractTraversable.filterImpl(Traversable.scala:108) at scala.collection.TraversableLike.filter(TraversableLike.scala:395) at scala.collection.TraversableLike.filter$(TraversableLike.scala:395) at scala.collection.AbstractTraversable.filter(Traversable.scala:108) at org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSource(DataSource.scala:629) at org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSourceV2(DataSource.scala:697) at org.apache.spark.sql.DataFrameWriter.lookupV2Provider(DataFrameWriter.scala:860) at org.apache.spark.sql.DataFrameWriter.saveAsTable(DataFrameWriter.scala:559) at org.apache.spark.sql.connect.planner.SparkConnectPlanner.handleWriteOperation(SparkConnectPlanner.scala:2326) at org.apache.spark.sql.connect.planner.SparkConnectPlanner.process(SparkConnectPlanner.scala:2091) at org.apache.spark.sql.connect.service.SparkConnectStreamHandler.handleCommand(SparkConnectStreamHandler.scala:120) at org.apache.spark.sql.connect.service.SparkConnectStreamHandler.$anonfun$handle$2(SparkConnectStreamHandler.scala:86) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:825) at org.apache.spark.sql.connect.service.SparkConnectStreamHandler.$anonfun$handle$1(SparkConnectStreamHandler.scala:53) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) at org.apache.spark.util.Utils$.withContextClassLoader(Utils.scala:209) at org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager$.withArtifactClassLoader(SparkConnectArtifactManager.scala:178) at org.apache.spark.sql.connect.service.SparkConnectStreamHandler.handle(SparkConnectStreamHandler.scala:48) at org.apache.spark.sql.connect.service.SparkConnectService.executePlan(SparkConnectService.scala:166) at org.apache.spark.connect.proto.SparkConnectServiceGrpc$MethodHandlers.invoke(SparkConnectServiceGrpc.java:611) at org.sparkproject.connect.grpc.io.grpc.stub.ServerCalls$UnaryServerCallHandler$UnaryServerCallListener.onHalfClose(ServerCalls.java:182) at org.sparkproject.connect.grpc.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.halfClosed(ServerCallImpl.java:352) at org.sparkproject.connect.grpc.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1HalfClosed.runInContext(ServerImpl.java:866) at org.sparkproject.connect.grpc.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37) at org.sparkproject.connect.grpc.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:750) Caused by: java.lang.NoClassDefFoundError: org/apache/hadoop/hive/ql/plan/FileSinkDesc at java.lang.Class.getDeclaredConstructors0(Native Method) at java.lang.Class.privateGetDeclaredConstructors(Class.java:2671) at java.lang.Class.getConstructor0(Class.java:3075) at java.lang.Class.newInstance(Class.java:412) at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:380) ... 40 more Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.ql.plan.FileSinkDesc at java.net.URLClassLoader.findClass(URLClassLoader.java:387) at java.lang.ClassLoader.loadClass(ClassLoader.java:419) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:352) at java.lang.ClassLoader.loadClass(ClassLoader.java:352) ... 45 more ``` So this PR proposal takes the initiative to clean up the `$sparkHome/sql/hive/target/$scalaDir/classes` and `$sparkHome/sql/hive/target/$scalaDir/test-classes` directories when `IntegrationTestUtils#isSparkHiveJarAvailable` is false to protect the above scenario. ### Does this PR introduce _any_ user-facing change? No, just for test. ### How was this patch tested? - Pass Github Actions - Manual test: The following command can reproduce the problem without this pr Maven ``` build/mvn clean install -DskipTests build/mvn test -pl connector/connect/client/jvm ``` SBT ``` build/sbt package build/sbt "connect-client-jvm/test" ``` **Before** Maven There are 13 test cases with similar failures ``` - recoverPartitions *** FAILED *** io.grpc.StatusRuntimeException: INTERNAL: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.sql.hive.execution.HiveFileFormat could not be instantiated at io.grpc.Status.asRuntimeException(Status.java:535) at io.grpc.stub.ClientCalls$BlockingResponseStream.hasNext(ClientCalls.java:660) at scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:45) at scala.collection.Iterator.toStream(Iterator.scala:1417) at scala.collection.Iterator.toStream$(Iterator.scala:1416) at scala.collection.AbstractIterator.toStream(Iterator.scala:1431) at scala.collection.TraversableOnce.toSeq(TraversableOnce.scala:354) at scala.collection.TraversableOnce.toSeq$(TraversableOnce.scala:354) at scala.collection.AbstractIterator.toSeq(Iterator.scala:1431) at org.apache.spark.sql.SparkSession.execute(SparkSession.scala:489) ... ``` SBT There are similar errors of sbt, and the test will unexpectedly aborted. **After** Both Maven and SBT no longer have similar test failures Closes apache#41282 from LuciferYang/SPARK-43647. Authored-by: yangjie01 <yangjie01@baidu.com> Signed-off-by: yangjie01 <yangjie01@baidu.com>
### What changes were proposed in this pull request? This pr fixs the issue that `setupHadoopConfForCompression` did not set isCompressed as expected due to we implicitly convert ShimFileSinkDesc to FileSinkDesc. This issue does not affect master branch since we removed ShimFileSinkDesc in #40848 ### Why are the changes needed? To make `hive.exec.compress.output` work as expected. ### Does this PR introduce _any_ user-facing change? yes, fix bug ### How was this patch tested? manually test ### Was this patch authored or co-authored using generative AI tooling? no Closes #45286 from ulysses-you/fix. Authored-by: ulysses-you <ulyssesyou18@gmail.com> Signed-off-by: youxiduo <youxiduo@corp.netease.com>
### What changes were proposed in this pull request? This pr fixs the issue that `setupHadoopConfForCompression` did not set isCompressed as expected due to we implicitly convert ShimFileSinkDesc to FileSinkDesc. This issue does not affect master branch since we removed ShimFileSinkDesc in apache#40848 ### Why are the changes needed? To make `hive.exec.compress.output` work as expected. ### Does this PR introduce _any_ user-facing change? yes, fix bug ### How was this patch tested? manually test ### Was this patch authored or co-authored using generative AI tooling? no Closes apache#45286 from ulysses-you/fix. Authored-by: ulysses-you <ulyssesyou18@gmail.com> Signed-off-by: youxiduo <youxiduo@corp.netease.com>
…erver` when `spark-hive_xxx.jar` is not in the classpath ### What changes were proposed in this pull request? This pr adds two new check condition sto the `launcher.AbstractCommandBuilder#buildClassPath` method: When `SPARK_PREPEND_CLASSES` is true, it no longer prepending the class path of the `sql/hive` module when `spark-hive_xxx.jar` is not in the classpath. The assumption here is that if `spark-hive_xxx.jar` is not in the classpath, then the `-Phive` profile was not used during package, and therefore the Hive-related jars(such as hive-exec-xx.jar) should also not be in the classpath. To avoid failure in loading the SPI in `DataSourceRegister` under `sql/hive`, so no longer prepend `sql/hive`. Meanwhile, due to the strong dependency of `sql/hive-thriftserver` on `sql/hive`, the prepend for `sql/hive-thriftserver` will also be excluded if `spark-hive_xxx.jar` is not in the classpath. On the other hand, if `spark-hive-thriftserver_xxx.jar` is not in the classpath, then the `-Phive-thriftserver` profile was not used during package, and therefore, jars such as hive-cli and hive-beeline should also not be included in the classpath. To avoid the inelegant startup failures of tools such as spark-sql, in this scenario, `sql/hive-thriftserver` will no longer be prepended to the classpath. ### Why are the changes needed? To fix some bad cases during development, one of them is as follows: ``` build/sbt clean package export SPARK_PREPEND_CLASSES=true bin/spark-shell ``` ``` Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 4.0.0-SNAPSHOT /_/ Using Scala version 2.13.14 (OpenJDK 64-Bit Server VM, Java 17.0.12) Type in expressions to have them evaluated. Type :help for more information. 24/09/06 17:27:54 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable Spark context Web UI available at http://172.22.200.248:4040 Spark context available as 'sc' (master = local[*], app id = local-1725614875132). Spark session available as 'spark'. scala> spark.sql("CREATE TABLE test_table (id BIGINT) USING parquet") java.util.ServiceConfigurationError: org.apache.spark.sql.sources.DataSourceRegister: org.apache.spark.sql.hive.execution.HiveFileFormat Unable to get public no-arg constructor at java.base/java.util.ServiceLoader.fail(ServiceLoader.java:586) at java.base/java.util.ServiceLoader.getConstructor(ServiceLoader.java:679) at java.base/java.util.ServiceLoader$LazyClassPathLookupIterator.hasNextService(ServiceLoader.java:1240) at java.base/java.util.ServiceLoader$LazyClassPathLookupIterator.hasNext(ServiceLoader.java:1273) at java.base/java.util.ServiceLoader$2.hasNext(ServiceLoader.java:1309) at java.base/java.util.ServiceLoader$3.hasNext(ServiceLoader.java:1393) at scala.collection.convert.JavaCollectionWrappers$JIteratorWrapper.hasNext(JavaCollectionWrappers.scala:46) at scala.collection.StrictOptimizedIterableOps.filterImpl(StrictOptimizedIterableOps.scala:225) at scala.collection.StrictOptimizedIterableOps.filterImpl$(StrictOptimizedIterableOps.scala:222) at scala.collection.convert.JavaCollectionWrappers$JIterableWrapper.filterImpl(JavaCollectionWrappers.scala:83) at scala.collection.StrictOptimizedIterableOps.filter(StrictOptimizedIterableOps.scala:218) at scala.collection.StrictOptimizedIterableOps.filter$(StrictOptimizedIterableOps.scala:218) at scala.collection.convert.JavaCollectionWrappers$JIterableWrapper.filter(JavaCollectionWrappers.scala:83) at org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSource(DataSource.scala:647) at org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSourceV2(DataSource.scala:727) at org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils$.getTableProvider(DataSourceV2Utils.scala:163) at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog.org$apache$spark$sql$catalyst$analysis$ResolveSessionCatalog$$isV2Provider(ResolveSessionCatalog.scala:666) at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog$$anonfun$apply$1.applyOrElse(ResolveSessionCatalog.scala:172) at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog$$anonfun$apply$1.applyOrElse(ResolveSessionCatalog.scala:54) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$3(AnalysisHelper.scala:138) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$1(AnalysisHelper.scala:138) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:386) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning(AnalysisHelper.scala:134) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning$(AnalysisHelper.scala:130) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsUpWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUp(AnalysisHelper.scala:111) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUp$(AnalysisHelper.scala:110) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsUp(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog.apply(ResolveSessionCatalog.scala:54) at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog.apply(ResolveSessionCatalog.scala:48) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:226) at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183) at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179) at scala.collection.immutable.List.foldLeft(List.scala:79) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:223) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:215) at scala.collection.immutable.List.foreach(List.scala:334) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:215) at org.apache.spark.sql.catalyst.analysis.Analyzer.org$apache$spark$sql$catalyst$analysis$Analyzer$$executeSameContext(Analyzer.scala:234) at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$execute$1(Analyzer.scala:230) at org.apache.spark.sql.catalyst.analysis.AnalysisContext$.withNewAnalysisContext(Analyzer.scala:186) at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:230) at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:201) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:186) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:186) at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:222) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:393) at org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:221) at org.apache.spark.sql.execution.QueryExecution.$anonfun$analyzed$1(QueryExecution.scala:92) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:234) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:608) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:234) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:742) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:233) at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:92) at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:89) at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:73) at org.apache.spark.sql.Dataset$.$anonfun$ofRows$3(Dataset.scala:120) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:742) at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:117) at org.apache.spark.sql.SparkSession.$anonfun$sql$4(SparkSession.scala:562) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:742) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:553) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:568) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:578) ... 42 elided Caused by: java.lang.NoClassDefFoundError: org/apache/hadoop/hive/ql/plan/FileSinkDesc at java.base/java.lang.Class.getDeclaredConstructors0(Native Method) at java.base/java.lang.Class.privateGetDeclaredConstructors(Class.java:3373) at java.base/java.lang.Class.getConstructor0(Class.java:3578) at java.base/java.lang.Class.getConstructor(Class.java:2271) at java.base/java.util.ServiceLoader$1.run(ServiceLoader.java:666) at java.base/java.util.ServiceLoader$1.run(ServiceLoader.java:663) at java.base/java.security.AccessController.doPrivileged(AccessController.java:569) at java.base/java.util.ServiceLoader.getConstructor(ServiceLoader.java:674) ... 108 more Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.ql.plan.FileSinkDesc at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:641) at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:188) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:525) ... 116 more ``` The aforementioned error is due to the fact that after #40848, the initialization of the SPI `org.apache.spark.sql.hive.execution.HiveFileFormat` within the `sql/hive` module requires `org.apache.hadoop.hive.ql.plan.FileSinkDesc`, but in the current scenario, the relevant jars are not present in the classpath. Therefore, the current pr opts to not prepend the classpath of `sql/hive` in this specific scenario. Another one is as follows: ``` build/sbt clean package -Phive // or build/sbt clean package export SPARK_PREPEND_CLASSES=true bin/spark-sql ``` ``` bin/spark-sql NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of assembly. WARNING: Using incubator modules: jdk.incubator.vector 24/09/09 00:28:26 ERROR SparkSubmit: Failed to load org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver java.lang.NoClassDefFoundError: org/apache/hadoop/hive/cli/CliDriver at java.base/java.lang.ClassLoader.defineClass1(Native Method) at java.base/java.lang.ClassLoader.defineClass(ClassLoader.java:1017) at java.base/java.security.SecureClassLoader.defineClass(SecureClassLoader.java:150) at java.base/jdk.internal.loader.BuiltinClassLoader.defineClass(BuiltinClassLoader.java:862) at java.base/jdk.internal.loader.BuiltinClassLoader.findClassOnClassPathOrNull(BuiltinClassLoader.java:760) at java.base/jdk.internal.loader.BuiltinClassLoader.loadClassOrNull(BuiltinClassLoader.java:681) at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:639) at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:188) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:579) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:525) at java.base/java.lang.Class.forName0(Native Method) at java.base/java.lang.Class.forName(Class.java:467) at org.apache.spark.util.SparkClassUtils.classForName(SparkClassUtils.scala:41) at org.apache.spark.util.SparkClassUtils.classForName$(SparkClassUtils.scala:36) at org.apache.spark.util.Utils$.classForName(Utils.scala:99) at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:992) at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:203) at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:226) at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:100) at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1136) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1145) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.cli.CliDriver at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:641) at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:188) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:525) ... 22 more Failed to load hive class. You need to build Spark with -Phive and -Phive-thriftserver. ``` The aforementioned failure occurred because, when compiling without the `-Phive` and `-Phive-thriftserver` profiles, the classpath lacked the necessary dependencies related to hive-cli. Therefore, in this scenario, `sql/hive-thriftserver` should not be prepended to the classpath either. ### Does this PR introduce _any_ user-facing change? No,this is only for developers ### How was this patch tested? 1. Pass GitHub Actions 2. Manually verify that the aforementioned test scenarios. The first scenario no longer reports errors: ``` Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 4.0.0-SNAPSHOT /_/ Using Scala version 2.13.14 (OpenJDK 64-Bit Server VM, Java 17.0.12) Type in expressions to have them evaluated. Type :help for more information. 24/09/06 17:45:24 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable 24/09/06 17:45:24 WARN Utils: Service 'SparkUI' could not bind on port 4040. Attempting port 4041. Spark context Web UI available at http://172.22.200.248:4041 Spark context available as 'sc' (master = local[*], app id = local-1725615924448). Spark session available as 'spark'. scala> spark.sql("CREATE TABLE test_table (id BIGINT) USING parquet") val res0: org.apache.spark.sql.DataFrame = [] ``` For the second scenario, although spark-sql will also fail to start, the error message appears to be simpler and clearer: ``` bin/spark-sql NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of assembly. WARNING: Using incubator modules: jdk.incubator.vector Error: Failed to load class org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver. Failed to load main class org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver. You need to build Spark with -Phive and -Phive-thriftserver. ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #48015 from LuciferYang/exclude-sql-hive-prepend. Lead-authored-by: yangjie01 <yangjie01@baidu.com> Co-authored-by: YangJie <yangjie01@baidu.com> Signed-off-by: yangjie01 <yangjie01@baidu.com>
…erver` when `spark-hive_xxx.jar` is not in the classpath ### What changes were proposed in this pull request? This pr adds two new check condition sto the `launcher.AbstractCommandBuilder#buildClassPath` method: When `SPARK_PREPEND_CLASSES` is true, it no longer prepending the class path of the `sql/hive` module when `spark-hive_xxx.jar` is not in the classpath. The assumption here is that if `spark-hive_xxx.jar` is not in the classpath, then the `-Phive` profile was not used during package, and therefore the Hive-related jars(such as hive-exec-xx.jar) should also not be in the classpath. To avoid failure in loading the SPI in `DataSourceRegister` under `sql/hive`, so no longer prepend `sql/hive`. Meanwhile, due to the strong dependency of `sql/hive-thriftserver` on `sql/hive`, the prepend for `sql/hive-thriftserver` will also be excluded if `spark-hive_xxx.jar` is not in the classpath. On the other hand, if `spark-hive-thriftserver_xxx.jar` is not in the classpath, then the `-Phive-thriftserver` profile was not used during package, and therefore, jars such as hive-cli and hive-beeline should also not be included in the classpath. To avoid the inelegant startup failures of tools such as spark-sql, in this scenario, `sql/hive-thriftserver` will no longer be prepended to the classpath. ### Why are the changes needed? To fix some bad cases during development, one of them is as follows: ``` build/sbt clean package export SPARK_PREPEND_CLASSES=true bin/spark-shell ``` ``` Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 4.0.0-SNAPSHOT /_/ Using Scala version 2.13.14 (OpenJDK 64-Bit Server VM, Java 17.0.12) Type in expressions to have them evaluated. Type :help for more information. 24/09/06 17:27:54 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable Spark context Web UI available at http://172.22.200.248:4040 Spark context available as 'sc' (master = local[*], app id = local-1725614875132). Spark session available as 'spark'. scala> spark.sql("CREATE TABLE test_table (id BIGINT) USING parquet") java.util.ServiceConfigurationError: org.apache.spark.sql.sources.DataSourceRegister: org.apache.spark.sql.hive.execution.HiveFileFormat Unable to get public no-arg constructor at java.base/java.util.ServiceLoader.fail(ServiceLoader.java:586) at java.base/java.util.ServiceLoader.getConstructor(ServiceLoader.java:679) at java.base/java.util.ServiceLoader$LazyClassPathLookupIterator.hasNextService(ServiceLoader.java:1240) at java.base/java.util.ServiceLoader$LazyClassPathLookupIterator.hasNext(ServiceLoader.java:1273) at java.base/java.util.ServiceLoader$2.hasNext(ServiceLoader.java:1309) at java.base/java.util.ServiceLoader$3.hasNext(ServiceLoader.java:1393) at scala.collection.convert.JavaCollectionWrappers$JIteratorWrapper.hasNext(JavaCollectionWrappers.scala:46) at scala.collection.StrictOptimizedIterableOps.filterImpl(StrictOptimizedIterableOps.scala:225) at scala.collection.StrictOptimizedIterableOps.filterImpl$(StrictOptimizedIterableOps.scala:222) at scala.collection.convert.JavaCollectionWrappers$JIterableWrapper.filterImpl(JavaCollectionWrappers.scala:83) at scala.collection.StrictOptimizedIterableOps.filter(StrictOptimizedIterableOps.scala:218) at scala.collection.StrictOptimizedIterableOps.filter$(StrictOptimizedIterableOps.scala:218) at scala.collection.convert.JavaCollectionWrappers$JIterableWrapper.filter(JavaCollectionWrappers.scala:83) at org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSource(DataSource.scala:647) at org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSourceV2(DataSource.scala:727) at org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils$.getTableProvider(DataSourceV2Utils.scala:163) at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog.org$apache$spark$sql$catalyst$analysis$ResolveSessionCatalog$$isV2Provider(ResolveSessionCatalog.scala:666) at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog$$anonfun$apply$1.applyOrElse(ResolveSessionCatalog.scala:172) at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog$$anonfun$apply$1.applyOrElse(ResolveSessionCatalog.scala:54) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$3(AnalysisHelper.scala:138) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$1(AnalysisHelper.scala:138) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:386) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning(AnalysisHelper.scala:134) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning$(AnalysisHelper.scala:130) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsUpWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUp(AnalysisHelper.scala:111) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUp$(AnalysisHelper.scala:110) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsUp(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog.apply(ResolveSessionCatalog.scala:54) at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog.apply(ResolveSessionCatalog.scala:48) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:226) at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183) at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179) at scala.collection.immutable.List.foldLeft(List.scala:79) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:223) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:215) at scala.collection.immutable.List.foreach(List.scala:334) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:215) at org.apache.spark.sql.catalyst.analysis.Analyzer.org$apache$spark$sql$catalyst$analysis$Analyzer$$executeSameContext(Analyzer.scala:234) at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$execute$1(Analyzer.scala:230) at org.apache.spark.sql.catalyst.analysis.AnalysisContext$.withNewAnalysisContext(Analyzer.scala:186) at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:230) at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:201) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:186) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:186) at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:222) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:393) at org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:221) at org.apache.spark.sql.execution.QueryExecution.$anonfun$analyzed$1(QueryExecution.scala:92) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:234) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:608) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:234) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:742) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:233) at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:92) at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:89) at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:73) at org.apache.spark.sql.Dataset$.$anonfun$ofRows$3(Dataset.scala:120) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:742) at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:117) at org.apache.spark.sql.SparkSession.$anonfun$sql$4(SparkSession.scala:562) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:742) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:553) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:568) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:578) ... 42 elided Caused by: java.lang.NoClassDefFoundError: org/apache/hadoop/hive/ql/plan/FileSinkDesc at java.base/java.lang.Class.getDeclaredConstructors0(Native Method) at java.base/java.lang.Class.privateGetDeclaredConstructors(Class.java:3373) at java.base/java.lang.Class.getConstructor0(Class.java:3578) at java.base/java.lang.Class.getConstructor(Class.java:2271) at java.base/java.util.ServiceLoader$1.run(ServiceLoader.java:666) at java.base/java.util.ServiceLoader$1.run(ServiceLoader.java:663) at java.base/java.security.AccessController.doPrivileged(AccessController.java:569) at java.base/java.util.ServiceLoader.getConstructor(ServiceLoader.java:674) ... 108 more Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.ql.plan.FileSinkDesc at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:641) at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:188) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:525) ... 116 more ``` The aforementioned error is due to the fact that after apache#40848, the initialization of the SPI `org.apache.spark.sql.hive.execution.HiveFileFormat` within the `sql/hive` module requires `org.apache.hadoop.hive.ql.plan.FileSinkDesc`, but in the current scenario, the relevant jars are not present in the classpath. Therefore, the current pr opts to not prepend the classpath of `sql/hive` in this specific scenario. Another one is as follows: ``` build/sbt clean package -Phive // or build/sbt clean package export SPARK_PREPEND_CLASSES=true bin/spark-sql ``` ``` bin/spark-sql NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of assembly. WARNING: Using incubator modules: jdk.incubator.vector 24/09/09 00:28:26 ERROR SparkSubmit: Failed to load org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver java.lang.NoClassDefFoundError: org/apache/hadoop/hive/cli/CliDriver at java.base/java.lang.ClassLoader.defineClass1(Native Method) at java.base/java.lang.ClassLoader.defineClass(ClassLoader.java:1017) at java.base/java.security.SecureClassLoader.defineClass(SecureClassLoader.java:150) at java.base/jdk.internal.loader.BuiltinClassLoader.defineClass(BuiltinClassLoader.java:862) at java.base/jdk.internal.loader.BuiltinClassLoader.findClassOnClassPathOrNull(BuiltinClassLoader.java:760) at java.base/jdk.internal.loader.BuiltinClassLoader.loadClassOrNull(BuiltinClassLoader.java:681) at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:639) at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:188) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:579) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:525) at java.base/java.lang.Class.forName0(Native Method) at java.base/java.lang.Class.forName(Class.java:467) at org.apache.spark.util.SparkClassUtils.classForName(SparkClassUtils.scala:41) at org.apache.spark.util.SparkClassUtils.classForName$(SparkClassUtils.scala:36) at org.apache.spark.util.Utils$.classForName(Utils.scala:99) at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:992) at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:203) at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:226) at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:100) at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1136) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1145) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.cli.CliDriver at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:641) at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:188) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:525) ... 22 more Failed to load hive class. You need to build Spark with -Phive and -Phive-thriftserver. ``` The aforementioned failure occurred because, when compiling without the `-Phive` and `-Phive-thriftserver` profiles, the classpath lacked the necessary dependencies related to hive-cli. Therefore, in this scenario, `sql/hive-thriftserver` should not be prepended to the classpath either. ### Does this PR introduce _any_ user-facing change? No,this is only for developers ### How was this patch tested? 1. Pass GitHub Actions 2. Manually verify that the aforementioned test scenarios. The first scenario no longer reports errors: ``` Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 4.0.0-SNAPSHOT /_/ Using Scala version 2.13.14 (OpenJDK 64-Bit Server VM, Java 17.0.12) Type in expressions to have them evaluated. Type :help for more information. 24/09/06 17:45:24 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable 24/09/06 17:45:24 WARN Utils: Service 'SparkUI' could not bind on port 4040. Attempting port 4041. Spark context Web UI available at http://172.22.200.248:4041 Spark context available as 'sc' (master = local[*], app id = local-1725615924448). Spark session available as 'spark'. scala> spark.sql("CREATE TABLE test_table (id BIGINT) USING parquet") val res0: org.apache.spark.sql.DataFrame = [] ``` For the second scenario, although spark-sql will also fail to start, the error message appears to be simpler and clearer: ``` bin/spark-sql NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of assembly. WARNING: Using incubator modules: jdk.incubator.vector Error: Failed to load class org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver. Failed to load main class org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver. You need to build Spark with -Phive and -Phive-thriftserver. ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes apache#48015 from LuciferYang/exclude-sql-hive-prepend. Lead-authored-by: yangjie01 <yangjie01@baidu.com> Co-authored-by: YangJie <yangjie01@baidu.com> Signed-off-by: yangjie01 <yangjie01@baidu.com>
…erver` when `spark-hive_xxx.jar` is not in the classpath ### What changes were proposed in this pull request? This pr adds two new check condition sto the `launcher.AbstractCommandBuilder#buildClassPath` method: When `SPARK_PREPEND_CLASSES` is true, it no longer prepending the class path of the `sql/hive` module when `spark-hive_xxx.jar` is not in the classpath. The assumption here is that if `spark-hive_xxx.jar` is not in the classpath, then the `-Phive` profile was not used during package, and therefore the Hive-related jars(such as hive-exec-xx.jar) should also not be in the classpath. To avoid failure in loading the SPI in `DataSourceRegister` under `sql/hive`, so no longer prepend `sql/hive`. Meanwhile, due to the strong dependency of `sql/hive-thriftserver` on `sql/hive`, the prepend for `sql/hive-thriftserver` will also be excluded if `spark-hive_xxx.jar` is not in the classpath. On the other hand, if `spark-hive-thriftserver_xxx.jar` is not in the classpath, then the `-Phive-thriftserver` profile was not used during package, and therefore, jars such as hive-cli and hive-beeline should also not be included in the classpath. To avoid the inelegant startup failures of tools such as spark-sql, in this scenario, `sql/hive-thriftserver` will no longer be prepended to the classpath. ### Why are the changes needed? To fix some bad cases during development, one of them is as follows: ``` build/sbt clean package export SPARK_PREPEND_CLASSES=true bin/spark-shell ``` ``` Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 4.0.0-SNAPSHOT /_/ Using Scala version 2.13.14 (OpenJDK 64-Bit Server VM, Java 17.0.12) Type in expressions to have them evaluated. Type :help for more information. 24/09/06 17:27:54 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable Spark context Web UI available at http://172.22.200.248:4040 Spark context available as 'sc' (master = local[*], app id = local-1725614875132). Spark session available as 'spark'. scala> spark.sql("CREATE TABLE test_table (id BIGINT) USING parquet") java.util.ServiceConfigurationError: org.apache.spark.sql.sources.DataSourceRegister: org.apache.spark.sql.hive.execution.HiveFileFormat Unable to get public no-arg constructor at java.base/java.util.ServiceLoader.fail(ServiceLoader.java:586) at java.base/java.util.ServiceLoader.getConstructor(ServiceLoader.java:679) at java.base/java.util.ServiceLoader$LazyClassPathLookupIterator.hasNextService(ServiceLoader.java:1240) at java.base/java.util.ServiceLoader$LazyClassPathLookupIterator.hasNext(ServiceLoader.java:1273) at java.base/java.util.ServiceLoader$2.hasNext(ServiceLoader.java:1309) at java.base/java.util.ServiceLoader$3.hasNext(ServiceLoader.java:1393) at scala.collection.convert.JavaCollectionWrappers$JIteratorWrapper.hasNext(JavaCollectionWrappers.scala:46) at scala.collection.StrictOptimizedIterableOps.filterImpl(StrictOptimizedIterableOps.scala:225) at scala.collection.StrictOptimizedIterableOps.filterImpl$(StrictOptimizedIterableOps.scala:222) at scala.collection.convert.JavaCollectionWrappers$JIterableWrapper.filterImpl(JavaCollectionWrappers.scala:83) at scala.collection.StrictOptimizedIterableOps.filter(StrictOptimizedIterableOps.scala:218) at scala.collection.StrictOptimizedIterableOps.filter$(StrictOptimizedIterableOps.scala:218) at scala.collection.convert.JavaCollectionWrappers$JIterableWrapper.filter(JavaCollectionWrappers.scala:83) at org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSource(DataSource.scala:647) at org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSourceV2(DataSource.scala:727) at org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils$.getTableProvider(DataSourceV2Utils.scala:163) at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog.org$apache$spark$sql$catalyst$analysis$ResolveSessionCatalog$$isV2Provider(ResolveSessionCatalog.scala:666) at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog$$anonfun$apply$1.applyOrElse(ResolveSessionCatalog.scala:172) at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog$$anonfun$apply$1.applyOrElse(ResolveSessionCatalog.scala:54) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$3(AnalysisHelper.scala:138) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:86) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$1(AnalysisHelper.scala:138) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:386) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning(AnalysisHelper.scala:134) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUpWithPruning$(AnalysisHelper.scala:130) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsUpWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUp(AnalysisHelper.scala:111) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsUp$(AnalysisHelper.scala:110) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsUp(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog.apply(ResolveSessionCatalog.scala:54) at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog.apply(ResolveSessionCatalog.scala:48) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:226) at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183) at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179) at scala.collection.immutable.List.foldLeft(List.scala:79) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:223) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:215) at scala.collection.immutable.List.foreach(List.scala:334) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:215) at org.apache.spark.sql.catalyst.analysis.Analyzer.org$apache$spark$sql$catalyst$analysis$Analyzer$$executeSameContext(Analyzer.scala:234) at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$execute$1(Analyzer.scala:230) at org.apache.spark.sql.catalyst.analysis.AnalysisContext$.withNewAnalysisContext(Analyzer.scala:186) at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:230) at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:201) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:186) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:186) at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:222) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:393) at org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:221) at org.apache.spark.sql.execution.QueryExecution.$anonfun$analyzed$1(QueryExecution.scala:92) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:234) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:608) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:234) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:742) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:233) at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:92) at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:89) at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:73) at org.apache.spark.sql.Dataset$.$anonfun$ofRows$3(Dataset.scala:120) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:742) at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:117) at org.apache.spark.sql.SparkSession.$anonfun$sql$4(SparkSession.scala:562) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:742) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:553) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:568) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:578) ... 42 elided Caused by: java.lang.NoClassDefFoundError: org/apache/hadoop/hive/ql/plan/FileSinkDesc at java.base/java.lang.Class.getDeclaredConstructors0(Native Method) at java.base/java.lang.Class.privateGetDeclaredConstructors(Class.java:3373) at java.base/java.lang.Class.getConstructor0(Class.java:3578) at java.base/java.lang.Class.getConstructor(Class.java:2271) at java.base/java.util.ServiceLoader$1.run(ServiceLoader.java:666) at java.base/java.util.ServiceLoader$1.run(ServiceLoader.java:663) at java.base/java.security.AccessController.doPrivileged(AccessController.java:569) at java.base/java.util.ServiceLoader.getConstructor(ServiceLoader.java:674) ... 108 more Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.ql.plan.FileSinkDesc at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:641) at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:188) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:525) ... 116 more ``` The aforementioned error is due to the fact that after apache#40848, the initialization of the SPI `org.apache.spark.sql.hive.execution.HiveFileFormat` within the `sql/hive` module requires `org.apache.hadoop.hive.ql.plan.FileSinkDesc`, but in the current scenario, the relevant jars are not present in the classpath. Therefore, the current pr opts to not prepend the classpath of `sql/hive` in this specific scenario. Another one is as follows: ``` build/sbt clean package -Phive // or build/sbt clean package export SPARK_PREPEND_CLASSES=true bin/spark-sql ``` ``` bin/spark-sql NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of assembly. WARNING: Using incubator modules: jdk.incubator.vector 24/09/09 00:28:26 ERROR SparkSubmit: Failed to load org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver java.lang.NoClassDefFoundError: org/apache/hadoop/hive/cli/CliDriver at java.base/java.lang.ClassLoader.defineClass1(Native Method) at java.base/java.lang.ClassLoader.defineClass(ClassLoader.java:1017) at java.base/java.security.SecureClassLoader.defineClass(SecureClassLoader.java:150) at java.base/jdk.internal.loader.BuiltinClassLoader.defineClass(BuiltinClassLoader.java:862) at java.base/jdk.internal.loader.BuiltinClassLoader.findClassOnClassPathOrNull(BuiltinClassLoader.java:760) at java.base/jdk.internal.loader.BuiltinClassLoader.loadClassOrNull(BuiltinClassLoader.java:681) at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:639) at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:188) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:579) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:525) at java.base/java.lang.Class.forName0(Native Method) at java.base/java.lang.Class.forName(Class.java:467) at org.apache.spark.util.SparkClassUtils.classForName(SparkClassUtils.scala:41) at org.apache.spark.util.SparkClassUtils.classForName$(SparkClassUtils.scala:36) at org.apache.spark.util.Utils$.classForName(Utils.scala:99) at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:992) at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:203) at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:226) at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:100) at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1136) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1145) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.cli.CliDriver at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:641) at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:188) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:525) ... 22 more Failed to load hive class. You need to build Spark with -Phive and -Phive-thriftserver. ``` The aforementioned failure occurred because, when compiling without the `-Phive` and `-Phive-thriftserver` profiles, the classpath lacked the necessary dependencies related to hive-cli. Therefore, in this scenario, `sql/hive-thriftserver` should not be prepended to the classpath either. ### Does this PR introduce _any_ user-facing change? No,this is only for developers ### How was this patch tested? 1. Pass GitHub Actions 2. Manually verify that the aforementioned test scenarios. The first scenario no longer reports errors: ``` Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 4.0.0-SNAPSHOT /_/ Using Scala version 2.13.14 (OpenJDK 64-Bit Server VM, Java 17.0.12) Type in expressions to have them evaluated. Type :help for more information. 24/09/06 17:45:24 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable 24/09/06 17:45:24 WARN Utils: Service 'SparkUI' could not bind on port 4040. Attempting port 4041. Spark context Web UI available at http://172.22.200.248:4041 Spark context available as 'sc' (master = local[*], app id = local-1725615924448). Spark session available as 'spark'. scala> spark.sql("CREATE TABLE test_table (id BIGINT) USING parquet") val res0: org.apache.spark.sql.DataFrame = [] ``` For the second scenario, although spark-sql will also fail to start, the error message appears to be simpler and clearer: ``` bin/spark-sql NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of assembly. WARNING: Using incubator modules: jdk.incubator.vector Error: Failed to load class org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver. Failed to load main class org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver. You need to build Spark with -Phive and -Phive-thriftserver. ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes apache#48015 from LuciferYang/exclude-sql-hive-prepend. Lead-authored-by: yangjie01 <yangjie01@baidu.com> Co-authored-by: YangJie <yangjie01@baidu.com> Signed-off-by: yangjie01 <yangjie01@baidu.com>
What changes were proposed in this pull request?
Remove
org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc
, which is used to address serializable issue oforg.apache.hadoop.hive.ql.plan.FileSinkDesc
Why are the changes needed?
HIVE-6171 changed
FileSinkDesc
's property fromString dirName
toPath dirName
, but thePath
is not serializable until HADOOP-13519 (got fixed in Hadoop 3.0.0).Since SPARK-42452 removed support for Hadoop2, we can remove this workaround now.
Does this PR introduce any user-facing change?
No.
How was this patch tested?
Pass GA.