Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

everywhere else, this is defined as a lazy val. Can you make it such that:

override lazy val statistics: Statistics  = {
  val sizePerRow = sink.schema.toAttributes.map(_.dataType.defaultSize).sum
  Statistics(sizeInBytes = sizePerRow * sink.allData.size)
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thanks @brkyvz. I'm afraid it should not be lazy val as the statistics should vary from batch to batch -- sink.allData.sizeshould vary from batch to batch. sizePerRow, however, does not vary from batch to batch, thus it's private val as it is now.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oh yeah! you're right! good catch!

}
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thanks! This will be much faster than a test where we actually join DataFrames.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

happy to do this

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 { _ =>
Expand Down