Re: Spark UI Source Code

2018-05-08 Thread Anshi Shrivastava
Hi Marcelo, Dev, Thanks for your response. I have used SparkListeners to fetch the metrics (the public REST API uses the same) but to monitor these metrics over time, I have to persist them (using KVStore library of spark). Is there a way to fetch data from this KVStore (which uses levelDb for

Re: Documenting the various DataFrame/SQL join types

2018-05-08 Thread Nicholas Chammas
OK great, I’m happy to take this on. Does it make sense to approach this by adding an example for each join type here (and perhaps also in the matching areas for Scala, Java, and R), and then referencing the

Re: Identifying specific persisted DataFrames via getPersistentRDDs()

2018-05-08 Thread Nicholas Chammas
That’s correct. I probably would have done better to title this thread something like “How to effectively track and release persisted DataFrames”. I jumped the gun in my initial email by referencing getPersistentRDDs() as a potential solution, but in theory the desired API is something like

Re: eager execution and debuggability

2018-05-08 Thread Reynold Xin
Yes would be great if possible but it’s non trivial (might be impossible to do in general; we already have stacktraces that point to line numbers when an error occur in UDFs but clearly that’s not sufficient). Also in environments like REPL it’s still more useful to show error as soon as it

Re: eager execution and debuggability

2018-05-08 Thread Nicholas Chammas
This may be technically impractical, but it would be fantastic if we could make it easier to debug Spark programs without needing to rely on eager execution. Sprinkling .count() and .checkpoint() at various points in my code is still a debugging technique I use, but it always makes me wish Spark

Re: [DISCUSS] Spark SQL internal data: InternalRow or UnsafeRow?

2018-05-08 Thread Ryan Blue
Because the InternalRow representation is already in the v2 public API, we’re already working on cleaning up classes and documenting the internal representation. The JIRA issue is SPARK-23657 and my PR is #21242 . The confusion I want to clear up is

Re: [DISCUSS] Spark SQL internal data: InternalRow or UnsafeRow?

2018-05-08 Thread Reynold Xin
IIRC we switched all internals to UnsafeRow for simplicity. It is easier to serialize UnsafeRows, compute hash codes, etc. At some point we had a bug with unioning two plans producing different types of rows, so we forced the conversion at input. Can't your "wish" be satisfied by having the

Re: [DISCUSS] Spark SQL internal data: InternalRow or UnsafeRow?

2018-05-08 Thread Ryan Blue
Is the goal to design an API so the consumers of the API can directly produces what Spark expects internally, to cut down perf cost? No. That has already been done. The problem on the API side is that it makes little sense to force implementers to create UnsafeRow when it almost certainly causes

Re: [DISCUSS] Spark SQL internal data: InternalRow or UnsafeRow?

2018-05-08 Thread Reynold Xin
What the internal operators do are strictly internal. To take one step back, is the goal to design an API so the consumers of the API can directly produces what Spark expects internally, to cut down perf cost? On Tue, May 8, 2018 at 1:22 PM Ryan Blue wrote: > While

Re: eager execution and debuggability

2018-05-08 Thread Ryan Blue
I've opened SPARK-24215 to track this. On Tue, May 8, 2018 at 3:58 PM, Reynold Xin wrote: > Yup. Sounds great. This is something simple Spark can do and provide huge > value to the end users. > > > On Tue, May 8, 2018 at 3:53 PM Ryan Blue wrote: > >>

Re: eager execution and debuggability

2018-05-08 Thread Reynold Xin
Yup. Sounds great. This is something simple Spark can do and provide huge value to the end users. On Tue, May 8, 2018 at 3:53 PM Ryan Blue wrote: > Would be great if it is something more turn-key. > > We can easily add the __repr__ and _repr_html_ methods and behavior to >

Re: eager execution and debuggability

2018-05-08 Thread Ryan Blue
Would be great if it is something more turn-key. We can easily add the __repr__ and _repr_html_ methods and behavior to PySpark classes. We could also add a configuration property to determine whether the dataset evaluation is eager or not. That would make it turn-key for anyone running PySpark

Re: eager execution and debuggability

2018-05-08 Thread Reynold Xin
s/underestimated/overestimated/ On Tue, May 8, 2018 at 3:44 PM Reynold Xin wrote: > Marco, > > There is understanding how Spark works, and there is finding bugs early in > their own program. One can perfectly understand how Spark works and still > find it valuable to get

Re: eager execution and debuggability

2018-05-08 Thread Reynold Xin
Marco, There is understanding how Spark works, and there is finding bugs early in their own program. One can perfectly understand how Spark works and still find it valuable to get feedback asap, and that's why we built eager analysis in the first place. Also I'm afraid you've significantly

Re: eager execution and debuggability

2018-05-08 Thread Marco Gaido
I am not sure how this is useful. For students, it is important to understand how Spark works. This can be critical in many decision they have to take (whether and what to cache for instance) in order to have performant Spark application. Creating a eager execution probably can help them having

[DISCUSS] Spark SQL internal data: InternalRow or UnsafeRow?

2018-05-08 Thread Ryan Blue
While moving the new data source API to InternalRow, I noticed a few odd things: - Spark scans always produce UnsafeRow, but that data is passed around as InternalRow with explicit casts. - Operators expect InternalRow and nearly all codegen works with InternalRow (I’ve tested this

Re: eager execution and debuggability

2018-05-08 Thread Ryan Blue
At Netflix, we use Jupyter notebooks and consoles for interactive sessions. For anyone interested, this mode of interaction is really easy to add in Jupyter and PySpark. You would just define a different *repr_html* or *repr* method for Dataset that runs a take(10) or take(100) and formats the

Re: eager execution and debuggability

2018-05-08 Thread Koert Kuipers
yeah we run into this all the time with new hires. they will send emails explaining there is an error in the .write operation and they are debugging the writing to disk, focusing on that piece of code :) unrelated, but another frequent cause for confusion is cascading errors. like the

eager execution and debuggability

2018-05-08 Thread Reynold Xin
Similar to the thread yesterday about improving ML/DL integration, I'm sending another email on what I've learned recently from Spark users. I recently talked to some educators that have been teaching Spark in their (top-tier) university classes. They are some of the most important users for

Re: Integrating ML/DL frameworks with Spark

2018-05-08 Thread Nan Zhu
.how I skipped the last part On Tue, May 8, 2018 at 11:16 AM, Reynold Xin wrote: > Yes, Nan, totally agree. To be on the same page, that's exactly what I > wrote wasn't it? > > On Tue, May 8, 2018 at 11:14 AM Nan Zhu wrote: > >> besides

Re: Integrating ML/DL frameworks with Spark

2018-05-08 Thread Reynold Xin
Yes, Nan, totally agree. To be on the same page, that's exactly what I wrote wasn't it? On Tue, May 8, 2018 at 11:14 AM Nan Zhu wrote: > besides that, one of the things which is needed by multiple frameworks is > to schedule tasks in a single wave > > i.e. > > if some

Re: Integrating ML/DL frameworks with Spark

2018-05-08 Thread Nan Zhu
besides that, one of the things which is needed by multiple frameworks is to schedule tasks in a single wave i.e. if some frameworks like xgboost/mxnet requires 50 parallel workers, Spark is desired to provide a capability to ensure that either we run 50 tasks at once, or we should quit the

Re: Integrating ML/DL frameworks with Spark

2018-05-08 Thread Reynold Xin
I think that's what Xiangrui was referring to. Instead of retrying a single task, retry the entire stage, and the entire stage of tasks need to be scheduled all at once. On Tue, May 8, 2018 at 8:53 AM Shivaram Venkataraman < shiva...@eecs.berkeley.edu> wrote: > >> >>>- Fault tolerance and

Re: Integrating ML/DL frameworks with Spark

2018-05-08 Thread Naveen Swamy
I am committer on the MXNet project and very interested in working on Integrating with Spark. I am wondering how would training proceed in case of 1) training is done on one host with multiple GPUs -- I don't know if Spark's capabilities can leveraged here 2) distributed training with data

Re: Identifying specific persisted DataFrames via getPersistentRDDs()

2018-05-08 Thread Mark Hamstra
If I am understanding you correctly, you're just saying that the problem is that you know what you want to keep, not what you want to throw away, and that there is no unpersist DataFrames call based on that what-to-keep information. On Tue, May 8, 2018 at 6:00 AM, Nicholas Chammas

Re: Integrating ML/DL frameworks with Spark

2018-05-08 Thread Shivaram Venkataraman
> > > >>- Fault tolerance and execution model: Spark assumes fine-grained >>task recovery, i.e. if something fails, only that task is rerun. This >>doesn’t match the execution model of distributed ML/DL frameworks that are >>typically MPI-based, and rerunning a single task would

Re: Documenting the various DataFrame/SQL join types

2018-05-08 Thread Reynold Xin
Would be great to document. Probably best with examples. On Tue, May 8, 2018 at 6:13 AM Nicholas Chammas wrote: > The documentation for DataFrame.join() > > lists all the

Documenting the various DataFrame/SQL join types

2018-05-08 Thread Nicholas Chammas
The documentation for DataFrame.join() lists all the join types we support: - inner - cross - outer - full - full_outer - left - left_outer - right - right_outer - left_semi

Re: Identifying specific persisted DataFrames via getPersistentRDDs()

2018-05-08 Thread Nicholas Chammas
I certainly can, but the problem I’m facing is that of how best to track all the DataFrames I no longer want to persist. I create and persist various DataFrames throughout my pipeline. Spark is already tracking all this for me, and exposing some of that tracking information via

Re: Integrating ML/DL frameworks with Spark

2018-05-08 Thread Jörn Franke
Hi, You misunderstood me. I exactly wanted to say that Spark should be aware of them. So I agree with you. The point is to have also the yarn GPU/fpga scheduling as an option aside a potential spark GPU/fpga scheduler. For the other proposal - yes the interfaces are slow, but one has to think