Re: Which OutputCommitter to use for S3?

2015-03-25 Thread Pei-Lun Lee
I updated the PR for SPARK-6352 to be more like SPARK-3595. I added a new setting "spark.sql.parquet.output.committer.class" in hadoop configuration to allow custom implementation of ParquetOutputCommitter. Can someone take a look at the PR? On Mon, Mar 16, 2015 at 5:23 PM, Pei-Lun Lee wrote: >

Re: hadoop input/output format advanced control

2015-03-25 Thread Aaron Davidson
Should we mention that you should synchronize on HadoopRDD.CONFIGURATION_INSTANTIATION_LOCK to avoid a possible race condition in cloning Hadoop Configuration objects prior to Hadoop 2.7.0? :) On Wed, Mar 25, 2015 at 7:16 PM, Patrick Wendell wrote: > Great - that's even easier. Maybe we could ha

Haskell language Spark support

2015-03-25 Thread danilo2
Hi! I'm a haskell developer and I have created many haskell libraries in my life and some GHC extensions. I would like to create Haskell binding for Spark. Where can I find any documentation / sources describing the first steps in creation of a new language binding? I would be very thankful for an

Re: hadoop input/output format advanced control

2015-03-25 Thread Patrick Wendell
Great - that's even easier. Maybe we could have a simple example in the doc. On Wed, Mar 25, 2015 at 7:06 PM, Sandy Ryza wrote: > Regarding Patrick's question, you can just do "new Configuration(oldConf)" > to get a cloned Configuration object and add any new properties to it. > > -Sandy > > On W

Re: hadoop input/output format advanced control

2015-03-25 Thread Sandy Ryza
Regarding Patrick's question, you can just do "new Configuration(oldConf)" to get a cloned Configuration object and add any new properties to it. -Sandy On Wed, Mar 25, 2015 at 4:42 PM, Imran Rashid wrote: > Hi Nick, > > I don't remember the exact details of these scenarios, but I think the use

Re: Using CUDA within Spark / boosting linear algebra

2015-03-25 Thread Evan R. Sparks
Yeah, much more reasonable - nice to know that we can get full GPU performance from breeze/netlib-java - meaning there's no compelling performance reason to switch out our current linear algebra library (at least as far as this benchmark is concerned). Instead, it looks like a user guide for confi

RE: Using CUDA within Spark / boosting linear algebra

2015-03-25 Thread Ulanov, Alexander
As everyone suggested, the results were too good to be true, so I double-checked them. It turns that nvblas did not do multiplication due to parameter NVBLAS_TILE_DIM from "nvblas.conf" and returned zero matrix. My previously posted results with nvblas are matrices copying only. The default NVB

Re: lower&upperBound not working/spark 1/3

2015-03-25 Thread alessandro.andrioni
If I'm reading this comment[1] correctly, this is expected behavior: the lower and upper bounds are used to make the partitioning more efficient, not to limit the data returned. >/** > * Given a partitioning schematic (a column of integral type, a number of > * partitions, and upper and lower boun

RE: Using CUDA within Spark / boosting linear algebra

2015-03-25 Thread Ulanov, Alexander
John, Thanks for your suggestion, it really seems strange. Though right now I have no idea what's wrong since I use exactly the same script for testing. I will appreciate any suggestions. Best regards, Alexander -Original Message- From: jfcanny [mailto:ca...@berkeley.edu] Sent: Wednes

RE: Using CUDA within Spark / boosting linear algebra

2015-03-25 Thread Ulanov, Alexander
Sure, I will write a how-to after I re-check the results. -Original Message- From: Sam Halliday [mailto:sam.halli...@gmail.com] Sent: Wednesday, March 25, 2015 3:04 PM To: Evan R. Sparks; dev@spark.apache.org Subject: Re: Using CUDA within Spark / boosting linear algebra If you write it

Re: Using CUDA within Spark / boosting linear algebra

2015-03-25 Thread Reza Zadeh
These are awesome (and surprising) results, Alex. I've been following this thread and really surprised by the improvement over BIDMat-cuda, almost 20x faster. Any chance you could send scripts or github gist for reproduction? Thanks, Reza On Wed, Mar 25, 2015 at 2:31 PM, Ulanov, Alexander wrote

Re: Using CUDA within Spark / boosting linear algebra

2015-03-25 Thread Sam Halliday
That would be a difficult task that would only benefit users of netlib-java. MultiBLAS is easily implemented (although a lot of boilerplate) and benefits all BLAS users on the system. If anyone knows of a funding route for it, I'd love to hear from them, because it's too much work for me to take o

Re: Using CUDA within Spark / boosting linear algebra

2015-03-25 Thread Dmitriy Lyubimov
Sam, whould it be easier to hack netlib-java to allow multiple (configurable) library contexts? And so enable 3rd party configurations and optimizers to make their own choices until then? On Wed, Mar 25, 2015 at 3:07 PM, Sam Halliday wrote: > Yeah, MultiBLAS... it is dynamic. > > Except, I hav

Re: Using CUDA within Spark / boosting linear algebra

2015-03-25 Thread jfcanny
Alex, I think you should recheck your numbers. Both BIDMat and nvblas are wrappers for cublas. The speeds are identical, except on machines that have multiple GPUs which nvblas exploits and cublas doesnt. It would be a good idea to add a column with Gflop throughput. Your numbers for BIDMat 10k

RE: Using CUDA within Spark / boosting linear algebra

2015-03-25 Thread Sam Halliday
Yeah, MultiBLAS... it is dynamic. Except, I haven't written it yet :-P On 25 Mar 2015 22:06, "Ulanov, Alexander" wrote: > Netlib knows nothing about GPU (or CPU), it just uses cblas symbols from > the provided libblas.so.3 library at the runtime. So, you can switch at the > runtime by providing

RE: Using CUDA within Spark / boosting linear algebra

2015-03-25 Thread Ulanov, Alexander
Netlib knows nothing about GPU (or CPU), it just uses cblas symbols from the provided libblas.so.3 library at the runtime. So, you can switch at the runtime by providing another library. Sam, please suggest if there is another way. From: Dmitriy Lyubimov [mailto:dlie...@gmail.com] Sent: Wednesda

Re: Using CUDA within Spark / boosting linear algebra

2015-03-25 Thread Sam Halliday
If you write it up I'll add it to the netlib-java wiki :-) BTW, does it automatically flip between cpu/GPU? I've a project called MultiBLAS which was going to do this, it should be easy (but boring to write) On 25 Mar 2015 22:00, "Evan R. Sparks" wrote: > Alex - great stuff, and the nvblas numbe

Re: Using CUDA within Spark / boosting linear algebra

2015-03-25 Thread Evan R. Sparks
Alex - great stuff, and the nvblas numbers are pretty remarkable (almost too good... did you check the results for correctness? - also, is it possible that the "unified memory model" of nvblas is somehow hiding pci transfer time?) this last bit (getting nvblas + netlib-java to play together) sound

Re: Using CUDA within Spark / boosting linear algebra

2015-03-25 Thread Dmitriy Lyubimov
Alexander, does using netlib imply that one cannot switch between CPU and GPU blas alternatives at will at the same time? the choice is always determined by linking aliternatives to libblas.so, right? On Wed, Mar 25, 2015 at 2:31 PM, Ulanov, Alexander wrote: > Hi again, > > I finally managed to

Re: LogisticGradient Design

2015-03-25 Thread Debasish Das
Cool...Thanks...It will be great if they move in two code paths just for the sake of code clean-up On Wed, Mar 25, 2015 at 2:37 PM, DB Tsai wrote: > I did the benchmark when I used the if-else statement to switch the > binary & multinomial logistic loss and gradient, and there is no > performanc

Re: LogisticGradient Design

2015-03-25 Thread DB Tsai
I did the benchmark when I used the if-else statement to switch the binary & multinomial logistic loss and gradient, and there is no performance hit at all. However, I'm refactoring the LogisticGradient code so the addBias and scaling can be done in LogisticGradient instead of the input dataset to

RE: Using CUDA within Spark / boosting linear algebra

2015-03-25 Thread Ulanov, Alexander
Hi again, I finally managed to use nvblas within Spark+netlib-java. It has exceptional performance for big matrices with Double, faster than BIDMat-cuda with Float. But for smaller matrices, if you will copy them to/from GPU, OpenBlas or MKL might be a better choice. This correlates with origin

Re: Jira Issues

2015-03-25 Thread Igor Costa
Thank you guys for the info. Actually was a problem with my id on apache. Rather than need to be logged in to view issues. I'm browsing some issues now. Best Regards Igor Costa www.igorcosta.com www.igorcosta.org On Wed, Mar 25, 2015 at 5:58 PM, Sean Owen wrote:

Re: Jira Issues

2015-03-25 Thread Sean Owen
It's just the standard Apache JIRA, nothing separate. I'd say JIRA is used to track issues, bugs, features, but Github is where the concrete changes to implement those things are discussed and merged. So for a non-trivial issue, you'd want to describe the issue in general in JIRA, and then open a

Re: Jira Issues

2015-03-25 Thread Reynold Xin
Igor, Welcome -- everything is open here: https://issues.apache.org/jira/browse/SPARK You should be able to see them even if you are not an ASF member. On Wed, Mar 25, 2015 at 1:51 PM, Igor Costa wrote: > Hi there Guys. > > I want to be more collaborative to Spark, but I have two questions. >

Re: Jira Issues

2015-03-25 Thread Ted Yu
Issues are tracked on Apache JIRA: https://issues.apache.org/jira/browse/SPARK/?selectedTab=com.atlassian.jira.jira-projects-plugin:summary-panel Cheers On Wed, Mar 25, 2015 at 1:51 PM, Igor Costa wrote: > Hi there Guys. > > I want to be more collaborative to Spark, but I have two questions. >

Jira Issues

2015-03-25 Thread Igor Costa
Hi there Guys. I want to be more collaborative to Spark, but I have two questions. Issues are used in Github or jira Issues? If so on Jira, Is there a way I can get in to see the issues? I've tried to login but no success. I'm PMC from another Apache project, flex.apache.org Best Regards I

Re: hadoop input/output format advanced control

2015-03-25 Thread Imran Rashid
Hi Nick, I don't remember the exact details of these scenarios, but I think the user wanted a lot more control over how the files got grouped into partitions, to group the files together by some arbitrary function. I didn't think that was possible w/ CombineFileInputFormat, but maybe there is a w

Re: hadoop input/output format advanced control

2015-03-25 Thread Koert Kuipers
yeah fair enough On Wed, Mar 25, 2015 at 2:41 PM, Patrick Wendell wrote: > Yeah I agree that might have been nicer, but I think for consistency > with the input API's maybe we should do the same thing. We can also > give an example of how to clone sc.hadoopConfiguration and then set > some new v

Re: LogisticGradient Design

2015-03-25 Thread Joseph Bradley
It would be nice to see how big a performance hit we take from combining binary & multiclass logistic loss/gradient. If it's not a big hit, then it might be simpler from an outside API perspective to keep them in 1 class (even if it's more complicated within). Joseph On Wed, Mar 25, 2015 at 8:15

Re: Experience using binary packages on various Hadoop distros

2015-03-25 Thread Marcelo Vanzin
Hey Patrick, The only issue I've seen so far has been the YARN container ID issue. That can be technically be described as a breakage in forwards compatibility in YARN. The APIs didn't break, but the data transferred through YARN's protocol has, and the old library cannot understand the data sent

Re: functools.partial as UserDefinedFunction

2015-03-25 Thread Davies Liu
It’s good to support functools.partial, could you file a JIRA for it? On Wednesday, March 25, 2015 at 5:42 AM, Karlson wrote: > > Hi all, > > passing a functools.partial-function as a UserDefinedFunction to > DataFrame.select raises an AttributeException, because functools.partial > does

Re: hadoop input/output format advanced control

2015-03-25 Thread Patrick Wendell
Yeah I agree that might have been nicer, but I think for consistency with the input API's maybe we should do the same thing. We can also give an example of how to clone sc.hadoopConfiguration and then set some new values: val conf = sc.hadoopConfiguration.clone() .set("k1", "v1") .set("k2", "v

jenkins upgraded to 1.606....

2015-03-25 Thread shane knapp
...due to some big security fixes: https://wiki.jenkins-ci.org/display/SECURITY/Jenkins+Security+Advisory+2015-03-23 :) shane

LogisticGradient Design

2015-03-25 Thread Debasish Das
Hi, Right now LogisticGradient implements both binary and multi-class in the same class using an if-else statement which is a bit convoluted. For Generalized matrix factorization, if the data has distinct ratings I want to use LeastSquareGradient (regression has given best results to date) but if

Re: mllib.recommendation Design

2015-03-25 Thread Debasish Das
Hi Xiangrui, I am facing some minor issues in implementing Alternating Nonlinear Minimization as documented in this JIRA due to the ALS code being in ml package: https://issues.apache.org/jira/browse/SPARK-6323 I need to use Vectors.fromBreeze / Vectors.toBreeze but they are package private on ml

Re: Understanding shuffle file name conflicts

2015-03-25 Thread Cheng Lian
Ah, I see where I'm wrong here. What are reused here are the shuffle map output files themselves, rather than the file paths. No new shuffle map output files are generated for the 2nd job. Thanks! Really need to walk through Spark core code again :) Cheng On 3/25/15 9:31 PM, Shao, Saisai wrot

Re: hadoop input/output format advanced control

2015-03-25 Thread Koert Kuipers
my personal preference would be something like a Map[String, String] that only reflects the changes you want to make the Configuration for the given input/output format (so system wide defaults continue to come from sc.hadoopConfiguration), similarly to what cascading/scalding did, but am arbitrary

RE: Understanding shuffle file name conflicts

2015-03-25 Thread Shao, Saisai
Hi Cheng, I think your scenario is acceptable for Spark's shuffle mechanism and will not occur shuffle file name conflicts. From my understanding I think the code snippet you mentioned is the same RDD graph, just running twice, these two jobs will generate 3 stages, map stage and collect stag

functools.partial as UserDefinedFunction

2015-03-25 Thread Karlson
Hi all, passing a functools.partial-function as a UserDefinedFunction to DataFrame.select raises an AttributeException, because functools.partial does not have the attribute __name__. Is there any alternative to relying on __name__ in pyspark/sql/functions.py:126 ?

Re: Can't assembly YARN project with SBT

2015-03-25 Thread Zoltán Zvara
Hi! It seems that the problem of "unable to load YARN support" present only when I run my job from code and by not using the spark-submit script. IMO this is related to SPARK-5144 . I'm running QueueStream example with a single change: sparkConf.se

Re: Understanding shuffle file name conflicts

2015-03-25 Thread Cheng Lian
Hi Jerry & Josh It has been a while since the last time I looked into Spark core shuffle code, maybe I’m wrong here. But the shuffle ID is created along with ShuffleDependency, which is part of the RDD DAG. So if we submit multiple jobs over the same RDD DAG, I think the shuffle IDs in these

Can't assembly YARN project with SBT

2015-03-25 Thread Zoltán Zvara
Hi! I'm using the latest IntelliJ and I can't compile the yarn project into the Spark assembly fat JAR. That is why I'm getting a SparkException with message "Unable to load YARN support". The yarn project is also missing from SBT tasks and I can't add it. How can I force sbt to include? Thanks!

Re: Understanding shuffle file name conflicts

2015-03-25 Thread Saisai Shao
DIskBlockManager doesn't need to know the app id, all it need to do is to create a folder with a unique name (UUID based) and then put all the shuffle files into it. you can see the code in DiskBlockManager as below, it will create a bunch unique folders when initialized, these folders are app spe

Re: Understanding shuffle file name conflicts

2015-03-25 Thread Kannan Rajah
Josh & Saisai, When I say I am using a hardcoded location for shuffle files, I mean that I am not using DiskBlockManager.getFile API because that uses the directories created locally on the node. But for my use case, I need to look at creating those shuffle files on HDFS. I will take a closer look