dongjoon-hyun commented on a change in pull request #24005: [SPARK-27085][SQL] Migrate CSV to File Data Source V2 URL: https://github.com/apache/spark/pull/24005#discussion_r268047064
########## File path: sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala ########## @@ -470,22 +471,24 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with Befo } test("SPARK-25237 compute correct input metrics in FileScanRDD") { - withTempPath { p => - val path = p.getAbsolutePath - spark.range(1000).repartition(1).write.csv(path) - val bytesReads = new mutable.ArrayBuffer[Long]() - val bytesReadListener = new SparkListener() { - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - bytesReads += taskEnd.taskMetrics.inputMetrics.bytesRead + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "csv") { + withTempPath { p => + val path = p.getAbsolutePath + spark.range(1000).repartition(1).write.csv(path) + val bytesReads = new mutable.ArrayBuffer[Long]() + val bytesReadListener = new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + bytesReads += taskEnd.taskMetrics.inputMetrics.bytesRead + } + } + sparkContext.addSparkListener(bytesReadListener) + try { + spark.read.csv(path).limit(1).collect() + sparkContext.listenerBus.waitUntilEmpty(1000L) + assert(bytesReads.sum === 7860) + } finally { + sparkContext.removeSparkListener(bytesReadListener) } - } - sparkContext.addSparkListener(bytesReadListener) - try { - spark.read.csv(path).limit(1).collect() - sparkContext.listenerBus.waitUntilEmpty(1000L) - assert(bytesReads.sum === 7860) - } finally { - sparkContext.removeSparkListener(bytesReadListener) } } } Review comment: If this fails due to the number of bytes, can we have test coverage for both? Note that only `try` part is changed. ```scala test("SPARK-25237 compute correct input metrics in FileScanRDD") { withTempPath { p => val path = p.getAbsolutePath spark.range(1000).repartition(1).write.csv(path) val bytesReads = new mutable.ArrayBuffer[Long]() val bytesReadListener = new SparkListener() { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { bytesReads += taskEnd.taskMetrics.inputMetrics.bytesRead } } sparkContext.addSparkListener(bytesReadListener) try { Seq(("csv", 7860), ("", 3930)).foreach { case (list, sum) => bytesReads.clear() withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> list) { spark.read.csv(path).limit(1).collect() sparkContext.listenerBus.waitUntilEmpty(1000L) assert(bytesReads.sum === sum) } } } finally { sparkContext.removeSparkListener(bytesReadListener) } } } ``` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org