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-49012][SQL][BUILD] Add bouncycastle-related test dependencies to the hive-thriftserver module to fix the Maven daily test #47496

Closed
wants to merge 1 commit into from

Conversation

LuciferYang
Copy link
Contributor

@LuciferYang LuciferYang commented Jul 26, 2024

What changes were proposed in this pull request?

This pr add bouncycastle-related test dependencies to the hive-thrift module to fix the Maven daily test.

Why are the changes needed?

sql-on-files.sql added the following statement in #47480, which caused the Maven daily test to fail

CREATE TABLE sql_on_files.test_orc USING ORC AS SELECT 1;

- sql-on-files.sql *** FAILED ***
  "" did not contain "Exception" Exception did not match for query #6
  CREATE TABLE sql_on_files.test_orc USING ORC AS SELECT 1, expected: , but got: java.sql.SQLException
  org.apache.hive.service.cli.HiveSQLException: Error running query: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 8542.0 failed 1 times, most recent failure: Lost task 0.0 in stage 8542.0 (TID 8594) (localhost executor driver): java.lang.NoClassDefFoundError: org/bouncycastle/jce/provider/BouncyCastleProvider
    at test.org.apache.spark.sql.execution.datasources.orc.FakeKeyProvider$Factory.createProvider(FakeKeyProvider.java:127)
    at org.apache.hadoop.crypto.key.KeyProviderFactory.get(KeyProviderFactory.java:96)
    at org.apache.hadoop.crypto.key.KeyProviderFactory.getProviders(KeyProviderFactory.java:68)
    at org.apache.orc.impl.HadoopShimsCurrent.createKeyProvider(HadoopShimsCurrent.java:97)
    at org.apache.orc.impl.HadoopShimsCurrent.getHadoopKeyProvider(HadoopShimsCurrent.java:131)
    at org.apache.orc.impl.CryptoUtils$HadoopKeyProviderFactory.create(CryptoUtils.java:158)
    at org.apache.orc.impl.CryptoUtils.getKeyProvider(CryptoUtils.java:141)
    at org.apache.orc.impl.WriterImpl.setupEncryption(WriterImpl.java:1015)
    at org.apache.orc.impl.WriterImpl.<init>(WriterImpl.java:164)
    at org.apache.orc.OrcFile.createWriter(OrcFile.java:1078)
    at org.apache.spark.sql.execution.datasources.orc.OrcOutputWriter.<init>(OrcOutputWriter.scala:49)
    at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat$$anon$1.newInstance(OrcFileFormat.scala:89)
    at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:180)
    at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.<init>(FileFormatDataWriter.scala:165)
    at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:391)
    at org.apache.spark.sql.execution.datasources.WriteFilesExec.$anonfun$doExecuteWrite$1(WriteFiles.scala:107)
    at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:901)
    at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:901)
    at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
    at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
    at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
    at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93)
    at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:171)
    at org.apache.spark.scheduler.Task.run(Task.scala:146)
    at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$5(Executor.scala:644)
    at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64)
    at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61)
    at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:99)
    at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:647)
    at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
    at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
    at java.base/java.lang.Thread.run(Thread.java:840)
  Caused by: java.lang.ClassNotFoundException: org.bouncycastle.jce.provider.BouncyCastleProvider
    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)
    ... 32 more

Because we have configured hadoop.security.key.provider.path as test:/// in the parent pom.xml,

spark/pom.xml

Lines 3165 to 3166 in 5ccf9ba

<!--SPARK-42934: Need by `OrcEncryptionSuite` -->
<spark.hadoop.hadoop.security.key.provider.path>test:///</spark.hadoop.hadoop.security.key.provider.path>

KeyProviderFactory#getProviders will use FakeKeyProvider$Factory to create instances of FakeKeyProvider.

test.org.apache.spark.sql.execution.datasources.orc.FakeKeyProvider$Factory

During the initialization of FakeKeyProvider, it first initializes its superclass org.apache.hadoop.crypto.key.KeyProvider, which leads to the loading of the BouncyCastleProvider class. Therefore, we need to add bouncycastle-related test dependencies in the hive-thrift module.

Does this PR introduce any user-facing change?

No

How was this patch tested?

Manual Test with this pr.

build/mvn -Phive -Phive-thriftserver clean install -DskipTests
build/mvn -Phive -Phive-thriftserver clean install -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite -pl sql/hive-thriftserver
Run completed in 6 minutes, 52 seconds.
Total number of tests run: 243
Suites: completed 2, aborted 0
Tests: succeeded 243, failed 0, canceled 0, ignored 20, pending 0
All tests passed.

Was this patch authored or co-authored using generative AI tooling?

No

@LuciferYang LuciferYang changed the title Add bouncyCastle-related test dependencies to the hive-thrift module to fix the Maven daily test. [WIP] Add bouncyCastle-related test dependencies to the hive-thrift module to fix the Maven daily test. Jul 26, 2024
@LuciferYang LuciferYang changed the title [WIP] Add bouncyCastle-related test dependencies to the hive-thrift module to fix the Maven daily test. [SPARK-49012][SQL][BUILD] Add bouncyCastle-related test dependencies to the hive-thrift module to fix the Maven daily test. Jul 26, 2024
@LuciferYang LuciferYang marked this pull request as draft July 26, 2024 09:36
@LuciferYang LuciferYang changed the title [SPARK-49012][SQL][BUILD] Add bouncyCastle-related test dependencies to the hive-thrift module to fix the Maven daily test. [SPARK-49012][SQL][BUILD] Add bouncycastle-related test dependencies to the hive-thrift module to fix the Maven daily test. Jul 26, 2024
@LuciferYang LuciferYang changed the title [SPARK-49012][SQL][BUILD] Add bouncycastle-related test dependencies to the hive-thrift module to fix the Maven daily test. [SPARK-49012][SQL][BUILD] Add bouncycastle-related test dependencies to the hive-thrift module to fix the Maven daily test Jul 26, 2024
@LuciferYang LuciferYang marked this pull request as ready for review July 26, 2024 14:17
@LuciferYang
Copy link
Contributor Author

cc @HyukjinKwon @yaooqinn

@LuciferYang
Copy link
Contributor Author

spark/pom.xml

Lines 3165 to 3166 in 5ccf9ba

<!--SPARK-42934: Need by `OrcEncryptionSuite` -->
<spark.hadoop.hadoop.security.key.provider.path>test:///</spark.hadoop.hadoop.security.key.provider.path>

(Test / javaOptions) += "-Dspark.hadoop.hadoop.security.key.provider.path=test:///",

also cc @dongjoon-hyun

It looks like the configuration above might lead to test scenarios that write to Orc, other than OrcEncryptionSuite, to also use FakeKeyProvider. Is this within expectations? Should we consider reducing the effective scope of this configuration?

@dongjoon-hyun dongjoon-hyun changed the title [SPARK-49012][SQL][BUILD] Add bouncycastle-related test dependencies to the hive-thrift module to fix the Maven daily test [SPARK-49012][SQL][BUILD] Add bouncycastle-related test dependencies to the hive-thrift server module to fix the Maven daily test Jul 26, 2024
@dongjoon-hyun dongjoon-hyun changed the title [SPARK-49012][SQL][BUILD] Add bouncycastle-related test dependencies to the hive-thrift server module to fix the Maven daily test [SPARK-49012][SQL][BUILD] Add bouncycastle-related test dependencies to the hive-thriftserver module to fix the Maven daily test Jul 26, 2024
Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

Do you know why this happens from yesterday, @LuciferYang ?

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented Jul 26, 2024

I agree with adding this because it looks correct. However, let's figure out why this causes a problem from yesterday, @LuciferYang and @yaooqinn . If you don't mind, let's hold on this PR before merging.

Never mind. I'm not sure which part is new in the following syntax. Let's merge this first.

CREATE TABLE sql_on_files.test_orc USING ORC AS SELECT 1

The second approach can be done later.

-Dspark.hadoop.hadoop.security.key.provider.path=test:///

@dongjoon-hyun
Copy link
Member

I updated my comment. I'm going to merge this right now.

@LuciferYang
Copy link
Contributor Author

@dongjoon-hyun As I mentioned in the PR description:

  1. sql-on-files.sql added CREATE TABLE sql_on_files.test_orc USING ORC AS SELECT 1; yesterday
  2. we have configured hadoop.security.key.provider.path as test:/// in the parent pom.xml, so, the statement writing to the Orc table will make initialization of FakeKeyProvider and FakeKeyProvider first initializes its superclass org.apache.hadoop.crypto.key.KeyProvider, which leads to the loading of the BouncyCastleProvider class

@dongjoon-hyun
Copy link
Member

Thank you, @LuciferYang and @yaooqinn .

To @LuciferYang , yes, what I asked is why CREATE TABLE sql_on_files.test_orc USING ORC AS SELECT 1; is special in the FakeKeyProvider usage. The hive-thrift server module has ORC-related test cases already, doesn't it? In any way, it was just my curiosity. This PR itself is correct~

@LuciferYang
Copy link
Contributor Author

LuciferYang commented Jul 26, 2024

Yes, that's a good question, but it seems that no other test cases in the hive-thriftserver module have created and written data in Orc format before yesterday ...

image

@yaooqinn
Copy link
Member

It looks like Orc test suites are in sql/core, and sql/hive module, maybe it's the first time for sql/hive-thriftserver to touch orc tests, especially DDLs.

@dongjoon-hyun
Copy link
Member

Oh, got it.

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented Jul 26, 2024

I thought ThriftServerQueryTestSuite covers all data sources because it extends SQLQueryTestSuite.

class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServer with Logging {

However, indeed, yesterday's one is the first one to add ORC to inputs directory. Thank you for letting me know this.

$ git grep -i ' orc'
analyzer-results/sql-on-files.sql.out:CREATE TABLE sql_on_files.test_orc USING ORC AS SELECT 1
analyzer-results/sql-on-files.sql.out:SELECT * FROM orc.``
analyzer-results/sql-on-files.sql.out:SELECT * FROM orc.`/file/not/found`
analyzer-results/sql-on-files.sql.out:SELECT * FROM orc.`${spark.sql.warehouse.dir}/sql_on_files.db/test_orc`
analyzer-results/sql-on-files.sql.out:+- Relation [1#x] orc
inputs/sql-on-files.sql:-- ORC
inputs/sql-on-files.sql:CREATE TABLE sql_on_files.test_orc USING ORC AS SELECT 1;
inputs/sql-on-files.sql:SELECT * FROM orc.``;
inputs/sql-on-files.sql:SELECT * FROM orc.`/file/not/found`;
inputs/sql-on-files.sql:SELECT * FROM orc.`${spark.sql.warehouse.dir}/sql_on_files.db/test_orc`;
results/sql-on-files.sql.out:CREATE TABLE sql_on_files.test_orc USING ORC AS SELECT 1
results/sql-on-files.sql.out:SELECT * FROM orc.``
results/sql-on-files.sql.out:SELECT * FROM orc.`/file/not/found`
results/sql-on-files.sql.out:SELECT * FROM orc.`${spark.sql.warehouse.dir}/sql_on_files.db/test_orc`

ilicmarkodb pushed a commit to ilicmarkodb/spark that referenced this pull request Jul 29, 2024
…to the `hive-thriftserver` module to fix the Maven daily test

### What changes were proposed in this pull request?
This pr add bouncycastle-related test dependencies to the `hive-thrift` module to fix the Maven daily test.

### Why are the changes needed?
`sql-on-files.sql` added the following statement in apache#47480, which caused the Maven daily test to fail

https://github.com/apache/spark/blob/2363aec0c14ead24ade2bfa23478a4914f179c00/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql#L10

- https://github.com/apache/spark/actions/runs/10094638521/job/27943309504
- https://github.com/apache/spark/actions/runs/10095571472/job/27943298802

```
- sql-on-files.sql *** FAILED ***
  "" did not contain "Exception" Exception did not match for query apache#6
  CREATE TABLE sql_on_files.test_orc USING ORC AS SELECT 1, expected: , but got: java.sql.SQLException
  org.apache.hive.service.cli.HiveSQLException: Error running query: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 8542.0 failed 1 times, most recent failure: Lost task 0.0 in stage 8542.0 (TID 8594) (localhost executor driver): java.lang.NoClassDefFoundError: org/bouncycastle/jce/provider/BouncyCastleProvider
    at test.org.apache.spark.sql.execution.datasources.orc.FakeKeyProvider$Factory.createProvider(FakeKeyProvider.java:127)
    at org.apache.hadoop.crypto.key.KeyProviderFactory.get(KeyProviderFactory.java:96)
    at org.apache.hadoop.crypto.key.KeyProviderFactory.getProviders(KeyProviderFactory.java:68)
    at org.apache.orc.impl.HadoopShimsCurrent.createKeyProvider(HadoopShimsCurrent.java:97)
    at org.apache.orc.impl.HadoopShimsCurrent.getHadoopKeyProvider(HadoopShimsCurrent.java:131)
    at org.apache.orc.impl.CryptoUtils$HadoopKeyProviderFactory.create(CryptoUtils.java:158)
    at org.apache.orc.impl.CryptoUtils.getKeyProvider(CryptoUtils.java:141)
    at org.apache.orc.impl.WriterImpl.setupEncryption(WriterImpl.java:1015)
    at org.apache.orc.impl.WriterImpl.<init>(WriterImpl.java:164)
    at org.apache.orc.OrcFile.createWriter(OrcFile.java:1078)
    at org.apache.spark.sql.execution.datasources.orc.OrcOutputWriter.<init>(OrcOutputWriter.scala:49)
    at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat$$anon$1.newInstance(OrcFileFormat.scala:89)
    at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:180)
    at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.<init>(FileFormatDataWriter.scala:165)
    at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:391)
    at org.apache.spark.sql.execution.datasources.WriteFilesExec.$anonfun$doExecuteWrite$1(WriteFiles.scala:107)
    at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:901)
    at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:901)
    at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
    at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
    at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
    at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93)
    at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:171)
    at org.apache.spark.scheduler.Task.run(Task.scala:146)
    at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$5(Executor.scala:644)
    at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64)
    at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61)
    at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:99)
    at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:647)
    at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
    at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
    at java.base/java.lang.Thread.run(Thread.java:840)
  Caused by: java.lang.ClassNotFoundException: org.bouncycastle.jce.provider.BouncyCastleProvider
    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)
    ... 32 more
```

Because we have configured `hadoop.security.key.provider.path` as `test:///` in the parent `pom.xml`,

https://github.com/apache/spark/blob/5ccf9ba958f492c1eb4dde22a647ba75aba63d8e/pom.xml#L3165-L3166

`KeyProviderFactory#getProviders` will use `FakeKeyProvider$Factory` to create instances of `FakeKeyProvider`.

https://github.com/apache/spark/blob/5ccf9ba958f492c1eb4dde22a647ba75aba63d8e/sql/core/src/test/resources/META-INF/services/org.apache.hadoop.crypto.key.KeyProviderFactory#L18

During the initialization of `FakeKeyProvider`, it first initializes its superclass `org.apache.hadoop.crypto.key.KeyProvider`, which leads to the loading of the `BouncyCastleProvider` class. Therefore, we need to add bouncycastle-related test dependencies in the `hive-thrift` module.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manual Test with this pr.

```
build/mvn -Phive -Phive-thriftserver clean install -DskipTests
build/mvn -Phive -Phive-thriftserver clean install -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite -pl sql/hive-thriftserver
```

```
Run completed in 6 minutes, 52 seconds.
Total number of tests run: 243
Suites: completed 2, aborted 0
Tests: succeeded 243, failed 0, canceled 0, ignored 20, pending 0
All tests passed.
```

### Was this patch authored or co-authored using generative AI tooling?
No

Closes apache#47496 from LuciferYang/thrift-bouncycastle.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
fusheng9399 pushed a commit to fusheng9399/spark that referenced this pull request Aug 6, 2024
…to the `hive-thriftserver` module to fix the Maven daily test

### What changes were proposed in this pull request?
This pr add bouncycastle-related test dependencies to the `hive-thrift` module to fix the Maven daily test.

### Why are the changes needed?
`sql-on-files.sql` added the following statement in apache#47480, which caused the Maven daily test to fail

https://github.com/apache/spark/blob/2363aec0c14ead24ade2bfa23478a4914f179c00/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql#L10

- https://github.com/apache/spark/actions/runs/10094638521/job/27943309504
- https://github.com/apache/spark/actions/runs/10095571472/job/27943298802

```
- sql-on-files.sql *** FAILED ***
  "" did not contain "Exception" Exception did not match for query apache#6
  CREATE TABLE sql_on_files.test_orc USING ORC AS SELECT 1, expected: , but got: java.sql.SQLException
  org.apache.hive.service.cli.HiveSQLException: Error running query: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 8542.0 failed 1 times, most recent failure: Lost task 0.0 in stage 8542.0 (TID 8594) (localhost executor driver): java.lang.NoClassDefFoundError: org/bouncycastle/jce/provider/BouncyCastleProvider
    at test.org.apache.spark.sql.execution.datasources.orc.FakeKeyProvider$Factory.createProvider(FakeKeyProvider.java:127)
    at org.apache.hadoop.crypto.key.KeyProviderFactory.get(KeyProviderFactory.java:96)
    at org.apache.hadoop.crypto.key.KeyProviderFactory.getProviders(KeyProviderFactory.java:68)
    at org.apache.orc.impl.HadoopShimsCurrent.createKeyProvider(HadoopShimsCurrent.java:97)
    at org.apache.orc.impl.HadoopShimsCurrent.getHadoopKeyProvider(HadoopShimsCurrent.java:131)
    at org.apache.orc.impl.CryptoUtils$HadoopKeyProviderFactory.create(CryptoUtils.java:158)
    at org.apache.orc.impl.CryptoUtils.getKeyProvider(CryptoUtils.java:141)
    at org.apache.orc.impl.WriterImpl.setupEncryption(WriterImpl.java:1015)
    at org.apache.orc.impl.WriterImpl.<init>(WriterImpl.java:164)
    at org.apache.orc.OrcFile.createWriter(OrcFile.java:1078)
    at org.apache.spark.sql.execution.datasources.orc.OrcOutputWriter.<init>(OrcOutputWriter.scala:49)
    at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat$$anon$1.newInstance(OrcFileFormat.scala:89)
    at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:180)
    at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.<init>(FileFormatDataWriter.scala:165)
    at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:391)
    at org.apache.spark.sql.execution.datasources.WriteFilesExec.$anonfun$doExecuteWrite$1(WriteFiles.scala:107)
    at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:901)
    at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:901)
    at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
    at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
    at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
    at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93)
    at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:171)
    at org.apache.spark.scheduler.Task.run(Task.scala:146)
    at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$5(Executor.scala:644)
    at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64)
    at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61)
    at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:99)
    at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:647)
    at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
    at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
    at java.base/java.lang.Thread.run(Thread.java:840)
  Caused by: java.lang.ClassNotFoundException: org.bouncycastle.jce.provider.BouncyCastleProvider
    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)
    ... 32 more
```

Because we have configured `hadoop.security.key.provider.path` as `test:///` in the parent `pom.xml`,

https://github.com/apache/spark/blob/5ccf9ba958f492c1eb4dde22a647ba75aba63d8e/pom.xml#L3165-L3166

`KeyProviderFactory#getProviders` will use `FakeKeyProvider$Factory` to create instances of `FakeKeyProvider`.

https://github.com/apache/spark/blob/5ccf9ba958f492c1eb4dde22a647ba75aba63d8e/sql/core/src/test/resources/META-INF/services/org.apache.hadoop.crypto.key.KeyProviderFactory#L18

During the initialization of `FakeKeyProvider`, it first initializes its superclass `org.apache.hadoop.crypto.key.KeyProvider`, which leads to the loading of the `BouncyCastleProvider` class. Therefore, we need to add bouncycastle-related test dependencies in the `hive-thrift` module.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manual Test with this pr.

```
build/mvn -Phive -Phive-thriftserver clean install -DskipTests
build/mvn -Phive -Phive-thriftserver clean install -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite -pl sql/hive-thriftserver
```

```
Run completed in 6 minutes, 52 seconds.
Total number of tests run: 243
Suites: completed 2, aborted 0
Tests: succeeded 243, failed 0, canceled 0, ignored 20, pending 0
All tests passed.
```

### Was this patch authored or co-authored using generative AI tooling?
No

Closes apache#47496 from LuciferYang/thrift-bouncycastle.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
attilapiros pushed a commit to attilapiros/spark that referenced this pull request Oct 4, 2024
…to the `hive-thriftserver` module to fix the Maven daily test

### What changes were proposed in this pull request?
This pr add bouncycastle-related test dependencies to the `hive-thrift` module to fix the Maven daily test.

### Why are the changes needed?
`sql-on-files.sql` added the following statement in apache#47480, which caused the Maven daily test to fail

https://github.com/apache/spark/blob/2363aec0c14ead24ade2bfa23478a4914f179c00/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql#L10

- https://github.com/apache/spark/actions/runs/10094638521/job/27943309504
- https://github.com/apache/spark/actions/runs/10095571472/job/27943298802

```
- sql-on-files.sql *** FAILED ***
  "" did not contain "Exception" Exception did not match for query #6
  CREATE TABLE sql_on_files.test_orc USING ORC AS SELECT 1, expected: , but got: java.sql.SQLException
  org.apache.hive.service.cli.HiveSQLException: Error running query: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 8542.0 failed 1 times, most recent failure: Lost task 0.0 in stage 8542.0 (TID 8594) (localhost executor driver): java.lang.NoClassDefFoundError: org/bouncycastle/jce/provider/BouncyCastleProvider
    at test.org.apache.spark.sql.execution.datasources.orc.FakeKeyProvider$Factory.createProvider(FakeKeyProvider.java:127)
    at org.apache.hadoop.crypto.key.KeyProviderFactory.get(KeyProviderFactory.java:96)
    at org.apache.hadoop.crypto.key.KeyProviderFactory.getProviders(KeyProviderFactory.java:68)
    at org.apache.orc.impl.HadoopShimsCurrent.createKeyProvider(HadoopShimsCurrent.java:97)
    at org.apache.orc.impl.HadoopShimsCurrent.getHadoopKeyProvider(HadoopShimsCurrent.java:131)
    at org.apache.orc.impl.CryptoUtils$HadoopKeyProviderFactory.create(CryptoUtils.java:158)
    at org.apache.orc.impl.CryptoUtils.getKeyProvider(CryptoUtils.java:141)
    at org.apache.orc.impl.WriterImpl.setupEncryption(WriterImpl.java:1015)
    at org.apache.orc.impl.WriterImpl.<init>(WriterImpl.java:164)
    at org.apache.orc.OrcFile.createWriter(OrcFile.java:1078)
    at org.apache.spark.sql.execution.datasources.orc.OrcOutputWriter.<init>(OrcOutputWriter.scala:49)
    at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat$$anon$1.newInstance(OrcFileFormat.scala:89)
    at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:180)
    at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.<init>(FileFormatDataWriter.scala:165)
    at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:391)
    at org.apache.spark.sql.execution.datasources.WriteFilesExec.$anonfun$doExecuteWrite$1(WriteFiles.scala:107)
    at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:901)
    at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:901)
    at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
    at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
    at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
    at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93)
    at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:171)
    at org.apache.spark.scheduler.Task.run(Task.scala:146)
    at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$5(Executor.scala:644)
    at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64)
    at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61)
    at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:99)
    at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:647)
    at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
    at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
    at java.base/java.lang.Thread.run(Thread.java:840)
  Caused by: java.lang.ClassNotFoundException: org.bouncycastle.jce.provider.BouncyCastleProvider
    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)
    ... 32 more
```

Because we have configured `hadoop.security.key.provider.path` as `test:///` in the parent `pom.xml`,

https://github.com/apache/spark/blob/5ccf9ba958f492c1eb4dde22a647ba75aba63d8e/pom.xml#L3165-L3166

`KeyProviderFactory#getProviders` will use `FakeKeyProvider$Factory` to create instances of `FakeKeyProvider`.

https://github.com/apache/spark/blob/5ccf9ba958f492c1eb4dde22a647ba75aba63d8e/sql/core/src/test/resources/META-INF/services/org.apache.hadoop.crypto.key.KeyProviderFactory#L18

During the initialization of `FakeKeyProvider`, it first initializes its superclass `org.apache.hadoop.crypto.key.KeyProvider`, which leads to the loading of the `BouncyCastleProvider` class. Therefore, we need to add bouncycastle-related test dependencies in the `hive-thrift` module.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manual Test with this pr.

```
build/mvn -Phive -Phive-thriftserver clean install -DskipTests
build/mvn -Phive -Phive-thriftserver clean install -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite -pl sql/hive-thriftserver
```

```
Run completed in 6 minutes, 52 seconds.
Total number of tests run: 243
Suites: completed 2, aborted 0
Tests: succeeded 243, failed 0, canceled 0, ignored 20, pending 0
All tests passed.
```

### Was this patch authored or co-authored using generative AI tooling?
No

Closes apache#47496 from LuciferYang/thrift-bouncycastle.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
himadripal pushed a commit to himadripal/spark that referenced this pull request Oct 19, 2024
…to the `hive-thriftserver` module to fix the Maven daily test

### What changes were proposed in this pull request?
This pr add bouncycastle-related test dependencies to the `hive-thrift` module to fix the Maven daily test.

### Why are the changes needed?
`sql-on-files.sql` added the following statement in apache#47480, which caused the Maven daily test to fail

https://github.com/apache/spark/blob/2363aec0c14ead24ade2bfa23478a4914f179c00/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql#L10

- https://github.com/apache/spark/actions/runs/10094638521/job/27943309504
- https://github.com/apache/spark/actions/runs/10095571472/job/27943298802

```
- sql-on-files.sql *** FAILED ***
  "" did not contain "Exception" Exception did not match for query apache#6
  CREATE TABLE sql_on_files.test_orc USING ORC AS SELECT 1, expected: , but got: java.sql.SQLException
  org.apache.hive.service.cli.HiveSQLException: Error running query: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 8542.0 failed 1 times, most recent failure: Lost task 0.0 in stage 8542.0 (TID 8594) (localhost executor driver): java.lang.NoClassDefFoundError: org/bouncycastle/jce/provider/BouncyCastleProvider
    at test.org.apache.spark.sql.execution.datasources.orc.FakeKeyProvider$Factory.createProvider(FakeKeyProvider.java:127)
    at org.apache.hadoop.crypto.key.KeyProviderFactory.get(KeyProviderFactory.java:96)
    at org.apache.hadoop.crypto.key.KeyProviderFactory.getProviders(KeyProviderFactory.java:68)
    at org.apache.orc.impl.HadoopShimsCurrent.createKeyProvider(HadoopShimsCurrent.java:97)
    at org.apache.orc.impl.HadoopShimsCurrent.getHadoopKeyProvider(HadoopShimsCurrent.java:131)
    at org.apache.orc.impl.CryptoUtils$HadoopKeyProviderFactory.create(CryptoUtils.java:158)
    at org.apache.orc.impl.CryptoUtils.getKeyProvider(CryptoUtils.java:141)
    at org.apache.orc.impl.WriterImpl.setupEncryption(WriterImpl.java:1015)
    at org.apache.orc.impl.WriterImpl.<init>(WriterImpl.java:164)
    at org.apache.orc.OrcFile.createWriter(OrcFile.java:1078)
    at org.apache.spark.sql.execution.datasources.orc.OrcOutputWriter.<init>(OrcOutputWriter.scala:49)
    at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat$$anon$1.newInstance(OrcFileFormat.scala:89)
    at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:180)
    at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.<init>(FileFormatDataWriter.scala:165)
    at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:391)
    at org.apache.spark.sql.execution.datasources.WriteFilesExec.$anonfun$doExecuteWrite$1(WriteFiles.scala:107)
    at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:901)
    at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:901)
    at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
    at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:374)
    at org.apache.spark.rdd.RDD.iterator(RDD.scala:338)
    at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93)
    at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:171)
    at org.apache.spark.scheduler.Task.run(Task.scala:146)
    at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$5(Executor.scala:644)
    at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64)
    at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61)
    at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:99)
    at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:647)
    at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
    at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
    at java.base/java.lang.Thread.run(Thread.java:840)
  Caused by: java.lang.ClassNotFoundException: org.bouncycastle.jce.provider.BouncyCastleProvider
    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)
    ... 32 more
```

Because we have configured `hadoop.security.key.provider.path` as `test:///` in the parent `pom.xml`,

https://github.com/apache/spark/blob/5ccf9ba958f492c1eb4dde22a647ba75aba63d8e/pom.xml#L3165-L3166

`KeyProviderFactory#getProviders` will use `FakeKeyProvider$Factory` to create instances of `FakeKeyProvider`.

https://github.com/apache/spark/blob/5ccf9ba958f492c1eb4dde22a647ba75aba63d8e/sql/core/src/test/resources/META-INF/services/org.apache.hadoop.crypto.key.KeyProviderFactory#L18

During the initialization of `FakeKeyProvider`, it first initializes its superclass `org.apache.hadoop.crypto.key.KeyProvider`, which leads to the loading of the `BouncyCastleProvider` class. Therefore, we need to add bouncycastle-related test dependencies in the `hive-thrift` module.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manual Test with this pr.

```
build/mvn -Phive -Phive-thriftserver clean install -DskipTests
build/mvn -Phive -Phive-thriftserver clean install -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite -pl sql/hive-thriftserver
```

```
Run completed in 6 minutes, 52 seconds.
Total number of tests run: 243
Suites: completed 2, aborted 0
Tests: succeeded 243, failed 0, canceled 0, ignored 20, pending 0
All tests passed.
```

### Was this patch authored or co-authored using generative AI tooling?
No

Closes apache#47496 from LuciferYang/thrift-bouncycastle.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants