RE: Using CUDA within Spark / boosting linear algebra

2016-01-21 Thread Allen Zhang


Hi Kazuaki,


Jcuda is actually a wrapper of the **pure** CUDA, as your wiki page shows that 
3.15x performance boost of logistic regression seems slower than BIDMat-cublas 
or pure CUDA.
Could you elaborate on why you chose Jcuda other then JNI to call CUDA directly?


Regards,
Allen Zhang








At 2016-01-21 19:34:14, "Kazuaki Ishizaki"  wrote:
Dear all,

 Hi Alexander,

 Using GPUs with Spark would be very exciting.  Small comment:
 Concerning your question earlier about keeping data stored on the
 GPU rather than having to move it between main memory and GPU
 memory on each iteration, I would guess this would be critical to
 getting good performance.  If you could do multiple local
 iterations before aggregating results, then the cost of data
 movement to the GPU could be amortized (and I believe that is done
 in practice).  Having Spark be aware of the GPU and using it as another 
 part of memory sounds like a much bigger undertaking.

 Joseph

As Joseph pointed out before, there are two potential issues to efficiently 
exploit GPUs in Spark.
(1) the cost of data movement between CPU and GPU
(2) the cost of encoding/decoding between current row-format and GPU-friendly 
column format

Our prototype http://kiszk.github.io/spark-gpu/addresses these two issues by 
supporting data partition caching in GPU device memory and by providing binary 
column storage for data partition. We really appreciate it if you would give us 
comments, suggestions, or feedback.

Best Regards
Kazuaki Ishizaki



From:"Ulanov, Alexander" 
To:Sam Halliday , John Canny 

Cc:Xiangrui Meng , "dev@spark.apache.org" 
, Joseph Bradley , "Evan R. 
Sparks" 
Date:2016/01/21 11:07
Subject:RE: Using CUDA within Spark / boosting linear algebra




Hi Everyone,
 
I’ve updated the benchmark and done experiments with new hardware with 2x 
Nvidia Tesla K80 (physically 4x Tesla K40) and 2x modern Haswell CPU Intel 
E5-2650 v3 @ 2.30GHz.
 
This time I computed average and median of 10 runs for each of experiment and 
approximated FLOPS.
 
Results are available at google docs (old experiments are in the other 2 
sheets):
https://docs.google.com/spreadsheets/d/1lWdVSuSragOobb0A_oeouQgHUMx378T9J5r7kwKSPkY/edit?usp=sharing
Benchmark code:
https://github.com/avulanov/scala-blas
 
Best regards, Alexander
 
 
From: Sam Halliday [mailto:sam.halli...@gmail.com]
Sent: Thursday, March 26, 2015 9:27 AM
To: John Canny
Cc: Xiangrui Meng; dev@spark.apache.org; Joseph Bradley; Evan R. Sparks; 
Ulanov, Alexander
Subject: Re: Using CUDA within Spark / boosting linear algebra
 

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 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. it was only 
relevant for 1/7 of our recent benchmarks, which should be a reasonable sample. 
What really matters is sparse BLAS performance. BIDMat is still an order of 
magnitude faster there. Those kernels are only in BIDMat, since NVIDIAs sparse 
BLAS dont perform well on power-law data.

Its also the case that the overall performance of an algorithm is determined by 
the slowest kernel, not the fastest. If the goal is to get closer to BIDMach's 
performance on typical problems, you need to make sure that every kernel goes 
at comparable speed. So the real question is how much faster MLLib routines do 
on a complete problem with/without GPU acceleration. For BIDMach, its close to 
a factor of 10. But that required running entirely on the GPU, and making sure 
every kernel is close to its limit.

-John

If you think nvblas would be helpful, you should try it in some end-to-end 
benchmarks.
On 3/25/15, 6:23 PM, Evan R. Sparks wrote:
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 configuring Spark/MLlib to use the 
right BLAS library will get us most of the way there. Or, would it make sense 
to finally ship openblas compiled for some common platforms (64-bit linux, 
windows, mac) directly with Spark - hopefully eliminating the jblas warnings 
once and for all for most users? (Licensing is BSD) Or am I missing something?
 
On Wed, Mar 25, 2015 at 6:03 PM, Ulanov, Alexander 

RE: Using CUDA within Spark / boosting linear algebra

2016-01-21 Thread Kazuaki Ishizaki
Dear all,

 Hi Alexander,

 Using GPUs with Spark would be very exciting.  Small comment:
 Concerning your question earlier about keeping data stored on the
 GPU rather than having to move it between main memory and GPU
 memory on each iteration, I would guess this would be critical to
 getting good performance.  If you could do multiple local
 iterations before aggregating results, then the cost of data
 movement to the GPU could be amortized (and I believe that is done
 in practice).  Having Spark be aware of the GPU and using it as 
another part of memory sounds like a much bigger undertaking.

 Joseph

As Joseph pointed out before, there are two potential issues to 
efficiently exploit GPUs in Spark.
(1) the cost of data movement between CPU and GPU
(2) the cost of encoding/decoding between current row-format and 
GPU-friendly column format

Our prototype http://kiszk.github.io/spark-gpu/ addresses these two issues 
by supporting data partition caching in GPU device memory and by providing 
binary column storage for data partition. We really appreciate it if you 
would give us comments, suggestions, or feedback.

Best Regards
Kazuaki Ishizaki



From:   "Ulanov, Alexander" 
To: Sam Halliday , John Canny 

Cc: Xiangrui Meng , "dev@spark.apache.org" 
, Joseph Bradley , "Evan R. 
Sparks" 
Date:   2016/01/21 11:07
Subject:RE: Using CUDA within Spark / boosting linear algebra



Hi Everyone,
 
I’ve updated the benchmark and done experiments with new hardware with 2x 
Nvidia Tesla K80 (physically 4x Tesla K40) and 2x modern Haswell CPU Intel 
E5-2650 v3 @ 2.30GHz.
 
This time I computed average and median of 10 runs for each of experiment 
and approximated FLOPS.
 
Results are available at google docs (old experiments are in the other 2 
sheets):
https://docs.google.com/spreadsheets/d/1lWdVSuSragOobb0A_oeouQgHUMx378T9J5r7kwKSPkY/edit?usp=sharing
Benchmark code:
https://github.com/avulanov/scala-blas
 
Best regards, Alexander
 
 
From: Sam Halliday [mailto:sam.halli...@gmail.com] 
Sent: Thursday, March 26, 2015 9:27 AM
To: John Canny
Cc: Xiangrui Meng; dev@spark.apache.org; Joseph Bradley; Evan R. Sparks; 
Ulanov, Alexander
Subject: Re: Using CUDA within Spark / boosting linear algebra
 
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 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. 
it was only relevant for 1/7 of our recent benchmarks, which should be a 
reasonable sample. What really matters is sparse BLAS performance. BIDMat 
is still an order of magnitude faster there. Those kernels are only in 
BIDMat, since NVIDIAs sparse BLAS dont perform well on power-law data. 

Its also the case that the overall performance of an algorithm is 
determined by the slowest kernel, not the fastest. If the goal is to get 
closer to BIDMach's performance on typical problems, you need to make sure 
that every kernel goes at comparable speed. So the real question is how 
much faster MLLib routines do on a complete problem with/without GPU 
acceleration. For BIDMach, its close to a factor of 10. But that required 
running entirely on the GPU, and making sure every kernel is close to its 
limit.

-John

If you think nvblas would be helpful, you should try it in some end-to-end 
benchmarks. 
On 3/25/15, 6:23 PM, Evan R. Sparks wrote:
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 configuring Spark/MLlib to use the 
right BLAS library will get us most of the way there. Or, would it make 
sense to finally ship openblas compiled for some common platforms (64-bit 
linux, windows, mac) directly with Spark - hopefully eliminating the jblas 
warnings once and for all for most users? (Licensing is BSD) Or am I 
missing something?
 
On Wed, Mar 25, 2015 at 6:03 PM, Ulanov, Alexander <
alexander.ula...@hp.com> wrote:
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 NVBLAS_TILE_DIM==2048 is too big for my graphic card/matrix size. 
I 

Re: How Spark utilize low-level architecture features?

2016-01-21 Thread Driesprong, Fokko
Hi Boric,

For the Spark Mllib package, which is build on top of Breeze
, which uses in turn netlib-java
. This netlib-java library can be
optimized for each system by compiling the specific architecture:

*To get optimal performance for a specific machine, it is best to compile
locally by grabbing the latest ATLAS or the latest OpenBLAS and following
the compilation instructions.*

For the rest, Spark focusses on adding more machines instead of using very
specific optimization procedures. Also optimizing your jobs (decreasing
communication between workers e.d.) might do the trick.

Cheers, Fokko.

2016-01-21 6:55 GMT+01:00 Boric Tan :

> Anyone could shed some light on this?
>
> Thanks,
> Boric
>
> On Tue, Jan 19, 2016 at 4:12 PM, Boric Tan 
> wrote:
>
>> Hi there,
>>
>> I am new to Spark, and would like to get some help to understand if Spark
>> can utilize the underlying architectures for better performance. If so, how
>> does it do it?
>>
>> For example, assume there is a cluster built with machines of different
>> CPUs, will Spark check the individual CPU information and use some
>> machine-specific setting for the tasks assigned to that machine? Or is it
>> totally dependent on the underlying JVM implementation to run the JAR file,
>> and therefor the JVM is the place to check if certain CPU features can be
>> used?
>>
>> Thanks,
>> Boric
>>
>
>


Spark SQL: Avoid shuffles when data is already partitioned on disk

2016-01-21 Thread Justin Uang
Hi,

If I had a df and I wrote it out via partitionBy("id"), presumably, when I
load in the df and do a groupBy("id"), a shuffle shouldn't be necessary
right? Effectively, we can load in the dataframe with a hash partitioner
already set, since each task can simply read all the folders where
id= where hash() % reducer_count == reducer_id. Is this an
optimization that is on the radar? This will be a huge boon in terms of
reducing the number of shuffles necessary if we're always joining on the
same columns.

Best,

Justin


Re: How Spark utilize low-level architecture features?

2016-01-21 Thread Steve Loughran

> On 19 Jan 2016, at 16:12, Boric Tan  wrote:
> 
> Hi there,
> 
> I am new to Spark, and would like to get some help to understand if Spark can 
> utilize the underlying architectures for better performance. If so, how does 
> it do it?
> 
> For example, assume there is a cluster built with machines of different CPUs, 
> will Spark check the individual CPU information and use some machine-specific 
> setting for the tasks assigned to that machine? Or is it totally dependent on 
> the underlying JVM implementation to run the JAR file, and therefor the JVM 
> is the place to check if certain CPU features can be used?
> 
> Thanks,
> Boric

You can't control where work is done based on CPU parts. Ideally your cluster 
is homogenous, or at least vary only in CPU performance and memory.

If some of your systems have GPUs and some don't, then in a YARN cluster, label 
the GPU parts and then use yarn queues or spark-submit to schedule the work 
only on those GPU systems.

The native libraries you load into JVMs are generally where CPU checks and 
features (e.g. x86 AES opcodes for encrypt/decrypt) would go

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



Generate Amplab queries set

2016-01-21 Thread sara mustafa
Hi,
I have downloaded the Amplab benchmark dataset from
s3n://big-data-benchmark/pavlo/text/tiny, but I don't know how to generate a
set of random mixed queries of different types like scan,aggregate and join. 

Thanks,



--
View this message in context: 
http://apache-spark-developers-list.1001551.n3.nabble.com/Generate-Amplab-queries-set-tp16071.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.

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



Re: How Spark utilize low-level architecture features?

2016-01-21 Thread borictan
Thanks for the explanation, Steve. 

I don't want to control where the work is done. What I wanted to understand
is if Spark could take advantage of the underlying architecture features.
For example, if the CPUs on the nodes support some improved vector
instructions, can the Spark jobs (if they have a lot of vector operations)
benefit from this? If yes, how does it happen, inside Spark, or the JVM
where the the job TAR is running on?

Also, for the GPU part you mentioned, labeling the GPU nodes, and scheduling
work to those GPU-enabled system does not mean the GPU computation power
will be utilized, right? The user has to provide CUDE codes
(openCL/CUDA/etc) and somehow link them to the system. Is my understanding
correct? 


Thanks,
Boric



--
View this message in context: 
http://apache-spark-developers-list.1001551.n3.nabble.com/How-Spark-utilize-low-level-architecture-features-tp16052p16072.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.

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



Re: How Spark utilize low-level architecture features?

2016-01-21 Thread borictan
Thanks, Fokko.

Yes, increasing the parallelism is one way to speed up the performance. On
the other hand, we are also looking for opportunities to harness the single
node hardware power to increase the single-node performance, which will help
the overall performance. 

Thanks,
Boric



--
View this message in context: 
http://apache-spark-developers-list.1001551.n3.nabble.com/How-Spark-utilize-low-level-architecture-features-tp16052p16073.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.

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



Re: Spark SQL: Avoid shuffles when data is already partitioned on disk

2016-01-21 Thread Reynold Xin
It is not necessary if you are using bucketing available in Spark 2.0. For
partitioning, it is still necessary because we do not assume each partition
is small, and as a result there is no guarantee all the records for a
partition end up in a single Spark task partition.


On Thu, Jan 21, 2016 at 3:22 AM, Justin Uang  wrote:

> Hi,
>
> If I had a df and I wrote it out via partitionBy("id"), presumably, when I
> load in the df and do a groupBy("id"), a shuffle shouldn't be necessary
> right? Effectively, we can load in the dataframe with a hash partitioner
> already set, since each task can simply read all the folders where
> id= where hash() % reducer_count == reducer_id. Is this an
> optimization that is on the radar? This will be a huge boon in terms of
> reducing the number of shuffles necessary if we're always joining on the
> same columns.
>
> Best,
>
> Justin
>


Re: Spark SQL: Avoid shuffles when data is already partitioned on disk

2016-01-21 Thread Reynold Xin
The original email was asking about data partitioning (Hive style) for
files, not in memory caching.

On Thursday, January 21, 2016, Takeshi Yamamuro 
wrote:

> You mean RDD#partitions are possibly split into multiple Spark task
> partitions?
> If so, the optimization below is wrong?
>
> Without opt.:
> 
> == Physical Plan ==
> TungstenAggregate(key=[col0#159],
> functions=[(sum(col1#160),mode=Final,isDistinct=false),(avg(col2#161),mode=Final,isDistinct=false)],
> output=[col0#159,sum(col1)#177,avg(col2)#178])
> +- TungstenAggregate(key=[col0#159],
> functions=[(sum(col1#160),mode=Partial,isDistinct=false),(avg(col2#161),mode=Partial,isDistinct=false)],
> output=[col0#159,sum#200,sum#201,count#202L])
>+- TungstenExchange hashpartitioning(col0#159,200), None
>   +- InMemoryColumnarTableScan [col0#159,col1#160,col2#161],
> InMemoryRelation [col0#159,col1#160,col2#161], true, 1,
> StorageLevel(true, true, false, true, 1), ConvertToUnsafe, None
>
> With opt.:
> 
> == Physical Plan ==
> TungstenAggregate(key=[col0#159],
> functions=[(sum(col1#160),mode=Complete,isDistinct=false),(avg(col2#161),mode=Final,isDistinct=false)],
> output=[col0#159,sum(col1)#177,avg(col2)#178])
> +- TungstenExchange hashpartitioning(col0#159,200), None
>   +- InMemoryColumnarTableScan [col0#159,col1#160,col2#161],
> InMemoryRelation [col0#159,col1#160,col2#161], true, 1,
> StorageLevel(true, true, false, true, 1), ConvertToUnsafe, None
>
>
>
> On Fri, Jan 22, 2016 at 12:13 PM, Reynold Xin  > wrote:
>
>> It is not necessary if you are using bucketing available in Spark 2.0.
>> For partitioning, it is still necessary because we do not assume each
>> partition is small, and as a result there is no guarantee all the records
>> for a partition end up in a single Spark task partition.
>>
>>
>> On Thu, Jan 21, 2016 at 3:22 AM, Justin Uang > > wrote:
>>
>>> Hi,
>>>
>>> If I had a df and I wrote it out via partitionBy("id"), presumably, when
>>> I load in the df and do a groupBy("id"), a shuffle shouldn't be necessary
>>> right? Effectively, we can load in the dataframe with a hash partitioner
>>> already set, since each task can simply read all the folders where
>>> id= where hash() % reducer_count == reducer_id. Is this an
>>> optimization that is on the radar? This will be a huge boon in terms of
>>> reducing the number of shuffles necessary if we're always joining on the
>>> same columns.
>>>
>>> Best,
>>>
>>> Justin
>>>
>>
>>
>
>
> --
> ---
> Takeshi Yamamuro
>


RE: Using CUDA within Spark / boosting linear algebra

2016-01-21 Thread Ulanov, Alexander
Hi Kazuaki,

Indeed, moving data to/from GPU is costly and this benchmark summarizes the 
costs for moving different data sizes with regards to matrices multiplication. 
These costs are paid for the convenience of using the standard BLAS API that 
Nvidia NVBLAS provides. The thing is that there are no code changes required 
(in Spark), one just needs to reference BLAS implementation with the system 
variable. Naturally, hardware-specific implementation will always be faster 
than default. The benchmark results show that fact by comparing jCuda (by means 
of BIDMat) and NVBLAS. However, it also shows that it worth using NVBLAS for 
large matrices because it can take advantage of several GPUs and it will be 
faster despite the copying overhead. That is also a known thing advertised by 
Nvidia.

By the way, I don't think that the column/row friendly format is an issue, 
because one can use transposed matrices to fit the required format. I believe 
that is just a software preference.

My suggestion with regards to your prototype would be to make comparisons with 
Spark's implementation of logistic regression (that does not take advantage of 
GPU) and also with BIDMach's (that takes advantage of GPUs). It will give the 
users a better understanding of your's implementation performance. Currently 
you compare it with Spark's example logistic regression implementation that is 
supposed to be a reference for learning Spark rather than benchmarking its 
performance.

Best regards, Alexander

From: Kazuaki Ishizaki [mailto:ishiz...@jp.ibm.com]
Sent: Thursday, January 21, 2016 3:34 AM
To: dev@spark.apache.org; Ulanov, Alexander; Joseph Bradley
Cc: John Canny; Evan R. Sparks; Xiangrui Meng; Sam Halliday
Subject: RE: Using CUDA within Spark / boosting linear algebra

Dear all,

 Hi Alexander,

 Using GPUs with Spark would be very exciting.  Small comment:
 Concerning your question earlier about keeping data stored on the
 GPU rather than having to move it between main memory and GPU
 memory on each iteration, I would guess this would be critical to
 getting good performance.  If you could do multiple local
 iterations before aggregating results, then the cost of data
 movement to the GPU could be amortized (and I believe that is done
 in practice).  Having Spark be aware of the GPU and using it as another 
 part of memory sounds like a much bigger undertaking.

 Joseph

As Joseph pointed out before, there are two potential issues to efficiently 
exploit GPUs in Spark.
(1) the cost of data movement between CPU and GPU
(2) the cost of encoding/decoding between current row-format and GPU-friendly 
column format

Our prototype http://kiszk.github.io/spark-gpu/addresses these two issues by 
supporting data partition caching in GPU device memory and by providing binary 
column storage for data partition. We really appreciate it if you would give us 
comments, suggestions, or feedback.

Best Regards
Kazuaki Ishizaki



From:"Ulanov, Alexander" 
>
To:Sam Halliday 
>, John Canny 
>
Cc:Xiangrui Meng >, 
"dev@spark.apache.org" 
>, Joseph Bradley 
>, "Evan R. Sparks" 
>
Date:2016/01/21 11:07
Subject:RE: Using CUDA within Spark / boosting linear algebra




Hi Everyone,

I've updated the benchmark and done experiments with new hardware with 2x 
Nvidia Tesla K80 (physically 4x Tesla K40) and 2x modern Haswell CPU Intel 
E5-2650 v3 @ 2.30GHz.

This time I computed average and median of 10 runs for each of experiment and 
approximated FLOPS.

Results are available at google docs (old experiments are in the other 2 
sheets):
https://docs.google.com/spreadsheets/d/1lWdVSuSragOobb0A_oeouQgHUMx378T9J5r7kwKSPkY/edit?usp=sharing
Benchmark code:
https://github.com/avulanov/scala-blas

Best regards, Alexander


From: Sam Halliday [mailto:sam.halli...@gmail.com]
Sent: Thursday, March 26, 2015 9:27 AM
To: John Canny
Cc: Xiangrui Meng; dev@spark.apache.org; Joseph 
Bradley; Evan R. Sparks; Ulanov, Alexander
Subject: Re: Using CUDA within Spark / boosting linear algebra


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 machine learning workloads: at least for 
non-image workloads in industry (and for image processing you would