New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
SPARK-5199. FS read metrics should support CombineFileSplits and track bytes from all FSs #4050
Conversation
Test build #25567 has started for PR 4050 at commit
|
@@ -219,6 +220,9 @@ class HadoopRDD[K, V]( | |||
val bytesReadCallback = if (split.inputSplit.value.isInstanceOf[FileSplit]) { | |||
SparkHadoopUtil.get.getFSBytesReadOnThreadCallback( | |||
split.inputSplit.value.asInstanceOf[FileSplit].getPath, jobConf) | |||
} else if (split.inputSplit.value.isInstanceOf[CombineFileSplit]) { | |||
SparkHadoopUtil.get.getFSBytesReadOnThreadCallback( | |||
split.inputSplit.value.asInstanceOf[CombineFileSplit].getPath(0), jobConf) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you push this logic down to the SparkHadoopUtil so that we don't duplicate it in two places (HadoopRDD and NewHadoopRDD).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The issue is that those are actually two different classes. There's a CombineFileSplit
for the old MR API (used by HadoopRDD
) and a CombineFileSplit
for the new one (used by NewHadoopRDD
).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, SparkHadoopUtil) can check for those classes. It can have a matcher on the 4 classes (2 new and 2 old). So the call from hadoopRdd would be something like:
SparkHadoopUtil.get.getFSBytesReadOnThreadCallback(split.inputSplit, jobConf)
Not a big deal i guess since in SparkHadoopUtil you'll have four cases but at least that logic is centralized.
This mostly LGTM. My only concern is with the proliferation of copy pasta between the HadoopRDD and NewHadoopRDD. |
Test build #25567 has finished for PR 4050 at commit
|
Test PASSed. |
@@ -219,6 +220,9 @@ class HadoopRDD[K, V]( | |||
val bytesReadCallback = if (split.inputSplit.value.isInstanceOf[FileSplit]) { | |||
SparkHadoopUtil.get.getFSBytesReadOnThreadCallback( | |||
split.inputSplit.value.asInstanceOf[FileSplit].getPath, jobConf) | |||
} else if (split.inputSplit.value.isInstanceOf[CombineFileSplit]) { | |||
SparkHadoopUtil.get.getFSBytesReadOnThreadCallback( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is it guaranteed that all paths in the CombineFileSplit
have the same filesystem?
Also one related question after I dug around a bit more. Hadoops FileSystem.getAllStatistics()
returns a list where you can only distinguish one file system from another via the scheme. What happens if two different hdfs://
filesystems are being read from within the same thread (for instance if two Hadoop RDD's are coalesced)? Is the assumption that this will never happen?
9962dd0
to
ff8a4cb
Compare
Test build #25848 has started for PR 4050 at commit
|
Test build #25848 has finished for PR 4050 at commit
|
Test PASSed. |
If we use a inputFormat that don‘t instanc of org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, FileSplit}, then we can't get information of input metrics. |
case _ => None | ||
val bytesReadCallback = inputMetrics.bytesReadCallback.orElse { | ||
val inputSplit = split.inputSplit.value | ||
if (inputSplit.isInstanceOf[FileSplit] || inputSplit.isInstanceOf[CombineFileSplit]) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this is fine as is, but fyi you can do the same thing in a pattern match:
split.inputSplit.value match {
case _: FileSplit | _: CombineFileSplit => SparkHadoopUtil.get.getFSBytesReadOnThreadCallback(jobConf)
case _ => None
}
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah, yours looks prettier, will switch it.
ff8a4cb
to
0d504f1
Compare
This is the desired behavior. The input metrics are currently only able to track the bytes read from a Hadoop-compatible file system. Many InputFormats (e.g. DBInputFormat) don't read from Hadoop-compatible file systems, so reporting "bytes read" would be misleading. |
Test build #26109 has started for PR 4050 at commit
|
val qualifiedPath = path.getFileSystem(conf).makeQualified(path) | ||
val scheme = qualifiedPath.toUri().getScheme() | ||
val stats = FileSystem.getAllStatistics().filter(_.getScheme().equals(scheme)) | ||
private def getFileSystemThreadStatistics(conf: Configuration): Seq[AnyRef] = { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
does this need to take a +conf+ object anymore... can we just remove the +conf+'s throughout this callstack?
Test build #26109 has finished for PR 4050 at commit
|
Test PASSed. |
Looking good, added some comments. One thing - could we change the title here to reflect the actual change (maybe we could even open a new JIRA or something). This now is a broader change, something like "Track Hadoop from all filesystems access inside of a task". Second, we don't actually add a test here for the case that was reported, notably the use of CompbineFileSpits. Can we explicitly test that to make sure we don't regress behavior? |
Edited the JIRA title and added tests for the CombineFileSplits. Tested both against Hadoop 2.3 (which doesn't support getFSBytesReadCallback) and Hadoop 2.5 (which does). |
Test build #26124 has started for PR 4050 at commit
|
Test build #26124 has finished for PR 4050 at commit
|
Test PASSed. |
Cool - thanks Sandy! |
Hey FYI, I think this patch is causing some problems. I got the following exception today:
|
@rxin this should be fixed by SPARK-5492 |
...mbineFileSplits