Re: Possible Hive problem with Spark 2.0.0 preview.

2016-05-20 Thread Doug Balog
Some more info I’m still digging.
I’m just trying to do  `spark.table(“db.table”).count`from a spark-shell
“db.table” is just a hive table.

At commit b67668b this worked just fine and it returned the number of rows in 
db.table.
Starting at ca99171  "[SPARK-15073][SQL] Hide SparkSession constructor from the 
public” it fails with 

org.apache.spark.sql.AnalysisException: Database ‘db' does not exist;
  at 
org.apache.spark.sql.catalyst.catalog.ExternalCatalog.requireDbExists(ExternalCatalog.scala:37)
  at 
org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.tableExists(InMemoryCatalog.scala:195)
  at 
org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.requireTableExists(InMemoryCatalog.scala:63)
  at 
org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.getTable(InMemoryCatalog.scala:186)
  at 
org.apache.spark.sql.catalyst.catalog.SessionCatalog.lookupRelation(SessionCatalog.scala:337)
  at org.apache.spark.sql.SparkSession.table(SparkSession.scala:524)
  at org.apache.spark.sql.SparkSession.table(SparkSession.scala:520)
  ... 48 elided

If I run 
"org.apache.spark.sql.SparkSession.builder.enableHiveSupport.getOrCreate.catalog.listDatabases.show(False)”

+-+---+---+
|name   
  
|description|locationUri|
+-+---+---+
|Database[name='default', description='default database', 
path='hdfs://ns/{CWD}/spark-warehouse']|
+-+---+—+


 Where CWD is the current working directory of where I started my spark-shell.

It looks like this commit causes spark.catalog to be the internal one instead 
of the Hive one. 

Michael, I dont this this is related to the HDFS configurations, they are in 
/etc/hadoop/conf on each of the nodes in the cluster. 

Arun, I was referring to these docs, 
http://home.apache.org/~pwendell/spark-releases/spark-2.0.0-preview-docs/sql-programming-guide.html
 they need to be updated to no refer to HiveContext. 

I don’t think HiveContext should be marked as private[Hive], it should be 
public. 
 
I’ll keep digging.

Doug


> On May 19, 2016, at 6:52 PM, Reynold Xin  wrote:
> 
> The old one is deprecated but should still work though.
> 
> 
> On Thu, May 19, 2016 at 3:51 PM, Arun Allamsetty  
> wrote:
> Hi Doug,
> 
> If you look at the API docs here: 
> http://home.apache.org/~pwendell/spark-releases/spark-2.0.0-preview-docs/api/scala/index.html#org.apache.spark.sql.hive.HiveContext,
>  you'll see
> Deprecate (Since version 2.0.0) Use SparkSession.builder.enableHiveSupport 
> instead
> So you probably need to use that.
> 
> Arun
> 
> On Thu, May 19, 2016 at 3:44 PM, Michael Armbrust  
> wrote:
> 1. “val sqlContext = new org.apache.spark.sql.hive.HiveContext(sc)”  doesn’t 
> work because “HiveContext not a member of org.apache.spark.sql.hive”  I 
> checked the documentation, and it looks like it should still work for 
> spark-2.0.0-preview-bin-hadoop2.7.tgz
> 
> HiveContext has been deprecated and moved to a 1.x compatibility package, 
> which you'll need to include explicitly.  Docs have not been updated yet.
>  
> 2. I also tried the new spark session, ‘spark.table(“db.table”)’, it fails 
> with a HDFS permission denied can’t write to “/user/hive/warehouse”
> 
> Where are the HDFS configurations located?  We might not be propagating them 
> correctly any more. 
> 
> 


-
To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
For additional commands, e-mail: dev-h...@spark.apache.org



Possible Hive problem with Spark 2.0.0 preview.

2016-05-19 Thread Doug Balog
I haven’t had time to really look into this problem, but I want to mention it. 
I downloaded 
http://people.apache.org/~pwendell/spark-releases/spark-2.0.0-preview-bin/spark-2.0.0-preview-bin-hadoop2.7.tgz
and tried to run it against our Secure Hadoop cluster and access a Hive table.

1. “val sqlContext = new org.apache.spark.sql.hive.HiveContext(sc)”  doesn’t 
work because “HiveContext not a member of org.apache.spark.sql.hive”  I checked 
the documentation, and it looks like it should still work for 
spark-2.0.0-preview-bin-hadoop2.7.tgz

2. I also tried the new spark session, ‘spark.table(“db.table”)’, it fails with 
a HDFS permission denied can’t write to “/user/hive/warehouse”

Is there a new config option that I missed ? 

I tried a  SNAPSHOT version, downloaded from Patricks apache’s dir  from Apr 
26th,  that worked the way I expected.
I’m going to go through the commits and see which one broke the change, but my 
builds are not running (no such method ConcurrentHashMap.keySet()) so I have to 
fix that problem first.

Thanks for any hints. 

Doug



-
To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
For additional commands, e-mail: dev-h...@spark.apache.org



Re: Possible bug on Spark Yarn Client (1.5.1) during kerberos mode ?

2015-10-22 Thread Doug Balog

> On Oct 21, 2015, at 8:45 PM, Chester Chen <ches...@alpinenow.com> wrote:
> 
> Doug
>   thanks for responding. 
>  >>I think Spark just needs to be compiled against 1.2.1
> 
>Can you elaborate on this, or specific command you are referring ? 
> 
>In our build.scala, I was including the following
> 
> "org.spark-project.hive" % "hive-exec" % "1.2.1.spark" intransitive()
> 
>I am not sure how the Spark compilation is directly related to this, 
> please explain.   

I was referring to this comment
https://issues.apache.org/jira/browse/SPARK-6906?focusedCommentId=14712336=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14712336
And the updated documentation,
http://spark.apache.org/docs/latest/sql-programming-guide.html#interacting-with-different-versions-of-hive-metastore

Perhaps I misunderstood your question and why you are trying to compile against 
a different version of Hive.

> 
>When we submit the spark job, the we call Spark Yarn Client.scala directly 
> ( not using spark-submit). 
>The client side is not depending on spark-assembly jar ( which is in the 
> hadoop cluster).  The job submission actually failed in the client side. 
> 
>Currently we get around this by replace the spark's hive-exec with apache 
> hive-exec. 
> 

Why are you using the Spark Yarn Client.scala directly and not using the 
SparkLauncher that was introduced in 1.4.0 ?


Doug

> 
> 
> On Wed, Oct 21, 2015 at 5:27 PM, Doug Balog <d...@balog.net> wrote:
> See comments below.
> 
> > On Oct 21, 2015, at 5:33 PM, Chester Chen <ches...@alpinenow.com> wrote:
> >
> > All,
> >
> > just to see if this happens to other as well.
> >
> >   This is tested against the
> >
> >spark 1.5.1 ( branch 1.5  with label 1.5.2-SNAPSHOT with commit on Tue 
> > Oct 6, 84f510c4fa06e43bd35e2dc8e1008d0590cbe266)
> >
> >Spark deployment mode : Spark-Cluster
> >
> >Notice that if we enable Kerberos mode, the spark yarn client fails with 
> > the following:
> >
> > Could not initialize class org.apache.hadoop.hive.ql.metadata.Hive
> > java.lang.NoClassDefFoundError: Could not initialize class 
> > org.apache.hadoop.hive.ql.metadata.Hive
> > 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.yarn.Client$.org$apache$spark$deploy$yarn$Client$$obtainTokenForHiveMetastore(Client.scala:1252)
> > at 
> > org.apache.spark.deploy.yarn.Client.prepareLocalResources(Client.scala:271)
> > at 
> > org.apache.spark.deploy.yarn.Client.createContainerLaunchContext(Client.scala:629)
> > at 
> > org.apache.spark.deploy.yarn.Client.submitApplication(Client.scala:119)
> > at org.apache.spark.deploy.yarn.Client.run(Client.scala:907)
> >
> >
> > Diving in Yarn Client.scala code and tested against different dependencies 
> > and notice the followings:  if  the kerberos mode is enabled, 
> > Client.obtainTokenForHiveMetastore() will try to use scala reflection to 
> > get Hive and HiveConf and method on these method.
> >
> >   val hiveClass = 
> > mirror.classLoader.loadClass("org.apache.hadoop.hive.ql.metadata.Hive")
> >   val hive = hiveClass.getMethod("get").invoke(null)
> >
> >   val hiveConf = hiveClass.getMethod("getConf").invoke(hive)
> >   val hiveConfClass = 
> > mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf")
> >
> >   val hiveConfGet = (param: String) => Option(hiveConfClass
> > .getMethod("get", classOf[java.lang.String])
> > .invoke(hiveConf, param))
> >
> >If the "org.spark-project.hive" % "hive-exec" % "1.2.1.spark" is used, 
> > then you will get above exception. But if we use the
> >"org.apache.hive" % "hive-exec" "0.13.1-cdh5.2.0"
> >  The above method will not throw exception.
> >
> >   Here some questions and comments
> > 0) is this a bug ?
> 
> I’m not an expert on this, but I think this might not be a bug.
> The Hive integration was redone for 1.5.0, see 
> https://issues.apache.org/jira/browse/SPARK-6906
> and I think Spark just needs to be compiled aga

Re: [VOTE] Release Apache Spark 1.5.0 (RC1)

2015-08-25 Thread Doug Balog
It works for me in cluster mode. 
I’m running on Hortonworks 2.2.4.12 in secure mode with Hive 0.14
I built with

./make-distribution —tgz -Phive -Phive-thriftserver -Phbase-provided -Pyarn 
-Phadoop-2.6 

Doug



 On Aug 25, 2015, at 4:56 PM, Tom Graves tgraves...@yahoo.com.INVALID wrote:
 
 Anyone using HiveContext with secure Hive with Spark 1.5 and have it working?
 
 We have a non standard version of hive but was pulling our hive jars and its 
 failing to authenticate.  It could be something in our hive version but 
 wondering if spark isn't forwarding credentials properly.
 
 Tom
 
 
 
 On Tuesday, August 25, 2015 1:56 PM, Tom Graves 
 tgraves...@yahoo.com.INVALID wrote:
 
 
 Is there a jira to update the sql hive docs?
 Spark SQL and DataFrames - Spark 1.5.0 Documentation
  
  
  
  
  
  
 Spark SQL and DataFrames - Spark 1.5.0 Documentation
 Spark SQL and DataFrame Guide Overview DataFrames Starting Point: SQLContext 
 Creating DataFrames DataFrame Operations Running SQL Queries Programmatically 
 Interoperating with RDDs
 View on people.apache.org
 Preview by Yahoo
  
 
 it still says default is 0.13.1 but pom file builds with hive 1.2.1-spark.
 
 Tom
 
 
 
 On Monday, August 24, 2015 4:06 PM, Sandy Ryza sandy.r...@cloudera.com 
 wrote:
 
 
 I see that there's an 1.5.0-rc2 tag in github now.  Is that the official RC2 
 tag to start trying out?
 
 -Sandy
 
 On Mon, Aug 24, 2015 at 7:23 AM, Sean Owen so...@cloudera.com wrote:
 PS Shixiong Zhu is correct that this one has to be fixed:
 https://issues.apache.org/jira/browse/SPARK-10168
 
 For example you can see assemblies like this are nearly empty:
 https://repository.apache.org/content/repositories/orgapachespark-1137/org/apache/spark/spark-streaming-flume-assembly_2.10/1.5.0-rc1/
 
 Just a publishing glitch but worth a few more eyes on.
 
 On Fri, Aug 21, 2015 at 5:28 PM, Sean Owen so...@cloudera.com wrote:
  Signatures, license, etc. look good. I'm getting some fairly
  consistent failures using Java 7 + Ubuntu 15 + -Pyarn -Phive
  -Phive-thriftserver -Phadoop-2.6 -- does anyone else see these? they
  are likely just test problems, but worth asking. Stack traces are at
  the end.
 
  There are currently 79 issues targeted for 1.5.0, of which 19 are
  bugs, of which 1 is a blocker. (1032 have been resolved for 1.5.0.)
  That's significantly better than at the last release. I presume a lot
  of what's still targeted is not critical and can now be
  untargeted/retargeted.
 
  It occurs to me that the flurry of planning that took place at the
  start of the 1.5 QA cycle a few weeks ago was quite helpful, and is
  the kind of thing that would be even more useful at the start of a
  release cycle. So would be great to do this for 1.6 in a few weeks.
  Indeed there are already 267 issues targeted for 1.6.0 -- a decent
  roadmap already.
 
 
  Test failures:
 
  Core
 
  - Unpersisting TorrentBroadcast on executors and driver in distributed
  mode *** FAILED ***
java.util.concurrent.TimeoutException: Can't find 2 executors before
  1 milliseconds elapsed
at 
  org.apache.spark.ui.jobs.JobProgressListener.waitUntilExecutorsUp(JobProgressListener.scala:561)
at 
  org.apache.spark.broadcast.BroadcastSuite.testUnpersistBroadcast(BroadcastSuite.scala:313)
at 
  org.apache.spark.broadcast.BroadcastSuite.org$apache$spark$broadcast$BroadcastSuite$$testUnpersistTorrentBroadcast(BroadcastSuite.scala:287)
at 
  org.apache.spark.broadcast.BroadcastSuite$$anonfun$16.apply$mcV$sp(BroadcastSuite.scala:165)
at 
  org.apache.spark.broadcast.BroadcastSuite$$anonfun$16.apply(BroadcastSuite.scala:165)
at 
  org.apache.spark.broadcast.BroadcastSuite$$anonfun$16.apply(BroadcastSuite.scala:165)
at 
  org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
at org.scalatest.Transformer.apply(Transformer.scala:22)
...
 
  Streaming
 
  - stop slow receiver gracefully *** FAILED ***
0 was not greater than 0 (StreamingContextSuite.scala:324)
 
  Kafka
 
  - offset recovery *** FAILED ***
The code passed to eventually never returned normally. Attempted 191
  times over 10.043196973 seconds. Last failure message:
  strings.forall({
  ((elem: Any) = DirectKafkaStreamSuite.collectedData.contains(elem))
}) was false. (DirectKafkaStreamSuite.scala:249)
 
  On Fri, Aug 21, 2015 at 5:37 AM, Reynold Xin r...@databricks.com wrote:
  Please vote on releasing the following candidate as Apache Spark version
  1.5.0!
 
  The vote is open until Monday, Aug 17, 2015 at 20:00 UTC and passes if a
  majority of at least 3 +1 PMC votes are cast.
 
  [ ] +1 Release this package as Apache Spark 1.5.0
  [ ] -1 Do not release this package because ...
 
  To learn more about Apache Spark, please see http://spark.apache.org/
 
 
  The tag to be voted on is v1.5.0-rc1:
  

Support for Hive 0.14 in secure mode on hadoop 2.6.0

2015-03-27 Thread Doug Balog
Hi, 
  I'm just wondering if anybody is working on supporting Hive 0.14 in secure 
mode on hadoop 2.6.0 ?
I see once Jira referring to it  
https://issues.apache.org/jira/browse/SPARK-5111
but it mentions no effort to move to 0.14.

Thanks,

Doug



-
To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
For additional commands, e-mail: dev-h...@spark.apache.org