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 in Jupyter.
For JVM languages, we could also add a dependency on jvm-repr and do the same thing. rb On Tue, May 8, 2018 at 3:47 PM, Reynold Xin <r...@databricks.com> wrote: > s/underestimated/overestimated/ > > On Tue, May 8, 2018 at 3:44 PM Reynold Xin <r...@databricks.com> 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 feedback asap, and that's why we built eager >> analysis in the first place. >> >> Also I'm afraid you've significantly underestimated the level of >> technical sophistication of users. In many cases they struggle to get >> anything to work, and performance optimization of their programs is >> secondary to getting things working. As John Ousterhout says, "the greatest >> performance improvement of all is when a system goes from not-working to >> working". >> >> I really like Ryan's approach. Would be great if it is something more >> turn-key. >> >> >> >> >> >> >> On Tue, May 8, 2018 at 2:35 PM Marco Gaido <marcogaid...@gmail.com> >> wrote: >> >>> 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 something running more easily, but let them also using Spark >>> knowing less about how it works, thus they are likely to write worse >>> application and to have more problems in debugging any kind of problem >>> which may later (in production) occur (therefore affecting their experience >>> with the tool). >>> >>> Moreover, as Ryan also mentioned, there are tools/ways to force the >>> execution, helping in the debugging phase. So they can achieve without a >>> big effort the same result, but with a big difference: they are aware of >>> what is really happening, which may help them later. >>> >>> Thanks, >>> Marco >>> >>> 2018-05-08 21:37 GMT+02:00 Ryan Blue <rb...@netflix.com.invalid>: >>> >>>> 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 result. >>>> >>>> That way, the output of a cell or console execution always causes the >>>> dataframe to run and get displayed for that immediate feedback. But, there >>>> is no change to Spark’s behavior because the action is run by the REPL, and >>>> only when a dataframe is a result of an execution in order to display it. >>>> Intermediate results wouldn’t be run, but that gives users a way to avoid >>>> too many executions and would still support method chaining in the >>>> dataframe API (which would be horrible with an aggressive execution model). >>>> >>>> There are ways to do this in JVM languages as well if you are using a >>>> Scala or Java interpreter (see jvm-repr >>>> <https://github.com/jupyter/jvm-repr>). This is actually what we do in >>>> our Spark-based SQL interpreter to display results. >>>> >>>> rb >>>> >>>> >>>> On Tue, May 8, 2018 at 12:05 PM, Koert Kuipers <ko...@tresata.com> >>>> wrote: >>>> >>>>> 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 FetchFailedException. they will be debugging the reducer >>>>> task not realizing the error happened before that, and the >>>>> FetchFailedException is not the root cause. >>>>> >>>>> >>>>> On Tue, May 8, 2018 at 2:52 PM, Reynold Xin <r...@databricks.com> >>>>> wrote: >>>>> >>>>>> 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 adoption because of the multiplicative effect they have on the future >>>>>> generation. >>>>>> >>>>>> To my surprise the single biggest ask they want is to enable eager >>>>>> execution mode on all operations for teaching and debuggability: >>>>>> >>>>>> (1) Most of the students are relatively new to programming, and they >>>>>> need multiple iterations to even get the most basic operation right. In >>>>>> these cases, in order to trigger an error, they would need to explicitly >>>>>> add actions, which is non-intuitive. >>>>>> >>>>>> (2) If they don't add explicit actions to every operation and there >>>>>> is a mistake, the error pops up somewhere later where an action is >>>>>> triggered. This is in a different position from the code that causes the >>>>>> problem, and difficult for students to correlate the two. >>>>>> >>>>>> I suspect in the real world a lot of Spark users also struggle in >>>>>> similar ways as these students. While eager execution is really not >>>>>> practical in big data, in learning environments or in development against >>>>>> small, sampled datasets it can be pretty helpful. >>>>>> >>>>>> >>>>>> >>>>>> >>>>>> >>>>>> >>>>>> >>>>>> >>>>>> >>>>>> >>>>> >>>> >>>> >>>> -- >>>> Ryan Blue >>>> Software Engineer >>>> Netflix >>>> >>> >>> -- Ryan Blue Software Engineer Netflix