Hopefully there could be some progress on SPARK-2420. It looks like shading may 
be the voted solution among downgrading.
Any idea when this will happen? Could it happen in Spark 1.1.1 or Spark 1.1.2? 
By the way, regarding bin/spark-sql? Is this more of a debugging tool for Spark 
job integrating with Hive? How does people use spark-sql? I'm trying to 
understand the rationale and motivation behind this script, any idea?

> Date: Thu, 21 Aug 2014 16:31:08 -0700
> Subject: Re: Hive From Spark
> From: van...@cloudera.com
> To: l...@yahoo-inc.com.invalid
> CC: user@spark.apache.org; u...@spark.incubator.apache.org; pwend...@gmail.com
> 
> Hi Du,
> 
> I don't believe the Guava change has made it to the 1.1 branch. The
> Guava doc says "hashInt" was added in 12.0, so what's probably
> happening is that you have and old version of Guava in your classpath
> before the Spark jars. (Hadoop ships with Guava 11, so that may be the
> source of your problem.)
> 
> On Thu, Aug 21, 2014 at 4:23 PM, Du Li <l...@yahoo-inc.com.invalid> wrote:
> > Hi,
> >
> > This guava dependency conflict problem should have been fixed as of
> > yesterday according to https://issues.apache.org/jira/browse/SPARK-2420
> >
> > However, I just got java.lang.NoSuchMethodError:
> > com.google.common.hash.HashFunction.hashInt(I)Lcom/google/common/hash/HashCode;
> > by the following code snippet and “mvn3 test” on Mac. I built the latest
> > version of spark (1.1.0-SNAPSHOT) and installed the jar files to the local
> > maven repo. From my pom file I explicitly excluded guava from almost all
> > possible dependencies, such as spark-hive_2.10-1.1.0.SNAPSHOT, and
> > hadoop-client. This snippet is abstracted from a larger project. So the
> > pom.xml includes many dependencies although not all are required by this
> > snippet. The pom.xml is attached.
> >
> > Anybody knows what to fix it?
> >
> > Thanks,
> > Du
> > -------
> >
> > package com.myself.test
> >
> > import org.scalatest._
> > import org.apache.hadoop.io.{NullWritable, BytesWritable}
> > import org.apache.spark.{SparkContext, SparkConf}
> > import org.apache.spark.SparkContext._
> >
> > class MyRecord(name: String) extends Serializable {
> >   def getWritable(): BytesWritable = {
> >     new
> > BytesWritable(Option(name).getOrElse("\\N").toString.getBytes("UTF-8"))
> >   }
> >
> >   final override def equals(that: Any): Boolean = {
> >     if( !that.isInstanceOf[MyRecord] )
> >       false
> >     else {
> >       val other = that.asInstanceOf[MyRecord]
> >       this.getWritable == other.getWritable
> >     }
> >   }
> > }
> >
> > class MyRecordTestSuite extends FunSuite {
> >   // construct an MyRecord by Consumer.schema
> >   val rec: MyRecord = new MyRecord("James Bond")
> >
> >   test("generated SequenceFile should be readable from spark") {
> >     val path = "./testdata/"
> >
> >     val conf = new SparkConf(false).setMaster("local").setAppName("test data
> > exchange with Hive")
> >     conf.set("spark.driver.host", "localhost")
> >     val sc = new SparkContext(conf)
> >     val rdd = sc.makeRDD(Seq(rec))
> >     rdd.map((x: MyRecord) => (NullWritable.get(), x.getWritable()))
> >       .saveAsSequenceFile(path)
> >
> >     val bytes = sc.sequenceFile(path, classOf[NullWritable],
> > classOf[BytesWritable]).first._2
> >     assert(rec.getWritable() == bytes)
> >
> >     sc.stop()
> >     System.clearProperty("spark.driver.port")
> >   }
> > }
> >
> >
> > From: Andrew Lee <alee...@hotmail.com>
> > Reply-To: "user@spark.apache.org" <user@spark.apache.org>
> > Date: Monday, July 21, 2014 at 10:27 AM
> > To: "user@spark.apache.org" <user@spark.apache.org>,
> > "u...@spark.incubator.apache.org" <u...@spark.incubator.apache.org>
> >
> > Subject: RE: Hive From Spark
> >
> > Hi All,
> >
> > Currently, if you are running Spark HiveContext API with Hive 0.12, it won't
> > work due to the following 2 libraries which are not consistent with Hive
> > 0.12 and Hadoop as well. (Hive libs aligns with Hadoop libs, and as a common
> > practice, they should be consistent to work inter-operable).
> >
> > These are under discussion in the 2 JIRA tickets:
> >
> > https://issues.apache.org/jira/browse/HIVE-7387
> >
> > https://issues.apache.org/jira/browse/SPARK-2420
> >
> > When I ran the command by tweaking the classpath and build for Spark
> > 1.0.1-rc3, I was able to create table through HiveContext, however, when I
> > fetch the data, due to incompatible API calls in Guava, it breaks. This is
> > critical since it needs to map the cllumns to the RDD schema.
> >
> > Hive and Hadoop are using an older version of guava libraries (11.0.1) where
> > Spark Hive is using guava 14.0.1+.
> > The community isn't willing to downgrade to 11.0.1 which is the current
> > version for Hadoop 2.2 and Hive 0.12.
> > Be aware of protobuf version as well in Hive 0.12 (it uses protobuf 2.4).
> >
> > scala>
> >
> > scala> import org.apache.spark.SparkContext
> > import org.apache.spark.SparkContext
> >
> > scala> import org.apache.spark.sql.hive._
> > import org.apache.spark.sql.hive._
> >
> > scala>
> >
> > scala> val hiveContext = new org.apache.spark.sql.hive.HiveContext(sc)
> > hiveContext: org.apache.spark.sql.hive.HiveContext =
> > org.apache.spark.sql.hive.HiveContext@34bee01a
> >
> > scala>
> >
> > scala> hiveContext.hql("CREATE TABLE IF NOT EXISTS src (key INT, value
> > STRING)")
> > res0: org.apache.spark.sql.SchemaRDD =
> > SchemaRDD[0] at RDD at SchemaRDD.scala:104
> > == Query Plan ==
> > <Native command: executed by Hive>
> >
> > scala> hiveContext.hql("LOAD DATA LOCAL INPATH
> > 'examples/src/main/resources/kv1.txt' INTO TABLE src")
> > res1: org.apache.spark.sql.SchemaRDD =
> > SchemaRDD[3] at RDD at SchemaRDD.scala:104
> > == Query Plan ==
> > <Native command: executed by Hive>
> >
> > scala>
> >
> > scala> // Queries are expressed in HiveQL
> >
> > scala> hiveContext.hql("FROM src SELECT key,
> > value").collect().foreach(println)
> > java.lang.NoSuchMethodError:
> > com.google.common.hash.HashFunction.hashInt(I)Lcom/google/common/hash/HashCode;
> > at
> > org.apache.spark.util.collection.OpenHashSet.org$apache$spark$util$collection$OpenHashSet$$hashcode(OpenHashSet.scala:261)
> > at
> > org.apache.spark.util.collection.OpenHashSet$mcI$sp.getPos$mcI$sp(OpenHashSet.scala:165)
> > at
> > org.apache.spark.util.collection.OpenHashSet$mcI$sp.contains$mcI$sp(OpenHashSet.scala:102)
> > at
> > org.apache.spark.util.SizeEstimator$$anonfun$visitArray$2.apply$mcVI$sp(SizeEstimator.scala:214)
> > at scala.collection.immutable.Range.foreach$mVc$sp(Range.scala:141)
> > at org.apache.spark.util.SizeEstimator$.visitArray(SizeEstimator.scala:210)
> > at
> > org.apache.spark.util.SizeEstimator$.visitSingleObject(SizeEstimator.scala:169)
> > at
> > org.apache.spark.util.SizeEstimator$.org$apache$spark$util$SizeEstimator$$estimate(SizeEstimator.scala:161)
> > at org.apache.spark.util.SizeEstimator$.estimate(SizeEstimator.scala:155)
> > at org.apache.spark.storage.MemoryStore.putValues(MemoryStore.scala:75)
> > at org.apache.spark.storage.MemoryStore.putValues(MemoryStore.scala:92)
> > at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:661)
> > at org.apache.spark.storage.BlockManager.put(BlockManager.scala:546)
> > at org.apache.spark.storage.BlockManager.putSingle(BlockManager.scala:812)
> > at org.apache.spark.broadcast.HttpBroadcast.<init>(HttpBroadcast.scala:52)
> > at
> > org.apache.spark.broadcast.HttpBroadcastFactory.newBroadcast(HttpBroadcastFactory.scala:35)
> > at
> > org.apache.spark.broadcast.HttpBroadcastFactory.newBroadcast(HttpBroadcastFactory.scala:29)
> > at
> > org.apache.spark.broadcast.BroadcastManager.newBroadcast(BroadcastManager.scala:62)
> > at org.apache.spark.SparkContext.broadcast(SparkContext.scala:776)
> > at org.apache.spark.sql.hive.HadoopTableReader.<init>(TableReader.scala:60)
> > at
> > org.apache.spark.sql.hive.execution.HiveTableScan.<init>(HiveTableScan.scala:70)
> > at
> > org.apache.spark.sql.hive.HiveStrategies$HiveTableScans$$anonfun$4.apply(HiveStrategies.scala:73)
> > at
> > org.apache.spark.sql.hive.HiveStrategies$HiveTableScans$$anonfun$4.apply(HiveStrategies.scala:73)
> > at
> > org.apache.spark.sql.SQLContext$SparkPlanner.pruneFilterProject(SQLContext.scala:280)
> > at
> > org.apache.spark.sql.hive.HiveStrategies$HiveTableScans$.apply(HiveStrategies.scala:69)
> > at
> > org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58)
> > at
> > org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58)
> > at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371)
> > at
> > org.apache.spark.sql.catalyst.planning.QueryPlanner.apply(QueryPlanner.scala:59)
> > at
> > org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan$lzycompute(SQLContext.scala:316)
> > at
> > org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan(SQLContext.scala:316)
> > at
> > org.apache.spark.sql.SQLContext$QueryExecution.executedPlan$lzycompute(SQLContext.scala:319)
> > at
> > org.apache.spark.sql.SQLContext$QueryExecution.executedPlan(SQLContext.scala:319)
> > at org.apache.spark.sql.SchemaRDD.collect(SchemaRDD.scala:420)
> > at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:19)
> > at $iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:24)
> > at $iwC$$iwC$$iwC$$iwC.<init>(<console>:26)
> > at $iwC$$iwC$$iwC.<init>(<console>:28)
> > at $iwC$$iwC.<init>(<console>:30)
> > at $iwC.<init>(<console>:32)
> > at <init>(<console>:34)
> > at .<init>(<console>:38)
> > 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:788)
> > at
> > org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1056)
> > at org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:614)
> > at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:645)
> > at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:609)
> > at org.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:796)
> > at
> > org.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:841)
> > at org.apache.spark.repl.SparkILoop.command(SparkILoop.scala:753)
> > at org.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:601)
> > at org.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:608)
> > at org.apache.spark.repl.SparkILoop.loop(SparkILoop.scala:611)
> > at
> > org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply$mcZ$sp(SparkILoop.scala:936)
> > at
> > org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply(SparkILoop.scala:884)
> > at
> > org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply(SparkILoop.scala:884)
> > at
> > scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135)
> > at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:884)
> > at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:982)
> > 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$.launch(SparkSubmit.scala:303)
> > at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:55)
> > at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
> >
> >
> >
> >> From: hao.ch...@intel.com
> >> To: user@spark.apache.org; u...@spark.incubator.apache.org
> >> Subject: RE: Hive From Spark
> >> Date: Mon, 21 Jul 2014 01:14:19 +0000
> >>
> >> JiaJia, I've checkout the latest 1.0 branch, and then do the following
> >> steps:
> >> SPAKR_HIVE=true sbt/sbt clean assembly
> >> cd examples
> >> ../bin/run-example sql.hive.HiveFromSpark
> >>
> >> It works well in my local
> >>
> >> From your log output, it shows "Invalid method name: 'get_table', seems an
> >> incompatible jar version or something wrong between the Hive metastore
> >> service and client, can you double check the jar versions of Hive metastore
> >> service or thrift?
> >>
> >>
> >> -----Original Message-----
> >> From: JiajiaJing [mailto:jj.jing0...@gmail.com]
> >> Sent: Saturday, July 19, 2014 7:29 AM
> >> To: u...@spark.incubator.apache.org
> >> Subject: RE: Hive From Spark
> >>
> >> Hi Cheng Hao,
> >>
> >> Thank you very much for your reply.
> >>
> >> Basically, the program runs on Spark 1.0.0 and Hive 0.12.0 .
> >>
> >> Some setups of the environment are done by running "SPARK_HIVE=true
> >> sbt/sbt assembly/assembly", including the jar in all the workers, and
> >> copying the hive-site.xml to spark's conf dir.
> >>
> >> And then run the program as: " ./bin/run-example
> >> org.apache.spark.examples.sql.hive.HiveFromSpark "
> >>
> >> It's good to know that this example runs well on your machine, could you
> >> please give me some insight about your have done as well?
> >>
> >> Thank you very much!
> >>
> >> Jiajia
> >>
> >>
> >>
> >>
> >>
> >>
> >>
> >> --
> >> View this message in context:
> >> http://apache-spark-user-list.1001560.n3.nabble.com/Hive-From-Spark-tp10110p10215.html
> >> Sent from the Apache Spark User List mailing list archive at Nabble.com.
> >
> >
> > ---------------------------------------------------------------------
> > To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
> > For additional commands, e-mail: user-h...@spark.apache.org
> 
> 
> 
> -- 
> Marcelo
> 
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
> For additional commands, e-mail: user-h...@spark.apache.org
> 
                                          

Reply via email to