I may be ignorant of other debugging methods in Spark but the best success I've had is using smaller datasets (if runs take a long time) and adding intermediate output steps. This is quite different from application development in non-distributed systems where a debugger is trivial to attach but I believe it's one of the trade offs on using a system like Spark for data processing, most SQL engines suffer from the same issue. If you do believe there is a bug in Spark using the explain function like Herman mentioned helps as well as looking at the Spark plan in the Spark UI
On Tue, Feb 13, 2024 at 9:24 AM Nicholas Chammas <nicholas.cham...@gmail.com> wrote: > OK, I figured it out. The details are in SPARK-47024 > <https://issues.apache.org/jira/browse/SPARK-47024> for anyone who’s > interested. > > It turned out to be a floating point arithmetic “bug”. The main reason I > was able to figure it out was because I’ve been investigating another, > unrelated bug (a real bug) related to floats, so these weird float corner > cases have been top of mind. > > If it weren't for that, I wonder how much progress I would have made. > Though I could inspect the generated code, I couldn’t figure out how to get > logging statements placed in the generated code to print somewhere I could > see them. > > Depending on how often we find ourselves debugging aggregates like this, > it would be really helpful if we added some way to trace the aggregation > buffer. > > In any case, mystery solved. Thank you for the pointer! > > > On Feb 12, 2024, at 8:39 AM, Herman van Hovell <her...@databricks.com> > wrote: > > There is no really easy way of getting the state of the aggregation > buffer, unless you are willing to modify the code generation and sprinkle > in some logging. > > What I would start with is dumping the generated code by calling > explain('codegen') on the DataFrame. That helped me to find similar issues > in most cases. > > HTH > > On Sun, Feb 11, 2024 at 11:26 PM Nicholas Chammas < > nicholas.cham...@gmail.com> wrote: > >> Consider this example: >> >> >>> from pyspark.sql.functions import sum>>> >> >>> spark.range(4).repartition(2).select(sum("id")).show()+-------+|sum(id)|+-------+| >> >>> 6 |+-------+ >> >> >> I’m trying to understand how this works because I’m investigating a bug >> in this kind of aggregate. >> >> I see that doProduceWithoutKeys >> <https://github.com/apache/spark/blob/d02fbba6491fd17dc6bfc1a416971af7544952f3/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregateCodegenSupport.scala#L98> >> and doConsumeWithoutKeys >> <https://github.com/apache/spark/blob/d02fbba6491fd17dc6bfc1a416971af7544952f3/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregateCodegenSupport.scala#L193> >> are >> called, and I believe they are responsible for computing a declarative >> aggregate like `sum`. But I’m not sure how I would debug the generated >> code, or the inputs that drive what code gets generated. >> >> Say you were running the above example and it was producing an incorrect >> result, and you knew the problem was somehow related to the sum. How would >> you troubleshoot it to identify the root cause? >> >> Ideally, I would like some way to track how the aggregation buffer >> mutates as the computation is executed, so I can see something roughly like: >> >> [0, 1, 2, 3] >> [1, 5] >> [6] >> >> >> Is there some way to trace a declarative aggregate like this? >> >> Nick >> >> >