[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-26 Thread pwendell
Github user pwendell commented on the pull request:

https://github.com/apache/spark/pull/4050#issuecomment-71548906
  
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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-26 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4050#issuecomment-71548814
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/26109/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-26 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4050#issuecomment-71548800
  
  [Test build #26109 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26109/consoleFull)
 for   PR 4050 at commit 
[`0d504f1`](https://github.com/apache/spark/commit/0d504f15330214c1b6065e439e4ef9414f57a379).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-26 Thread pwendell
Github user pwendell commented on a diff in the pull request:

https://github.com/apache/spark/pull/4050#discussion_r23568576
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala 
---
@@ -172,10 +172,8 @@ class SparkHadoopUtil extends Logging {
 }
   }
 
-  private def getFileSystemThreadStatistics(path: Path, conf: 
Configuration): Seq[AnyRef] = {
-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] = {
--- End diff --

does this need to take a +conf+ object anymore... can we just remove the 
+conf+'s throughout this callstack?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-26 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4050#issuecomment-71537595
  
  [Test build #26109 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26109/consoleFull)
 for   PR 4050 at commit 
[`0d504f1`](https://github.com/apache/spark/commit/0d504f15330214c1b6065e439e4ef9414f57a379).
 * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-26 Thread sryza
Github user sryza commented on the pull request:

https://github.com/apache/spark/pull/4050#issuecomment-71537583
  
> 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.

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. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-26 Thread sryza
Github user sryza commented on a diff in the pull request:

https://github.com/apache/spark/pull/4050#discussion_r23563465
  
--- Diff: core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala ---
@@ -218,13 +219,14 @@ class HadoopRDD[K, V](
 
   // Find a function that will return the FileSystem bytes read by 
this thread. Do this before
   // creating RecordReader, because RecordReader's constructor might 
read some bytes
-  val bytesReadCallback = inputMetrics.bytesReadCallback.orElse(
-split.inputSplit.value match {
-  case split: FileSplit =>
-
SparkHadoopUtil.get.getFSBytesReadOnThreadCallback(split.getPath, jobConf)
-  case _ => None
+  val bytesReadCallback = inputMetrics.bytesReadCallback.orElse {
+val inputSplit = split.inputSplit.value
+if (inputSplit.isInstanceOf[FileSplit] || 
inputSplit.isInstanceOf[CombineFileSplit]) {
--- End diff --

Ah, yours looks prettier, will switch it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-26 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/4050#discussion_r23555820
  
--- Diff: core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala ---
@@ -218,13 +219,14 @@ class HadoopRDD[K, V](
 
   // Find a function that will return the FileSystem bytes read by 
this thread. Do this before
   // creating RecordReader, because RecordReader's constructor might 
read some bytes
-  val bytesReadCallback = inputMetrics.bytesReadCallback.orElse(
-split.inputSplit.value match {
-  case split: FileSplit =>
-
SparkHadoopUtil.get.getFSBytesReadOnThreadCallback(split.getPath, jobConf)
-  case _ => None
+  val bytesReadCallback = inputMetrics.bytesReadCallback.orElse {
+val inputSplit = split.inputSplit.value
+if (inputSplit.isInstanceOf[FileSplit] || 
inputSplit.isInstanceOf[CombineFileSplit]) {
--- End diff --

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
}


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-24 Thread shenh062326
Github user shenh062326 commented on the pull request:

https://github.com/apache/spark/pull/4050#issuecomment-71347965
  
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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-20 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4050#issuecomment-70755268
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/25848/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-20 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4050#issuecomment-70755263
  
  [Test build #25848 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/25848/consoleFull)
 for   PR 4050 at commit 
[`ff8a4cb`](https://github.com/apache/spark/commit/ff8a4cba9361e03441ee95489632b51724d673bb).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-20 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4050#issuecomment-70745023
  
  [Test build #25848 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/25848/consoleFull)
 for   PR 4050 at commit 
[`ff8a4cb`](https://github.com/apache/spark/commit/ff8a4cba9361e03441ee95489632b51724d673bb).
 * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-17 Thread pwendell
Github user pwendell commented on a diff in the pull request:

https://github.com/apache/spark/pull/4050#discussion_r23131557
  
--- Diff: core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala ---
@@ -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(
--- End diff --

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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-14 Thread ksakellis
Github user ksakellis commented on a diff in the pull request:

https://github.com/apache/spark/pull/4050#discussion_r22985033
  
--- Diff: core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala ---
@@ -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)
--- End diff --

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.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-14 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4050#issuecomment-70005837
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/25567/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-14 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4050#issuecomment-70005832
  
  [Test build #25567 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/25567/consoleFull)
 for   PR 4050 at commit 
[`9962dd0`](https://github.com/apache/spark/commit/9962dd097425442d62778f72911c6320c812f153).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-14 Thread sryza
Github user sryza commented on a diff in the pull request:

https://github.com/apache/spark/pull/4050#discussion_r22972019
  
--- Diff: core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala ---
@@ -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)
--- End diff --

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).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-14 Thread ksakellis
Github user ksakellis commented on the pull request:

https://github.com/apache/spark/pull/4050#issuecomment-69996188
  
This mostly LGTM. My only concern is with the proliferation of copy pasta 
between the HadoopRDD and NewHadoopRDD.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-14 Thread ksakellis
Github user ksakellis commented on a diff in the pull request:

https://github.com/apache/spark/pull/4050#discussion_r22971793
  
--- Diff: core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala ---
@@ -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)
--- End diff --

Can you push this logic down to the SparkHadoopUtil so that we don't 
duplicate it in two places (HadoopRDD and NewHadoopRDD). 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-14 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4050#issuecomment-69994416
  
  [Test build #25567 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/25567/consoleFull)
 for   PR 4050 at commit 
[`9962dd0`](https://github.com/apache/spark/commit/9962dd097425442d62778f72911c6320c812f153).
 * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: SPARK-5199. Input metrics should show up for I...

2015-01-14 Thread sryza
GitHub user sryza opened a pull request:

https://github.com/apache/spark/pull/4050

SPARK-5199. Input metrics should show up for InputFormats that return Co...

...mbineFileSplits

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/sryza/spark sandy-spark-5199

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/4050.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #4050


commit 9962dd097425442d62778f72911c6320c812f153
Author: Sandy Ryza 
Date:   2015-01-14T21:17:02Z

SPARK-5199. Input metrics should show up for InputFormats that return 
CombineFileSplits




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org