Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
Prev Previous commit
fixed review comments
  • Loading branch information
sandeep-katta committed Jan 30, 2020
commit 34425ad1eae32fe1ca177ad359685c410a467760
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ class DataSourceRDD(
(iter, batchReader)
} else {
val rowReader = partitionReaderFactory.createReader(inputPartition)
val iter = new MetricsIterator(new PartitionIterator[InternalRow](rowReader))
val iter = new MetricsRowIterator(new PartitionIterator[InternalRow](rowReader))
(iter, rowReader)
}
context.addTaskCompletionListener[Unit](_ => reader.close())
Expand Down Expand Up @@ -105,7 +105,7 @@ private class MetricsHandler extends Logging with Serializable {
}
}

private class MetricsIterator[I](iter: Iterator[I]) extends Iterator[I] {
private abstract class MetricsIterator[I](iter: Iterator[I]) extends Iterator[I] {
protected val metricsHandler = new MetricsHandler

override def hasNext: Boolean = {
Expand All @@ -116,8 +116,11 @@ private class MetricsIterator[I](iter: Iterator[I]) extends Iterator[I] {
false
}
}
}

override def next(): I = {
private class MetricsRowIterator(
iter: Iterator[InternalRow]) extends MetricsIterator[InternalRow](iter) {
override def next(): InternalRow = {
val item = iter.next
metricsHandler.updateMetrics(1)
item
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -172,27 +172,29 @@ 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.

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
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)
}
}
sparkContext.addSparkListener(bytesReadListener)
try {
df.collect()
sparkContext.listenerBus.waitUntilEmpty()
assert(bytesReads.sum > 0)
assert(recordsRead.sum == 10)
} finally {
sparkContext.removeSparkListener(bytesReadListener)
}
}
}
Expand Down