[ https://issues.apache.org/jira/browse/SPARK-9206?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Sean Owen updated SPARK-9206: ----------------------------- Assignee: Dennis Huo > ClassCastException using HiveContext with GoogleHadoopFileSystem as > fs.defaultFS > -------------------------------------------------------------------------------- > > Key: SPARK-9206 > URL: https://issues.apache.org/jira/browse/SPARK-9206 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 1.4.1 > Reporter: Dennis Huo > Assignee: Dennis Huo > Fix For: 1.5.0 > > > Originally reported on StackOverflow: > http://stackoverflow.com/questions/31478955/googlehadoopfilesystem-cannot-be-cast-to-hadoop-filesystem > Google's "bdutil" command-line tool > (https://github.com/GoogleCloudPlatform/bdutil) is one of the main supported > ways of deploying Hadoop and Spark cluster on Google Cloud Platform, and has > default settings which configure fs.defaultFS to use the Google Cloud Storage > connector for Hadoop (and performs installation of the connector jarfile on > top of tarball-based Hadoop and Spark distributions). > Starting in Spark 1.4.1, taking a default bdutil-based Spark deployment, > running "spark-shell", and then trying to read a file with sqlContext like: > {code} > sqlContext.parquetFile("gs://my-bucket/my-file.parquet") > {code} > results in the following: > {noformat} > 15/07/20 20:59:14 DEBUG IsolatedClientLoader: shared class: > com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystem > java.lang.RuntimeException: java.lang.ClassCastException: > com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystem cannot be cast to > org.apache.hadoop.fs.FileSystem > at > org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:346) > at > org.apache.spark.sql.hive.client.ClientWrapper.<init>(ClientWrapper.scala:116) > at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) > at > sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57) > at > sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) > at java.lang.reflect.Constructor.newInstance(Constructor.java:526) > at > org.apache.spark.sql.hive.client.IsolatedClientLoader.liftedTree1$1(IsolatedClientLoader.scala:172) > at > org.apache.spark.sql.hive.client.IsolatedClientLoader.<init>(IsolatedClientLoader.scala:168) > at > org.apache.spark.sql.hive.HiveContext.metadataHive$lzycompute(HiveContext.scala:213) > at > org.apache.spark.sql.hive.HiveContext.metadataHive(HiveContext.scala:176) > at > org.apache.spark.sql.hive.HiveContext$$anon$2.<init>(HiveContext.scala:371) > at > org.apache.spark.sql.hive.HiveContext.catalog$lzycompute(HiveContext.scala:371) > at org.apache.spark.sql.hive.HiveContext.catalog(HiveContext.scala:370) > at > org.apache.spark.sql.hive.HiveContext$$anon$1.<init>(HiveContext.scala:383) > at > org.apache.spark.sql.hive.HiveContext.analyzer$lzycompute(HiveContext.scala:383) > at org.apache.spark.sql.hive.HiveContext.analyzer(HiveContext.scala:382) > at > org.apache.spark.sql.SQLContext$QueryExecution.assertAnalyzed(SQLContext.scala:931) > 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.baseRelationToDataFrame(SQLContext.scala:438) > at > org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:264) > at org.apache.spark.sql.SQLContext.parquetFile(SQLContext.scala:1099) > at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:19) > at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:24) > at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:26) > at $iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:28) > at $iwC$$iwC$$iwC$$iwC.<init>(<console>:30) > at $iwC$$iwC$$iwC.<init>(<console>:32) > at $iwC$$iwC.<init>(<console>:34) > at $iwC.<init>(<console>:36) > at <init>(<console>:38) > at .<init>(<console>:42) > at .<clinit>(<console>) > at .<init>(<console>:7) > at .<clinit>(<console>) > at $print(<console>) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:606) > at > org.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065) > at > org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1338) > at > org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840) > at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871) > at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819) > at > org.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:857) > at > org.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:902) > at org.apache.spark.repl.SparkILoop.command(SparkILoop.scala:814) > at org.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:657) > at org.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:665) > at > org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$loop(SparkILoop.scala:670) > at > org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply$mcZ$sp(SparkILoop.scala:997) > at > org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945) > at > org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945) > at > scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135) > at > org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$process(SparkILoop.scala:945) > at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:1059) > at org.apache.spark.repl.Main$.main(Main.scala:31) > at org.apache.spark.repl.Main.main(Main.scala) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:606) > at > org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:665) > at > org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:170) > at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:193) > at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:112) > at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) > Caused by: java.lang.ClassCastException: > com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystem cannot be cast to > org.apache.hadoop.fs.FileSystem > at > org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2595) > at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:91) > at > org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2630) > at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2612) > at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:370) > at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:169) > at > org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:342) > ... 67 more > {noformat} > This appears to be a combination of > https://github.com/apache/spark/commit/9ac8393663d759860c67799e000ec072ced76493 > and its related "isolated classloader" changes with the > IsolatedClientLoader.isSharedClass method including "com.google.*" alongside > java.lang.*, java.net.*, etc., as shared classes, presumably for inclusion of > Guava and possibly protobuf and gson libraries. > Unfortunately, this also includes the Hadoop extended libraries like > com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystem > (https://github.com/GoogleCloudPlatform/bigdata-interop) and > com.google.cloud.bigtable.* > (https://github.com/GoogleCloudPlatform/cloud-bigtable-client). > This can be reproduced by downloading bdutil from > https://github.com/GoogleCloudPlatform/bdutil, modifying > bdutil/extensions/spark/spark_env.sh to set SPARK_HADOOP2_TARBALL_URI to some > Spark 1.4.1 tarball URI (http URIs should work as well) and then deploying a > cluster with: > {code} > ./bdutil -p <your project> -b <your GCS bucket> -z us-central1-f -e hadoop2 > -e spark deploy > ./bdutil -p <your project> -b <your GCS bucket> -z us-central1-f -e hadoop2 > -e spark shell > {code} > The last command opens an SSH session; then type: > {code} > spark-shell > > sqlContext.parquetFile("gs://your-bucket/some/path/to/parquet/file.parquet") > {code} > The ClassCastException should then immediately get thrown. > The simple fix of simply excluding com.google.cloud.* from being "shared > classes" appears to work just fine in an end-to-end bdutil-based deployment. -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org