From 0f75257b50a611e069d406da8d72225bb4e73b51 Mon Sep 17 00:00:00 2001 From: dujunling Date: Sat, 25 Aug 2018 14:20:35 +0800 Subject: [PATCH 1/3] remove updateBytesReadWithFileSize because we use Hadoop FileSystem statistics to update the inputMetrics --- .../spark/sql/execution/datasources/FileScanRDD.scala | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 99fc78ff3e49..345c9d82ca0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -89,14 +89,6 @@ class FileScanRDD( inputMetrics.setBytesRead(existingBytesRead + getBytesReadCallback()) } - // If we can't get the bytes read from the FS stats, fall back to the file size, - // which may be inaccurate. - private def updateBytesReadWithFileSize(): Unit = { - if (currentFile != null) { - inputMetrics.incBytesRead(currentFile.length) - } - } - private[this] val files = split.asInstanceOf[FilePartition].files.toIterator private[this] var currentFile: PartitionedFile = null private[this] var currentIterator: Iterator[Object] = null @@ -139,7 +131,6 @@ class FileScanRDD( /** Advances to the next file. Returns true if a new non-empty iterator is available. */ private def nextIterator(): Boolean = { - updateBytesReadWithFileSize() if (files.hasNext) { currentFile = files.next() logInfo(s"Reading File $currentFile") @@ -208,7 +199,6 @@ class FileScanRDD( override def close(): Unit = { updateBytesRead() - updateBytesReadWithFileSize() InputFileBlockHolder.unset() } } From 53dd42c1facebf97044afb22b1f0894ec209f3bb Mon Sep 17 00:00:00 2001 From: dujunling Date: Mon, 27 Aug 2018 11:26:30 +0800 Subject: [PATCH 2/3] add ut --- .../datasources/FileSourceSuite.scala | 54 +++++++++++++++++++ 1 file changed, 54 insertions(+) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceSuite.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceSuite.scala new file mode 100644 index 000000000000..11242f6d7111 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceSuite.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.expressions.PredicateHelper +import org.apache.spark.sql.test.SharedSQLContext + + +class FileSourceSuite extends QueryTest with SharedSQLContext with PredicateHelper { + + test("[SPARK-25237] remove updateBytesReadWithFileSize in FileScanRdd") { + withTempPath { p => + val path = p.getAbsolutePath + spark.range(1000).selectExpr("id AS c0", "rand() AS c1").repartition(10).write.csv(path) + + val bytesReads = new ArrayBuffer[Long]() + val bytesReadListener = new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + bytesReads += taskEnd.taskMetrics.inputMetrics.bytesRead + } + } + // Avoid receiving earlier taskEnd events + spark.sparkContext.listenerBus.waitUntilEmpty(500) + + spark.sparkContext.addSparkListener(bytesReadListener) + + spark.read.csv(path).limit(1).collect() + + spark.sparkContext.listenerBus.waitUntilEmpty(500) + spark.sparkContext.removeSparkListener(bytesReadListener) + + assert(bytesReads.sum < 5000) + } + } +} From 1c326466fbd24c432184be6e53afec93369970c1 Mon Sep 17 00:00:00 2001 From: dujunling Date: Mon, 27 Aug 2018 11:33:46 +0800 Subject: [PATCH 3/3] ut --- .../spark/sql/execution/datasources/FileSourceSuite.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceSuite.scala index 11242f6d7111..10ad28594d8b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceSuite.scala @@ -31,6 +31,7 @@ class FileSourceSuite extends QueryTest with SharedSQLContext with PredicateHelp withTempPath { p => val path = p.getAbsolutePath spark.range(1000).selectExpr("id AS c0", "rand() AS c1").repartition(10).write.csv(path) + val df = spark.read.csv(path).limit(1) val bytesReads = new ArrayBuffer[Long]() val bytesReadListener = new SparkListener() { @@ -43,12 +44,12 @@ class FileSourceSuite extends QueryTest with SharedSQLContext with PredicateHelp spark.sparkContext.addSparkListener(bytesReadListener) - spark.read.csv(path).limit(1).collect() + df.collect() spark.sparkContext.listenerBus.waitUntilEmpty(500) spark.sparkContext.removeSparkListener(bytesReadListener) - assert(bytesReads.sum < 5000) + assert(bytesReads.sum < 3000) } } }