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 @@ -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
Expand All @@ -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 MetricsRowIterator(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]])
}
Expand All @@ -80,3 +70,68 @@ class DataSourceRDD(
castPartition(split).inputPartition.preferredLocations()
}
}

private 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()
}
}

private 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 abstract 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
}
}
}

private class MetricsRowIterator(
iter: Iterator[InternalRow]) extends MetricsIterator[InternalRow](iter) {
override def next(): InternalRow = {
val item = iter.next
metricsHandler.updateMetrics(1)
item
}
}

private class MetricsBatchIterator(
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: to be consistent, let's have a MetricsRowIterator, and the base class MetricsIterator doesn't need to implement next

Copy link
Contributor

Choose a reason for hiding this comment

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

or the base class can implement next as

protected def numRecords(element: T): Int

override def next(): I = {
  val item = iter.next
  metricsHandler.updateMetrics(numRecords(item))
  item
}

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
Expand Up @@ -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
Expand Down Expand Up @@ -167,4 +170,33 @@ class DataSourceV2ScanExecRedactionSuite extends DataSourceScanRedactionTest {
}
}
}

test("SPARK-30362: test input metrics for DSV2") {
Copy link
Contributor

Choose a reason for hiding this comment

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

to actually test DSV2, let's call withSQLConf in the test and set SQLConf.USE_V1_SOURCE_LIST to empty.

withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") {
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)
}
}
}
}
}
}