Re: Reading from HBase is too slow

2014-10-08 Thread Tao Xiao
Hi Sean, Do I need to specify the number of executors when submitting the job? I suppose the number of executors will be determined by the number of regions of the table. Just like a MapReduce job, you needn't specify the number of map tasks when reading from a HBase table. The script to su

Re: Spark SQL question: why build hashtable for both sides in HashOuterJoin?

2014-10-08 Thread Matei Zaharia
I'm pretty sure inner joins on Spark SQL already build only one of the sides. Take a look at ShuffledHashJoin, which calls HashJoin.joinIterators. Only outer joins do both, and it seems like we could optimize it for those that are not full. Matei On Oct 7, 2014, at 11:04 PM, Haopu Wang wrote

Re: Reading from HBase is too slow

2014-10-08 Thread Sean Owen
You do need to specify the number of executor cores to use. Executors are not like mappers. After all they may do much more in their lifetime than just read splits from HBase so would not make sense to determine it by something that the first line of the program does. On Oct 8, 2014 8:00 AM, "Tao X

Re: Spark SQL question: why build hashtable for both sides in HashOuterJoin?

2014-10-08 Thread Liquan Pei
I am working on a PR to leverage the HashJoin trait code to optimize the Left/Right outer join. It's already been tested locally and will send out the PR soon after some clean up. Thanks, Liquan On Wed, Oct 8, 2014 at 12:09 AM, Matei Zaharia wrote: > I'm pretty sure inner joins on Spark SQL alr

Mosek Solver with Apache Spark

2014-10-08 Thread Raghuveer Chanda
Hi, Has anyone tried Mosek Solver in Spark? I getting weird serialization errors. I came to know that Mosek uses shared libraries which may not be serialized. Is this the reason that they are not serialized or Is it working for anyone. -- Regards, Raghuveer Chanda 4th y

Re: Any issues with repartition?

2014-10-08 Thread Paul Wais
Looks like an OOM issue? Have you tried persisting your RDDs to allow disk writes? I've seen a lot of similar crashes in a Spark app that reads from HDFS and does joins. I.e. I've seen "java.io.IOException: Filesystem closed," "Executor lost," "FetchFailed," etc etc with non-deterministic crashe

SparkContext.wholeTextFiles() java.io.FileNotFoundException: File does not exist:

2014-10-08 Thread jan.zikes
Hi I'm trying to use sc.wholeTextFiles() on file that is stored amazon S3 I'm getting following Error:   14/10/08 06:09:50 INFO input.FileInputFormat: Total input paths to process : 1 14/10/08 06:09:50 INFO input.FileInputFormat: Total input paths to process : 1 Traceback (most recent call last):

Re: How to do broadcast join in SparkSQL

2014-10-08 Thread Jianshi Huang
Ok, currently there's cost-based optimization however Parquet statistics is not implemented... What's the good way if I want to join a big fact table with several tiny dimension tables in Spark SQL (1.1)? I wish we can allow user hint for the join. Jianshi On Wed, Oct 8, 2014 at 2:18 PM, Jiansh

Interactive interface tool for spark

2014-10-08 Thread Dai, Kevin
Hi, All We need an interactive interface tool for spark in which we can run spark job and plot graph to explorer the data interactively. Ipython notebook is good, but it only support python (we want one supporting scala)... BR, Kevin.

Re: return probability \ confidence instead of actual class

2014-10-08 Thread Adamantios Corais
ok let me rephrase my question once again. python-wise I am preferring .predict_proba(X) instead of .decision_function(X) since it is easier for me to interpret the results. as far as I can see, the latter functionality is already implemented in Spark (well, in version 0.9.2 for example I have to c

Re: return probability \ confidence instead of actual class

2014-10-08 Thread Sean Owen
Plain old SVMs don't produce an estimate of class probabilities; predict_proba() does some additional work to estimate class probabilities from the SVM output. Spark does not implement this right now. Spark implements the equivalent of decision_function (the wTx + b bit) but does not expose it, an

org/I0Itec/zkclient/serialize/ZkSerializer ClassNotFound

2014-10-08 Thread cjwebb
Hello, I have been developing a Spark Streaming application using Kafka, which runs successfully on my Macbook. I am now trying to run it on an AWS Ubuntu spark cluster... and I receive a ClassNotFoundException. Kafka 0.8.1.1 Spark 1.1.0 I am submitting the job like this: /opt/spark/bin/spark-su

Re: Interactive interface tool for spark

2014-10-08 Thread andy petrella
Heya You can check Zeppellin or my fork of the Scala notebook. I'm going this week end to push some efforts on the doc, because it supports for realtime graphing, Scala, SQL, dynamic loading of dependencies and I started this morning a widget to track the progress of the jobs. I'm quite happy with

spark-ec2 - HDFS doesn't start on AWS EC2 cluster

2014-10-08 Thread Jan Warchoł
Hi all, I'm trying to launch an EC2 cluster using spark-ec2 script, and it seems that the script fails to configure HDFS properly. What's most puzzling is that it did work perfectly on Sunday. Here's the command i'm using: ./spark-1.1.0-bin-hadoop2.4/ec2/spark-ec2 \ -k xx \ -i x

Re: spark-ec2 - HDFS doesn't start on AWS EC2 cluster

2014-10-08 Thread mrm
I have the same problems. I get asked the questions about hdfs, and then the cluster finishes launching, but I also have one executor in my application (the driver node). Anybody knows what is happening? -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/spark-

foreachPartition: write to multiple files

2014-10-08 Thread david
Hi, I want to write my RDDs to multiples files based on a key value. So, i used groupByKey and iterate over partitions. Here is a the code : rdd.map(f => (f.substring(0,4), f)).groupByKey().foreachPartition(iterator => iterator.map { case (key, values) => val fs: FileSystem = File

Re: GraphX: Types for the Nodes and Edges

2014-10-08 Thread Oshi
Also, 3) can a union operation be done when the list of attributes are different for each vertex type? Would really appreciate a basic example to create graph with multiple node types :) Oshi wrote > 1. vP and vA are RDDs, how do I convert them to vertexRDDs and perform the > union? > 2. Should

sparksql connect remote hive cluster

2014-10-08 Thread jamborta
Hi all, just wondering if is it possible to allow spark to connect to hive on another cluster located remotely? I have setup hive-site.xml and amended the hive-metatstore uri, also opened the port for zookeeper, webhdfs and hive metastore. It seems it connects to hive, then it fails with the fo

Re: Any issues with repartition?

2014-10-08 Thread jamborta
I am still puzzled on this. In my case the data is allowed to write to disk, and I usually get different errors if it is out of memory. My guess is that akka kills the executors for some reason. -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/Any-issues-wi

我正在 LinkedIn (领英) 拓展职业人脉,希望能与您建立联系

2014-10-08 Thread yaochunnan
Apache,您好: 我正在 LinkedIn (领英) 拓展职业人脉,希望能与您建立联系。 姚春楠 接受邀请: http://www.linkedin.com/blink?simpleRedirect=3kQdPANcjAOdP0PdjkVdjkOejkZh4BKrSBQonhFtCVF9zpBdzcUijBPfnBBiShBsC5EsOpQsSlRpRZBt6BSrCAZqSkConhzbmlQqnpKqiRQsSlRpORIrmkZpSVFqSdxsDgCpnhFtCV9pSlipn9Mfm4CdPcJr65Gc6YMcmAJoDBWsPpTbjRBfP9SbSkLrmZzbC

Spark and tree data structures

2014-10-08 Thread Silvina Caíno Lores
Hi all, I'd like to use an octree data structure in order to simplify several computations in a big data set. I've been wondering if Spark has any built-in options for such structures (the only thing I could find is the DecisionTree), specially if they make use of RDDs. I've also been exploring t

Re: spark-ec2 - HDFS doesn't start on AWS EC2 cluster

2014-10-08 Thread mrm
Has anybody found a workaround for this? It would be great if you could share it! -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/spark-ec2-HDFS-doesn-t-start-on-AWS-EC2-cluster-tp15921p15932.html Sent from the Apache Spark User List mailing list archive at

Re: SparkContext.wholeTextFiles() java.io.FileNotFoundException: File does not exist:

2014-10-08 Thread jan.zikes
My additional question is if this problem can be possibly caused by the fact that my file is bigger than RAM memory across the whole cluster?   __ Hi I'm trying to use sc.wholeTextFiles() on file that is stored amazon S3 I'm getting fol

Error reading from Kafka

2014-10-08 Thread Antonio Jesus Navarro
Hi, I'm trying to read from Kafka. I was able to do it correctly with this method. def createStream( ssc: StreamingContext, zkQuorum: String, groupId: String, topics: Map[String, Int], storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 ): ReceiverInp

RE: MLLib Linear regression

2014-10-08 Thread Sameer Tilak
Hi Xiangrui,Changing the default step size to 0.01 made a huge difference. The results make sense when I use A + B + C + D. MSE is ~0.07 and the outcome matches the domain knowledge. I was wondering is there any documentation on the parameters and when/how to vary them. > Date: Tue, 7 Oct 20

Re: foreachPartition: write to multiple files

2014-10-08 Thread david
Hi, I finally found a solution after reading the post : http://apache-spark-user-list.1001560.n3.nabble.com/how-to-split-RDD-by-key-and-save-to-different-path-td11887.html#a11983 -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/foreachPartition-write-to-

Re: Interactive interface tool for spark

2014-10-08 Thread Michael Allman
Hi Andy, This sounds awesome. Please keep us posted. Meanwhile, can you share a link to your project? I wasn't able to find it. Cheers, Michael On Oct 8, 2014, at 3:38 AM, andy petrella wrote: > Heya > > You can check Zeppellin or my fork of the Scala notebook. > I'm going this week end to

Re: How could I start new spark cluster with hadoop2.0.2

2014-10-08 Thread st553
Hi, Were you able to figure out how to choose a specific version? Im having the same issue. Thanks. -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/How-could-I-start-new-spark-cluster-with-hadoop2-0-2-tp10450p15939.html Sent from the Apache Spark User Lis

Re: Spark-Shell: OOM: GC overhead limit exceeded

2014-10-08 Thread sranga
Increasing the driver memory resolved this issue. Thanks to Nick for the hint. Here is how I am starting the shell: "spark-shell --driver-memory 4g --driver-cores 4 --master local" -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/Spark-Shell-OOM-GC-overhead-l

Re: SparkContext.wholeTextFiles() java.io.FileNotFoundException: File does not exist:

2014-10-08 Thread jan.zikes
One more update: I've realized that this problem is not only Python related. I've tried it also in Scala, but I'm still getting the same error, my scala code: val file = sc.wholeTextFiles("s3n://wiki-dump/wikiinput").first() __ My addit

Re: SparkContext.wholeTextFiles() java.io.FileNotFoundException: File does not exist:

2014-10-08 Thread Sean Owen
Take this as a bit of a guess, since I don't use S3 much and am only a bit aware of the Hadoop+S3 integration issues. But I know that S3's lack of proper directories causes a few issues when used with Hadoop, which wants to list directories. According to http://hadoop.apache.org/docs/r2.3.0/api/o

Re: Interactive interface tool for spark

2014-10-08 Thread Michael Allman
Ummm... what's helium? Link, plz? On Oct 8, 2014, at 9:13 AM, Stephen Boesch wrote: > @kevin, Michael, > Second that: interested in seeing the zeppelin. pls use helium though .. > > 2014-10-08 7:57 GMT-07:00 Michael Allman : > Hi Andy, > > This sounds awesome. Please keep us posted. Meanwhi

Re: spark-ec2 - HDFS doesn't start on AWS EC2 cluster

2014-10-08 Thread mrm
They reverted to a previous version of the spark-ec2 script and things are working again! -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/spark-ec2-HDFS-doesn-t-start-on-AWS-EC2-cluster-tp15921p15945.html Sent from the Apache Spark User List mailing list arc

Re: Interactive interface tool for spark

2014-10-08 Thread andy petrella
Sure! I'll post updates as well in the ML :-) I'm doing it on twitter for now (until doc is ready). The repo is there (branch spark) : https://github.com/andypetrella/scala-notebook/tree/spark Some tweets: * very first working stuff: https://twitter.com/noootsab/status/508758335982927872/photo/1

Re: spark-ec2 - HDFS doesn't start on AWS EC2 cluster

2014-10-08 Thread Nicholas Chammas
Yup, though to be clear, Josh reverted a change to a hosted script that spark-ec2 references. The spark-ec2 script y’all are running locally hasn’t changed, obviously. ​ On Wed, Oct 8, 2014 at 12:20 PM, mrm wrote: > They reverted to a previous version of the spark-ec2 script and things are > wor

Re: spark-ec2 - HDFS doesn't start on AWS EC2 cluster

2014-10-08 Thread Jan Warchoł
Thanks for explanation, i was going to ask exactly about this :) On Wed, Oct 8, 2014 at 6:23 PM, Nicholas Chammas wrote: > Yup, though to be clear, Josh reverted a change to a hosted script that > spark-ec2 references. The spark-ec2 script y’all are running locally > hasn’t changed, obviously. >

Re: MLLib Linear regression

2014-10-08 Thread Xiangrui Meng
The proper step size partially depends on the Lipschitz constant of the objective. You should let the machine try different combinations of parameters and select the best. We are working with people from AMPLab to make hyperparameter tunning easier in MLlib 1.2. For the theory, Nesterov's book "Int

Spark SQL HiveContext Projection Pushdown

2014-10-08 Thread Anand Mohan
We have our analytics infra built on Spark and Parquet. We are trying to replace some of our queries based on the direct Spark RDD API to SQL based either on Spark SQL/HiveQL. Our motivation was to take advantage of the transparent projection & predicate pushdown that's offered by Spark SQL and eli

meetup october 30-31st in SF

2014-10-08 Thread Jeremy Freeman
Hi all, We’re organizing a meetup October 30-31st in downtown SF that might be of interest to the Spark community. The focus is on large-scale data analysis and its role in neuroscience. It will feature several active Spark developers and users, including Xiangrui Meng, Josh Rosen, Reza Zadeh,

Broadcast Torrent fail - then the job dies

2014-10-08 Thread Steve Lewis
I am running on Windows 8 using Spark 1.1.0 in local mode with Hadoop 2.2 - I repeatedly see the following in my logs. I believe this happens in combineByKey 14/10/08 09:36:30 INFO executor.Executor: Running task 3.0 in stage 0.0 (TID 3) 14/10/08 09:36:30 INFO broadcast.TorrentBroadcast: Started

Re: Spark SQL HiveContext Projection Pushdown

2014-10-08 Thread Michael Armbrust
We are working to improve the integration here, but I can recommend the following when running spark 1.1: create an external table and set spark.sql.hive.convertMetastoreParquet=true Note that even with a HiveContext we don't support window functions yet. On Wed, Oct 8, 2014 at 10:41 AM, Anand

Re: Broadcast Torrent fail - then the job dies

2014-10-08 Thread Liquan Pei
Hi Lewis, For debugging purpose, can you try using HttpBroadCast to see if the error remains? You can enable HttpBroadCast by setting spark.broadcast.factory to org.apache.spark.broadcast.HttpBroadcastFactory in spark conf. Thanks, Liquan On Wed, Oct 8, 2014 at 11:21 AM, Steve Lewis wrote: > I

executors not created yarn-cluster mode

2014-10-08 Thread jamborta
Hi all, I have a setup that works fine in yarn-client mode, but when I change that to yarn-cluster, the executors don't get created, apart from the driver (it seems that it does not even appear in yarn's resource manager). There is nothing in the spark logs, either (even when debug is enabled). W

Is there a way to look at RDD's lineage? Or debug a fault-tolerance error?

2014-10-08 Thread Sung Hwan Chung
My job is not being fault-tolerant (e.g., when there's a fetch failure or something). The lineage of RDDs are constantly updated every iteration. However, I think that when there's a failure, the lineage information is not being correctly reapplied. It goes something like this: val rawRDD = read

Re: How to do broadcast join in SparkSQL

2014-10-08 Thread Michael Armbrust
Thanks for the input. We purposefully made sure that the config option did not make it into a release as it is not something that we are willing to support long term. That said we'll try and make this easier in the future either through hints or better support for statistics. In this particular

Re: Is there a way to look at RDD's lineage? Or debug a fault-tolerance error?

2014-10-08 Thread Akshat Aranya
Using a var for RDDs in this way is not going to work. In this example, tx1.zip(tx2) would create and RDD that depends on tx2, but then soon after that, you change what tx2 means, so you would end up having a circular dependency. On Wed, Oct 8, 2014 at 12:01 PM, Sung Hwan Chung wrote: > My job

Spark on YARN driver memory allocation bug?

2014-10-08 Thread Greg Hill
So, I think this is a bug, but I wanted to get some feedback before I reported it as such. On Spark on YARN, 1.1.0, if you specify the --driver-memory value to be higher than the memory available on the client machine, Spark errors out due to failing to allocate enough memory. This happens eve

Re: Is there a way to look at RDD's lineage? Or debug a fault-tolerance error?

2014-10-08 Thread Liquan Pei
There is a toDebugString method in rdd that will print a description of this RDD and its recursive dependencies for debugging. Thanks, Liquan On Wed, Oct 8, 2014 at 12:01 PM, Sung Hwan Chung wrote: > My job is not being fault-tolerant (e.g., when there's a fetch failure or > something). > > The

Re: Support for Parquet V2 in ParquetTableSupport?

2014-10-08 Thread Michael Armbrust
Thats a good question, I'm not sure if that will work. I will note that we are hoping to do some upgrades of our parquet support in the near future. On Tue, Oct 7, 2014 at 10:33 PM, Michael Allman wrote: > Hello, > > I was interested in testing Parquet V2 with Spark SQL, but noticed after > som

Re: Is there a way to look at RDD's lineage? Or debug a fault-tolerance error?

2014-10-08 Thread Sung Hwan Chung
There is no circular dependency. Its simply dropping references to prev RDDs because there is no need for it. I wonder if that messes up things up though internally for Spark due to losing references to intermediate RDDs. > On Oct 8, 2014, at 12:13 PM, Akshat Aranya wrote: > > Using a var for

Re: spark-ec2 - HDFS doesn't start on AWS EC2 cluster

2014-10-08 Thread Akhil Das
Revert the script to an older version. Thanks Best Regards On Wed, Oct 8, 2014 at 9:57 PM, Jan Warchoł wrote: > Thanks for explanation, i was going to ask exactly about this :) > > On Wed, Oct 8, 2014 at 6:23 PM, Nicholas Chammas < > nichola

Dedup

2014-10-08 Thread Ge, Yao (Y.)
I need to do deduplication processing in Spark. The current plan is to generate a tuple where key is the dedup criteria and value is the original input. I am thinking to use reduceByKey to discard duplicate values. If I do that, can I simply return the first argument or should I return a copy of

Re: Dedup

2014-10-08 Thread Nicholas Chammas
Multiple values may be different, yet still be considered duplicates depending on how the dedup criteria is selected. Is that correct? Do you care in that case what value you select for a given key? On Wed, Oct 8, 2014 at 3:37 PM, Ge, Yao (Y.) wrote: > I need to do deduplication processing in S

Re: Spark Monitoring with Ganglia

2014-10-08 Thread Otis Gospodnetic
Hi, If using Ganglia is not an absolute requirement, check out SPM for Spark -- http://blog.sematext.com/2014/10/07/apache-spark-monitoring/ It monitors all Spark metrics (i.e. you don't need to figure out what you need to monitor, how to get it, how to graph it, etc.)

Re: Spark on YARN driver memory allocation bug?

2014-10-08 Thread Andrew Or
Hi Greg, It does seem like a bug. What is the particular exception message that you see? Andrew 2014-10-08 12:12 GMT-07:00 Greg Hill : > So, I think this is a bug, but I wanted to get some feedback before I > reported it as such. On Spark on YARN, 1.1.0, if you specify the > --driver-memory v

Re: Running Spark cluster on local machine, cannot connect to master error

2014-10-08 Thread rrussell25
Theodore, did you ever get this resolved? I just ran into the same thing. Before digging, I figured I'd ask. -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/Running-Spark-cluster-on-local-machine-cannot-connect-to-master-error-tp12743p15972.html Sent from

Re: Running Spark cluster on local machine, cannot connect to master error

2014-10-08 Thread Andrew Or
Hi Russell and Theodore, This usually means your Master / Workers / client machine are running different versions of Spark. On a local machine, you may want to restart your master and workers (sbin/stop-all.sh, then sbin/start-all.sh). On a real cluster, you want to make sure that every node (incl

Re: executors not created yarn-cluster mode

2014-10-08 Thread Andrew Or
Hi Jamborta, It could be that your executors are requesting too much memory. I'm not sure why it works in client mode but not in cluster mode, however. Have you checked the RM logs for messages that complain about container memory requested being too high? How much memory is each of your container

protobuf error running spark on hadoop 2.4

2014-10-08 Thread Chuang Liu
Hi: I tried to build Spark (1.1.0) with hadoop 2.4.0, and ran a simple wordcount example using spark_shell on mesos. When I ran my application, I got following error that looks related to the mismatch of protobuf versions between hadoop cluster (protobuf 2.5) and spark (protobuf 4.1). I ran "mvn d

Re: executors not created yarn-cluster mode

2014-10-08 Thread jamborta
hi Andrew, Thanks for the reply, I tried to tune the memory, changed it as low as possible, no luck. My guess is that this issue is related to what is discussed here http://apache-spark-user-list.1001560.n3.nabble.com/Initial-job-has-not-accepted-any-resources-td11668.html that is the executo

Re: protobuf error running spark on hadoop 2.4

2014-10-08 Thread Debasish Das
I have faced this in the past and I have to put a profile -Phadoop2.3... mvn -Dhadoop.version=2.3.0-cdh5.1.0 -Phadoop-2.3 -Pyarn -DskipTests install On Wed, Oct 8, 2014 at 1:40 PM, Chuang Liu wrote: > Hi: > > I tried to build Spark (1.1.0) with hadoop 2.4.0, and ran a simple > wordcount example

Building pyspark with maven?

2014-10-08 Thread Stephen Boesch
The build instructions for pyspark appear to be: sbt/sbt assembly Given that maven is the preferred build tool since July 1, presumably I have overlooked the instructions for building via maven? Anyone please point it out? thanks

Re: Dedup

2014-10-08 Thread Flavio Pompermaier
Maybe you could implement something like this (i don't know if something similar already exists in spark): http://www.cs.berkeley.edu/~jnwang/papers/icde14_massjoin.pdf Best, Flavio On Oct 8, 2014 9:58 PM, "Nicholas Chammas" wrote: > Multiple values may be different, yet still be considered dup

Re: Building pyspark with maven?

2014-10-08 Thread Stephen Boesch
Looking more closely, inside the core/pom.xml there are a few references to the python build This question mostly has to do with my limited of knowledge of python environment . I will look up how to set up python module. It appears a hack is to add $SPARK_HOME/core/target/scala-/classes to th

Re: Support for Parquet V2 in ParquetTableSupport?

2014-10-08 Thread Michael Allman
I am planning to try upgrading spark sql to a newer version of parquet, too. I'll let you know if I make progress. Thanks, Michael On Oct 8, 2014, at 12:17 PM, Michael Armbrust wrote: > Thats a good question, I'm not sure if that will work. I will note that we > are hoping to do some upgra

Re: Interactive interface tool for spark

2014-10-08 Thread moon soo Lee
Hi, Please check Zeppelin, too. http://zeppelin-project.org https://github.com/nflabs/zeppelin Which is similar to scala notebook. Best, moon 2014년 10월 9일 목요일, andy petrella님이 작성한 메시지: > Sure! I'll post updates as well in the ML :-) > I'm doing it on twitter for now (until doc is ready). > >

Re: Building pyspark with maven?

2014-10-08 Thread Ted Yu
Have you looked at http://spark.apache.org/docs/latest/building-with-maven.html ? Especially http://spark.apache.org/docs/latest/building-with-maven.html#building-for-pyspark-on-yarn Cheers On Wed, Oct 8, 2014 at 2:01 PM, Stephen Boesch wrote: > The build instructions for pyspark appear to be:

Re: Broadcast Torrent fail - then the job dies

2014-10-08 Thread Steve Lewis
That converts the error to the following 14/10/08 13:27:40 INFO executor.Executor: Running task 3.0 in stage 0.0 (TID 3) 14/10/08 13:27:40 INFO broadcast.HttpBroadcast: Started reading broadcast variable 0 14/10/08 13:27:40 ERROR executor.Executor: Exception in task 1.0 in stage 0.0 (TID 1) jav

Re: Is there a way to look at RDD's lineage? Or debug a fault-tolerance error?

2014-10-08 Thread Sung Hwan Chung
One thing I didn't mention is that we actually do data.repartition before hand with shuffle. I found that this can actually introduce randomness to lineage steps, because data get shuffled to different partitions and lead to inconsistent behavior if your algorithm is dependent on the order at whic

coalesce with shuffle or repartition is not necessarily fault-tolerant

2014-10-08 Thread Sung Hwan Chung
I noticed that repartition will result in non-deterministic lineage because it'll result in changed orders for rows. So for instance, if you do things like: val data = read(...) val k = data.repartition(5) val h = k.repartition(5) It seems that this results in different ordering of rows for 'k'

Re: Interactive interface tool for spark

2014-10-08 Thread Kelvin Chu
Hi Andy, It sounds great! Quick questions: I have been using IPython + PySpark. I crunch the data by PySpark and then visualize the data by Python libraries like matplotlib and basemap. Could I still use these Python libraries in the Scala Notebook? If not, what is suggested approaches for visuali

Spark-SQL: SchemaRDD - ClassCastException

2014-10-08 Thread Ranga
Hi I am in the process of migrating some logic in pig scripts to Spark-SQL. As part of this process, I am creating a few "Select...Group By" query and registering them as tables using the SchemaRDD.registerAsTable feature. When using such a registered table in a subsequent "Select...Group By" quer

Re: Spark-SQL: SchemaRDD - ClassCastException

2014-10-08 Thread Michael Armbrust
Using SUM on a string should automatically cast the column. Also you can use CAST to change the datatype . What version of Spark are you running? This could be https://issues.apache.or

Re: GroupBy Key and then sort values with the group

2014-10-08 Thread chinchu
Sean, I am having a similar issue, but I have a lot of data for a group & I cannot materialize the iterable into a List or Seq in memory. [I tried & it runs into OOM]. is there any other way to do this ? I also tried a secondary-sort, with the key having the "group::time", but the problem with th

Re: Spark-SQL: SchemaRDD - ClassCastException

2014-10-08 Thread Ranga
Thanks Michael. Should the cast be done in the source RDD or while doing the SUM? To give a better picture here is the code sequence: val sourceRdd = sql("select ... from source-hive-table") sourceRdd.registerAsTable("sourceRDD") val aggRdd = sql("select c1, c2, sum(c3) from sourceRDD group by c1,

Re: Spark-SQL: SchemaRDD - ClassCastException

2014-10-08 Thread Michael Armbrust
Which version of Spark are you running? On Wed, Oct 8, 2014 at 4:18 PM, Ranga wrote: > Thanks Michael. Should the cast be done in the source RDD or while doing > the SUM? > To give a better picture here is the code sequence: > > val sourceRdd = sql("select ... from source-hive-table") > sourceRd

Re: Spark-SQL: SchemaRDD - ClassCastException

2014-10-08 Thread Ranga
Sorry. Its 1.1.0. After digging a bit more into this, it seems like the OpenCSV Deseralizer converts all the columns to a String type. This maybe throwing the execution off. Planning to create a class and map the rows to this custom class. Will keep this thread updated. On Wed, Oct 8, 2014 at 5:11

How to configure build.sbt for Spark 1.2.0

2014-10-08 Thread Arun Luthra
I built Spark 1.2.0 succesfully, but was unable to build my Spark program under 1.2.0 with sbt assembly & my build.sbt file. It contains: I tried: "org.apache.spark" %% "spark-sql" % "1.2.0", "org.apache.spark" %% "spark-core" % "1.2.0", and "org.apache.spark" %% "spark-sql" % "1.2.0

RE: Error reading from Kafka

2014-10-08 Thread Shao, Saisai
Hi, I think you have to change the code like this to specify the type info, like this: val kafkaStream: ReceiverInputDStream[(String, String)] = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( ssc, kafkaParams, topicMap, StorageLevel.MEMORY_AND_DISK_S

Re: How to configure build.sbt for Spark 1.2.0

2014-10-08 Thread Pat McDonough
Hey Arun, Since this build depends on unpublished builds of spark (1.2.0-SNAPSHOT), you'll need to first build spark and "publish-local" so your application build can find those SNAPSHOTs in your local repo. Just append "publish-local" to your sbt command where you build Spark. -Pat On Wed, O

Re: How to configure build.sbt for Spark 1.2.0

2014-10-08 Thread Arun Luthra
Hi Pat, Couple of points: 1) I must have done something naive like: git clone git://github.com/apache/spark.git -b branch-1.2.0 because "git branch" is telling me I'm on the "master" branch, and I see that branch-1.2.0 doesn't exist (https://github.com/apache/spark). Nevertheless, when I compile

Re: Reading from HBase is too slow

2014-10-08 Thread Tao Xiao
Sean, I did specify the number of cores to use as follows: ... ... val sparkConf = new SparkConf() .setAppName("<<< Reading HBase >>>") .set("spark.cores.max", "32") val sc = new SparkContext(sparkConf) ... ... But that does not solve the problem --- only 2 workers are allocate

Spark SQL parser bug?

2014-10-08 Thread Mohammed Guller
Hi - When I run the following Spark SQL query in Spark-shell ( version 1.1.0) : val rdd = sqlContext.sql("SELECT a FROM x WHERE ts >= '2012-01-01T00:00:00' AND ts <= '2012-03-31T23:59:59' ") it gives the following error: rdd: org.apache.spark.sql.SchemaRDD = SchemaRDD[294] at RDD at SchemaRDD.s

JavaPairDStream saveAsTextFile

2014-10-08 Thread SA
HI, I am looking at the documentation for Java API for Streams. The scala library has option to save file locally, but the Java version doesnt seem to. The only option i see is saveAsHadoopFiles. Is there a reason why this option was left out from Java API? http://spark.apache.org/docs/1.0.0/a

Re: java.io.IOException Error in task deserialization

2014-10-08 Thread Sung Hwan Chung
This is also happening to me on a regular basis, when the job is large with relatively large serialized objects used in each RDD lineage. A bad thing about this is that this exception always stops the whole job. On Fri, Sep 26, 2014 at 11:17 AM, Brad Miller wrote: > FWIW I suspect that each cou

Re: Spark-SQL: SchemaRDD - ClassCastException

2014-10-08 Thread Ranga
This is a bit strange. When I print the schema for the RDD, it reflects the correct data type for each column. But doing any kind of mathematical calculation seems to result in ClassCastException. Here is a sample that results in the exception: select c1, c2 ... cast (c18 as int) * cast (c21 as int

Re: Dedup

2014-10-08 Thread Sonal Goyal
What is your data like? Are you looking at exact matching or are you interested in nearly same records? Do you need to merge similar records to get a canonical value? Best Regards, Sonal Nube Technologies On Thu, Oct 9, 2014 at 2:

Re: coalesce with shuffle or repartition is not necessarily fault-tolerant

2014-10-08 Thread Patrick Wendell
IIRC - the random is seeded with the index, so it will always produce the same result for the same index. Maybe I don't totally follow though. Could you give a small example of how this might change the RDD ordering in a way that you don't expect? In general repartition() will not preserve the orde

Re: sparksql connect remote hive cluster

2014-10-08 Thread Patrick Wendell
Spark will need to connect both to the hive metastore and to all HDFS nodes (NN and DN's). If that is all in place then it should work. In this case it looks like maybe it can't connect to a datanode in HDFS to get the raw data. Keep in mind that the performance might not be very good if you are tr

Re: How to configure build.sbt for Spark 1.2.0

2014-10-08 Thread Sean Owen
There is not yet a 1.2.0 branch; there is no 1.2.0 release. master is 1.2.0-SNAPSHOT, not 1.2.0. Your final command is correct, but it's redundant to 'package' and then throw that away with another 'clean'. Just the final command with '... clean install' is needed. On Thu, Oct 9, 2014 at 2:12 AM,

Re: JavaPairDStream saveAsTextFile

2014-10-08 Thread Sean Owen
Yeah it's not there. I imagine it was simply never added, and that there's not a good reaosn it couldn't be. On Thu, Oct 9, 2014 at 4:53 AM, SA wrote: > HI, > > I am looking at the documentation for Java API for Streams. The scala > library has option to save file locally, but the Java version d

Re: JavaPairDStream saveAsTextFile

2014-10-08 Thread Mayur Rustagi
Thats a cryptic way to say thr should be a Jira for it :) Mayur Rustagi Ph: +1 (760) 203 3257 http://www.sigmoidanalytics.com @mayur_rustagi On Thu, Oct 9, 2014 at 11:46 AM, Sean Owen wrote: > Yeah it's not there. I imagine it was simply never added, and tha

Re: Dedup

2014-10-08 Thread Akhil Das
If you are looking to eliminate duplicate rows (or similar) then you can define a key from the data and on that key you can do reduceByKey. Thanks Best Regards On Thu, Oct 9, 2014 at 10:30 AM, Sonal Goyal wrote: > What is your data like? Are you looking at exact matching or are you > interested

Re: coalesce with shuffle or repartition is not necessarily fault-tolerant

2014-10-08 Thread Sung Hwan Chung
Let's say you have some rows in a dataset (say X partitions initially). A B C D E . . . . You repartition to Y > X, then it seems that any of the following could be valid: partition 1 partition 2 A B .

Re: coalesce with shuffle or repartition is not necessarily fault-tolerant

2014-10-08 Thread Sean Owen
Yes, I think this another operation that is not deterministic even for the same RDD. If a partition is lost and recalculated the ordering can be different in the partition. Sorting the RDD makes the ordering deterministic. On Thu, Oct 9, 2014 at 7:51 AM, Sung Hwan Chung wrote: > Let's say you hav