-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-30362][Core] Update InputMetrics in DataSourceRDD #27021
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,10 +17,15 @@ | |
|
|
||
| package org.apache.spark.sql.execution.datasources.v2 | ||
|
|
||
| import scala.language.existentials | ||
|
|
||
| import org.apache.spark._ | ||
| import org.apache.spark.deploy.SparkHadoopUtil | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} | ||
| import org.apache.spark.sql.vectorized.ColumnarBatch | ||
|
|
||
| class DataSourceRDDPartition(val index: Int, val inputPartition: InputPartition) | ||
| extends Partition with Serializable | ||
|
|
@@ -47,31 +52,16 @@ class DataSourceRDD( | |
|
|
||
| override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { | ||
| val inputPartition = castPartition(split).inputPartition | ||
| val reader: PartitionReader[_] = if (columnarReads) { | ||
| partitionReaderFactory.createColumnarReader(inputPartition) | ||
| val (iter, reader) = if (columnarReads) { | ||
| val batchReader = partitionReaderFactory.createColumnarReader(inputPartition) | ||
| val iter = new MetricsBatchIterator(new PartitionIterator[ColumnarBatch](batchReader)) | ||
| (iter, batchReader) | ||
| } else { | ||
| partitionReaderFactory.createReader(inputPartition) | ||
| val rowReader = partitionReaderFactory.createReader(inputPartition) | ||
| val iter = new MetricsIterator(new PartitionIterator[InternalRow](rowReader)) | ||
| (iter, rowReader) | ||
| } | ||
|
|
||
| context.addTaskCompletionListener[Unit](_ => reader.close()) | ||
| val iter = new Iterator[Any] { | ||
| private[this] var valuePrepared = false | ||
|
|
||
| override def hasNext: Boolean = { | ||
| if (!valuePrepared) { | ||
| valuePrepared = reader.next() | ||
| } | ||
| valuePrepared | ||
| } | ||
|
|
||
| override def next(): Any = { | ||
| if (!hasNext) { | ||
| throw new java.util.NoSuchElementException("End of stream") | ||
| } | ||
| valuePrepared = false | ||
| reader.get() | ||
| } | ||
| } | ||
| // TODO: SPARK-25083 remove the type erasure hack in data source scan | ||
| new InterruptibleIterator(context, iter.asInstanceOf[Iterator[InternalRow]]) | ||
| } | ||
|
|
@@ -80,3 +70,65 @@ class DataSourceRDD( | |
| castPartition(split).inputPartition.preferredLocations() | ||
| } | ||
| } | ||
|
|
||
| class PartitionIterator[T](reader: PartitionReader[T]) extends Iterator[T] { | ||
| private[this] var valuePrepared = false | ||
|
|
||
| override def hasNext: Boolean = { | ||
| if (!valuePrepared) { | ||
| valuePrepared = reader.next() | ||
| } | ||
| valuePrepared | ||
| } | ||
|
|
||
| override def next(): T = { | ||
| if (!hasNext) { | ||
| throw new java.util.NoSuchElementException("End of stream") | ||
| } | ||
| valuePrepared = false | ||
| reader.get() | ||
| } | ||
| } | ||
|
|
||
| class MetricsHandler extends Logging with Serializable { | ||
| private val inputMetrics = TaskContext.get().taskMetrics().inputMetrics | ||
| private val startingBytesRead = inputMetrics.bytesRead | ||
| private val getBytesRead = SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() | ||
|
|
||
| def updateMetrics(numRows: Int, force: Boolean = false): Unit = { | ||
| inputMetrics.incRecordsRead(numRows) | ||
| val shouldUpdateBytesRead = | ||
| inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0 | ||
| if (shouldUpdateBytesRead || force) { | ||
| inputMetrics.setBytesRead(startingBytesRead + getBytesRead()) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private class MetricsIterator[I](iter: Iterator[I]) extends Iterator[I] { | ||
| protected val metricsHandler = new MetricsHandler | ||
|
|
||
| override def hasNext: Boolean = { | ||
| if (iter.hasNext) { | ||
| true | ||
| } else { | ||
| metricsHandler.updateMetrics(0, force = true) | ||
| false | ||
| } | ||
| } | ||
|
|
||
| override def next(): I = { | ||
| val item = iter.next | ||
| metricsHandler.updateMetrics(1) | ||
| item | ||
| } | ||
| } | ||
|
|
||
| private class MetricsBatchIterator( | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: to be consistent, let's have a
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. or the base class can implement |
||
| iter: Iterator[ColumnarBatch]) extends MetricsIterator[ColumnarBatch](iter) { | ||
| override def next(): ColumnarBatch = { | ||
| val batch: ColumnarBatch = iter.next | ||
| metricsHandler.updateMetrics(batch.numRows) | ||
| batch | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,9 +16,12 @@ | |
| */ | ||
| package org.apache.spark.sql.execution | ||
|
|
||
| import scala.collection.mutable | ||
|
|
||
| import org.apache.hadoop.fs.Path | ||
|
|
||
| import org.apache.spark.SparkConf | ||
| import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} | ||
| import org.apache.spark.sql.{DataFrame, QueryTest} | ||
| import org.apache.spark.sql.execution.datasources.v2.BatchScanExec | ||
| import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan | ||
|
|
@@ -167,4 +170,31 @@ class DataSourceV2ScanExecRedactionSuite extends DataSourceScanRedactionTest { | |
| } | ||
| } | ||
| } | ||
|
|
||
| test("SPARK-30362: test input metrics for DSV2") { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. to actually test DSV2, let's call |
||
| Seq("json", "orc", "parquet").foreach { format => | ||
| withTempPath { path => | ||
| val dir = path.getCanonicalPath | ||
| spark.range(0, 10).write.format(format).save(dir) | ||
| val df = spark.read.format(format).load(dir) | ||
| val bytesReads = new mutable.ArrayBuffer[Long]() | ||
| val recordsRead = new mutable.ArrayBuffer[Long]() | ||
| val bytesReadListener = new SparkListener() { | ||
| override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { | ||
| bytesReads += taskEnd.taskMetrics.inputMetrics.bytesRead | ||
| recordsRead += taskEnd.taskMetrics.inputMetrics.recordsRead | ||
| } | ||
| } | ||
| sparkContext.addSparkListener(bytesReadListener) | ||
| try { | ||
| df.collect() | ||
| sparkContext.listenerBus.waitUntilEmpty() | ||
| assert(bytesReads.sum > 0) | ||
| assert(recordsRead.sum == 10) | ||
| } finally { | ||
| sparkContext.removeSparkListener(bytesReadListener) | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should these new classes be
privateinstead? I don't see a need to expose them.