diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index 48d9791faf1e9..613c7ccdd226a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -27,7 +27,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LeafNode +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics} import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -212,4 +212,8 @@ class MemorySink(val schema: StructType, outputMode: OutputMode) extends Sink wi */ case class MemoryPlan(sink: MemorySink, output: Seq[Attribute]) extends LeafNode { def this(sink: MemorySink) = this(sink, sink.schema.toAttributes) + + private val sizePerRow = sink.schema.toAttributes.map(_.dataType.defaultSize).sum + + override def statistics: Statistics = Statistics(sizePerRow * sink.allData.size) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala index 310d75630272b..4e9fba9dbaa1e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala @@ -187,6 +187,22 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { query.stop() } + test("MemoryPlan statistics") { + implicit val schema = new StructType().add(new StructField("value", IntegerType)) + val sink = new MemorySink(schema, InternalOutputModes.Append) + val plan = new MemoryPlan(sink) + + // Before adding data, check output + checkAnswer(sink.allData, Seq.empty) + assert(plan.statistics.sizeInBytes === 0) + + sink.addBatch(0, 1 to 3) + assert(plan.statistics.sizeInBytes === 12) + + sink.addBatch(1, 4 to 6) + assert(plan.statistics.sizeInBytes === 24) + } + ignore("stress test") { // Ignore the stress test as it takes several minutes to run (0 until 1000).foreach { _ =>