Filed SPARK-29450 [1] and raised a patch [2]. Please let me know if you would like to be assigned as a reporter of SPARK-29450.
1. https://issues.apache.org/jira/browse/SPARK-29450 2. https://github.com/apache/spark/pull/26104 On Sun, Oct 13, 2019 at 4:06 PM Jungtaek Lim <kabhwan.opensou...@gmail.com> wrote: > Thanks for reporting. That might be possible it could be intentionally > excluded as it could cause some confusion before introducing empty batch > (given output rows are irrelevant to the input rows in current batch), but > given we have empty batch I'm not seeing the reason why we don't deal with > it. I'll file and submit a patch. > > Btw, there's a metric bug with empty batch as well - see SPARK-29314 [1] > which I've submitted a patch recently. > > Thanks, > Jungtaek Lim (HeartSaVioR) > > 1. http://issues.apache.org/jira/browse/SPARK-29314 > > > On Sun, Oct 13, 2019 at 1:12 AM Jacek Laskowski <ja...@japila.pl> wrote: > >> Hi, >> >> I use Spark 2.4.4 >> >> I've just noticed that the number of output rows metric >> of StateStoreSaveExec physical operator does not seem to be measured for >> Append output mode. In other words, whatever happens before or >> after StateStoreSaveExec operator the metric is always 0. >> >> It is measured for the other modes - Complete and Update. >> >> See >> https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala#L329-L365 >> >> Is this intentional? Why? >> >> Pozdrawiam, >> Jacek Laskowski >> ---- >> https://about.me/JacekLaskowski >> The Internals of Spark SQL https://bit.ly/spark-sql-internals >> The Internals of Spark Structured Streaming >> https://bit.ly/spark-structured-streaming >> The Internals of Apache Kafka https://bit.ly/apache-kafka-internals >> Follow me at https://twitter.com/jaceklaskowski >> >>