spark-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Tomasz Gawęda <tomasz.gaw...@outlook.com>
Subject Re: eager execution and debuggability
Date Mon, 14 May 2018 13:46:35 GMT
Hi,

>I agree, it would be great if we could make the errors more clear about where the error
happened (user code or in Spark code) and what assumption was violated. The problem is that
this is a really hard thing to do generally, like Reynold said. I think we should look for
individual cases where we can improve feedback so we can take a deeper look.


Huge +1 from my side. In current project we see that it's very hard to see why program failed,
what was wrong and in which column. Sometimes it's easier to use UDFs, because at least you
can wrote custom assertions in validation-sensitive functions. It's getting worse in WholeStageCodegen,
where generated code does not look like the code we wrote. If you know Spark a bit more you
have intuition where to search for errors, but I saw that my friends felt "lost" sometimes
and needed my help and tips how to check where is error. More explicit assertions would make
debugging much easier even for more experienced developers.

For performance investigation, it would be great to have job info in SQL Visualization - i.e.,
possibility to "expand" nodes and see how many jobs was triggered and number of tasks and
their duration. Now it's hard to debug it, especially for newbies.

Pozdrawiam / Best regards,
Tomek Gawęda

On 2018-05-10 18:31, Ryan Blue wrote:
> it would be fantastic if we could make it easier to debug Spark programs without needing
to rely on eager execution.

I agree, it would be great if we could make the errors more clear about where the error happened
(user code or in Spark code) and what assumption was violated. The problem is that this is
a really hard thing to do generally, like Reynold said. I think we should look for individual
cases where we can improve feedback so we can take a deeper look.

For example, we have an error case where users get a `NullPointerException` in generated code.
This was a huge pain to track down the first time, but the problem is almost always that the
user registered a UDF that returns an object and Spark inferred that it would be non-null
but the user's code returns null. In these cases, we could add better error messages to generated
code, like "Column 'x = some_udf(y)' is required, but the value was null". That would be really
useful.

> I used to use an evaluate(dataframe) -> DataFrame function that simply forces the
materialization of a dataframe.

We have one of these, too. `display` that will run a dataframe and format it for notebooks
(html and text output). We also have a `materialize` method that materializes a dataframe
or RDD, like people use `count` for, but that returns the materialized RDD so we can reuse
it from the last shuffle (we use this to avoid caching). It would be great if it were easier
to reuse the RDDs materialized by these calls, or even automatic. Right now, if you run `show`,
Spark doesn't know that a dataframe was materialized and won't reuse the results unless you
keep a reference to it.

We also have a problem where a dataframe used multiple times will cause several table scans
when the filters or projected columns change. That's because each action optimizes the dataframe
without knowing about the next. I'd love to hear ideas on how to fix this.

On Wed, May 9, 2018 at 5:39 AM, Tim Hunter <timhunter@databricks.com<mailto:timhunter@databricks.com>>
wrote:
The repr() trick is neat when working on a notebook. When working in a library, I used to
use an evaluate(dataframe) -> DataFrame function that simply forces the materialization
of a dataframe. As Reynold mentions, this is very convenient when working on a lot of chained
UDFs, and it is a standard trick in lazy environments and languages.

Tim

On Wed, May 9, 2018 at 3:26 AM, Reynold Xin <rxin@databricks.com<mailto:rxin@databricks.com>>
wrote:
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 occurs, rather than showing it potentially 30 lines later.

On Tue, May 8, 2018 at 7:22 PM Nicholas Chammas <nicholas.chammas@gmail.com<mailto:nicholas.chammas@gmail.com>>
wrote:

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 could point more directly to the offending transformation when something
goes wrong.

Is it somehow possible to have each individual operator (is that the correct term?) in a DAG
include metadata pointing back to the line of code that generated the operator? That way when
an action triggers an error, the failing operation can point to the relevant line of code
— even if it’s a transformation — and not just the action on the tail end that triggered
the error.

I don’t know how feasible this is, but addressing it would directly solve the issue of linking
failures to the responsible transformation, as opposed to leaving the user to break up a chain
of transformations with several debug actions. And this would benefit new and experienced
users alike.

Nick

2018년 5월 8일 (화) 오후 7:09, Ryan Blue rblue@netflix.com.invalid<http://mailto:rblue@netflix.com.invalid>님이
작성:

I've opened SPARK-24215 to track this.

On Tue, May 8, 2018 at 3:58 PM, Reynold Xin <rxin@databricks.com<mailto:rxin@databricks.com>>
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 <rblue@netflix.com<mailto:rblue@netflix.com>>
wrote:

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 <rxin@databricks.com<mailto:rxin@databricks.com>>
wrote:
s/underestimated/overestimated/

On Tue, May 8, 2018 at 3:44 PM Reynold Xin <rxin@databricks.com<mailto:rxin@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 <marcogaido91@gmail.com<mailto:marcogaido91@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 <rblue@netflix.com.invalid<mailto:rblue@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 <koert@tresata.com<mailto:koert@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.
[https://ssl.gstatic.com/ui/v1/icons/mail/images/cleardot.gif]


On Tue, May 8, 2018 at 2:52 PM, Reynold Xin <rxin@databricks.com<mailto:rxin@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



--
Ryan Blue
Software Engineer
Netflix
​




--
Ryan Blue
Software Engineer
Netflix

Mime
View raw message