Skip to content
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-7853] [SQL] Fixes a class loader issue in Spark SQL #6435

Closed
wants to merge 4 commits into from

Conversation

liancheng
Copy link
Contributor

This PR is based on PR #6396 authored by @chenghao-intel. Essentially, Spark SQL should use context classloader to load SerDe classes.

@yhuai helped updating the test case, and I fixed a bug in the original CliSuite: while testing the CLI tool with runCliWithin, we don't append \n to the last query, thus the last query is never executed.

Original PR description is pasted below.


bin/spark-sql --jars ./sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar
CREATE TABLE t1(a string, b string) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe';

Throws exception like

15/05/26 00:16:33 ERROR SparkSQLDriver: Failed in [CREATE TABLE t1(a string, b string) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe']
org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Cannot validate serde: org.apache.hive.hcatalog.data.JsonSerDe
        at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:333)
        at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:310)
        at org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:139)
        at org.apache.spark.sql.hive.client.ClientWrapper.runHive(ClientWrapper.scala:310)
        at org.apache.spark.sql.hive.client.ClientWrapper.runSqlHive(ClientWrapper.scala:300)
        at org.apache.spark.sql.hive.HiveContext.runSqlHive(HiveContext.scala:457)
        at org.apache.spark.sql.hive.execution.HiveNativeCommand.run(HiveNativeCommand.scala:33)
        at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:57)
        at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:57)
        at org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:68)
        at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:88)
        at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:88)
        at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:148)
        at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:87)
        at org.apache.spark.sql.SQLContext$QueryExecution.toRdd$lzycompute(SQLContext.scala:922)
        at org.apache.spark.sql.SQLContext$QueryExecution.toRdd(SQLContext.scala:922)
        at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:147)
        at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:131)
        at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:51)
        at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:727)
        at org.apache.spark.sql.hive.thriftserver.AbstractSparkSQLDriver.run(AbstractSparkSQLDriver.scala:57)


test("Commands using SerDe provided in --jars") {
val jarFile =
"../hive/src/test/resources/hive-hcatalog-core-0.13.1.jar"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Relative path probably cause problem, as the PWD may not the root path of the spark source code. Get that as what we did for dataFilePath ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is because hive-hcatalog-core-0.13.1.jar is under another module (hive). Also, when a test case of a module is executed from SBT and Maven, the working directory is always the directory of that module.

@SparkQA
Copy link

SparkQA commented May 27, 2015

Test build #33583 has finished for PR 6435 at commit d4c4845.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@yhuai
Copy link
Contributor

yhuai commented May 27, 2015

Thanks @chenghao-intel and @liancheng. I am merging it to branch 1.4 and master.

asfgit pushed a commit that referenced this pull request May 27, 2015
This PR is based on PR #6396 authored by chenghao-intel. Essentially, Spark SQL should use context classloader to load SerDe classes.

yhuai helped updating the test case, and I fixed a bug in the original `CliSuite`: while testing the CLI tool with `runCliWithin`, we don't append `\n` to the last query, thus the last query is never executed.

Original PR description is pasted below.

----

```
bin/spark-sql --jars ./sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar
CREATE TABLE t1(a string, b string) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe';
```

Throws exception like

```
15/05/26 00:16:33 ERROR SparkSQLDriver: Failed in [CREATE TABLE t1(a string, b string) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe']
org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Cannot validate serde: org.apache.hive.hcatalog.data.JsonSerDe
        at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:333)
        at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:310)
        at org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:139)
        at org.apache.spark.sql.hive.client.ClientWrapper.runHive(ClientWrapper.scala:310)
        at org.apache.spark.sql.hive.client.ClientWrapper.runSqlHive(ClientWrapper.scala:300)
        at org.apache.spark.sql.hive.HiveContext.runSqlHive(HiveContext.scala:457)
        at org.apache.spark.sql.hive.execution.HiveNativeCommand.run(HiveNativeCommand.scala:33)
        at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:57)
        at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:57)
        at org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:68)
        at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:88)
        at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:88)
        at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:148)
        at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:87)
        at org.apache.spark.sql.SQLContext$QueryExecution.toRdd$lzycompute(SQLContext.scala:922)
        at org.apache.spark.sql.SQLContext$QueryExecution.toRdd(SQLContext.scala:922)
        at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:147)
        at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:131)
        at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:51)
        at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:727)
        at org.apache.spark.sql.hive.thriftserver.AbstractSparkSQLDriver.run(AbstractSparkSQLDriver.scala:57)
```

Author: Cheng Hao <hao.cheng@intel.com>
Author: Cheng Lian <lian@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #6435 from liancheng/classLoader and squashes the following commits:

d4c4845 [Cheng Lian] Fixes CliSuite
75e80e2 [Yin Huai] Update the fix.
fd26533 [Cheng Hao] scalastyle
dd78775 [Cheng Hao] workaround for classloader of IsolatedClientLoader

(cherry picked from commit db3fd05)
Signed-off-by: Yin Huai <yhuai@databricks.com>
@asfgit asfgit closed this in db3fd05 May 27, 2015
asfgit pushed a commit that referenced this pull request May 29, 2015
https://issues.apache.org/jira/browse/SPARK-7853

This fixes the problem introduced by my change in #6435, which causes that Hive Context fails to create in spark shell because of the class loader issue.

Author: Yin Huai <yhuai@databricks.com>

Closes #6459 from yhuai/SPARK-7853 and squashes the following commits:

37ad33e [Yin Huai] Do not use hiveQlTable at all.
47cdb6d [Yin Huai] Move hiveconf.set to the end of setConf.
005649b [Yin Huai] Update comment.
35d86f3 [Yin Huai] Access TTable directly to make sure Hive will not internally use any metastore utility functions.
3737766 [Yin Huai] Recursively find all jars.

(cherry picked from commit 572b62c)
Signed-off-by: Yin Huai <yhuai@databricks.com>
asfgit pushed a commit that referenced this pull request May 29, 2015
https://issues.apache.org/jira/browse/SPARK-7853

This fixes the problem introduced by my change in #6435, which causes that Hive Context fails to create in spark shell because of the class loader issue.

Author: Yin Huai <yhuai@databricks.com>

Closes #6459 from yhuai/SPARK-7853 and squashes the following commits:

37ad33e [Yin Huai] Do not use hiveQlTable at all.
47cdb6d [Yin Huai] Move hiveconf.set to the end of setConf.
005649b [Yin Huai] Update comment.
35d86f3 [Yin Huai] Access TTable directly to make sure Hive will not internally use any metastore utility functions.
3737766 [Yin Huai] Recursively find all jars.
jeanlyn pushed a commit to jeanlyn/spark that referenced this pull request Jun 12, 2015
This PR is based on PR apache#6396 authored by chenghao-intel. Essentially, Spark SQL should use context classloader to load SerDe classes.

yhuai helped updating the test case, and I fixed a bug in the original `CliSuite`: while testing the CLI tool with `runCliWithin`, we don't append `\n` to the last query, thus the last query is never executed.

Original PR description is pasted below.

----

```
bin/spark-sql --jars ./sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar
CREATE TABLE t1(a string, b string) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe';
```

Throws exception like

```
15/05/26 00:16:33 ERROR SparkSQLDriver: Failed in [CREATE TABLE t1(a string, b string) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe']
org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Cannot validate serde: org.apache.hive.hcatalog.data.JsonSerDe
        at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:333)
        at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:310)
        at org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:139)
        at org.apache.spark.sql.hive.client.ClientWrapper.runHive(ClientWrapper.scala:310)
        at org.apache.spark.sql.hive.client.ClientWrapper.runSqlHive(ClientWrapper.scala:300)
        at org.apache.spark.sql.hive.HiveContext.runSqlHive(HiveContext.scala:457)
        at org.apache.spark.sql.hive.execution.HiveNativeCommand.run(HiveNativeCommand.scala:33)
        at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:57)
        at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:57)
        at org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:68)
        at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:88)
        at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:88)
        at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:148)
        at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:87)
        at org.apache.spark.sql.SQLContext$QueryExecution.toRdd$lzycompute(SQLContext.scala:922)
        at org.apache.spark.sql.SQLContext$QueryExecution.toRdd(SQLContext.scala:922)
        at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:147)
        at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:131)
        at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:51)
        at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:727)
        at org.apache.spark.sql.hive.thriftserver.AbstractSparkSQLDriver.run(AbstractSparkSQLDriver.scala:57)
```

Author: Cheng Hao <hao.cheng@intel.com>
Author: Cheng Lian <lian@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes apache#6435 from liancheng/classLoader and squashes the following commits:

d4c4845 [Cheng Lian] Fixes CliSuite
75e80e2 [Yin Huai] Update the fix.
fd26533 [Cheng Hao] scalastyle
dd78775 [Cheng Hao] workaround for classloader of IsolatedClientLoader
jeanlyn pushed a commit to jeanlyn/spark that referenced this pull request Jun 12, 2015
https://issues.apache.org/jira/browse/SPARK-7853

This fixes the problem introduced by my change in apache#6435, which causes that Hive Context fails to create in spark shell because of the class loader issue.

Author: Yin Huai <yhuai@databricks.com>

Closes apache#6459 from yhuai/SPARK-7853 and squashes the following commits:

37ad33e [Yin Huai] Do not use hiveQlTable at all.
47cdb6d [Yin Huai] Move hiveconf.set to the end of setConf.
005649b [Yin Huai] Update comment.
35d86f3 [Yin Huai] Access TTable directly to make sure Hive will not internally use any metastore utility functions.
3737766 [Yin Huai] Recursively find all jars.
nemccarthy pushed a commit to nemccarthy/spark that referenced this pull request Jun 19, 2015
This PR is based on PR apache#6396 authored by chenghao-intel. Essentially, Spark SQL should use context classloader to load SerDe classes.

yhuai helped updating the test case, and I fixed a bug in the original `CliSuite`: while testing the CLI tool with `runCliWithin`, we don't append `\n` to the last query, thus the last query is never executed.

Original PR description is pasted below.

----

```
bin/spark-sql --jars ./sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar
CREATE TABLE t1(a string, b string) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe';
```

Throws exception like

```
15/05/26 00:16:33 ERROR SparkSQLDriver: Failed in [CREATE TABLE t1(a string, b string) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe']
org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Cannot validate serde: org.apache.hive.hcatalog.data.JsonSerDe
        at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:333)
        at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:310)
        at org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:139)
        at org.apache.spark.sql.hive.client.ClientWrapper.runHive(ClientWrapper.scala:310)
        at org.apache.spark.sql.hive.client.ClientWrapper.runSqlHive(ClientWrapper.scala:300)
        at org.apache.spark.sql.hive.HiveContext.runSqlHive(HiveContext.scala:457)
        at org.apache.spark.sql.hive.execution.HiveNativeCommand.run(HiveNativeCommand.scala:33)
        at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:57)
        at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:57)
        at org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:68)
        at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:88)
        at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:88)
        at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:148)
        at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:87)
        at org.apache.spark.sql.SQLContext$QueryExecution.toRdd$lzycompute(SQLContext.scala:922)
        at org.apache.spark.sql.SQLContext$QueryExecution.toRdd(SQLContext.scala:922)
        at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:147)
        at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:131)
        at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:51)
        at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:727)
        at org.apache.spark.sql.hive.thriftserver.AbstractSparkSQLDriver.run(AbstractSparkSQLDriver.scala:57)
```

Author: Cheng Hao <hao.cheng@intel.com>
Author: Cheng Lian <lian@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes apache#6435 from liancheng/classLoader and squashes the following commits:

d4c4845 [Cheng Lian] Fixes CliSuite
75e80e2 [Yin Huai] Update the fix.
fd26533 [Cheng Hao] scalastyle
dd78775 [Cheng Hao] workaround for classloader of IsolatedClientLoader
nemccarthy pushed a commit to nemccarthy/spark that referenced this pull request Jun 19, 2015
https://issues.apache.org/jira/browse/SPARK-7853

This fixes the problem introduced by my change in apache#6435, which causes that Hive Context fails to create in spark shell because of the class loader issue.

Author: Yin Huai <yhuai@databricks.com>

Closes apache#6459 from yhuai/SPARK-7853 and squashes the following commits:

37ad33e [Yin Huai] Do not use hiveQlTable at all.
47cdb6d [Yin Huai] Move hiveconf.set to the end of setConf.
005649b [Yin Huai] Update comment.
35d86f3 [Yin Huai] Access TTable directly to make sure Hive will not internally use any metastore utility functions.
3737766 [Yin Huai] Recursively find all jars.
sunchao pushed a commit that referenced this pull request Feb 27, 2023
…uiltin' Hive version for metadata client

### What changes were proposed in this pull request?
When using the 'builtin' Hive version for the Hive metadata client, do not create a separate classloader, and rather continue to use the overall user/application classloader (regardless of Java version). This standardizes the behavior for all Java versions with that of Java 9+. See SPARK-42539 for more details on why this approach was chosen.

### Why are the changes needed?
Please see a much more detailed description in SPARK-42539. The tl;dr is that user-provided JARs (such as `hive-exec-2.3.8.jar`) take precedence over Spark/system JARs when constructing the classloader used by `IsolatedClientLoader` on Java 8 in 'builtin' mode, which can cause unexpected behavior and/or breakages. This violates the expectation that, unless user-first classloader mode is used, Spark JARs should be prioritized over user JARs. It also seems that this separate classloader was unnecessary from the start, since the intent of 'builtin' mode is to use the JARs already existing on the regular classloader (as alluded to [here](#24057 (comment))). The isolated clientloader was originally added in #5876 in 2015. This bit in the PR description is the only mention of the behavior for "builtin":
> attempt to discover the jars that were used to load Spark SQL and use those. This option is only valid when using the execution version of Hive.

I can't follow the logic here; the user classloader clearly has all of the necessary Hive JARs, since that's where we're getting the JAR URLs from, so we could just use that directly instead of grabbing the URLs. When this was initially added, it only used the JARs from the user classloader, not any of its parents, which I suspect was the motivating factor (to try to avoid more Spark classes being duplicated inside of the isolated classloader, I guess). But that was changed a month later anyway in #6435 / #6459, so I think this may have basically been deadcode from the start. It has also caused at least one issue over the years, e.g. SPARK-21428, which disables the new-classloader behavior in the case of running inside of a CLI session.

### Does this PR introduce _any_ user-facing change?
No, except to protect Spark itself from potentially being broken by bad user JARs.

### How was this patch tested?
This includes a new unit test in `HiveUtilsSuite` which demonstrates the issue and shows that this approach resolves it. It has also been tested on a live cluster running Java 8 and Hive communication functionality continues to work as expected.

Closes #40144 from xkrogen/xkrogen/SPARK-42539/hive-isolatedclientloader-builtin-user-jar-conflict-fix/java9strategy.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: Chao Sun <sunchao@apple.com>
sunchao pushed a commit that referenced this pull request Feb 27, 2023
…uiltin' Hive version for metadata client

### What changes were proposed in this pull request?
When using the 'builtin' Hive version for the Hive metadata client, do not create a separate classloader, and rather continue to use the overall user/application classloader (regardless of Java version). This standardizes the behavior for all Java versions with that of Java 9+. See SPARK-42539 for more details on why this approach was chosen.

### Why are the changes needed?
Please see a much more detailed description in SPARK-42539. The tl;dr is that user-provided JARs (such as `hive-exec-2.3.8.jar`) take precedence over Spark/system JARs when constructing the classloader used by `IsolatedClientLoader` on Java 8 in 'builtin' mode, which can cause unexpected behavior and/or breakages. This violates the expectation that, unless user-first classloader mode is used, Spark JARs should be prioritized over user JARs. It also seems that this separate classloader was unnecessary from the start, since the intent of 'builtin' mode is to use the JARs already existing on the regular classloader (as alluded to [here](#24057 (comment))). The isolated clientloader was originally added in #5876 in 2015. This bit in the PR description is the only mention of the behavior for "builtin":
> attempt to discover the jars that were used to load Spark SQL and use those. This option is only valid when using the execution version of Hive.

I can't follow the logic here; the user classloader clearly has all of the necessary Hive JARs, since that's where we're getting the JAR URLs from, so we could just use that directly instead of grabbing the URLs. When this was initially added, it only used the JARs from the user classloader, not any of its parents, which I suspect was the motivating factor (to try to avoid more Spark classes being duplicated inside of the isolated classloader, I guess). But that was changed a month later anyway in #6435 / #6459, so I think this may have basically been deadcode from the start. It has also caused at least one issue over the years, e.g. SPARK-21428, which disables the new-classloader behavior in the case of running inside of a CLI session.

### Does this PR introduce _any_ user-facing change?
No, except to protect Spark itself from potentially being broken by bad user JARs.

### How was this patch tested?
This includes a new unit test in `HiveUtilsSuite` which demonstrates the issue and shows that this approach resolves it. It has also been tested on a live cluster running Java 8 and Hive communication functionality continues to work as expected.

Closes #40144 from xkrogen/xkrogen/SPARK-42539/hive-isolatedclientloader-builtin-user-jar-conflict-fix/java9strategy.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: Chao Sun <sunchao@apple.com>
xkrogen added a commit to xkrogen/spark that referenced this pull request Feb 28, 2023
…uiltin' Hive version for metadata client

When using the 'builtin' Hive version for the Hive metadata client, do not create a separate classloader, and rather continue to use the overall user/application classloader (regardless of Java version). This standardizes the behavior for all Java versions with that of Java 9+. See SPARK-42539 for more details on why this approach was chosen.

Please see a much more detailed description in SPARK-42539. The tl;dr is that user-provided JARs (such as `hive-exec-2.3.8.jar`) take precedence over Spark/system JARs when constructing the classloader used by `IsolatedClientLoader` on Java 8 in 'builtin' mode, which can cause unexpected behavior and/or breakages. This violates the expectation that, unless user-first classloader mode is used, Spark JARs should be prioritized over user JARs. It also seems that this separate classloader was unnecessary from the start, since the intent of 'builtin' mode is to use the JARs already existing on the regular classloader (as alluded to [here](apache#24057 (comment))). The isolated clientloader was originally added in apache#5876 in 2015. This bit in the PR description is the only mention of the behavior for "builtin":
> attempt to discover the jars that were used to load Spark SQL and use those. This option is only valid when using the execution version of Hive.

I can't follow the logic here; the user classloader clearly has all of the necessary Hive JARs, since that's where we're getting the JAR URLs from, so we could just use that directly instead of grabbing the URLs. When this was initially added, it only used the JARs from the user classloader, not any of its parents, which I suspect was the motivating factor (to try to avoid more Spark classes being duplicated inside of the isolated classloader, I guess). But that was changed a month later anyway in apache#6435 / apache#6459, so I think this may have basically been deadcode from the start. It has also caused at least one issue over the years, e.g. SPARK-21428, which disables the new-classloader behavior in the case of running inside of a CLI session.

No, except to protect Spark itself from potentially being broken by bad user JARs.

This includes a new unit test in `HiveUtilsSuite` which demonstrates the issue and shows that this approach resolves it. It has also been tested on a live cluster running Java 8 and Hive communication functionality continues to work as expected.
sunchao pushed a commit that referenced this pull request Mar 1, 2023
…uiltin' Hive version for metadata client

### What changes were proposed in this pull request?
When using the 'builtin' Hive version for the Hive metadata client, do not create a separate classloader, and rather continue to use the overall user/application classloader (regardless of Java version). This standardizes the behavior for all Java versions with that of Java 9+. See SPARK-42539 for more details on why this approach was chosen.

Please note that this is a re-submit of #40144. That one introduced test failures, and potentially a real issue, because the PR works by setting `isolationOn = false` for `builtin` mode. In addition to adjusting the classloader, `HiveClientImpl` relies on `isolationOn` to determine if it should use an isolated copy of `SessionState`, so the PR inadvertently switched to using a shared `SessionState` object. I think we do want to continue to have the isolated session state even in `builtin` mode, so this adds a new flag `sessionStateIsolationOn` which controls whether the session state should be isolated, _separately_ from the `isolationOn` flag which controls whether the classloader should be isolated. Default behavior is for `sessionStateIsolationOn` to be set equal to `isolationOn`, but for `builtin` mode, we override it to enable session state isolated even though classloader isolation is turned off.

### Why are the changes needed?
Please see a much more detailed description in SPARK-42539. The tl;dr is that user-provided JARs (such as `hive-exec-2.3.8.jar`) take precedence over Spark/system JARs when constructing the classloader used by `IsolatedClientLoader` on Java 8 in 'builtin' mode, which can cause unexpected behavior and/or breakages. This violates the expectation that, unless user-first classloader mode is used, Spark JARs should be prioritized over user JARs. It also seems that this separate classloader was unnecessary from the start, since the intent of 'builtin' mode is to use the JARs already existing on the regular classloader (as alluded to [here](#24057 (comment))). The isolated clientloader was originally added in #5876 in 2015. This bit in the PR description is the only mention of the behavior for "builtin":
> attempt to discover the jars that were used to load Spark SQL and use those. This option is only valid when using the execution version of Hive.

I can't follow the logic here; the user classloader clearly has all of the necessary Hive JARs, since that's where we're getting the JAR URLs from, so we could just use that directly instead of grabbing the URLs. When this was initially added, it only used the JARs from the user classloader, not any of its parents, which I suspect was the motivating factor (to try to avoid more Spark classes being duplicated inside of the isolated classloader, I guess). But that was changed a month later anyway in #6435 / #6459, so I think this may have basically been deadcode from the start. It has also caused at least one issue over the years, e.g. SPARK-21428, which disables the new-classloader behavior in the case of running inside of a CLI session.

### Does this PR introduce _any_ user-facing change?
No, except to protect Spark itself from potentially being broken by bad user JARs.

### How was this patch tested?
This includes a new unit test in `HiveUtilsSuite` which demonstrates the issue and shows that this approach resolves it. It has also been tested on a live cluster running Java 8 and Hive communication functionality continues to work as expected.

Unit tests failing in #40144 have been locally tested (`HiveUtilsSuite`, `HiveSharedStateSuite`, `HiveCliSessionStateSuite`, `JsonHadoopFsRelationSuite`).

Closes #40224 from xkrogen/xkrogen/SPARK-42539/hive-isolatedclientloader-builtin-user-jar-conflict-fix/take2.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: Chao Sun <sunchao@apple.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
4 participants