You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Steve Loughran (Jira)" <ji...@apache.org> on 2022/02/02 17:16:00 UTC

[jira] [Commented] (SPARK-37771) Race condition in withHiveState and limited logic in IsolatedClientLoader result in ClassNotFoundException

    [ https://issues.apache.org/jira/browse/SPARK-37771?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17485973#comment-17485973 ] 

Steve Loughran commented on SPARK-37771:
----------------------------------------

[~ivan.sadikov] -any update here?

> Race condition in withHiveState and limited logic in IsolatedClientLoader result in ClassNotFoundException
> ----------------------------------------------------------------------------------------------------------
>
>                 Key: SPARK-37771
>                 URL: https://issues.apache.org/jira/browse/SPARK-37771
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core
>    Affects Versions: 3.1.0, 3.1.2, 3.2.0
>            Reporter: Ivan Sadikov
>            Priority: Major
>
> There is a race condition between creating a Hive client and loading classes that do not appear in shared prefixes config. For example, we confirmed that the code fails for the following configuration:
> {code:java}
> spark.sql.hive.metastore.version 0.13.0
> spark.sql.hive.metastore.jars maven
> spark.sql.hive.metastore.sharedPrefixes <string that does not include com.amazonaws prefix>
> spark.hadoop.fs.s3a.impl org.apache.hadoop.fs.s3a.S3AFileSystem{code}
> And code: 
> {code:java}
> -- Prerequisite commands to set up the table
> -- drop table if exists ivan_test_2;
> -- create table ivan_test_2 (a int, part string) using csv location 's3://bucket/hive-test' partitioned by (part);
> -- insert into ivan_test_2 values (1, 'a'); 
> -- Command that triggers failure
> ALTER TABLE ivan_test_2 ADD PARTITION (part='b') LOCATION 's3://bucket/hive-test'{code}
>  
> Stacktrace (line numbers might differ):
> {code:java}
> 21/12/22 04:37:05 DEBUG IsolatedClientLoader: shared class: org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider
> 21/12/22 04:37:05 DEBUG IsolatedClientLoader: shared class: org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider
> 21/12/22 04:37:05 DEBUG IsolatedClientLoader: hive class: com.amazonaws.auth.EnvironmentVariableCredentialsProvider - null
> 21/12/22 04:37:05 ERROR S3AFileSystem: Failed to initialize S3AFileSystem for path s3://bucket/hive-test
> java.io.IOException: From option fs.s3a.aws.credentials.provider java.lang.ClassNotFoundException: Class com.amazonaws.auth.EnvironmentVariableCredentialsProvider not found
>     at org.apache.hadoop.fs.s3a.S3AUtils.loadAWSProviderClasses(S3AUtils.java:725)
>     at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:688)
>     at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:411)
>     at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3469)
>     at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:174)
>     at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3574)
>     at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521)
>     at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540)
>     at org.apache.hadoop.fs.Path.getFileSystem(Path.java:365)
>     at org.apache.hadoop.hive.metastore.Warehouse.getFs(Warehouse.java:112)
>     at org.apache.hadoop.hive.metastore.Warehouse.getDnsPath(Warehouse.java:144)
>     at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.createLocationForAddedPartition(HiveMetaStore.java:1993)
>     at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.add_partitions_core(HiveMetaStore.java:1865)
>     at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.add_partitions_req(HiveMetaStore.java:1910)
>     at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>     at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>     at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>     at java.lang.reflect.Method.invoke(Method.java:498)
>     at org.apache.hadoop.hive.metastore.RetryingHMSHandler.invoke(RetryingHMSHandler.java:105)
>     at com.sun.proxy.$Proxy58.add_partitions_req(Unknown Source)
>     at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.add_partitions(HiveMetaStoreClient.java:457)
>     at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>     at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>     at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>     at java.lang.reflect.Method.invoke(Method.java:498)
>     at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:89)
>     at com.sun.proxy.$Proxy59.add_partitions(Unknown Source)
>     at org.apache.hadoop.hive.ql.metadata.Hive.createPartitions(Hive.java:1514)
>     at org.apache.spark.sql.hive.client.Shim_v0_13.createPartitions(HiveShim.scala:773)
>     at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$createPartitions$1(HiveClientImpl.scala:683)
>     at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
>     at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:346)
>     at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$retryLocked$1(HiveClientImpl.scala:247)
>     at org.apache.spark.sql.hive.client.HiveClientImpl.synchronizeOnObject(HiveClientImpl.scala:283)
>     at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:239)
>     at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:326)
>     at org.apache.spark.sql.hive.client.HiveClientImpl.createPartitions(HiveClientImpl.scala:676)
>     at org.apache.spark.sql.hive.client.PoolingHiveClient.$anonfun$createPartitions$1(PoolingHiveClient.scala:345)
>     at org.apache.spark.sql.hive.client.PoolingHiveClient.$anonfun$createPartitions$1$adapted(PoolingHiveClient.scala:344)
>     at org.apache.spark.sql.hive.client.PoolingHiveClient.withHiveClient(PoolingHiveClient.scala:112)
>     at org.apache.spark.sql.hive.client.PoolingHiveClient.createPartitions(PoolingHiveClient.scala:344)
>     at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$createPartitions$1(HiveExternalCatalog.scala:1170)
>     at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
>     at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$withClient$2(HiveExternalCatalog.scala:150)
>     at org.apache.spark.sql.hive.HiveExternalCatalog.maybeSynchronized(HiveExternalCatalog.scala:111)
>     at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$withClient$1(HiveExternalCatalog.scala:149)
>     at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:148)
>     at org.apache.spark.sql.hive.HiveExternalCatalog.createPartitions(HiveExternalCatalog.scala:1152)
>     at org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener.createPartitions(ExternalCatalogWithListener.scala:213)
>     at org.apache.spark.sql.catalyst.catalog.SessionCatalogImpl.createPartitions(SessionCatalog.scala:1552)
>     at org.apache.spark.sql.execution.command.AlterTableAddPartitionCommand.$anonfun$run$16(ddl.scala:530)
>     at org.apache.spark.sql.execution.command.AlterTableAddPartitionCommand.$anonfun$run$16$adapted(ddl.scala:529)
>     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 org.apache.spark.sql.execution.command.AlterTableAddPartitionCommand.run(ddl.scala:529)
>     ...
>     at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.ClassNotFoundException: Class com.amazonaws.auth.EnvironmentVariableCredentialsProvider not found
>     at org.apache.hadoop.conf.Configuration.getClassByName(Configuration.java:2571)
>     at org.apache.hadoop.fs.s3a.S3AUtils.loadAWSProviderClasses(S3AUtils.java:722)
>     ... 118 more{code}
>  
> Based on the stacktrace, we call {{withHiveState}} method which sets {{IsolatedClientLoader.classLoader}} class loader for Hadoop configuration. All of the Hadoop configuration and file system code runs as a closure within {{{}withHiveState{}}}.
> Hadoop configuration uses the set class loader to load classes with {{conf.getClassByNameOrNull()}} method. When isolated class loader tries to load the class and the class is not shared (com.amazonaws prefix is not shared), it will only look up the jars that are loaded with the class loader without checking the base class loader.
> To mitigate the issue, you need to set:
>  * {{spark.sql.hive.metastore.sharedPrefixes <string that includes com.amazonaws prefix>}}
> Or update credentials providers to not load {{{}com.amazonaws.auth.EnvironmentVariableCredentialsProvider{}}}.
>  
> I think we should revisit the mechanism of sharing classes in IsolatedClassLoader and maybe only enforcing so that Hive classes are loaded first instead of in isolation.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org