[ https://issues.apache.org/jira/browse/SPARK-9974?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Apache Spark reassigned SPARK-9974: ----------------------------------- Assignee: (was: Apache Spark) > SBT build: com.twitter:parquet-hadoop-bundle:1.6.0 is not packaged into the > assembly jar > ---------------------------------------------------------------------------------------- > > Key: SPARK-9974 > URL: https://issues.apache.org/jira/browse/SPARK-9974 > Project: Spark > Issue Type: Bug > Components: Build, SQL > Affects Versions: 1.5.0 > Reporter: Cheng Lian > Priority: Blocker > > One of the consequence of this issue is that Parquet tables created in Hive > are not accessible from Spark SQL built with SBT. Maven build is OK. > Git commit: > [69930310115501f0de094fe6f5c6c60dade342bd|https://github.com/apache/spark/commit/69930310115501f0de094fe6f5c6c60dade342bd] > Build with SBT and check the assembly jar for > {{parquet.hadoop.api.WriteSupport}}: > {noformat} > $ ./build/sbt -Phive -Phive-thriftserver -Phadoop-1 -Dhadoop.version=1.2.1 > clean assembly/assembly > ... > $ jar tf > assembly/target/scala-2.10/spark-assembly-1.5.0-SNAPSHOT-hadoop1.2.1.jar | > fgrep "parquet/hadoop/api" > org/apache/parquet/hadoop/api/ > org/apache/parquet/hadoop/api/DelegatingReadSupport.class > org/apache/parquet/hadoop/api/DelegatingWriteSupport.class > org/apache/parquet/hadoop/api/InitContext.class > org/apache/parquet/hadoop/api/ReadSupport$ReadContext.class > org/apache/parquet/hadoop/api/ReadSupport.class > org/apache/parquet/hadoop/api/WriteSupport$FinalizedWriteContext.class > org/apache/parquet/hadoop/api/WriteSupport$WriteContext.class > org/apache/parquet/hadoop/api/WriteSupport.class > {noformat} > Only classes of {{org.apache.parquet:parquet-mr:1.7.0}}. Note that classes in > {{com.twitter:parquet-hadoop-bundle:1.6.0}} are not under the {{org.apache}} > namespace. > Build with Maven and check the assembly jar for > {{parquet.hadoop.api.WriteSupport}}: > {noformat} > $ ./build/mvn -Phive -Phive-thriftserver -Phadoop-1 -Dhadoop.version=1.2.1 > -DskipTests clean package > $ jar tf > assembly/target/scala-2.10/spark-assembly-1.5.0-SNAPSHOT-hadoop1.2.1.jar | > fgrep "parquet/hadoop/api" > org/apache/parquet/hadoop/api/ > org/apache/parquet/hadoop/api/DelegatingReadSupport.class > org/apache/parquet/hadoop/api/DelegatingWriteSupport.class > org/apache/parquet/hadoop/api/InitContext.class > org/apache/parquet/hadoop/api/ReadSupport$ReadContext.class > org/apache/parquet/hadoop/api/ReadSupport.class > org/apache/parquet/hadoop/api/WriteSupport$FinalizedWriteContext.class > org/apache/parquet/hadoop/api/WriteSupport$WriteContext.class > org/apache/parquet/hadoop/api/WriteSupport.class > parquet/hadoop/api/ > parquet/hadoop/api/DelegatingReadSupport.class > parquet/hadoop/api/DelegatingWriteSupport.class > parquet/hadoop/api/InitContext.class > parquet/hadoop/api/ReadSupport$ReadContext.class > parquet/hadoop/api/ReadSupport.class > parquet/hadoop/api/WriteSupport$FinalizedWriteContext.class > parquet/hadoop/api/WriteSupport$WriteContext.class > parquet/hadoop/api/WriteSupport.class > {noformat} > Expected classes are packaged properly. > To reproduce the Parquet table access issue, first create a Parquet table > with Hive (say 0.13.1): > {noformat} > hive> CREATE TABLE parquet_test STORED AS PARQUET AS SELECT 1; > {noformat} > Build Spark assembly jar with the SBT command above, start {{spark-shell}}: > {noformat} > scala> sqlContext.table("parquet_test").show() > 15/08/14 17:52:50 INFO HiveMetaStore: 0: get_table : db=default > tbl=parquet_test > 15/08/14 17:52:50 INFO audit: ugi=lian ip=unknown-ip-addr cmd=get_table > : db=default tbl=parquet_test > java.lang.NoClassDefFoundError: parquet/hadoop/api/WriteSupport > at java.lang.Class.forName0(Native Method) > at java.lang.Class.forName(Class.java:270) > at > org.apache.hadoop.hive.ql.metadata.Table.getOutputFormatClass(Table.java:328) > at > org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$getTableOption$1$$anonfun$2.apply(ClientWrapper.scala:320) > at > org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$getTableOption$1$$anonfun$2.apply(ClientWrapper.scala:303) > at scala.Option.map(Option.scala:145) > at > org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$getTableOption$1.apply(ClientWrapper.scala:303) > at > org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$getTableOption$1.apply(ClientWrapper.scala:298) > at > org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$withHiveState$1.apply(ClientWrapper.scala:256) > at > org.apache.spark.sql.hive.client.ClientWrapper.retryLocked(ClientWrapper.scala:211) > at > org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:248) > at > org.apache.spark.sql.hive.client.ClientWrapper.getTableOption(ClientWrapper.scala:298) > at > org.apache.spark.sql.hive.client.ClientInterface$class.getTable(ClientInterface.scala:123) > at > org.apache.spark.sql.hive.client.ClientWrapper.getTable(ClientWrapper.scala:60) > at > org.apache.spark.sql.hive.HiveMetastoreCatalog.lookupRelation(HiveMetastoreCatalog.scala:397) > at > org.apache.spark.sql.hive.HiveContext$$anon$1.org$apache$spark$sql$catalyst$analysis$OverrideCatalog$$super$lookupRelation(HiveContext.scala:403) > at > org.apache.spark.sql.catalyst.analysis.OverrideCatalog$$anonfun$lookupRelation$3.apply(Catalog.scala:170) > at > org.apache.spark.sql.catalyst.analysis.OverrideCatalog$$anonfun$lookupRelation$3.apply(Catalog.scala:170) > at scala.Option.getOrElse(Option.scala:120) > at > org.apache.spark.sql.catalyst.analysis.OverrideCatalog$class.lookupRelation(Catalog.scala:170) > at > org.apache.spark.sql.hive.HiveContext$$anon$1.lookupRelation(HiveContext.scala:403) > at org.apache.spark.sql.SQLContext.table(SQLContext.scala:806) > at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:20) > at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:25) > at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:27) > at $iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:29) > at $iwC$$iwC$$iwC$$iwC.<init>(<console>:31) > at $iwC$$iwC$$iwC.<init>(<console>:33) > at $iwC$$iwC.<init>(<console>:35) > at $iwC.<init>(<console>:37) > at <init>(<console>:39) > at .<init>(<console>:43) > 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:1340) > 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:672) > at > org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:180) > at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:205) > at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:120) > at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) > Caused by: java.lang.ClassNotFoundException: parquet.hadoop.api.WriteSupport > at > scala.tools.nsc.interpreter.AbstractFileClassLoader.findClass(AbstractFileClassLoader.scala:83) > at java.lang.ClassLoader.loadClass(ClassLoader.java:425) > at java.lang.ClassLoader.loadClass(ClassLoader.java:358) > at > org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1.doLoadClass(IsolatedClientLoader.scala:169) > at > org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1.loadClass(IsolatedClientLoader.scala:153) > at java.lang.ClassLoader.loadClass(ClassLoader.java:358) > ... 68 more > {noformat} -- 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