Thanks Wenchen. That makes a lot of sense now (after you made the point about AnalysisBarrier that I've been seeing here and there, but haven't spent much time to explore yet, but turned out important).
Pozdrawiam, Jacek Laskowski ---- https://about.me/JacekLaskowski Mastering Spark SQL https://bit.ly/mastering-spark-sql Spark Structured Streaming https://bit.ly/spark-structured-streaming Mastering Kafka Streams https://bit.ly/mastering-kafka-streams Follow me at https://twitter.com/jaceklaskowski On Thu, Jan 4, 2018 at 11:57 AM, Wenchen Fan <cloud0...@gmail.com> wrote: > First of all, I think you know that `QueryExecution` is a developer API > right? By definition `QueryExecution.logical` is the input plan, which can > even be unresolved. Developers should be aware of it and do not apply > operations that need the plan to be resolved. Obviously `LogicalPlan.stats` > needs the plan to be resolved. > > For this particular case, we can make it work by defining `computeStats` > in `AnalysisBarrier`. But it's also OK to just leave it as it is, as this > doesn't break any real use cases. > > On Thu, Jan 4, 2018 at 4:36 PM, Jacek Laskowski <ja...@japila.pl> wrote: > >> Hi, >> >> I use Spark from the master today. >> >> $ ./bin/spark-shell --version >> Welcome to >> ____ __ >> / __/__ ___ _____/ /__ >> _\ \/ _ \/ _ `/ __/ '_/ >> /___/ .__/\_,_/_/ /_/\_\ version 2.3.0-SNAPSHOT >> /_/ >> >> Using Scala version 2.11.8, Java HotSpot(TM) 64-Bit Server VM, 1.8.0_152 >> Branch master >> Compiled by user jacek on 2018-01-04T05:44:05Z >> Revision 7d045c5f00e2c7c67011830e2169a4e130c3ace8 >> >> Can anyone explain why some queries have stats in logical plan while >> others don't (and I had to use analyzed logical plan)? >> >> I can explain the difference using the code, but I don't know why there >> is the difference. >> >> spark.range(1000).write.parquet("/tmp/p1000") >> // The stats are available in logical plan (in logical "phase") >> scala> spark.read.parquet("/tmp/p1000").queryExecution.logical.stats >> res21: org.apache.spark.sql.catalyst.plans.logical.Statistics = >> Statistics(sizeInBytes=6.9 KB, hints=none) >> >> // logical plan fails, but it worked fine above --> WHY?! >> val names = Seq((1, "one"), (2, "two")).toDF("id", "name") >> scala> names.queryExecution.logical.stats >> java.lang.UnsupportedOperationException >> at org.apache.spark.sql.catalyst.plans.logical.LeafNode.compute >> Stats(LogicalPlan.scala:232) >> at org.apache.spark.sql.catalyst.plans.logical.statsEstimation. >> SizeInBytesOnlyStatsPlanVisitor$.default(SizeInBytesOnlyStat >> sPlanVisitor.scala:55) >> at org.apache.spark.sql.catalyst.plans.logical.statsEstimation. >> SizeInBytesOnlyStatsPlanVisitor$.default(SizeInBytesOnlyStat >> sPlanVisitor.scala:27) >> >> // analyzed logical plan works fine >> scala> names.queryExecution.analyzed.stats >> res23: org.apache.spark.sql.catalyst.plans.logical.Statistics = >> Statistics(sizeInBytes=48.0 B, hints=none) >> >> Pozdrawiam, >> Jacek Laskowski >> ---- >> https://about.me/JacekLaskowski >> Mastering Spark SQL https://bit.ly/mastering-spark-sql >> Spark Structured Streaming https://bit.ly/spark-structured-streaming >> Mastering Kafka Streams https://bit.ly/mastering-kafka-streams >> Follow me at https://twitter.com/jaceklaskowski >> > >