Re: Understanding shuffle file name conflicts

2015-03-26 Thread Kannan Rajah
Thanks folks. I understood the workflow. I noticed there is some code in Worker.scala that creates app specific local dir. -- Kannan On Wed, Mar 25, 2015 at 7:33 AM, Cheng Lian wrote: > Ah, I see where I'm wrong here. What are reused here are the shuffle map > output files themselves, rather t

Re: Building spark 1.2 from source requires more dependencies

2015-03-26 Thread Pala M Muthaia
+spark-dev Yes, the dependencies are there. I guess my question is how come the build is succeeding in the mainline then, without adding these dependencies? On Thu, Mar 26, 2015 at 3:44 PM, Ted Yu wrote: > Looking at output from dependency:tree, servlet-api is brought in by the > following: > >

RE: Storing large data for MLlib machine learning

2015-03-26 Thread Ulanov, Alexander
Thanks, Jeremy! I also work with time series data right now, so your suggestions are really relevant. However, we want to handle not the raw data, but already processed and prepared for machine learning. Initially, we also wanted to have our own simple binary format, but we could not agree on h

Re: RDD.map does not allowed to preservesPartitioning?

2015-03-26 Thread Zhan Zhang
Thanks all for the quick response. Thanks. Zhan Zhang On Mar 26, 2015, at 3:14 PM, Patrick Wendell wrote: > I think we have a version of mapPartitions that allows you to tell > Spark the partitioning is preserved: > > https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/

Re: RDD.map does not allowed to preservesPartitioning?

2015-03-26 Thread Patrick Wendell
I think we have a version of mapPartitions that allows you to tell Spark the partitioning is preserved: https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/RDD.scala#L639 We could also add a map function that does same. Or you can just write your map using an iter

Re: RDD.map does not allowed to preservesPartitioning?

2015-03-26 Thread Jonathan Coveney
This is just a deficiency of the api, imo. I agree: mapValues could definitely be a function (K, V)=>V1. The option isn't set by the function, it's on the RDD. So you could look at the code and do this. https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/RDD.scala

Re: Storing large data for MLlib machine learning

2015-03-26 Thread Evan R. Sparks
Protobufs are great for serializing individual records - but parquet is good for efficiently storing a whole bunch of these objects. Matt Massie has a good (slightly dated) blog post on using Spark+Parquet+Avro (and you can pretty much s/Avro/Protobuf/) describing how they all work together here:

Re: Storing large data for MLlib machine learning

2015-03-26 Thread Jeremy Freeman
Hi Ulvanov, great question, we’ve encountered it frequently with scientific data (e.g. time series). Agreed text is inefficient for dense arrays, and we also found HDF5+Spark to be a pain. Our strategy has been flat binary files with fixed length records. Loading these is now supported in Spar

Re: RDD.map does not allowed to preservesPartitioning?

2015-03-26 Thread Zhan Zhang
Thanks Jonathan. You are right regarding rewrite the example. I mean providing such option to developer so that it is controllable. The example may seems silly, and I don’t know the use cases. But for example, if I also want to operate both the key and value part to generate some new value with

RE: Storing large data for MLlib machine learning

2015-03-26 Thread Ulanov, Alexander
Thanks, Evan. What do you think about Protobuf? Twitter has a library to manage protobuf files in hdfs https://github.com/twitter/elephant-bird From: Evan R. Sparks [mailto:evan.spa...@gmail.com] Sent: Thursday, March 26, 2015 2:34 PM To: Stephen Boesch Cc: Ulanov, Alexander; dev@spark.apache.or

Re: RDD.map does not allowed to preservesPartitioning?

2015-03-26 Thread Jonathan Coveney
I believe if you do the following: sc.parallelize(List(1,2,3,4,5,5,6,6,7,8,9,10,2,4)).map((_,1)).reduceByKey(_+_).mapValues(_+1).reduceByKey(_+_).toDebugString (8) MapPartitionsRDD[34] at reduceByKey at :23 [] | MapPartitionsRDD[33] at mapValues at :23 [] | ShuffledRDD[32] at reduceByKey at :

RDD.map does not allowed to preservesPartitioning?

2015-03-26 Thread Zhan Zhang
Hi Folks, Does anybody know what is the reason not allowing preserverPartitioning in RDD.map? Do I miss something here? Following example involves two shuffles. I think if preservePartitioning is allowed, we can avoid the second one, right? val r1 = sc.parallelize(List(1,2,3,4,5,5,6,6,7,8,9,1

Re: Storing large data for MLlib machine learning

2015-03-26 Thread Evan R. Sparks
On binary file formats - I looked at HDF5+Spark a couple of years ago and found it barely JVM-friendly and very Hadoop-unfriendly (e.g. the APIs needed filenames as input, you couldn't pass it anything like an InputStream). I don't know if it has gotten any better. Parquet plays much more nicely a

RE: Storing large data for MLlib machine learning

2015-03-26 Thread Ulanov, Alexander
Thanks for suggestion, but libsvm is a format for sparse data storing in text file and I have dense vectors. In my opinion, text format is not appropriate for storing large dense vectors due to overhead related to parsing from string to digits and also storing digits as strings is not efficient.

Re: Storing large data for MLlib machine learning

2015-03-26 Thread Stephen Boesch
There are some convenience methods you might consider including: MLUtils.loadLibSVMFile and MLUtils.loadLabeledPoint 2015-03-26 14:16 GMT-07:00 Ulanov, Alexander : > Hi, > > Could you suggest what would be the reasonable file format to store > feature vector data for machine learni

Storing large data for MLlib machine learning

2015-03-26 Thread Ulanov, Alexander
Hi, Could you suggest what would be the reasonable file format to store feature vector data for machine learning in Spark MLlib? Are there any best practices for Spark? My data is dense feature vectors with labels. Some of the requirements are that the format should be easy loaded/serialized,

Re: Using CUDA within Spark / boosting linear algebra

2015-03-26 Thread Sean Owen
The license issue is with libgfortran, rather than OpenBLAS. (FWIW I am going through the motions to get OpenBLAS set up by default on CDH in the near future, and the hard part is just handling libgfortran.) On Thu, Mar 26, 2015 at 4:07 PM, Evan R. Sparks wrote: > Alright Sam - you are the exper

Re: Using CUDA within Spark / boosting linear algebra

2015-03-26 Thread Sam Halliday
John, I have to disagree with you there. Dense matrices come up a lot in industry, although your personal experience may be different. On 26 Mar 2015 16:20, "John Canny" wrote: > I mentioned this earlier in the thread, but I'll put it out again. Dense > BLAS are not very important for most mach

Re: Using CUDA within Spark / boosting linear algebra

2015-03-26 Thread John Canny
I mentioned this earlier in the thread, but I'll put it out again. Dense BLAS are not very important for most machine learning workloads: at least for non-image workloads in industry (and for image processing you would probably want a deep learning/SGD solution with convolution kernels). e.g. i

Re: Using CUDA within Spark / boosting linear algebra

2015-03-26 Thread Evan R. Sparks
Alright Sam - you are the expert here. If the GPL issues are unavoidable, that's fine - what is the exact bit of code that is GPL? The suggestion to use OpenBLAS is not to say it's the best option, but that it's a *free, reasonable default* for many users - keep in mind the most common deployment

Re: functools.partial as UserDefinedFunction

2015-03-26 Thread Karlson
Hi, I've filed a JIRA (https://issues.apache.org/jira/browse/SPARK-6553) and suggested a fix (https://github.com/apache/spark/pull/5206). On 2015-03-25 19:49, Davies Liu wrote: It’s good to support functools.partial, could you file a JIRA for it? On Wednesday, March 25, 2015 at 5:42 AM, Ka

Re: Using CUDA within Spark / boosting linear algebra

2015-03-26 Thread Sam Halliday
Btw, OpenBLAS requires GPL runtime binaries which are typically considered "system libraries" (and these fall under something similar to the Java classpath exception rule)... so it's basically impossible to distribute OpenBLAS the way you're suggesting, sorry. Indeed, there is work ongoing in Spark

Re: Using CUDA within Spark / boosting linear algebra

2015-03-26 Thread Sam Halliday
I'm not at all surprised ;-) I fully expect the GPU performance to get better automatically as the hardware improves. Netlib natives still need to be shipped separately. I'd also oppose any move to make Open BLAS the default - is not always better and I think natives really need DevOps buy-in. It'

Can I call aggregate UDF in DataFrame?

2015-03-26 Thread Haopu Wang
Specifically there are only 5 aggregate functions in class org.apache.spark.sql.GroupedData: sum/max/min/mean/count. Can I plugin a function to calculate stddev? Thank you! - To unsubscribe, e-mail: dev-unsubscr...@spark.apache