Skip to content

Conversation

@dujunling
Copy link

…tatistics to update the inputMetrics

What changes were proposed in this pull request?

In FileScanRdd, we will update inputMetrics's bytesRead using updateBytesRead every 1000 rows or when close the iterator.

but when close the iterator, we will invoke updateBytesReadWithFileSize to increase the inputMetrics's bytesRead with file's length.

this will result in the inputMetrics's bytesRead is wrong when run the query with limit such as select * from table limit 1.

because we do not support for Hadoop 2.5 and earlier now, we always get the bytesRead from Hadoop FileSystem statistics other than files's length.

How was this patch tested?

manual test

@dujunling
Copy link
Author

@wzhfy

@wzhfy
Copy link
Contributor

wzhfy commented Aug 25, 2018

ok to test

@wzhfy
Copy link
Contributor

wzhfy commented Aug 25, 2018

this seems to be caused by removing support for Hadoop 2.5 and earlier? cc original authors @cloud-fan @srowen to make sure

@wzhfy
Copy link
Contributor

wzhfy commented Aug 25, 2018

test this please


override def close(): Unit = {
updateBytesRead()
updateBytesReadWithFileSize()
Copy link
Member

Choose a reason for hiding this comment

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

If we just remove this updateBytesReadWithFileSize, the issue in the description can be solved? We need to remove updateBytesReadWithFileSize in the line 142, too?

Copy link
Author

Choose a reason for hiding this comment

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

Yes, before SPARK-19464, there will only one works between updateBytesRead and updateBytesReadWithFileSize. If the hadoop version is 2.5 or earlier, updateBytesReadWithFileSize works, If the hadoop version is 2.6 or later, updateBytesRead works.

Copy link
Author

Choose a reason for hiding this comment

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

When there are one more files in the partition, the inputMetrics is wrong when updateBytesReadWithFileSize in the line 142 is exist.

Copy link
Member

Choose a reason for hiding this comment

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

aha, I see.

@maropu
Copy link
Member

maropu commented Aug 25, 2018

Is it difficult to add tests for checking the metric in the case select * from t limit 1?

@maropu
Copy link
Member

maropu commented Aug 25, 2018

btw, can you clean up the title and the description..?

@dujunling
Copy link
Author

While metris suites are in core test , fileScanRdd should be in sql test, it is difficult to add tests to check the input metrics in sql module

@maropu
Copy link
Member

maropu commented Aug 25, 2018

I'm not sure we can test the case though, for example, how about the sequence below?


import org.apache.spark.TaskContext
spark.range(10).selectExpr("id AS c0", "rand() AS c1").write.parquet("/tmp/t1")
val df = spark.read.parquet("/tmp/t1")

val fileScanRdd = df.repartition(1).queryExecution.executedPlan.children(0).children(0).execute()

fileScanRdd.mapPartitions { part =>
  println(s"Initial bytesRead=${TaskContext.get.taskMetrics().inputMetrics.bytesRead}")

  TaskContext.get.addTaskCompletionListener[Unit] { taskCtx =>
    // Check if the metric is correct?
    println(s"Total bytesRead=${TaskContext.get.taskMetrics().inputMetrics.bytesRead}")
  }
  part
}.collect

@srowen
Copy link
Member

srowen commented Aug 25, 2018

It's OK to assume Hadoop 2.6+ only. In fact 2.6 is quite old anyway.

dujunling added 2 commits August 27, 2018 11:26
@dujunling
Copy link
Author

@maropu I have added a ut to check the inputMetrics

spark.sparkContext.listenerBus.waitUntilEmpty(500)
spark.sparkContext.removeSparkListener(bytesReadListener)

assert(bytesReads.sum < 3000)
Copy link
Member

Choose a reason for hiding this comment

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

The data above could be made deterministic so that you can assert the bytes read more exactly. I wonder if it's important to make sure the bytes read are exact, rather than just close, given that the change above would change the metric only a little I think.

You can just track the sum rather than all values written, but it doesn't matter.

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)
Copy link
Member

Choose a reason for hiding this comment

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

I think a single partition is ok for this test.

@wzhfy
Copy link
Contributor

wzhfy commented Aug 31, 2018

ok to test

@SparkQA
Copy link

SparkQA commented Aug 31, 2018

Test build #95508 has finished for PR 22232 at commit 1c32646.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@srowen
Copy link
Member

srowen commented Sep 3, 2018

@dujunling I personally think this can be merged, but only if the test is tightened up

@maropu
Copy link
Member

maropu commented Sep 3, 2018

@srowen I could take this over or do follow-up if the author is still inactive.

@srowen
Copy link
Member

srowen commented Sep 3, 2018

I think that's fine @maropu ; we can always apportion credit appropriately later.

asfgit pushed a commit that referenced this pull request Sep 7, 2018
## What changes were proposed in this pull request?
This pr removed the method `updateBytesReadWithFileSize` in `FileScanRDD` because it computes input metrics by file size supported in Hadoop 2.5 and earlier. The current Spark does not support the versions, so it causes wrong input metric numbers.

This is rework from #22232.

Closes #22232

## How was this patch tested?
Added tests in `FileBasedDataSourceSuite`.

Closes #22324 from maropu/pr22232-2.

Lead-authored-by: dujunling <[email protected]>
Co-authored-by: Takeshi Yamamuro <[email protected]>
Signed-off-by: Sean Owen <[email protected]>
(cherry picked from commit ed249db)
Signed-off-by: Sean Owen <[email protected]>
@asfgit asfgit closed this in ed249db Sep 7, 2018
otterc pushed a commit to linkedin/spark that referenced this pull request Mar 22, 2023
This pr removed the method `updateBytesReadWithFileSize` in `FileScanRDD` because it computes input metrics by file size supported in Hadoop 2.5 and earlier. The current Spark does not support the versions, so it causes wrong input metric numbers.

This is rework from apache#22232.

Closes apache#22232

Added tests in `FileBasedDataSourceSuite`.

Closes apache#22324 from maropu/pr22232-2.

Lead-authored-by: dujunling <[email protected]>
Co-authored-by: Takeshi Yamamuro <[email protected]>
Signed-off-by: Sean Owen <[email protected]>
(cherry picked from commit ed249db)

Ref: LIHADOOP-41272

RB=1446834
BUG=LIHADOOP-41272
G=superfriends-reviewers
R=fli,mshen,yezhou,edlu
A=fli
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants